diff --git a/app-conf/AggregatorConf.xml b/app-conf/AggregatorConf.xml index 420146405..23586d587 100644 --- a/app-conf/AggregatorConf.xml +++ b/app-conf/AggregatorConf.xml @@ -35,11 +35,9 @@ spark - org.apache.spark.SparkMetricsAggregator - diff --git a/app-conf/FetcherConf.xml b/app-conf/FetcherConf.xml index e0651a6ec..d00b247f1 100644 --- a/app-conf/FetcherConf.xml +++ b/app-conf/FetcherConf.xml @@ -56,18 +56,9 @@ --> + spark - org.apache.spark.deploy.history.SparkFSFetcher - + com.linkedin.drelephant.spark.fetchers.SparkFetcher diff --git a/app-conf/HeuristicConf.xml b/app-conf/HeuristicConf.xml index 8e897c295..9dbb6b1bb 100644 --- a/app-conf/HeuristicConf.xml +++ b/app-conf/HeuristicConf.xml @@ -158,65 +158,27 @@ spark - Spark Configuration Best Practice - com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic - views.html.help.spark.helpBestProperties - + Spark Configuration + com.linkedin.drelephant.spark.heuristics.ConfigurationHeuristic + views.html.help.spark.helpConfigurationHeuristic - spark - Spark Memory Limit - com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic - views.html.help.spark.helpMemoryLimit - + Spark Executor Metrics + com.linkedin.drelephant.spark.heuristics.ExecutorsHeuristic + views.html.help.spark.helpExecutorsHeuristic - spark - Spark Stage Runtime - com.linkedin.drelephant.spark.heuristics.StageRuntimeHeuristic - views.html.help.spark.helpStageRuntime - + Spark Job Metrics + com.linkedin.drelephant.spark.heuristics.JobsHeuristic + views.html.help.spark.helpJobsHeuristic - - - spark - Spark Job Runtime - com.linkedin.drelephant.spark.heuristics.JobRuntimeHeuristic - views.html.help.spark.helpJobRuntime - - - - - spark - Spark Executor Load Balance - com.linkedin.drelephant.spark.heuristics.ExecutorLoadHeuristic - views.html.help.spark.helpExecutorLoad - - - spark - Spark Event Log Limit - com.linkedin.drelephant.spark.heuristics.EventLogLimitHeuristic - views.html.help.spark.helpEventLogLimit + Spark Stage Metrics + com.linkedin.drelephant.spark.heuristics.StagesHeuristic + views.html.help.spark.helpStagesHeuristic diff --git a/app/com/linkedin/drelephant/ElephantContext.java b/app/com/linkedin/drelephant/ElephantContext.java index 1788ca7fe..35e99876b 100644 --- a/app/com/linkedin/drelephant/ElephantContext.java +++ b/app/com/linkedin/drelephant/ElephantContext.java @@ -47,7 +47,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.log4j.Logger; -import org.apache.spark.SparkMetricsAggregator; import org.w3c.dom.Document; import play.api.Play; import play.api.templates.Html; diff --git a/app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java b/app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java index 27c3e3acf..2c88f3d7b 100644 --- a/app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java +++ b/app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java @@ -26,32 +26,32 @@ public class HadoopAggregatedData { private long totalDelay = 0; /** - * Returns the resource usage of the job - * @return The resource usage of the job + * Returns the resource usage (in MBSeconds) of the job + * @return The resource usage (in MBSeconds) of the job */ public long getResourceUsed() { return resourceUsed; } /** - * Setter for the resource usage of the job - * @param resourceUsed The resource usage of the job + * Setter for the resource usage (in MBSeconds) of the job + * @param resourceUsed The resource usage (in MBSeconds) of the job */ public void setResourceUsed(long resourceUsed) { this.resourceUsed = resourceUsed; } /** - * Returns the wasted resources of the job - * @return The wasted resources of the job + * Returns the wasted resources (in MBSeconds) of the job + * @return The wasted resources (in MBSeconds) of the job */ public long getResourceWasted() { return resourceWasted; } /** - * Setter for the wasted resources - * @param resourceWasted The wasted resources of the job + * Setter for the wasted resources (in MBSeconds) + * @param resourceWasted The wasted resources (in MBSeconds) of the job */ public void setResourceWasted(long resourceWasted) { this.resourceWasted = resourceWasted; diff --git a/app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java b/app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java index ac7d9e72e..caf2d8be6 100644 --- a/app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java +++ b/app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java @@ -29,6 +29,10 @@ public class HeuristicResultDetails { private String _value; private String _details; + public HeuristicResultDetails(String name, String value) { + this(name, value, null); + } + public HeuristicResultDetails(String name, String value, String details) { this._name = name; this._value = value; diff --git a/app/com/linkedin/drelephant/analysis/SeverityThresholds.scala b/app/com/linkedin/drelephant/analysis/SeverityThresholds.scala new file mode 100644 index 000000000..7359c7cac --- /dev/null +++ b/app/com/linkedin/drelephant/analysis/SeverityThresholds.scala @@ -0,0 +1,53 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.analysis + +import com.linkedin.drelephant.util.Utils + + +/** + * A convenience case class for containing severity thresholds and calculating severity. + */ +case class SeverityThresholds(low: Number, moderate: Number, severe: Number, critical: Number, ascending: Boolean) { + if (ascending) { + require(low.doubleValue <= moderate.doubleValue) + require(moderate.doubleValue <= severe.doubleValue) + require(severe.doubleValue <= critical.doubleValue) + } else { + require(low.doubleValue >= moderate.doubleValue) + require(moderate.doubleValue >= severe.doubleValue) + require(severe.doubleValue >= critical.doubleValue) + } + + def severityOf(value: Number): Severity = if (ascending) { + Severity.getSeverityAscending(value, low, moderate, severe, critical) + } else { + Severity.getSeverityDescending(value, low, moderate, severe, critical) + } +} + +object SeverityThresholds { + val NUM_THRESHOLDS = 4 + + /** Returns a SeverityThresholds object from a Dr. Elephant configuration string parseable by Utils.getParam(String, int). */ + def parse( + rawString: String, + ascending: Boolean + ): Option[SeverityThresholds] = Option(Utils.getParam(rawString, NUM_THRESHOLDS)).map { thresholds => + SeverityThresholds(low = thresholds(0), moderate = thresholds(1), severe = thresholds(2), critical = thresholds(3), ascending) + } +} diff --git a/app/com/linkedin/drelephant/math/Statistics.java b/app/com/linkedin/drelephant/math/Statistics.java index c8e1995a8..de608278f 100644 --- a/app/com/linkedin/drelephant/math/Statistics.java +++ b/app/com/linkedin/drelephant/math/Statistics.java @@ -98,7 +98,10 @@ public static long percentile(List values, int percentile) { } Collections.sort(values); - int position = (int) Math.ceil(values.size() * percentile / 100); + + // Use Nearest Rank method. + // https://en.wikipedia.org/wiki/Percentile#The_Nearest_Rank_method + int position = (int) Math.ceil(values.size() * percentile / 100.0); // should never happen. if (position == 0) { diff --git a/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala b/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala new file mode 100644 index 000000000..648de94fa --- /dev/null +++ b/app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala @@ -0,0 +1,120 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark + +import com.linkedin.drelephant.analysis.{HadoopAggregatedData, HadoopApplicationData, HadoopMetricsAggregator} +import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData +import com.linkedin.drelephant.math.Statistics +import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.util.MemoryFormatUtils +import org.apache.commons.io.FileUtils +import org.apache.log4j.Logger +import scala.util.Try + + +class SparkMetricsAggregator(private val aggregatorConfigurationData: AggregatorConfigurationData) + extends HadoopMetricsAggregator { + import SparkMetricsAggregator._ + + private val logger: Logger = Logger.getLogger(classOf[SparkMetricsAggregator]) + + private val allocatedMemoryWasteBufferPercentage: Double = + Option(aggregatorConfigurationData.getParamMap.get(ALLOCATED_MEMORY_WASTE_BUFFER_PERCENTAGE_KEY)) + .flatMap { value => Try(value.toDouble).toOption } + .getOrElse(DEFAULT_ALLOCATED_MEMORY_WASTE_BUFFER_PERCENTAGE) + + private val hadoopAggregatedData: HadoopAggregatedData = new HadoopAggregatedData() + + override def getResult(): HadoopAggregatedData = hadoopAggregatedData + + override def aggregate(data: HadoopApplicationData): Unit = data match { + case (data: SparkApplicationData) => aggregate(data) + case _ => throw new IllegalArgumentException("data should be SparkApplicationData") + } + + private def aggregate(data: SparkApplicationData): Unit = for { + executorInstances <- executorInstancesOf(data) + executorMemoryBytes <- executorMemoryBytesOf(data) + } { + val applicationDurationMillis = applicationDurationMillisOf(data) + val totalExecutorTaskTimeMillis = totalExecutorTaskTimeMillisOf(data) + + val resourcesAllocatedMBSeconds = + aggregateResourcesAllocatedMBSeconds(executorInstances, executorMemoryBytes, applicationDurationMillis) + val resourcesUsedMBSeconds = aggregateResourcesUsedMBSeconds(executorMemoryBytes, totalExecutorTaskTimeMillis) + + val resourcesWastedMBSeconds = + ((BigDecimal(resourcesAllocatedMBSeconds) * (1.0 - allocatedMemoryWasteBufferPercentage)) - BigDecimal(resourcesUsedMBSeconds)) + .toBigInt + + if (resourcesUsedMBSeconds.isValidLong) { + hadoopAggregatedData.setResourceUsed(resourcesUsedMBSeconds.toLong) + } else { + logger.info(s"resourcesUsedMBSeconds exceeds Long.MaxValue: ${resourcesUsedMBSeconds}") + } + + if (resourcesWastedMBSeconds.isValidLong) { + hadoopAggregatedData.setResourceWasted(resourcesWastedMBSeconds.toLong) + } else { + logger.info(s"resourcesWastedMBSeconds exceeds Long.MaxValue: ${resourcesWastedMBSeconds}") + } + } + + private def aggregateResourcesUsedMBSeconds(executorMemoryBytes: Long, totalExecutorTaskTimeMillis: BigInt): BigInt = { + val bytesMillis = BigInt(executorMemoryBytes) * totalExecutorTaskTimeMillis + (bytesMillis / (BigInt(FileUtils.ONE_MB) * BigInt(Statistics.SECOND_IN_MS))) + } + + private def aggregateResourcesAllocatedMBSeconds( + executorInstances: Int, + executorMemoryBytes: Long, + applicationDurationMillis: Long + ): BigInt = { + val bytesMillis = BigInt(executorInstances) * BigInt(executorMemoryBytes) * BigInt(applicationDurationMillis) + (bytesMillis / (BigInt(FileUtils.ONE_MB) * BigInt(Statistics.SECOND_IN_MS))) + } + + private def executorInstancesOf(data: SparkApplicationData): Option[Int] = { + val appConfigurationProperties = data.appConfigurationProperties + appConfigurationProperties.get(SPARK_EXECUTOR_INSTANCES_KEY).map(_.toInt) + } + + private def executorMemoryBytesOf(data: SparkApplicationData): Option[Long] = { + val appConfigurationProperties = data.appConfigurationProperties + appConfigurationProperties.get(SPARK_EXECUTOR_MEMORY_KEY).map(MemoryFormatUtils.stringToBytes) + } + + private def applicationDurationMillisOf(data: SparkApplicationData): Long = { + require(data.applicationInfo.attempts.nonEmpty) + val lastApplicationAttemptInfo = data.applicationInfo.attempts.last + lastApplicationAttemptInfo.endTime.getTime - lastApplicationAttemptInfo.startTime.getTime + } + + private def totalExecutorTaskTimeMillisOf(data: SparkApplicationData): BigInt = { + data.executorSummaries.map { executorSummary => BigInt(executorSummary.totalDuration) }.sum + } +} + +object SparkMetricsAggregator { + /** The percentage of allocated memory we expect to waste because of overhead. */ + val DEFAULT_ALLOCATED_MEMORY_WASTE_BUFFER_PERCENTAGE = 0.5D + + val ALLOCATED_MEMORY_WASTE_BUFFER_PERCENTAGE_KEY = "allocated_memory_waste_buffer_percentage" + + val SPARK_EXECUTOR_INSTANCES_KEY = "spark.executor.instances" + val SPARK_EXECUTOR_MEMORY_KEY = "spark.executor.memory" +} diff --git a/app/com/linkedin/drelephant/spark/data/SparkApplicationData.scala b/app/com/linkedin/drelephant/spark/data/SparkApplicationData.scala new file mode 100644 index 000000000..6e6ac5972 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/data/SparkApplicationData.scala @@ -0,0 +1,70 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.data + +import java.util.Properties + +import scala.collection.JavaConverters + +import com.linkedin.drelephant.analysis.{ApplicationType, HadoopApplicationData} +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, ExecutorSummary, JobData, StageData} + + +case class SparkApplicationData( + appId: String, + appConfigurationProperties: Map[String, String], + applicationInfo: ApplicationInfo, + jobDatas: Seq[JobData], + stageDatas: Seq[StageData], + executorSummaries: Seq[ExecutorSummary] +) extends HadoopApplicationData { + import SparkApplicationData._ + import JavaConverters._ + + override def getApplicationType(): ApplicationType = APPLICATION_TYPE + + override def getConf(): Properties = { + val properties = new Properties() + properties.putAll(appConfigurationProperties.asJava) + properties + } + + override def getAppId(): String = appId + + // This instance will always have data, or at least the data the Spark REST API gives us. + override def isEmpty(): Boolean = false +} + +object SparkApplicationData { + val APPLICATION_TYPE = new ApplicationType("SPARK") + + def apply( + appId: String, + restDerivedData: SparkRestDerivedData, + logDerivedData: Option[SparkLogDerivedData] + ): SparkApplicationData = { + val appConfigurationProperties: Map[String, String] = + logDerivedData + .flatMap { _.environmentUpdate.environmentDetails.get("Spark Properties").map(_.toMap) } + .getOrElse(Map.empty) + val applicationInfo = restDerivedData.applicationInfo + val jobDatas = restDerivedData.jobDatas + val stageDatas = restDerivedData.stageDatas + val executorSummaries = restDerivedData.executorSummaries + apply(appId, appConfigurationProperties, applicationInfo, jobDatas, stageDatas, executorSummaries) + } +} diff --git a/app/com/linkedin/drelephant/spark/data/SparkEnvironmentData.java b/app/com/linkedin/drelephant/spark/data/SparkEnvironmentData.java deleted file mode 100644 index 05ce4e9dc..000000000 --- a/app/com/linkedin/drelephant/spark/data/SparkEnvironmentData.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.data; - -import java.util.Properties; - - -/** - * This data class holds Spark environment data (Spark properties, JVM properties and etc.) - */ -public class SparkEnvironmentData { - private final Properties _sparkProperties; - private final Properties _systemProperties; - - public SparkEnvironmentData() { - _sparkProperties = new Properties(); - _systemProperties = new Properties(); - } - - public void addSparkProperty(String key, String value) { - _sparkProperties.put(key, value); - } - - public void addSystemProperty(String key, String value) { - _systemProperties.put(key, value); - } - - public String getSparkProperty(String key) { - return _sparkProperties.getProperty(key); - } - - public String getSparkProperty(String key, String defaultValue) { - String val = getSparkProperty(key); - if (val == null) { - return defaultValue; - } - return val; - } - - public String getSystemProperty(String key) { - return _systemProperties.getProperty(key); - } - - public Properties getSparkProperties() { - return _sparkProperties; - } - - public Properties getSystemProperties() { - return _systemProperties; - } - - @Override - public String toString() { - return _sparkProperties.toString() + "\n\n\n" + _systemProperties.toString(); - } -} diff --git a/app/com/linkedin/drelephant/spark/data/SparkExecutorData.java b/app/com/linkedin/drelephant/spark/data/SparkExecutorData.java deleted file mode 100644 index df56dc213..000000000 --- a/app/com/linkedin/drelephant/spark/data/SparkExecutorData.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.data; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - - -/** - * This class contains Spark executor information. - */ -public class SparkExecutorData { - public static final String EXECUTOR_DRIVER_NAME = "driver"; - - public static class ExecutorInfo { - public String execId; - public String hostPort; - public int rddBlocks = 0; - public long memUsed = 0L; - public long maxMem = 0L; - public long diskUsed = 0L; - - public int activeTasks = 0; - public int completedTasks = 0; - public int failedTasks = 0; - public int totalTasks = 0; - public long duration = 0L; - public long inputBytes = 0L; - public long outputBytes = 0L; - public long shuffleRead = 0L; - public long shuffleWrite = 0L; - - public String toString() { - return "{execId: " + execId + ", hostPort:" + hostPort + " , rddBlocks: " + rddBlocks + ", memUsed: " + memUsed - + ", maxMem: " + maxMem + ", diskUsed: " + diskUsed + ", totalTasks" + totalTasks + ", tasksActive: " - + activeTasks + ", tasksComplete: " + completedTasks + ", tasksFailed: " + failedTasks + ", duration: " - + duration + ", inputBytes: " + inputBytes + ", outputBytes:" + outputBytes + ", shuffleRead: " + shuffleRead - + ", shuffleWrite: " + shuffleWrite + "}"; - } - } - - private final Map _executorInfoMap = new HashMap(); - - public void setExecutorInfo(String executorId, ExecutorInfo info) { - _executorInfoMap.put(executorId, info); - } - - public ExecutorInfo getExecutorInfo(String executorId) { - return _executorInfoMap.get(executorId); - } - - public Set getExecutors() { - return _executorInfoMap.keySet(); - } -} diff --git a/app/com/linkedin/drelephant/spark/data/SparkGeneralData.java b/app/com/linkedin/drelephant/spark/data/SparkGeneralData.java deleted file mode 100644 index 11afdd088..000000000 --- a/app/com/linkedin/drelephant/spark/data/SparkGeneralData.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.data; - -import java.util.Set; - - -/** - * This class holds Spark application information - */ -public class SparkGeneralData { - private Set _adminAcls; - private Set _viewAcls; - private String _applicationId; - private String _applicationName; - private String _sparkUser; - private long _startTime; - private long _endTime; - - public Set getAdminAcls() { - return _adminAcls; - } - - public void setAdminAcls(Set adminAcls) { - _adminAcls = adminAcls; - } - - public Set getViewAcls() { - return _viewAcls; - } - - public void setViewAcls(Set viewAcls) { - _viewAcls = viewAcls; - } - - public String getApplicationId() { - return _applicationId; - } - - public void setApplicationId(String applicationId) { - _applicationId = applicationId; - } - - public String getApplicationName() { - return _applicationName; - } - - public void setApplicationName(String applicationName) { - _applicationName = applicationName; - } - - public String getSparkUser() { - return _sparkUser; - } - - public void setSparkUser(String sparkUser) { - _sparkUser = sparkUser; - } - - public long getStartTime() { - return _startTime; - } - - public void setStartTime(long startTime) { - _startTime = startTime; - } - - public long getEndTime() { - return _endTime; - } - - public void setEndTime(long endTime) { - _endTime = endTime; - } -} diff --git a/app/com/linkedin/drelephant/spark/data/SparkJobProgressData.java b/app/com/linkedin/drelephant/spark/data/SparkJobProgressData.java deleted file mode 100644 index 417212a79..000000000 --- a/app/com/linkedin/drelephant/spark/data/SparkJobProgressData.java +++ /dev/null @@ -1,265 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.data; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - - -/** - * This class represents information contained in a job runtime process. - */ -public class SparkJobProgressData { - private static final Logger logger = Logger.getLogger(SparkJobProgressData.class); - private final Map _jobIdToInfo = new HashMap(); - private final Set _completedJobs = new HashSet(); - private final Set _failedJobs = new HashSet(); - - private final Map _stageIdToInfo = new HashMap(); - private final Set _completedStages = new HashSet(); - private final Set _failedStages = new HashSet(); - - public void addJobInfo(int jobId, JobInfo info) { - _jobIdToInfo.put(jobId, info); - } - - public void addCompletedJob(int jobId) { - _completedJobs.add(jobId); - } - - public void addFailedJob(int jobId) { - _failedJobs.add(jobId); - } - - public void addStageInfo(int stageId, int attemptId, StageInfo info) { - _stageIdToInfo.put(new StageAttemptId(stageId, attemptId), info); - } - - public void addCompletedStages(int stageId, int attemptId) { - _completedStages.add(new StageAttemptId(stageId, attemptId)); - } - - public void addFailedStages(int stageId, int attemptId) { - _failedStages.add(new StageAttemptId(stageId, attemptId)); - } - - public Set getCompletedJobs() { - return _completedJobs; - } - - public Set getFailedJobs() { - return _failedJobs; - } - - private static double getFailureRate(int numCompleted, int numFailed) { - int num = numCompleted + numFailed; - - if (num == 0) { - return 0d; - } - - return numFailed * 1.0d / num; - } - - public double getJobFailureRate() { - return getFailureRate(_completedJobs.size(), _failedJobs.size()); - } - - public double getStageFailureRate() { - return getFailureRate(_completedStages.size(), _failedStages.size()); - } - - public JobInfo getJobInfo(int jobId) { - return _jobIdToInfo.get(jobId); - } - - public StageInfo getStageInfo(int stageId, int attemptId) { - return _stageIdToInfo.get(new StageAttemptId(stageId, attemptId)); - } - - public Set getCompletedStages() { - return _completedStages; - } - - public Set getFailedStages() { - return _failedStages; - } - - /** - * Job itself does not have a name, it will use its latest stage as the name. - * - * @param jobId - * @return - */ - public String getJobDescription(int jobId) { - List stageIds = _jobIdToInfo.get(jobId).stageIds; - int id = -1; - for (int stageId : stageIds) { - id = Math.max(id, stageId); - } - if (id == -1) { - logger.error("Spark Job id [" + jobId + "] does not contain any stage."); - return null; - } - return _stageIdToInfo.get(new StageAttemptId(id, 0)).name; - } - - public List getFailedJobDescriptions() { - List result = new ArrayList(); - for (int id : _failedJobs) { - result.add(getJobDescription(id)); - } - return result; - } - - // For debug purpose - public String toString() { - StringBuilder s = new StringBuilder(); - s.append("JobInfo: ["); - - for (Map.Entry entry : _jobIdToInfo.entrySet()) { - s.append("{id:" + entry.getKey() + ", value: " + entry.getValue() + "}"); - } - - s.append("]\nStageInfo: ["); - for (Map.Entry entry : _stageIdToInfo.entrySet()) { - s.append("{id:" + entry.getKey() + ", value: " + entry.getValue() + "}"); - } - s.append("]"); - - return s.toString(); - } - - public static class StageAttemptId { - public int stageId; - public int attemptId; - - public StageAttemptId(int stageId, int attemptId) { - this.stageId = stageId; - this.attemptId = attemptId; - } - - @Override - public int hashCode() { - return new Integer(stageId).hashCode() * 31 + new Integer(attemptId).hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof StageAttemptId) { - StageAttemptId other = (StageAttemptId) obj; - return stageId == other.stageId && attemptId == other.attemptId; - } - return false; - } - - public String toString() { - return "id: " + stageId + " # attemptId: " + attemptId; - } - } - - public static class JobInfo { - public int jobId; - public String jobGroup; - public long startTime; - public long endTime; - public final List stageIds = new ArrayList(); - - /* Tasks */ - public int numTasks = 0; - public int numActiveTasks = 0; - public int numCompletedTasks = 0; - public int numSkippedTasks = 0; - public int numFailedTasks = 0; - - /* Stages */ - public int numActiveStages = 0; - // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: - public final Set completedStageIndices = new HashSet(); - public int numSkippedStages = 0; - public int numFailedStages = 0; - - public void addStageId(int stageId) { - stageIds.add(stageId); - } - - public double getFailureRate() { - return SparkJobProgressData.getFailureRate(numCompletedTasks, numFailedTasks); - } - - public String toString() { - return String.format("{jobId:%s, jobGroup:%s, startTime:%s, endTime:%s, numTask:%s, numActiveTasks:%s, " - + "numCompletedTasks:%s, numSkippedTasks:%s, numFailedTasks:%s, numActiveStages:%s, " - + "completedStageIndices:%s, stages:%s, numSkippedStages:%s, numFailedStages:%s}", jobId, jobGroup, - startTime, endTime, numTasks, numActiveTasks, numCompletedTasks, numSkippedTasks, numFailedTasks, - numActiveStages, getListString(completedStageIndices), getListString(stageIds), numSkippedStages, - numFailedStages); - } - } - - public static class StageInfo { - public int numActiveTasks; - public int numCompleteTasks; - public final Set completedIndices = new HashSet(); - public int numFailedTasks; - - // Total accumulated executor runtime - public long executorRunTime; - // Total stage duration - public long duration; - - // Note, currently calculating I/O speed on stage level does not make sense - // since we do not have information about specific I/O time. - public long inputBytes = 0; - public long outputBytes = 0; - public long shuffleReadBytes = 0; - public long shuffleWriteBytes = 0; - public long memoryBytesSpilled = 0; - public long diskBytesSpilled = 0; - - public String name; - public String description; - - public double getFailureRate() { - return SparkJobProgressData.getFailureRate(numCompleteTasks, numFailedTasks); - } - - // TODO: accumulables info seem to be unnecessary, might might be useful later on - // sample code from Spark source: var accumulables = new HashMap[Long, AccumulableInfo] - - @Override - public String toString() { - return String.format("{numActiveTasks:%s, numCompleteTasks:%s, completedIndices:%s, numFailedTasks:%s," - + " executorRunTime:%s, inputBytes:%s, outputBytes:%s, shuffleReadBytes:%s, shuffleWriteBytes:%s," - + " memoryBytesSpilled:%s, diskBytesSpilled:%s, name:%s, description:%s}", - numActiveTasks, numCompleteTasks, getListString(completedIndices), numFailedTasks, executorRunTime, - inputBytes, outputBytes, shuffleReadBytes, shuffleWriteBytes, memoryBytesSpilled, diskBytesSpilled, name, - description); - } - } - - private static String getListString(Collection collection) { - return "[" + StringUtils.join(collection, ",") + "]"; - } -} diff --git a/test/com/linkedin/drelephant/spark/heuristics/JobRuntimeHeuristicTest.java b/app/com/linkedin/drelephant/spark/data/SparkLogDerivedData.scala similarity index 65% rename from test/com/linkedin/drelephant/spark/heuristics/JobRuntimeHeuristicTest.java rename to app/com/linkedin/drelephant/spark/data/SparkLogDerivedData.scala index 7a6b5ac59..537234a5c 100644 --- a/test/com/linkedin/drelephant/spark/heuristics/JobRuntimeHeuristicTest.java +++ b/app/com/linkedin/drelephant/spark/data/SparkLogDerivedData.scala @@ -14,17 +14,11 @@ * the License. */ -package com.linkedin.drelephant.spark.heuristics; +package com.linkedin.drelephant.spark.data -import junit.framework.TestCase; +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate -/** - * This class tests JobRuntimeHeuristic - */ -public class JobRuntimeHeuristicTest extends TestCase { - - public void test() { - - } +case class SparkLogDerivedData(environmentUpdate: SparkListenerEnvironmentUpdate) { + def appConfigurationProperties: Map[String, String] = environmentUpdate.environmentDetails("Spark Properties").toMap } diff --git a/app/com/linkedin/drelephant/spark/data/SparkApplicationData.java b/app/com/linkedin/drelephant/spark/data/SparkRestDerivedData.scala similarity index 53% rename from app/com/linkedin/drelephant/spark/data/SparkApplicationData.java rename to app/com/linkedin/drelephant/spark/data/SparkRestDerivedData.scala index 2583a364b..721c4f3c9 100644 --- a/app/com/linkedin/drelephant/spark/data/SparkApplicationData.java +++ b/app/com/linkedin/drelephant/spark/data/SparkRestDerivedData.scala @@ -14,25 +14,14 @@ * the License. */ -package com.linkedin.drelephant.spark.data; +package com.linkedin.drelephant.spark.data -import com.linkedin.drelephant.analysis.HadoopApplicationData; +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, ExecutorSummary, JobData, StageData} -/** - * This holds a collection of all SparkApplicationData - */ -public interface SparkApplicationData extends HadoopApplicationData { - - public boolean isThrottled(); - - public SparkGeneralData getGeneralData(); - - public SparkEnvironmentData getEnvironmentData(); - - public SparkExecutorData getExecutorData(); - - public SparkJobProgressData getJobProgressData(); - - public SparkStorageData getStorageData(); -} +case class SparkRestDerivedData( + applicationInfo: ApplicationInfo, + jobDatas: Seq[JobData], + stageDatas: Seq[StageData], + executorSummaries: Seq[ExecutorSummary] +) diff --git a/app/com/linkedin/drelephant/spark/data/SparkStorageData.java b/app/com/linkedin/drelephant/spark/data/SparkStorageData.java deleted file mode 100644 index d821d4acd..000000000 --- a/app/com/linkedin/drelephant/spark/data/SparkStorageData.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.data; - -import java.util.List; -import org.apache.spark.storage.RDDInfo; -import org.apache.spark.storage.StorageStatus; - - -/** - * This class holds information related to Spark storage (RDDs specifically) information. - */ -public class SparkStorageData { - private List _rddInfoList; - private List _storageStatusList; - - public List getRddInfoList() { - return _rddInfoList; - } - - public void setRddInfoList(List rddInfoList) { - _rddInfoList = rddInfoList; - } - - public List getStorageStatusList() { - return _storageStatusList; - } - - public void setStorageStatusList(List storageStatusList) { - _storageStatusList = storageStatusList; - } -} diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala new file mode 100644 index 000000000..0cf703f23 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala @@ -0,0 +1,101 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.fetchers + +import scala.async.Async +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.duration.{Duration, SECONDS} +import scala.util.Try +import scala.util.control.NonFatal + +import com.linkedin.drelephant.analysis.{AnalyticJob, ElephantFetcher} +import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData +import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.util.SparkUtils +import org.apache.hadoop.conf.Configuration +import org.apache.log4j.Logger +import org.apache.spark.SparkConf + + +/** + * A fetcher that gets Spark-related data from a combination of the Spark monitoring REST API and Spark event logs. + */ +class SparkFetcher(fetcherConfigurationData: FetcherConfigurationData) + extends ElephantFetcher[SparkApplicationData] { + import SparkFetcher._ + import ExecutionContext.Implicits.global + + private val logger: Logger = Logger.getLogger(classOf[SparkFetcher]) + + private[fetchers] lazy val hadoopConfiguration: Configuration = new Configuration() + + private[fetchers] lazy val sparkUtils: SparkUtils = SparkUtils + + private[fetchers] lazy val sparkConf: SparkConf = { + val sparkConf = new SparkConf() + sparkUtils.getDefaultPropertiesFile(sparkUtils.defaultEnv) match { + case Some(filename) => sparkConf.setAll(sparkUtils.getPropertiesFromFile(filename)) + case None => throw new IllegalStateException("can't find Spark conf; please set SPARK_HOME or SPARK_CONF_DIR") + } + sparkConf + } + + private[fetchers] lazy val sparkRestClient: SparkRestClient = new SparkRestClient(sparkConf) + + private[fetchers] lazy val sparkLogClient: Option[SparkLogClient] = { + val eventLogEnabled = sparkConf.getBoolean(SPARK_EVENT_LOG_ENABLED_KEY, false) + if (eventLogEnabled) Some(new SparkLogClient(hadoopConfiguration, sparkConf)) else None + } + + override def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { + val appId = analyticJob.getAppId + logger.info(s"Fetching data for ${appId}") + try { + Await.result(doFetchData(sparkRestClient, sparkLogClient, appId), DEFAULT_TIMEOUT) + } catch { + case NonFatal(e) => + logger.error(s"Failed fetching data for ${appId}", e) + throw e + } + } +} + +object SparkFetcher { + import Async.{async, await} + + val SPARK_EVENT_LOG_ENABLED_KEY = "spark.eventLog.enabled" + val DEFAULT_TIMEOUT = Duration(30, SECONDS) + + private def doFetchData( + sparkRestClient: SparkRestClient, + sparkLogClient: Option[SparkLogClient], + appId: String + )( + implicit ec: ExecutionContext + ): Future[SparkApplicationData] = async { + val restDerivedData = await(sparkRestClient.fetchData(appId)) + val lastAttemptId = restDerivedData.applicationInfo.attempts.maxBy { _.startTime }.attemptId + + // Would use .map but await doesn't like that construction. + val logDerivedData = sparkLogClient match { + case Some(sparkLogClient) => Some(await(sparkLogClient.fetchData(appId, lastAttemptId))) + case None => None + } + + SparkApplicationData(appId, restDerivedData, logDerivedData) + } +} diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala new file mode 100644 index 000000000..75fcbff78 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala @@ -0,0 +1,237 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.fetchers + +import java.io.{BufferedInputStream, FileNotFoundException, InputStream} +import java.net.URI + +import scala.async.Async +import scala.collection.mutable.HashMap +import scala.concurrent.{ExecutionContext, Future} +import scala.io.Source + +import com.linkedin.drelephant.spark.data.SparkLogDerivedData +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.log4j.Logger +import org.apache.spark.SparkConf +import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec, LZFCompressionCodec, SnappyCompressionCodec} +import org.apache.spark.scheduler.{SparkListenerEnvironmentUpdate, SparkListenerEvent} +import org.json4s.{DefaultFormats, JsonAST} +import org.json4s.jackson.JsonMethods + + +/** + * A client for getting data from the Spark event logs, using the location configured for spark.eventLog.dir. + * + * This client uses webhdfs to access the location, even if spark.eventLog.dir is provided as an hdfs URL. + * + * The codecs used by this client use JNI, which results in some weird classloading issues (at least when testing in the console), + * so some of the client's implementation is non-lazy or synchronous when needed. + */ +class SparkLogClient(hadoopConfiguration: Configuration, sparkConf: SparkConf) { + import SparkLogClient._ + import Async.async + + private val logger: Logger = Logger.getLogger(classOf[SparkLogClient]) + + private[fetchers] val webhdfsEventLogUri: URI = { + val eventLogUri = sparkConf.getOption(SPARK_EVENT_LOG_DIR_KEY).map(new URI(_)) + val dfsNamenodeHttpAddress = Option(hadoopConfiguration.get(HADOOP_DFS_NAMENODE_HTTP_ADDRESS_KEY)) + (eventLogUri, dfsNamenodeHttpAddress) match { + case (Some(eventLogUri), _) if eventLogUri.getScheme == "webhdfs" => + eventLogUri + case (Some(eventLogUri), Some(dfsNamenodeHttpAddress)) if eventLogUri.getScheme == "hdfs" => + val dfsNamenodeHttpUri = new URI(null, dfsNamenodeHttpAddress, null, null, null) + new URI(s"webhdfs://${eventLogUri.getHost}:${dfsNamenodeHttpUri.getPort}${eventLogUri.getPath}") + case _ => + throw new IllegalArgumentException( + s"""|${SPARK_EVENT_LOG_DIR_KEY} must be provided as webhdfs:// or hdfs://; + |if hdfs, ${HADOOP_DFS_NAMENODE_HTTP_ADDRESS_KEY} must also be provided for port""".stripMargin.replaceAll("\n", " ") + ) + } + } + + private[fetchers] lazy val fs: FileSystem = FileSystem.get(webhdfsEventLogUri, hadoopConfiguration) + + private lazy val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", defaultValue = false) + private lazy val compressionCodec = if (shouldCompress) Some(compressionCodecFromConf(sparkConf)) else None + private lazy val compressionCodecShortName = compressionCodec.map(shortNameOfCompressionCodec) + + def fetchData(appId: String, attemptId: Option[String])(implicit ec: ExecutionContext): Future[SparkLogDerivedData] = { + val logPath = getLogPath(webhdfsEventLogUri, appId, attemptId, compressionCodecShortName) + logger.info(s"looking for logs at ${logPath}") + + val codec = compressionCodecForLogPath(sparkConf, logPath) + + // Limit scope of async. + async { + resource.managed { openEventLog(sparkConf, logPath, fs) } + .acquireAndGet { in => findDerivedData(codec.map { _.compressedInputStream(in) }.getOrElse(in)) } + } + } +} + +object SparkLogClient { + import JsonAST._ + + val SPARK_EVENT_LOG_DIR_KEY = "spark.eventLog.dir" + val HADOOP_DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address" + + private implicit val formats: DefaultFormats = DefaultFormats + + def findDerivedData(in: InputStream, eventsLimit: Option[Int] = None): SparkLogDerivedData = { + val events = eventsLimit.map { getEvents(in).take(_) }.getOrElse { getEvents(in) } + + var environmentUpdate: Option[SparkListenerEnvironmentUpdate] = None + while (events.hasNext && environmentUpdate.isEmpty) { + val event = events.next + event match { + case Some(eu: SparkListenerEnvironmentUpdate) => environmentUpdate = Some(eu) + case _ => {} // Do nothing. + } + } + + environmentUpdate + .map(SparkLogDerivedData(_)) + .getOrElse { throw new IllegalArgumentException("Spark event log doesn't have Spark properties") } + } + + private def getEvents(in: InputStream): Iterator[Option[SparkListenerEvent]] = getLines(in).map(lineToEvent) + + private def getLines(in: InputStream): Iterator[String] = Source.fromInputStream(in).getLines + + private def lineToEvent(line: String): Option[SparkListenerEvent] = sparkEventFromJson(JsonMethods.parse(line)) + + // Below this line are modified utility methods from: + // + // https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala + // https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala + // https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/util/Utils.scala + + private val IN_PROGRESS = ".inprogress" + private val DEFAULT_COMPRESSION_CODEC = "snappy" + + private val compressionCodecClassNamesByShortName = Map( + "lz4" -> classOf[LZ4CompressionCodec].getName, + "lzf" -> classOf[LZFCompressionCodec].getName, + "snappy" -> classOf[SnappyCompressionCodec].getName + ) + + // A cache for compression codecs to avoid creating the same codec many times + private val compressionCodecMap = HashMap.empty[String, CompressionCodec] + + private def compressionCodecFromConf(conf: SparkConf): CompressionCodec = { + val codecName = conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC) + loadCompressionCodec(conf, codecName) + } + + private def loadCompressionCodec(conf: SparkConf, codecName: String): CompressionCodec = { + val codecClass = compressionCodecClassNamesByShortName.getOrElse(codecName.toLowerCase, codecName) + val classLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getClass.getClassLoader) + val codec = try { + val ctor = Class.forName(codecClass, true, classLoader).getConstructor(classOf[SparkConf]) + Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) + } catch { + case e: ClassNotFoundException => None + case e: IllegalArgumentException => None + } + codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. ")) + } + + private def shortNameOfCompressionCodec(compressionCodec: CompressionCodec): String = { + val codecName = compressionCodec.getClass.getName + if (compressionCodecClassNamesByShortName.contains(codecName)) { + codecName + } else { + compressionCodecClassNamesByShortName + .collectFirst { case (k, v) if v == codecName => k } + .getOrElse { throw new IllegalArgumentException(s"No short name for codec $codecName.") } + } + } + + private def getLogPath( + logBaseDir: URI, + appId: String, + appAttemptId: Option[String], + compressionCodecName: Option[String] = None + ): Path = { + val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) + val codec = compressionCodecName.map("." + _).getOrElse("") + if (appAttemptId.isDefined) { + new Path(base + "_" + sanitize(appAttemptId.get) + codec) + } else { + new Path(base + codec) + } + } + + private def openEventLog(conf: SparkConf, logPath: Path, fs: FileSystem): InputStream = { + // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain + // IOException when a file does not exist, so try our best to throw a proper exception. + if (!fs.exists(logPath)) { + throw new FileNotFoundException(s"File ${logPath} does not exist.") + } + + new BufferedInputStream(fs.open(logPath)) + } + + private def compressionCodecForLogPath(conf: SparkConf, logPath: Path): Option[CompressionCodec] = { + // Compression codec is encoded as an extension, e.g. app_123.lzf + // Since we sanitize the app ID to not include periods, it is safe to split on it + val logBaseName = logPath.getName.stripSuffix(IN_PROGRESS) + logBaseName.split("\\.").tail.lastOption.map { codecName => + compressionCodecMap.getOrElseUpdate(codecName, loadCompressionCodec(conf, codecName)) + } + } + + private def sanitize(str: String): String = { + str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase + } + + private def sparkEventFromJson(json: JValue): Option[SparkListenerEvent] = { + val environmentUpdate = getFormattedClassName(SparkListenerEnvironmentUpdate) + + (json \ "Event").extract[String] match { + case `environmentUpdate` => Some(environmentUpdateFromJson(json)) + case _ => None + } + } + + private def getFormattedClassName(obj: AnyRef): String = obj.getClass.getSimpleName.replace("$", "") + + private def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { + val environmentDetails = Map[String, Seq[(String, String)]]( + "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, + "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, + "System Properties" -> mapFromJson(json \ "System Properties").toSeq, + "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) + SparkListenerEnvironmentUpdate(environmentDetails) + } + + private def mapFromJson(json: JValue): Map[String, String] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, JString(v)) => (k, v) }.toMap + } + + /** Return an option that translates JNothing to None */ + private def jsonOption(json: JValue): Option[JValue] = { + json match { + case JNothing => None + case value: JValue => Some(value) + } + } +} diff --git a/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala b/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala new file mode 100644 index 000000000..48adb9d78 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala @@ -0,0 +1,158 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.fetchers + +import java.net.URI +import java.text.SimpleDateFormat +import java.util.{Calendar, SimpleTimeZone} + +import scala.async.Async +import scala.concurrent.{ExecutionContext, Future} +import scala.util.control.NonFatal + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper +import com.linkedin.drelephant.spark.data.SparkRestDerivedData +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo, ExecutorSummary, JobData, StageData} +import javax.ws.rs.client.{Client, ClientBuilder, WebTarget} +import javax.ws.rs.core.MediaType +import org.apache.log4j.Logger +import org.apache.spark.SparkConf + + +/** + * A client for getting data from the Spark monitoring REST API, e.g. . + * + * Jersey classloading seems to be brittle (at least when testing in the console), so some of the implementation is non-lazy + * or synchronous when needed. + */ +class SparkRestClient(sparkConf: SparkConf) { + import SparkRestClient._ + import Async.{async, await} + + private val logger: Logger = Logger.getLogger(classOf[SparkRestClient]) + + private val client: Client = ClientBuilder.newClient() + + private val historyServerUri: URI = sparkConf.getOption(HISTORY_SERVER_ADDRESS_KEY) match { + case Some(historyServerAddress) => + val baseUri = new URI(s"http://${historyServerAddress}") + require(baseUri.getPath == "") + baseUri + case None => + throw new IllegalArgumentException("spark.yarn.historyServer.address not provided; can't use Spark REST API") + } + + private val apiTarget: WebTarget = client.target(historyServerUri).path(API_V1_MOUNT_PATH) + + def fetchData(appId: String)(implicit ec: ExecutionContext): Future[SparkRestDerivedData] = { + val appTarget = apiTarget.path(s"applications/${appId}") + logger.info(s"calling REST API at ${appTarget.getUri}") + + val applicationInfo = getApplicationInfo(appTarget) + + // Limit scope of async. + async { + val lastAttemptId = applicationInfo.attempts.maxBy { _.startTime }.attemptId + lastAttemptId match { + case Some(attemptId) => { + val attemptTarget = appTarget.path(attemptId) + val futureJobDatas = async { getJobDatas(attemptTarget) } + val futureStageDatas = async { getStageDatas(attemptTarget) } + val futureExecutorSummaries = async { getExecutorSummaries(attemptTarget) } + SparkRestDerivedData( + applicationInfo, + await(futureJobDatas), + await(futureStageDatas), + await(futureExecutorSummaries) + ) + } + case None => throw new IllegalArgumentException("Spark REST API has no attempt information") + } + } + } + + private def getApplicationInfo(appTarget: WebTarget): ApplicationInfo = { + try { + get(appTarget, SparkRestObjectMapper.readValue[ApplicationInfo]) + } catch { + case NonFatal(e) => { + logger.error(s"error reading ${appTarget.getUri}", e) + throw e + } + } + } + + private def getJobDatas(attemptTarget: WebTarget): Seq[JobData] = { + val target = attemptTarget.path("jobs") + try { + get(target, SparkRestObjectMapper.readValue[Seq[JobData]]) + } catch { + case NonFatal(e) => { + logger.error(s"error reading ${target.getUri}", e) + throw e + } + } + } + + private def getStageDatas(attemptTarget: WebTarget): Seq[StageData] = { + val target = attemptTarget.path("stages") + try { + get(target, SparkRestObjectMapper.readValue[Seq[StageData]]) + } catch { + case NonFatal(e) => { + logger.error(s"error reading ${target.getUri}", e) + throw e + } + } + } + + private def getExecutorSummaries(attemptTarget: WebTarget): Seq[ExecutorSummary] = { + val target = attemptTarget.path("executors") + try { + get(target, SparkRestObjectMapper.readValue[Seq[ExecutorSummary]]) + } catch { + case NonFatal(e) => { + logger.error(s"error reading ${target.getUri}", e) + throw e + } + } + } +} + +object SparkRestClient { + val HISTORY_SERVER_ADDRESS_KEY = "spark.yarn.historyServer.address" + val API_V1_MOUNT_PATH = "api/v1" + + val SparkRestObjectMapper = { + val dateFormat = { + val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'") + val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT")) + iso8601.setCalendar(cal) + iso8601 + } + + val objectMapper = new ObjectMapper() with ScalaObjectMapper + objectMapper.setDateFormat(dateFormat) + objectMapper.registerModule(DefaultScalaModule) + objectMapper + } + + def get[T](webTarget: WebTarget, converter: String => T): T = + converter(webTarget.request(MediaType.APPLICATION_JSON).get(classOf[String])) +} diff --git a/app/com/linkedin/drelephant/spark/fetchers/statusapiv1/statusapiv1.scala b/app/com/linkedin/drelephant/spark/fetchers/statusapiv1/statusapiv1.scala new file mode 100644 index 000000000..d586d4b0f --- /dev/null +++ b/app/com/linkedin/drelephant/spark/fetchers/statusapiv1/statusapiv1.scala @@ -0,0 +1,250 @@ +/* + * Originally from + * https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/status/api/v1/api.scala + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * Modifications copyright 2016 LinkedIn Corp. + * + * To keep up to date, please copy + * https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/status/api/v1/api.scala + * and maintain in this package. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package com.linkedin.drelephant.spark.fetchers.statusapiv1 + +import java.util.Date + +import scala.collection.Map + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.status.api.v1.StageStatus + +class ApplicationInfo( + val id: String, + val name: String, + val attempts: Seq[ApplicationAttemptInfo]) + +class ApplicationAttemptInfo( + val attemptId: Option[String], + val startTime: Date, + val endTime: Date, + val sparkUser: String, + val completed: Boolean = false) + +class ExecutorStageSummary( + val taskTime : Long, + val failedTasks : Int, + val succeededTasks : Int, + val inputBytes : Long, + val outputBytes : Long, + val shuffleRead : Long, + val shuffleWrite : Long, + val memoryBytesSpilled : Long, + val diskBytesSpilled : Long) + +class ExecutorSummary( + val id: String, + val hostPort: String, + val rddBlocks: Int, + val memoryUsed: Long, + val diskUsed: Long, + val activeTasks: Int, + val failedTasks: Int, + val completedTasks: Int, + val totalTasks: Int, + val totalDuration: Long, + val totalInputBytes: Long, + val totalShuffleRead: Long, + val totalShuffleWrite: Long, + val maxMemory: Long, + val executorLogs: Map[String, String]) + +class JobData( + val jobId: Int, + val name: String, + val description: Option[String], + val submissionTime: Option[Date], + val completionTime: Option[Date], + val stageIds: Seq[Int], + val jobGroup: Option[String], + val status: JobExecutionStatus, + val numTasks: Int, + val numActiveTasks: Int, + val numCompletedTasks: Int, + val numSkippedTasks: Int, + val numFailedTasks: Int, + val numActiveStages: Int, + val numCompletedStages: Int, + val numSkippedStages: Int, + val numFailedStages: Int) + +// Q: should Tachyon size go in here as well? currently the UI only shows it on the overall storage +// page ... does anybody pay attention to it? +class RDDStorageInfo( + val id: Int, + val name: String, + val numPartitions: Int, + val numCachedPartitions: Int, + val storageLevel: String, + val memoryUsed: Long, + val diskUsed: Long, + val dataDistribution: Option[Seq[RDDDataDistribution]], + val partitions: Option[Seq[RDDPartitionInfo]]) + +class RDDDataDistribution( + val address: String, + val memoryUsed: Long, + val memoryRemaining: Long, + val diskUsed: Long) + +class RDDPartitionInfo( + val blockName: String, + val storageLevel: String, + val memoryUsed: Long, + val diskUsed: Long, + val executors: Seq[String]) + +class StageData( + val status: StageStatus, + val stageId: Int, + val attemptId: Int, + val numActiveTasks: Int , + val numCompleteTasks: Int, + val numFailedTasks: Int, + + val executorRunTime: Long, + + val inputBytes: Long, + val inputRecords: Long, + val outputBytes: Long, + val outputRecords: Long, + val shuffleReadBytes: Long, + val shuffleReadRecords: Long, + val shuffleWriteBytes: Long, + val shuffleWriteRecords: Long, + val memoryBytesSpilled: Long, + val diskBytesSpilled: Long, + + val name: String, + val details: String, + val schedulingPool: String, + + val accumulatorUpdates: Seq[AccumulableInfo], + val tasks: Option[Map[Long, TaskData]], + val executorSummary: Option[Map[String, ExecutorStageSummary]]) + +class TaskData( + val taskId: Long, + val index: Int, + val attempt: Int, + val launchTime: Date, + val executorId: String, + val host: String, + val taskLocality: String, + val speculative: Boolean, + val accumulatorUpdates: Seq[AccumulableInfo], + val errorMessage: Option[String] = None, + val taskMetrics: Option[TaskMetrics] = None) + +class TaskMetrics( + val executorDeserializeTime: Long, + val executorRunTime: Long, + val resultSize: Long, + val jvmGcTime: Long, + val resultSerializationTime: Long, + val memoryBytesSpilled: Long, + val diskBytesSpilled: Long, + val inputMetrics: Option[InputMetrics], + val outputMetrics: Option[OutputMetrics], + val shuffleReadMetrics: Option[ShuffleReadMetrics], + val shuffleWriteMetrics: Option[ShuffleWriteMetrics]) + +class InputMetrics( + val bytesRead: Long, + val recordsRead: Long) + +class OutputMetrics( + val bytesWritten: Long, + val recordsWritten: Long) + +class ShuffleReadMetrics( + val remoteBlocksFetched: Int, + val localBlocksFetched: Int, + val fetchWaitTime: Long, + val remoteBytesRead: Long, + val totalBlocksFetched: Int, + val recordsRead: Long) + +class ShuffleWriteMetrics( + val bytesWritten: Long, + val writeTime: Long, + val recordsWritten: Long) + +class TaskMetricDistributions( + val quantiles: IndexedSeq[Double], + + val executorDeserializeTime: IndexedSeq[Double], + val executorRunTime: IndexedSeq[Double], + val resultSize: IndexedSeq[Double], + val jvmGcTime: IndexedSeq[Double], + val resultSerializationTime: IndexedSeq[Double], + val memoryBytesSpilled: IndexedSeq[Double], + val diskBytesSpilled: IndexedSeq[Double], + + val inputMetrics: Option[InputMetricDistributions], + val outputMetrics: Option[OutputMetricDistributions], + val shuffleReadMetrics: Option[ShuffleReadMetricDistributions], + val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions]) + +class InputMetricDistributions( + val bytesRead: IndexedSeq[Double], + val recordsRead: IndexedSeq[Double]) + +class OutputMetricDistributions( + val bytesWritten: IndexedSeq[Double], + val recordsWritten: IndexedSeq[Double]) + +class ShuffleReadMetricDistributions( + val readBytes: IndexedSeq[Double], + val readRecords: IndexedSeq[Double], + val remoteBlocksFetched: IndexedSeq[Double], + val localBlocksFetched: IndexedSeq[Double], + val fetchWaitTime: IndexedSeq[Double], + val remoteBytesRead: IndexedSeq[Double], + val totalBlocksFetched: IndexedSeq[Double]) + +class ShuffleWriteMetricDistributions( + val writeBytes: IndexedSeq[Double], + val writeRecords: IndexedSeq[Double], + val writeTime: IndexedSeq[Double]) + +class AccumulableInfo( + val id: Long, + val name: String, + val update: Option[String], + val value: String) diff --git a/app/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristic.java b/app/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristic.java deleted file mode 100644 index 46c0609c2..000000000 --- a/app/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristic.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkEnvironmentData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import com.linkedin.drelephant.util.Utils; -import java.util.Arrays; -import java.util.Map; -import org.apache.log4j.Logger; - - -/** - * This heuristic rule check some of the most commonly set spark properties and make sure the user is following - * a best convention of them. - */ -public class BestPropertiesConventionHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(BestPropertiesConventionHeuristic.class); - - public static final String SPARK_SERIALIZER = "spark.serializer"; - public static final String SPARK_DRIVER_MEMORY = "spark.driver.memory"; - public static final String SPARK_SHUFFLE_MANAGER = "spark.shuffle.manager"; - public static final String SPARK_EXECUTOR_CORES = "spark.executor.cores"; - - // Severity parameters. - private static final String NUM_CORE_SEVERITY = "num_core_severity"; - private static final String DRIVER_MEM_SEVERITY = "driver_memory_severity_in_gb"; - - // Default value of parameters - private double[] numCoreLimit= {2d}; // Spark Executor Cores - private double[] driverMemLimits = {4d, 4d, 8d, 8d}; // Spark Driver Memory - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confNumCoreLimit = Utils.getParam(paramMap.get(NUM_CORE_SEVERITY), numCoreLimit.length); - if (confNumCoreLimit != null) { - numCoreLimit = confNumCoreLimit; - } - logger.info(heuristicName + " will use " + NUM_CORE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(numCoreLimit)); - - double[] confDriverMemLimits = Utils.getParam(paramMap.get(DRIVER_MEM_SEVERITY), driverMemLimits.length); - if (confDriverMemLimits != null) { - driverMemLimits = confDriverMemLimits; - } - logger.info(heuristicName + " will use " + DRIVER_MEM_SEVERITY + " with the following threshold settings: " - + Arrays.toString(driverMemLimits)); - for (int i = 0; i < driverMemLimits.length; i++) { - driverMemLimits[i] = (double) MemoryFormatUtils.stringToBytes(Double.toString(driverMemLimits[i]) + "G"); - } - } - - public BestPropertiesConventionHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkEnvironmentData env = data.getEnvironmentData(); - String sparkSerializer = env.getSparkProperty(SPARK_SERIALIZER); - String sparkDriverMemory = env.getSparkProperty(SPARK_DRIVER_MEMORY); - String sparkShuffleManager = env.getSparkProperty(SPARK_SHUFFLE_MANAGER); - String sparkExecutorCores = env.getSparkProperty(SPARK_EXECUTOR_CORES); - int coreNum = sparkExecutorCores == null ? 1 : Integer.parseInt(sparkExecutorCores); - - Severity kryoSeverity = - binarySeverity("org.apache.spark.serializer.KryoSerializer", sparkSerializer, true, Severity.MODERATE); - Severity driverMemSeverity = getDriverMemorySeverity(MemoryFormatUtils.stringToBytes(sparkDriverMemory)); - Severity sortSeverity = binarySeverity("sort", sparkShuffleManager, true, Severity.MODERATE); - Severity executorCoreSeverity = getCoreNumSeverity(coreNum); - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), Severity.max(kryoSeverity, driverMemSeverity, sortSeverity, - executorCoreSeverity), 0); - - result.addResultDetail(SPARK_SERIALIZER, propertyToString(sparkSerializer)); - result.addResultDetail(SPARK_DRIVER_MEMORY, propertyToString(sparkDriverMemory)); - result.addResultDetail(SPARK_SHUFFLE_MANAGER, propertyToString(sparkShuffleManager)); - result.addResultDetail(SPARK_EXECUTOR_CORES, propertyToString(sparkExecutorCores)); - - return result; - } - - private Severity getCoreNumSeverity(int cores) { - if (cores > numCoreLimit[0]) { - return Severity.CRITICAL; - } else { - return Severity.NONE; - } - } - - private Severity getDriverMemorySeverity(long mem) { - return Severity.getSeverityAscending( - mem, driverMemLimits[0], driverMemLimits[1], driverMemLimits[2], driverMemLimits[3]); - } - - private static Severity binarySeverity(String expectedValue, String actualValue, boolean ignoreNull, - Severity severity) { - if (actualValue == null) { - if (ignoreNull) { - return Severity.NONE; - } else { - return severity; - } - } - - if (actualValue.equals(expectedValue)) { - return Severity.NONE; - } else { - return severity; - } - } - - private static String propertyToString(String val) { - return val == null ? "Not presented. Using default" : val; - } -} diff --git a/app/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristic.scala b/app/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristic.scala new file mode 100644 index 000000000..c0c33bc3e --- /dev/null +++ b/app/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristic.scala @@ -0,0 +1,132 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import scala.collection.JavaConverters +import scala.util.Try + +import com.linkedin.drelephant.analysis.{HeuristicResultDetails, Heuristic, HeuristicResult, Severity} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.spark.data.SparkApplicationData +import com.linkedin.drelephant.util.MemoryFormatUtils + + +/** + * A heuristic based on an app's known configuration. + * + * The results from this heuristic primarily inform users about key app configuration settings, including + * driver memory, executor cores, executor instances, executor memory, and the serializer. + */ +class ConfigurationHeuristic(private val heuristicConfigurationData: HeuristicConfigurationData) + extends Heuristic[SparkApplicationData] { + import ConfigurationHeuristic._ + import JavaConverters._ + + val serializerIfNonNullRecommendation: String = + Option(heuristicConfigurationData.getParamMap.get(SERIALIZER_IF_NON_NULL_RECOMMENDATION_KEY)) + .getOrElse(DEFAULT_SERIALIZER_IF_NON_NULL_RECOMMENDATION) + + val serializerIfNonNullSeverityIfRecommendationUnmet: Severity = + DEFAULT_SERIALIZER_IF_NON_NULL_SEVERITY_IF_RECOMMENDATION_UNMET + + override def getHeuristicConfData(): HeuristicConfigurationData = heuristicConfigurationData + + override def apply(data: SparkApplicationData): HeuristicResult = { + val evaluator = new Evaluator(this, data) + + def formatProperty(property: Option[String]): String = + property.getOrElse("Not presented. Using default.") + + val resultDetails = Seq( + new HeuristicResultDetails( + SPARK_DRIVER_MEMORY_KEY, + formatProperty(evaluator.driverMemoryBytes.map(MemoryFormatUtils.bytesToString)) + ), + new HeuristicResultDetails( + SPARK_EXECUTOR_MEMORY_KEY, + formatProperty(evaluator.executorMemoryBytes.map(MemoryFormatUtils.bytesToString)) + ), + new HeuristicResultDetails( + SPARK_EXECUTOR_INSTANCES_KEY, + formatProperty(evaluator.executorInstances.map(_.toString)) + ), + new HeuristicResultDetails( + SPARK_EXECUTOR_CORES_KEY, + formatProperty(evaluator.executorCores.map(_.toString)) + ), + new HeuristicResultDetails( + SPARK_SERIALIZER_KEY, + formatProperty(evaluator.serializer) + ) + ) + val result = new HeuristicResult( + heuristicConfigurationData.getClassName, + heuristicConfigurationData.getHeuristicName, + evaluator.severity, + 0, + resultDetails.asJava + ) + result + } +} + +object ConfigurationHeuristic { + val DEFAULT_SERIALIZER_IF_NON_NULL_RECOMMENDATION = "org.apache.spark.serializer.KryoSerializer" + val DEFAULT_SERIALIZER_IF_NON_NULL_SEVERITY_IF_RECOMMENDATION_UNMET = Severity.MODERATE + + val SERIALIZER_IF_NON_NULL_RECOMMENDATION_KEY = "serializer_if_non_null_recommendation" + + val SPARK_DRIVER_MEMORY_KEY = "spark.driver.memory" + val SPARK_EXECUTOR_MEMORY_KEY = "spark.executor.memory" + val SPARK_EXECUTOR_INSTANCES_KEY = "spark.executor.instances" + val SPARK_EXECUTOR_CORES_KEY = "spark.executor.cores" + val SPARK_SERIALIZER_KEY = "spark.serializer" + + class Evaluator(configurationHeuristic: ConfigurationHeuristic, data: SparkApplicationData) { + lazy val appConfigurationProperties: Map[String, String] = + data.appConfigurationProperties + + lazy val driverMemoryBytes: Option[Long] = + Try(getProperty(SPARK_DRIVER_MEMORY_KEY).map(MemoryFormatUtils.stringToBytes)).getOrElse(None) + + lazy val executorMemoryBytes: Option[Long] = + Try(getProperty(SPARK_EXECUTOR_MEMORY_KEY).map(MemoryFormatUtils.stringToBytes)).getOrElse(None) + + lazy val executorInstances: Option[Int] = + Try(getProperty(SPARK_EXECUTOR_INSTANCES_KEY).map(_.toInt)).getOrElse(None) + + lazy val executorCores: Option[Int] = + Try(getProperty(SPARK_EXECUTOR_CORES_KEY).map(_.toInt)).getOrElse(None) + + lazy val serializer: Option[String] = getProperty(SPARK_SERIALIZER_KEY) + + lazy val serializerSeverity: Severity = serializer match { + case None => Severity.NONE + case Some(`serializerIfNonNullRecommendation`) => Severity.NONE + case Some(_) => serializerIfNonNullSeverityIfRecommendationUnmet + } + + lazy val severity: Severity = serializerSeverity + + private val serializerIfNonNullRecommendation: String = configurationHeuristic.serializerIfNonNullRecommendation + + private val serializerIfNonNullSeverityIfRecommendationUnmet: Severity = + configurationHeuristic.serializerIfNonNullSeverityIfRecommendationUnmet + + private def getProperty(key: String): Option[String] = appConfigurationProperties.get(key) + } +} diff --git a/app/com/linkedin/drelephant/spark/heuristics/EventLogLimitHeuristic.java b/app/com/linkedin/drelephant/spark/heuristics/EventLogLimitHeuristic.java deleted file mode 100644 index 201b756c2..000000000 --- a/app/com/linkedin/drelephant/spark/heuristics/EventLogLimitHeuristic.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; - - -/** - * This is a safeguard heuristic rule that makes sure if a log size passes the limit, we do not automatically - * approve it. - */ -public class EventLogLimitHeuristic implements Heuristic { - private HeuristicConfigurationData _heuristicConfData; - - public EventLogLimitHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - Severity severity = getSeverity(data); - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, 0); - if (severity == Severity.CRITICAL) { - result.addResultDetail("Large Log File", "Spark job's event log passes the limit. No actual log data is fetched." - + " All other heuristic rules will not make sense.", null); - } - return result; - } - - private Severity getSeverity(SparkApplicationData data) { - if (data.isThrottled()) { - return Severity.CRITICAL; - } else { - return Severity.NONE; - } - } -} diff --git a/app/com/linkedin/drelephant/spark/heuristics/ExecutorLoadHeuristic.java b/app/com/linkedin/drelephant/spark/heuristics/ExecutorLoadHeuristic.java deleted file mode 100644 index 086d5fe33..000000000 --- a/app/com/linkedin/drelephant/spark/heuristics/ExecutorLoadHeuristic.java +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.math.Statistics; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkExecutorData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import com.linkedin.drelephant.util.Utils; -import java.util.Arrays; -import java.util.Map; -import java.util.Set; -import org.apache.log4j.Logger; - -import static com.linkedin.drelephant.spark.data.SparkExecutorData.EXECUTOR_DRIVER_NAME; - - -/** - * This heuristic rule observes load details of executors - */ -public class ExecutorLoadHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(ExecutorLoadHeuristic.class); - private static final long MEMORY_OBSERVATION_THRESHOLD = MemoryFormatUtils.stringToBytes("1 MB"); - - // Severity parameters. - private static final String LOOSER_METRIC_DEV_SEVERITY = "looser_metric_deviation_severity"; - private static final String METRIC_DEV_SEVERITY = "metric_deviation_severity"; - - // Default value of parameters - private double[] looserMetDevLimits = {0.8d, 1d, 1.2d, 1.4d}; // Max deviation from avg. - private double[] metDevLimits = {0.4d, 0.6d, 0.8d, 1.0d}; - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confLooserMetDevLimits = Utils.getParam(paramMap.get(LOOSER_METRIC_DEV_SEVERITY), - looserMetDevLimits.length); - if (confLooserMetDevLimits != null) { - looserMetDevLimits = confLooserMetDevLimits; - } - logger.info(heuristicName + " will use " + LOOSER_METRIC_DEV_SEVERITY + " with the following threshold settings: " - + Arrays.toString(looserMetDevLimits)); - - double[] confMetDevLimits = Utils.getParam(paramMap.get(METRIC_DEV_SEVERITY), metDevLimits.length); - if (confMetDevLimits != null) { - metDevLimits = confMetDevLimits; - } - logger.info(heuristicName + " will use " + METRIC_DEV_SEVERITY + " with the following threshold settings: " - + Arrays.toString(metDevLimits)); - } - - public ExecutorLoadHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - private class ValueObserver { - private final long[] _values; - private Long _min; - private Long _max; - private Long _avg; - - public ValueObserver(int size) { - _values = new long[size]; - } - - public void set(int index, long value) { - _values[index] = value; - if (_min == null) { - _min = value; - } else { - _min = Math.min(value, _min); - } - - if (_max == null) { - _max = value; - } else { - _max = Math.max(value, _max); - } - } - - public long getMin() { - return _min == null ? 0L : _min; - } - - public long getMax() { - return _max == null ? 0L : _max; - } - - public long getAvg() { - if (_avg == null) { - if (_values == null) { - return 0L; - } - _avg = Statistics.average(_values); - } - return _avg; - } - - /** - * Max(|max-avg|, |min-avg|) / avg - * - * @return - */ - public double getDeviationFactor() { - long avg = getAvg(); - if (avg == 0) { - return 0d; - } - long diff = Math.max(getMax() - avg, avg - getMin()); - return diff * 1.0d / avg; - } - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkExecutorData executorData = data.getExecutorData(); - Set executors = executorData.getExecutors(); - - int numNonDriverExe = executors.size(); - if (executors.contains(EXECUTOR_DRIVER_NAME)) { - numNonDriverExe -= 1; - } - ValueObserver peakMems = new ValueObserver(numNonDriverExe); - ValueObserver durations = new ValueObserver(numNonDriverExe); - ValueObserver inputBytes = new ValueObserver(numNonDriverExe); - ValueObserver outputBytes = new ValueObserver(numNonDriverExe); - - int i = 0; - for (String exeId : executors) { - if (!exeId.equals(EXECUTOR_DRIVER_NAME)) { - SparkExecutorData.ExecutorInfo info = executorData.getExecutorInfo(exeId); - - /* Ignore the memory variation and consider it as 0 if it is too small - * The deviation of memory usage in KB level is too fluctuating to track. - */ - if (info.memUsed < MEMORY_OBSERVATION_THRESHOLD) { - peakMems.set(i, 0L); - } else { - peakMems.set(i, info.memUsed); - } - - durations.set(i, info.duration); - inputBytes.set(i, info.inputBytes); - outputBytes.set(i, info.outputBytes); - i += 1; - } - } - - Severity severity = Severity.max(getLooserMetricDeviationSeverity(peakMems), getMetricDeviationSeverity(durations), - getMetricDeviationSeverity(inputBytes), getLooserMetricDeviationSeverity(outputBytes)); - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, 0); - - result.addResultDetail("Average peak storage memory", - String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(peakMems.getAvg()), - MemoryFormatUtils.bytesToString(peakMems.getMin()), MemoryFormatUtils.bytesToString(peakMems.getMax()))); - result.addResultDetail("Average runtime", - String.format("%s (%s~%s)", Statistics.readableTimespan(durations.getAvg()), - Statistics.readableTimespan(durations.getMin()), Statistics.readableTimespan(durations.getMax()))); - result.addResultDetail("Average input size", - String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(inputBytes.getAvg()), - MemoryFormatUtils.bytesToString(inputBytes.getMin()), MemoryFormatUtils.bytesToString(inputBytes.getMax()))); - result.addResultDetail("Average output size", - String.format("%s (%s~%s)", MemoryFormatUtils.bytesToString(outputBytes.getAvg()), - MemoryFormatUtils.bytesToString(outputBytes.getMin()), MemoryFormatUtils.bytesToString(outputBytes.getMax()))); - - return result; - } - - /** - * Some metrics by nature could deviate a bit more than other metrics. This method basically allows some metrics - * to have looser severity thresholds. - * - * @param ob - * @return the corresponding severity - */ - private Severity getLooserMetricDeviationSeverity(ValueObserver ob) { - double diffFactor = ob.getDeviationFactor(); - return Severity.getSeverityAscending( - diffFactor, looserMetDevLimits[0], looserMetDevLimits[1], looserMetDevLimits[2], looserMetDevLimits[3]); - } - - private Severity getMetricDeviationSeverity(ValueObserver ob) { - double diffFactor = ob.getDeviationFactor(); - return Severity.getSeverityAscending( - diffFactor, metDevLimits[0], metDevLimits[1], metDevLimits[2], metDevLimits[3]); - } - -} diff --git a/app/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristic.scala b/app/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristic.scala new file mode 100644 index 000000000..39bd524e2 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristic.scala @@ -0,0 +1,228 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import scala.collection.JavaConverters +import scala.collection.mutable.ArrayBuffer + +import com.linkedin.drelephant.analysis.{Heuristic, HeuristicResult, HeuristicResultDetails, Severity, SeverityThresholds} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.math.Statistics +import com.linkedin.drelephant.spark.data.SparkApplicationData +import com.linkedin.drelephant.spark.fetchers.statusapiv1.ExecutorSummary +import com.linkedin.drelephant.util.MemoryFormatUtils + + +/** + * A heuristic based on metrics for a Spark app's executors. + * + * This heuristic concerns the distribution (min, 25p, median, 75p, max) of key executor metrics including input bytes, + * shuffle read bytes, shuffle write bytes, storage memory used, and task time. The max-to-median ratio determines the + * severity of any particular metric. + */ +class ExecutorsHeuristic(private val heuristicConfigurationData: HeuristicConfigurationData) + extends Heuristic[SparkApplicationData] { + import ExecutorsHeuristic._ + import JavaConverters._ + + val maxToMedianRatioSeverityThresholds: SeverityThresholds = + SeverityThresholds.parse(heuristicConfigurationData.getParamMap.get(MAX_TO_MEDIAN_RATIO_SEVERITY_THRESHOLDS_KEY), ascending = true) + .getOrElse(DEFAULT_MAX_TO_MEDIAN_RATIO_SEVERITY_THRESHOLDS) + + val ignoreMaxBytesLessThanThreshold: Long = + Option(heuristicConfigurationData.getParamMap.get(IGNORE_MAX_BYTES_LESS_THAN_THRESHOLD_KEY)) + .map(MemoryFormatUtils.stringToBytes) + .getOrElse(DEFAULT_IGNORE_MAX_BYTES_LESS_THAN_THRESHOLD) + + val ignoreMaxMillisLessThanThreshold: Long = + Option(heuristicConfigurationData.getParamMap.get(IGNORE_MAX_MILLIS_LESS_THAN_THRESHOLD_KEY)) + .map(_.toLong) + .getOrElse(DEFAULT_IGNORE_MAX_MILLIS_LESS_THAN_THRESHOLD) + + override def getHeuristicConfData(): HeuristicConfigurationData = heuristicConfigurationData + + override def apply(data: SparkApplicationData): HeuristicResult = { + val evaluator = new Evaluator(this, data) + + def formatDistribution(distribution: Distribution, longFormatter: Long => String, separator: String = ", "): String = { + val labels = Seq( + s"min: ${longFormatter(distribution.min)}", + s"p25: ${longFormatter(distribution.p25)}", + s"median: ${longFormatter(distribution.median)}", + s"p75: ${longFormatter(distribution.p75)}", + s"max: ${longFormatter(distribution.max)}" + ) + labels.mkString(separator) + } + + def formatDistributionBytes(distribution: Distribution): String = + formatDistribution(distribution, MemoryFormatUtils.bytesToString) + + def formatDistributionDuration(distribution: Distribution): String = + formatDistribution(distribution, Statistics.readableTimespan) + + val resultDetails = Seq( + new HeuristicResultDetails( + "Total executor storage memory allocated", + MemoryFormatUtils.bytesToString(evaluator.totalStorageMemoryAllocated) + ), + new HeuristicResultDetails( + "Total executor storage memory used", + MemoryFormatUtils.bytesToString(evaluator.totalStorageMemoryUsed) + ), + new HeuristicResultDetails( + "Executor storage memory utilization rate", + f"${evaluator.storageMemoryUtilizationRate}%1.3f" + ), + new HeuristicResultDetails( + "Executor storage memory used distribution", + formatDistributionBytes(evaluator.storageMemoryUsedDistribution) + ), + new HeuristicResultDetails( + "Executor task time distribution", + formatDistributionDuration(evaluator.taskTimeDistribution) + ), + new HeuristicResultDetails( + "Executor input bytes distribution", + formatDistributionBytes(evaluator.inputBytesDistribution) + ), + new HeuristicResultDetails( + "Executor shuffle read bytes distribution", + formatDistributionBytes(evaluator.shuffleReadBytesDistribution) + ), + new HeuristicResultDetails( + "Executor shuffle write bytes distribution", + formatDistributionBytes(evaluator.shuffleWriteBytesDistribution) + ) + ) + val result = new HeuristicResult( + heuristicConfigurationData.getClassName, + heuristicConfigurationData.getHeuristicName, + evaluator.severity, + 0, + resultDetails.asJava + ) + result + } +} + +object ExecutorsHeuristic { + import JavaConverters._ + import scala.concurrent.duration._ + + val DEFAULT_MAX_TO_MEDIAN_RATIO_SEVERITY_THRESHOLDS: SeverityThresholds = SeverityThresholds( + low = math.pow(10, 0.125), // ~1.334 + moderate = math.pow(10, 0.25), // ~1.778 + severe = math.pow(10, 0.5), // ~3.162 + critical = 10, + ascending = true + ) + + val DEFAULT_IGNORE_MAX_BYTES_LESS_THAN_THRESHOLD: Long = MemoryFormatUtils.stringToBytes("100 MB") + + val DEFAULT_IGNORE_MAX_MILLIS_LESS_THAN_THRESHOLD: Long = Duration(5, MINUTES).toMillis + + val MAX_TO_MEDIAN_RATIO_SEVERITY_THRESHOLDS_KEY: String = "max_to_median_ratio_severity_thresholds" + + val IGNORE_MAX_BYTES_LESS_THAN_THRESHOLD_KEY: String = "ignore_max_bytes_less_than_threshold" + + val IGNORE_MAX_MILLIS_LESS_THAN_THRESHOLD_KEY: String = "ignore_max_millis_less_than_threshold" + + class Evaluator(executorsHeuristic: ExecutorsHeuristic, data: SparkApplicationData) { + lazy val executorSummaries: Seq[ExecutorSummary] = data.executorSummaries + + lazy val totalStorageMemoryAllocated: Long = executorSummaries.map { _.maxMemory }.sum + + lazy val totalStorageMemoryUsed: Long = executorSummaries.map { _.memoryUsed }.sum + + lazy val storageMemoryUtilizationRate: Double = totalStorageMemoryUsed.toDouble / totalStorageMemoryAllocated.toDouble + + lazy val storageMemoryUsedDistribution: Distribution = + Distribution(executorSummaries.map { _.memoryUsed }) + + lazy val storageMemoryUsedSeverity: Severity = + severityOfDistribution(storageMemoryUsedDistribution, ignoreMaxBytesLessThanThreshold) + + lazy val taskTimeDistribution: Distribution = + Distribution(executorSummaries.map { _.totalDuration }) + + lazy val taskTimeSeverity: Severity = + severityOfDistribution(taskTimeDistribution, ignoreMaxMillisLessThanThreshold) + + lazy val inputBytesDistribution: Distribution = + Distribution(executorSummaries.map { _.totalInputBytes }) + + lazy val inputBytesSeverity: Severity = + severityOfDistribution(inputBytesDistribution, ignoreMaxBytesLessThanThreshold) + + lazy val shuffleReadBytesDistribution: Distribution = + Distribution(executorSummaries.map { _.totalShuffleRead }) + + lazy val shuffleReadBytesSeverity: Severity = + severityOfDistribution(shuffleReadBytesDistribution, ignoreMaxBytesLessThanThreshold) + + lazy val shuffleWriteBytesDistribution: Distribution = + Distribution(executorSummaries.map { _.totalShuffleWrite }) + + lazy val shuffleWriteBytesSeverity: Severity = + severityOfDistribution(shuffleWriteBytesDistribution, ignoreMaxBytesLessThanThreshold) + + lazy val severity: Severity = Severity.max( + storageMemoryUsedSeverity, + taskTimeSeverity, + inputBytesSeverity, + shuffleReadBytesSeverity, + shuffleWriteBytesSeverity + ) + + private[heuristics] def severityOfDistribution( + distribution: Distribution, + ignoreMaxLessThanThreshold: Long, + severityThresholds: SeverityThresholds = maxToMedianRatioSeverityThresholds + ): Severity = { + if (distribution.max < ignoreMaxLessThanThreshold) { + Severity.NONE + } else if (distribution.median == 0L) { + severityThresholds.severityOf(Long.MaxValue) + } else { + severityThresholds.severityOf(BigDecimal(distribution.max) / BigDecimal(distribution.median)) + } + } + + private lazy val maxToMedianRatioSeverityThresholds = executorsHeuristic.maxToMedianRatioSeverityThresholds + + private lazy val ignoreMaxBytesLessThanThreshold = executorsHeuristic.ignoreMaxBytesLessThanThreshold + + private lazy val ignoreMaxMillisLessThanThreshold = executorsHeuristic.ignoreMaxMillisLessThanThreshold + } + + case class Distribution(min: Long, p25: Long, median: Long, p75: Long, max: Long) + + object Distribution { + def apply(values: Seq[Long]): Distribution = { + val sortedValues = values.sorted + val sortedValuesAsJava = sortedValues.map(Long.box).to[ArrayBuffer].asJava + Distribution( + sortedValues.min, + p25 = Statistics.percentile(sortedValuesAsJava, 25), + Statistics.median(sortedValuesAsJava), + p75 = Statistics.percentile(sortedValuesAsJava, 75), + sortedValues.max + ) + } + } +} diff --git a/app/com/linkedin/drelephant/spark/heuristics/JobRuntimeHeuristic.java b/app/com/linkedin/drelephant/spark/heuristics/JobRuntimeHeuristic.java deleted file mode 100644 index 0e3cfed98..000000000 --- a/app/com/linkedin/drelephant/spark/heuristics/JobRuntimeHeuristic.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkJobProgressData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.Utils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - - -/** - * This heuristic rule observes the runtime characteristics of the spark application run. - */ -public class JobRuntimeHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(JobRuntimeHeuristic.class); - - // Severity parameters. - private static final String AVG_JOB_FAILURE_SEVERITY = "avg_job_failure_rate_severity"; - private static final String SINGLE_JOB_FAILURE_SEVERITY = "single_job_failure_rate_severity"; - - // Default value of parameters - private double[] avgJobFailureLimits = {0.1d, 0.3d, 0.5d, 0.5d}; // The avg job failure rate - private double[] jobFailureLimits = {0.0d, 0.3d, 0.5d, 0.5d}; - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confAvgJobFailureLimits = Utils.getParam(paramMap.get(AVG_JOB_FAILURE_SEVERITY), - avgJobFailureLimits.length); - if (confAvgJobFailureLimits != null) { - avgJobFailureLimits = confAvgJobFailureLimits; - } - logger.info(heuristicName + " will use " + AVG_JOB_FAILURE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(avgJobFailureLimits)); - - double[] confJobFailureLimits = Utils.getParam(paramMap.get(SINGLE_JOB_FAILURE_SEVERITY), - jobFailureLimits.length); - if (confJobFailureLimits != null) { - jobFailureLimits = confJobFailureLimits; - } - logger.info(heuristicName + " will use " + SINGLE_JOB_FAILURE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(jobFailureLimits)); - } - - public JobRuntimeHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkJobProgressData jobProgressData = data.getJobProgressData(); - Severity endSeverity = Severity.NONE; - - Set completedJobs = jobProgressData.getCompletedJobs(); - Set failedJobs = jobProgressData.getFailedJobs(); - - // Average job failure rate - double avgJobFailureRate = jobProgressData.getJobFailureRate(); - Severity jobFailureRateSeverity = getAvgJobFailureRateSeverity(avgJobFailureRate); - endSeverity = Severity.max(endSeverity, jobFailureRateSeverity); - - // For each completed individual job - List highFailureRateJobs = new ArrayList(); - for (int jobId : completedJobs) { - SparkJobProgressData.JobInfo job = jobProgressData.getJobInfo(jobId); - double jobFailureRate = job.getFailureRate(); - Severity severity = getSingleJobFailureRateSeverity(jobFailureRate); - if (severity.getValue() > Severity.MODERATE.getValue()) { - highFailureRateJobs.add( - jobProgressData.getJobDescription(jobId) + " (task failure rate:" + String.format("%1.3f", jobFailureRate) - + ")"); - } - endSeverity = Severity.max(endSeverity, severity); - } - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), endSeverity, 0); - - result.addResultDetail("Spark completed jobs number", String.valueOf(completedJobs.size())); - result.addResultDetail("Spark failed jobs number", String.valueOf(failedJobs.size())); - result.addResultDetail("Spark failed jobs list", getJobsAsString(jobProgressData.getFailedJobDescriptions())); - result.addResultDetail("Spark average job failure rate", String.format("%.3f", avgJobFailureRate)); - result.addResultDetail("Spark jobs with high task failure rate", getJobsAsString(highFailureRateJobs)); - - return result; - } - - private Severity getAvgJobFailureRateSeverity(double rate) { - return Severity.getSeverityAscending( - rate, avgJobFailureLimits[0], avgJobFailureLimits[1], avgJobFailureLimits[2], avgJobFailureLimits[3]); - } - - private Severity getSingleJobFailureRateSeverity(double rate) { - return Severity.getSeverityAscending( - rate, jobFailureLimits[0], jobFailureLimits[1], jobFailureLimits[2], jobFailureLimits[3]); - } - - private static String getJobsAsString(Collection names) { - return StringUtils.join(names, "\n"); - } -} diff --git a/app/com/linkedin/drelephant/spark/heuristics/JobsHeuristic.scala b/app/com/linkedin/drelephant/spark/heuristics/JobsHeuristic.scala new file mode 100644 index 000000000..7014acfe8 --- /dev/null +++ b/app/com/linkedin/drelephant/spark/heuristics/JobsHeuristic.scala @@ -0,0 +1,148 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import scala.collection.JavaConverters + +import com.linkedin.drelephant.analysis.{Heuristic, HeuristicResult, HeuristicResultDetails, Severity, SeverityThresholds} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.spark.data.SparkApplicationData +import com.linkedin.drelephant.spark.fetchers.statusapiv1.JobData +import org.apache.spark.JobExecutionStatus + + +/** + * A heuristic based on metrics for a Spark app's jobs. + * + * This heuristic reports job failures and high task failure rates for each job. + */ +class JobsHeuristic(private val heuristicConfigurationData: HeuristicConfigurationData) + extends Heuristic[SparkApplicationData] { + import JobsHeuristic._ + import JavaConverters._ + + val jobFailureRateSeverityThresholds: SeverityThresholds = + SeverityThresholds.parse(heuristicConfigurationData.getParamMap.get(JOB_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY), ascending = true) + .getOrElse(DEFAULT_JOB_FAILURE_RATE_SEVERITY_THRESHOLDS) + + val taskFailureRateSeverityThresholds: SeverityThresholds = + SeverityThresholds.parse(heuristicConfigurationData.getParamMap.get(TASK_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY), ascending = true) + .getOrElse(DEFAULT_TASK_FAILURE_RATE_SEVERITY_THRESHOLDS) + + override def getHeuristicConfData(): HeuristicConfigurationData = heuristicConfigurationData + + override def apply(data: SparkApplicationData): HeuristicResult = { + val evaluator = new Evaluator(this, data) + + def formatFailedJobs(failedJobs: Seq[JobData]): String = failedJobs.map(formatFailedJob).mkString("\n") + + def formatFailedJob(jobData: JobData): String = f"job ${jobData.jobId}, ${jobData.name}" + + def formatJobsWithHighTaskFailureRates(jobsWithHighTaskFailureRates: Seq[(JobData, Double)]): String = + jobsWithHighTaskFailureRates + .map { case (jobData, taskFailureRate) => formatJobWithHighTaskFailureRate(jobData, taskFailureRate) } + .mkString("\n") + + def formatJobWithHighTaskFailureRate(jobData: JobData, taskFailureRate: Double): String = + f"job ${jobData.jobId}, ${jobData.name} (task failure rate: ${taskFailureRate}%1.3f)" + + val resultDetails = Seq( + new HeuristicResultDetails("Spark completed jobs count", evaluator.numCompletedJobs.toString), + new HeuristicResultDetails("Spark failed jobs count", evaluator.numFailedJobs.toString), + new HeuristicResultDetails("Spark failed jobs list", formatFailedJobs(evaluator.failedJobs)), + new HeuristicResultDetails("Spark job failure rate", f"${evaluator.jobFailureRate.getOrElse(0.0D)}%.3f"), + new HeuristicResultDetails( + "Spark jobs with high task failure rates", + formatJobsWithHighTaskFailureRates(evaluator.jobsWithHighTaskFailureRates) + ) + ) + val result = new HeuristicResult( + heuristicConfigurationData.getClassName, + heuristicConfigurationData.getHeuristicName, + evaluator.severity, + 0, + resultDetails.asJava + ) + result + } +} + +object JobsHeuristic { + /** The default severity thresholds for the rate of an application's jobs failing. */ + val DEFAULT_JOB_FAILURE_RATE_SEVERITY_THRESHOLDS = + SeverityThresholds(low = 0.1D, moderate = 0.3D, severe = 0.5D, critical = 0.5D, ascending = true) + + /** The default severity thresholds for the rate of a job's tasks failing. */ + val DEFAULT_TASK_FAILURE_RATE_SEVERITY_THRESHOLDS = + SeverityThresholds(low = 0.1D, moderate = 0.3D, severe = 0.5D, critical = 0.5D, ascending = true) + + val JOB_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY = "job_failure_rate_severity_thresholds" + + val TASK_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY = "job_task_failure_rate_severity_thresholds" + + class Evaluator(jobsHeuristic: JobsHeuristic, data: SparkApplicationData) { + lazy val jobDatas: Seq[JobData] = data.jobDatas + + lazy val numCompletedJobs: Int = jobDatas.count { _.status == JobExecutionStatus.SUCCEEDED } + + lazy val numFailedJobs: Int = jobDatas.count { _.status == JobExecutionStatus.FAILED } + + lazy val failedJobs: Seq[JobData] = jobDatas.filter { _.status == JobExecutionStatus.FAILED } + + lazy val jobFailureRate: Option[Double] = { + // Currently, the calculation assumes there are no jobs with UNKNOWN or RUNNING state. + val numJobs = numCompletedJobs + numFailedJobs + if (numJobs == 0) None else Some(numFailedJobs.toDouble / numJobs.toDouble) + } + + lazy val jobsWithHighTaskFailureRates: Seq[(JobData, Double)] = + jobsWithHighTaskFailureRateSeverities.map { case (jobData, taskFailureRate, _) => (jobData, taskFailureRate) } + + lazy val severity: Severity = Severity.max((jobFailureRateSeverity +: taskFailureRateSeverities): _*) + + private lazy val jobFailureRateSeverityThresholds = jobsHeuristic.jobFailureRateSeverityThresholds + + private lazy val taskFailureRateSeverityThresholds = jobsHeuristic.taskFailureRateSeverityThresholds + + private lazy val jobFailureRateSeverity: Severity = + jobFailureRateSeverityThresholds.severityOf(jobFailureRate.getOrElse[Double](0.0D)) + + private lazy val jobsWithHighTaskFailureRateSeverities: Seq[(JobData, Double, Severity)] = + jobsAndTaskFailureRateSeverities.filter { case (_, _, severity) => severity.getValue > Severity.MODERATE.getValue } + + private lazy val jobsAndTaskFailureRateSeverities: Seq[(JobData, Double, Severity)] = for { + jobData <- jobDatas + (taskFailureRate, severity) = taskFailureRateAndSeverityOf(jobData) + } yield (jobData, taskFailureRate, severity) + + private lazy val taskFailureRateSeverities: Seq[Severity] = + jobsAndTaskFailureRateSeverities.map { case (_, _, severity) => severity } + + private def taskFailureRateAndSeverityOf(jobData: JobData): (Double, Severity) = { + val taskFailureRate = taskFailureRateOf(jobData).getOrElse(0.0D) + (taskFailureRate, taskFailureRateSeverityThresholds.severityOf(taskFailureRate)) + } + + private def taskFailureRateOf(jobData: JobData): Option[Double] = { + // Currently, the calculation doesn't include skipped or active tasks. + val numCompletedTasks = jobData.numCompletedTasks + val numFailedTasks = jobData.numFailedTasks + val numTasks = numCompletedTasks + numFailedTasks + if (numTasks == 0) None else Some(numFailedTasks.toDouble / numTasks.toDouble) + } + } +} diff --git a/app/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristic.java b/app/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristic.java deleted file mode 100644 index 6224b787a..000000000 --- a/app/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristic.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkEnvironmentData; -import com.linkedin.drelephant.spark.data.SparkExecutorData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import com.linkedin.drelephant.util.Utils; -import java.util.Arrays; -import java.util.Map; -import org.apache.log4j.Logger; -import org.apache.spark.SparkConf; - -import static com.linkedin.drelephant.spark.data.SparkExecutorData.EXECUTOR_DRIVER_NAME; - - -/** - * This heuristic checks for memory consumption. - */ -public class MemoryLimitHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(MemoryLimitHeuristic.class); - - public static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; - public static final String SPARK_DRIVER_MEMORY = "spark.driver.memory"; - public static final String SPARK_EXECUTOR_INSTANCES = "spark.executor.instances"; - - public static final String SPARK_STORAGE_MEMORY_FRACTION = "spark.storage.memoryFraction"; - public static final double DEFAULT_SPARK_STORAGE_MEMORY_FRACTION = 0.6d; - - // Severity parameters. - private static final String MEM_UTILIZATION_SEVERITY = "mem_util_severity"; - private static final String TOTAL_MEM_SEVERITY = "total_mem_severity_in_tb"; - - // Default value of parameters - private double[] memUtilLimits = {0.8d, 0.6d, 0.4d, 0.2d}; - private double[] totalMemLimits = {0.5d, 1d, 1.5d, 2d}; // Peak Memory / Total Storage Memory - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - if(paramMap.get(MEM_UTILIZATION_SEVERITY) != null) { - double[] confMemUtilLimits = Utils.getParam(paramMap.get(MEM_UTILIZATION_SEVERITY), memUtilLimits.length); - if (confMemUtilLimits != null) { - memUtilLimits = confMemUtilLimits; - } - } - logger.info(heuristicName + " will use " + MEM_UTILIZATION_SEVERITY + " with the following threshold settings: " - + Arrays.toString(memUtilLimits)); - - if(paramMap.get(TOTAL_MEM_SEVERITY) != null) { - double[] confTotalMemLimits = Utils.getParam(paramMap.get(TOTAL_MEM_SEVERITY), totalMemLimits.length); - if (confTotalMemLimits != null) { - totalMemLimits = confTotalMemLimits; - } - } - logger.info(heuristicName + " will use " + TOTAL_MEM_SEVERITY + " with the following threshold settings: " - + Arrays.toString(totalMemLimits)); - for (int i = 0; i < totalMemLimits.length; i++) { - totalMemLimits[i] = MemoryFormatUtils.stringToBytes(totalMemLimits[i] + "T"); - } - } - - public MemoryLimitHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - - int executorNum = Integer.parseInt(data.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_INSTANCES, "0")); - long perExecutorMem = - MemoryFormatUtils.stringToBytes(data.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_MEMORY, "0")); - - long totalExecutorMem = executorNum * perExecutorMem; - - long totalStorageMem = getTotalStorageMem(data); - long totalDriverMem = getTotalDriverMem(data); - long peakMem = getStoragePeakMemory(data); - - Severity totalMemorySeverity = getTotalMemorySeverity(totalExecutorMem); - Severity memoryUtilizationServerity = getMemoryUtilizationSeverity(peakMem, totalStorageMem); - - HeuristicResult result = - new HeuristicResult(_heuristicConfData.getClassName(), _heuristicConfData.getHeuristicName(), - Severity.max(totalMemorySeverity, memoryUtilizationServerity), 0); - - result.addResultDetail("Total executor memory allocated", String - .format("%s (%s x %s)", MemoryFormatUtils.bytesToString(totalExecutorMem), - MemoryFormatUtils.bytesToString(perExecutorMem), executorNum)); - result.addResultDetail("Total driver memory allocated", MemoryFormatUtils.bytesToString(totalDriverMem)); - result.addResultDetail("Total memory allocated for storage", MemoryFormatUtils.bytesToString(totalStorageMem)); - result.addResultDetail("Total memory used at peak", MemoryFormatUtils.bytesToString(peakMem)); - result.addResultDetail("Memory utilization rate", String.format("%1.3f", peakMem * 1.0 / totalStorageMem)); - return result; - } - - /** - * Get the total driver memory - * - * @param data The spark application data that contains the information - * @return The memory in bytes - */ - private static long getTotalDriverMem(SparkApplicationData data) { - long bytes = MemoryFormatUtils.stringToBytes(data.getEnvironmentData().getSparkProperty(SPARK_DRIVER_MEMORY)); - // spark.driver.memory might not be present, in which case we would infer it from the executor data - if (bytes == 0L) { - SparkExecutorData.ExecutorInfo info = data.getExecutorData().getExecutorInfo(EXECUTOR_DRIVER_NAME); - if (info == null) { - logger.error("Application id [" + data.getGeneralData().getApplicationId() - + "] does not contain driver memory configuration info and also does not contain executor driver info." - + " Unable to detect is driver memory usage."); - return 0L; - } - // This maxmium memory only counts in memory for storage - bytes = (long) (info.maxMem / getStorageMemoryFraction(data.getEnvironmentData())); - } - - return bytes; - } - - /** - * Get the storage memory fraction ratio used for storage - * - * @param data The spark environment data - * @return the memory fraction - */ - private static double getStorageMemoryFraction(SparkEnvironmentData data) { - String ratio = data.getSparkProperty(SPARK_STORAGE_MEMORY_FRACTION); - if (ratio == null) { - ratio = new SparkConf().get(SPARK_STORAGE_MEMORY_FRACTION, String.valueOf(DEFAULT_SPARK_STORAGE_MEMORY_FRACTION)); - } - return Double.parseDouble(ratio); - } - - /** - * Get the peak storage memory used during all running time of the spark application - * - * @param data The spark application data that contains the information - * @return The memory in bytes - */ - private static long getStoragePeakMemory(SparkApplicationData data) { - SparkExecutorData executorData = data.getExecutorData(); - long mem = 0L; - for (String id : executorData.getExecutors()) { - mem += executorData.getExecutorInfo(id).memUsed; - } - return mem; - } - - /** - * Get the total memory allocated for storage - * - * @param data The spark application data that contains the information - * @return The memory in bytes - */ - private static long getTotalStorageMem(SparkApplicationData data) { - SparkExecutorData executorData = data.getExecutorData(); - long totalStorageMem = 0L; - for (String id : executorData.getExecutors()) { - totalStorageMem += executorData.getExecutorInfo(id).maxMem; - } - return totalStorageMem; - } - - public Severity getTotalMemorySeverity(long memory) { - return Severity.getSeverityAscending(memory, totalMemLimits[0], totalMemLimits[1], totalMemLimits[2], - totalMemLimits[3]); - } - - private Severity getMemoryUtilizationSeverity(long peakMemory, long totalStorageMemory) { - double fraction = peakMemory * 1.0 / totalStorageMemory; - if (totalStorageMemory < MemoryFormatUtils.stringToBytes("500 GB")) { - return Severity.NONE; - } else { - return Severity.getSeverityDescending( - fraction, memUtilLimits[0], memUtilLimits[1], memUtilLimits[2], memUtilLimits[3]); - } - } -} diff --git a/app/com/linkedin/drelephant/spark/heuristics/StageRuntimeHeuristic.java b/app/com/linkedin/drelephant/spark/heuristics/StageRuntimeHeuristic.java deleted file mode 100644 index 4856248e8..000000000 --- a/app/com/linkedin/drelephant/spark/heuristics/StageRuntimeHeuristic.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.Heuristic; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.math.Statistics; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkJobProgressData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.Utils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - - -/** - * Spark heuristic that detects stage runtime anomalies. - * - */ -public class StageRuntimeHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(StageRuntimeHeuristic.class); - - // Severity parameters - private static final String STAGE_FAILURE_SEVERITY = "stage_failure_rate_severity"; - private static final String SINGLE_STAGE_FAILURE_SEVERITY = "single_stage_tasks_failure_rate_severity"; - private static final String STAGE_RUNTIME_SEVERITY = "stage_runtime_severity_in_min"; - - // Default value of parameters - private double[] stageFailRateLimits = {0.3d, 0.3d, 0.5d, 0.5d}; - private double[] singleStageFailLimits = {0.0d, 0.3d, 0.5d, 0.5d}; - private double[] stageRuntimeLimits = {15, 30, 60, 60}; - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confStageFailRateLimits = Utils.getParam(paramMap.get(STAGE_FAILURE_SEVERITY), - stageFailRateLimits.length); - if (confStageFailRateLimits != null) { - stageFailRateLimits = confStageFailRateLimits; - } - logger.info(heuristicName + " will use " + STAGE_FAILURE_SEVERITY + " with the following threshold settings: " - + Arrays.toString(stageFailRateLimits)); - - double[] confSingleFailLimits = Utils.getParam(paramMap.get(SINGLE_STAGE_FAILURE_SEVERITY), - singleStageFailLimits.length); - if (confSingleFailLimits != null) { - singleStageFailLimits = confSingleFailLimits; - } - logger.info(heuristicName + " will use " + SINGLE_STAGE_FAILURE_SEVERITY + " with the following threshold" - + " settings: " + Arrays.toString(singleStageFailLimits)); - - double[] confStageRuntimeLimits = Utils.getParam(paramMap.get(STAGE_RUNTIME_SEVERITY), stageRuntimeLimits.length); - if (confStageRuntimeLimits != null) { - stageRuntimeLimits = confStageRuntimeLimits; - } - logger.info(heuristicName + " will use " + STAGE_RUNTIME_SEVERITY + " with the following threshold settings: " - + Arrays.toString(stageRuntimeLimits)); - for (int i = 0; i < stageRuntimeLimits.length; i++) { - stageRuntimeLimits[i] = stageRuntimeLimits[i] * Statistics.MINUTE_IN_MS; - } - } - - public StageRuntimeHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } - - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } - - @Override - public HeuristicResult apply(SparkApplicationData data) { - SparkJobProgressData jobProgressData = data.getJobProgressData(); - Severity endSeverity = Severity.NONE; - - Set completedStages = jobProgressData.getCompletedStages(); - Set failedStages = jobProgressData.getFailedStages(); - - // Average stage failure rate - double avgStageFailureRate = jobProgressData.getStageFailureRate(); - endSeverity = Severity.max(endSeverity, getStageFailureRateSeverity(avgStageFailureRate)); - - // For each completed stage, the task failure rate - List problematicStages = new ArrayList(); - - for (SparkJobProgressData.StageAttemptId id : completedStages) { - SparkJobProgressData.StageInfo info = jobProgressData.getStageInfo(id.stageId, id.attemptId); - double stageTasksFailureRate = info.getFailureRate(); - Severity tasksFailureRateSeverity = getSingleStageTasksFailureRate(stageTasksFailureRate); - - if (tasksFailureRateSeverity.getValue() > Severity.MODERATE.getValue()) { - problematicStages.add(String.format("%s (task failure rate: %1.3f)", id, stageTasksFailureRate)); - } - - long duration = info.duration; - Severity runtimeSeverity = getStageRuntimeSeverity(duration); - if (runtimeSeverity.getValue() > Severity.MODERATE.getValue()) { - problematicStages - .add(String.format("%s (runtime: %s)", id, Statistics.readableTimespan(duration))); - } - - endSeverity = Severity.max(endSeverity, tasksFailureRateSeverity, runtimeSeverity); - } - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), endSeverity, 0); - - result.addResultDetail("Spark stage completed", String.valueOf(completedStages.size())); - result.addResultDetail("Spark stage failed", String.valueOf(failedStages.size())); - result.addResultDetail("Spark average stage failure rate", String.format("%.3f", avgStageFailureRate)); - result.addResultDetail("Spark problematic stages", getStagesAsString(problematicStages)); - - return result; - } - - private Severity getStageRuntimeSeverity(long runtime) { - return Severity.getSeverityAscending( - runtime, stageRuntimeLimits[0], stageRuntimeLimits[1], stageRuntimeLimits[2], stageRuntimeLimits[3]); - } - - private Severity getStageFailureRateSeverity(double rate) { - return Severity.getSeverityAscending( - rate, stageFailRateLimits[0], stageFailRateLimits[1], stageFailRateLimits[2], stageFailRateLimits[3]); - } - - private Severity getSingleStageTasksFailureRate(double rate) { - return Severity.getSeverityAscending( - rate, singleStageFailLimits[0], singleStageFailLimits[1], singleStageFailLimits[2], singleStageFailLimits[3]); - } - - private static String getStagesAsString(Collection names) { - return StringUtils.join(names, "\n"); - } -} diff --git a/app/com/linkedin/drelephant/spark/heuristics/StagesHeuristic.scala b/app/com/linkedin/drelephant/spark/heuristics/StagesHeuristic.scala new file mode 100644 index 000000000..dd92f814f --- /dev/null +++ b/app/com/linkedin/drelephant/spark/heuristics/StagesHeuristic.scala @@ -0,0 +1,210 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import com.linkedin.drelephant.spark.fetchers.statusapiv1.ExecutorSummary +import scala.collection.JavaConverters +import scala.concurrent.duration +import scala.concurrent.duration.Duration + +import com.linkedin.drelephant.analysis.{Heuristic, HeuristicResult, HeuristicResultDetails, Severity, SeverityThresholds} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.math.Statistics +import com.linkedin.drelephant.spark.data.SparkApplicationData +import com.linkedin.drelephant.spark.fetchers.statusapiv1.StageData +import org.apache.spark.status.api.v1.StageStatus + + +/** + * A heuristic based on metrics for a Spark app's stages. + * + * This heuristic reports stage failures, high task failure rates for each stage, and long average executor runtimes for + * each stage. + */ +class StagesHeuristic(private val heuristicConfigurationData: HeuristicConfigurationData) + extends Heuristic[SparkApplicationData] { + import StagesHeuristic._ + import JavaConverters._ + + val stageFailureRateSeverityThresholds: SeverityThresholds = + SeverityThresholds.parse(heuristicConfigurationData.getParamMap.get(STAGE_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY), ascending = true) + .getOrElse(DEFAULT_STAGE_FAILURE_RATE_SEVERITY_THRESHOLDS) + + val taskFailureRateSeverityThresholds: SeverityThresholds = + SeverityThresholds.parse(heuristicConfigurationData.getParamMap.get(TASK_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY), ascending = true) + .getOrElse(DEFAULT_TASK_FAILURE_RATE_SEVERITY_THRESHOLDS) + + val stageRuntimeMillisSeverityThresholds: SeverityThresholds = + SeverityThresholds.parse(heuristicConfigurationData.getParamMap.get(STAGE_RUNTIME_MINUTES_SEVERITY_THRESHOLDS_KEY), ascending = true) + .map(minutesSeverityThresholdsToMillisSeverityThresholds) + .getOrElse(DEFAULT_STAGE_RUNTIME_MILLIS_SEVERITY_THRESHOLDS) + + override def getHeuristicConfData(): HeuristicConfigurationData = heuristicConfigurationData + + override def apply(data: SparkApplicationData): HeuristicResult = { + val evaluator = new Evaluator(this, data) + + def formatStagesWithHighTaskFailureRates(stagesWithHighTaskFailureRates: Seq[(StageData, Double)]): String = + stagesWithHighTaskFailureRates + .map { case (stageData, taskFailureRate) => formatStageWithHighTaskFailureRate(stageData, taskFailureRate) } + .mkString("\n") + + def formatStageWithHighTaskFailureRate(stageData: StageData, taskFailureRate: Double): String = + f"stage ${stageData.stageId}, attempt ${stageData.attemptId} (task failure rate: ${taskFailureRate}%1.3f)" + + def formatStagesWithLongAverageExecutorRuntimes(stagesWithLongAverageExecutorRuntimes: Seq[(StageData, Long)]): String = + stagesWithLongAverageExecutorRuntimes + .map { case (stageData, runtime) => formatStageWithLongRuntime(stageData, runtime) } + .mkString("\n") + + def formatStageWithLongRuntime(stageData: StageData, runtime: Long): String = + f"stage ${stageData.stageId}, attempt ${stageData.attemptId} (runtime: ${Statistics.readableTimespan(runtime)})" + + val resultDetails = Seq( + new HeuristicResultDetails("Spark completed stages count", evaluator.numCompletedStages.toString), + new HeuristicResultDetails("Spark failed stages count", evaluator.numFailedStages.toString), + new HeuristicResultDetails("Spark stage failure rate", f"${evaluator.stageFailureRate.getOrElse(0.0D)}%.3f"), + new HeuristicResultDetails( + "Spark stages with high task failure rates", + formatStagesWithHighTaskFailureRates(evaluator.stagesWithHighTaskFailureRates) + ), + new HeuristicResultDetails( + "Spark stages with long average executor runtimes", + formatStagesWithLongAverageExecutorRuntimes(evaluator.stagesWithLongAverageExecutorRuntimes) + ) + ) + val result = new HeuristicResult( + heuristicConfigurationData.getClassName, + heuristicConfigurationData.getHeuristicName, + evaluator.severity, + 0, + resultDetails.asJava + ) + result + } + +} + +object StagesHeuristic { + /** The default severity thresholds for the rate of an application's stages failing. */ + val DEFAULT_STAGE_FAILURE_RATE_SEVERITY_THRESHOLDS = + SeverityThresholds(low = 0.1D, moderate = 0.3D, severe = 0.5D, critical = 0.5D, ascending = true) + + /** The default severity thresholds for the rate of a stage's tasks failing. */ + val DEFAULT_TASK_FAILURE_RATE_SEVERITY_THRESHOLDS = + SeverityThresholds(low = 0.1D, moderate = 0.3D, severe = 0.5D, critical = 0.5D, ascending = true) + + /** The default severity thresholds for a stage's runtime. */ + val DEFAULT_STAGE_RUNTIME_MILLIS_SEVERITY_THRESHOLDS = SeverityThresholds( + low = Duration("15min").toMillis, + moderate = Duration("30min").toMillis, + severe = Duration("45min").toMillis, + critical = Duration("60min").toMillis, + ascending = true + ) + + val STAGE_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY = "stage_failure_rate_severity_thresholds" + val TASK_FAILURE_RATE_SEVERITY_THRESHOLDS_KEY = "stage_task_failure_rate_severity_thresholds" + val STAGE_RUNTIME_MINUTES_SEVERITY_THRESHOLDS_KEY = "stage_runtime_minutes_severity_thresholds" + + val SPARK_EXECUTOR_INSTANCES_KEY = "spark.executor.instances" + + class Evaluator(stagesHeuristic: StagesHeuristic, data: SparkApplicationData) { + lazy val stageDatas: Seq[StageData] = data.stageDatas + + lazy val appConfigurationProperties: Map[String, String] = + data.appConfigurationProperties + + lazy val executorSummaries: Seq[ExecutorSummary] = data.executorSummaries + + lazy val numCompletedStages: Int = stageDatas.count { _.status == StageStatus.COMPLETE } + + lazy val numFailedStages: Int = stageDatas.count { _.status == StageStatus.FAILED } + + lazy val stageFailureRate: Option[Double] = { + val numStages = numCompletedStages + numFailedStages + if (numStages == 0) None else Some(numFailedStages.toDouble / numStages.toDouble) + } + + lazy val stagesWithHighTaskFailureRates: Seq[(StageData, Double)] = + stagesWithHighTaskFailureRateSeverities.map { case (stageData, taskFailureRate, _) => (stageData, taskFailureRate) } + + lazy val stagesWithLongAverageExecutorRuntimes: Seq[(StageData, Long)] = + stagesAndAverageExecutorRuntimeSeverities + .collect { case (stageData, runtime, severity) if severity.getValue > Severity.MODERATE.getValue => (stageData, runtime) } + + lazy val severity: Severity = Severity.max((stageFailureRateSeverity +: (taskFailureRateSeverities ++ runtimeSeverities)): _*) + + private lazy val stageFailureRateSeverityThresholds = stagesHeuristic.stageFailureRateSeverityThresholds + + private lazy val taskFailureRateSeverityThresholds = stagesHeuristic.taskFailureRateSeverityThresholds + + private lazy val stageRuntimeMillisSeverityThresholds = stagesHeuristic.stageRuntimeMillisSeverityThresholds + + private lazy val stageFailureRateSeverity: Severity = + stageFailureRateSeverityThresholds.severityOf(stageFailureRate.getOrElse[Double](0.0D)) + + private lazy val stagesWithHighTaskFailureRateSeverities: Seq[(StageData, Double, Severity)] = + stagesAndTaskFailureRateSeverities.filter { case (_, _, severity) => severity.getValue > Severity.MODERATE.getValue } + + private lazy val stagesAndTaskFailureRateSeverities: Seq[(StageData, Double, Severity)] = for { + stageData <- stageDatas + (taskFailureRate, severity) = taskFailureRateAndSeverityOf(stageData) + } yield (stageData, taskFailureRate, severity) + + private lazy val taskFailureRateSeverities: Seq[Severity] = + stagesAndTaskFailureRateSeverities.map { case (_, _, severity) => severity } + + private lazy val stagesAndAverageExecutorRuntimeSeverities: Seq[(StageData, Long, Severity)] = for { + stageData <- stageDatas + (runtime, severity) = averageExecutorRuntimeAndSeverityOf(stageData) + } yield (stageData, runtime, severity) + + private lazy val runtimeSeverities: Seq[Severity] = stagesAndAverageExecutorRuntimeSeverities.map { case (_, _, severity) => severity } + + private lazy val executorInstances: Int = + appConfigurationProperties.get(SPARK_EXECUTOR_INSTANCES_KEY).map(_.toInt).getOrElse(executorSummaries.size) + + private def taskFailureRateAndSeverityOf(stageData: StageData): (Double, Severity) = { + val taskFailureRate = taskFailureRateOf(stageData).getOrElse(0.0D) + (taskFailureRate, taskFailureRateSeverityThresholds.severityOf(taskFailureRate)) + } + + private def taskFailureRateOf(stageData: StageData): Option[Double] = { + // Currently, the calculation doesn't include skipped or active tasks. + val numCompleteTasks = stageData.numCompleteTasks + val numFailedTasks = stageData.numFailedTasks + val numTasks = numCompleteTasks + numFailedTasks + if (numTasks == 0) None else Some(numFailedTasks.toDouble / numTasks.toDouble) + } + + private def averageExecutorRuntimeAndSeverityOf(stageData: StageData): (Long, Severity) = { + val averageExecutorRuntime = stageData.executorRunTime / executorInstances + (averageExecutorRuntime, stageRuntimeMillisSeverityThresholds.severityOf(averageExecutorRuntime)) + } + } + + def minutesSeverityThresholdsToMillisSeverityThresholds( + minutesSeverityThresholds: SeverityThresholds + ): SeverityThresholds = SeverityThresholds( + Duration(minutesSeverityThresholds.low.longValue, duration.MINUTES).toMillis, + Duration(minutesSeverityThresholds.moderate.longValue, duration.MINUTES).toMillis, + Duration(minutesSeverityThresholds.severe.longValue, duration.MINUTES).toMillis, + Duration(minutesSeverityThresholds.critical.longValue, duration.MINUTES).toMillis, + minutesSeverityThresholds.ascending + ) +} diff --git a/app/com/linkedin/drelephant/util/InfoExtractor.java b/app/com/linkedin/drelephant/util/InfoExtractor.java index d03e98baa..7c529acf4 100644 --- a/app/com/linkedin/drelephant/util/InfoExtractor.java +++ b/app/com/linkedin/drelephant/util/InfoExtractor.java @@ -20,7 +20,6 @@ import com.linkedin.drelephant.configurations.scheduler.SchedulerConfiguration; import com.linkedin.drelephant.configurations.scheduler.SchedulerConfigurationData; import com.linkedin.drelephant.schedulers.Scheduler; -import com.linkedin.drelephant.spark.data.SparkApplicationData; import java.lang.reflect.InvocationTargetException; import java.util.List; @@ -108,8 +107,6 @@ public static void loadInfo(AppResult result, HadoopApplicationData data) { Properties properties = new Properties(); if (data instanceof MapReduceApplicationData) { properties = retrieveMapreduceProperties((MapReduceApplicationData) data); - } else if (data instanceof SparkApplicationData) { - properties = retrieveSparkProperties((SparkApplicationData) data); } Scheduler scheduler = getSchedulerInstance(data.getAppId(), properties); @@ -158,33 +155,6 @@ public static Properties retrieveMapreduceProperties(MapReduceApplicationData ap return appData.getConf(); } - /** - * Retrieve the spark properties from SPARK_EXTRA_JAVA_OPTIONS - * - * @param appData the Spark Application Data - * @return The retrieved Spark properties - */ - public static Properties retrieveSparkProperties(SparkApplicationData appData) { - String prop = appData.getEnvironmentData().getSparkProperty(SPARK_EXTRA_JAVA_OPTIONS); - Properties properties = new Properties(); - if (prop != null) { - try { - Map javaOptions = Utils.parseJavaOptions(prop); - for (String key : javaOptions.keySet()) { - properties.setProperty(key, unescapeString(javaOptions.get(key))); - } - logger.info("Parsed options:" + properties.toString()); - } catch (IllegalArgumentException e) { - logger.error("Encountered error while parsing java options into urls: " + e.getMessage()); - } - } else { - logger.error("Unable to retrieve the scheduler info for application [" + - appData.getGeneralData().getApplicationId() + "]. It does not contain [" + SPARK_EXTRA_JAVA_OPTIONS - + "] property in its spark properties."); - } - return properties; - } - /** * A temporary solution that SPARK 1.2 need to escape '&' with '\&' in its javaOptions. * This is the reverse process that recovers the escaped string. diff --git a/app/com/linkedin/drelephant/util/SparkUtils.scala b/app/com/linkedin/drelephant/util/SparkUtils.scala new file mode 100644 index 000000000..3a0354070 --- /dev/null +++ b/app/com/linkedin/drelephant/util/SparkUtils.scala @@ -0,0 +1,62 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.util + +import java.io.{File, FileInputStream, InputStreamReader} +import java.util.Properties + +import scala.collection.JavaConverters + + +trait SparkUtils { + import JavaConverters._ + + def defaultEnv: Map[String, String] + + // Below this line are modified utility methods from + // https://github.com/apache/spark/blob/v1.4.1/core/src/main/scala/org/apache/spark/util/Utils.scala + + /** Return the path of the default Spark properties file. */ + def getDefaultPropertiesFile(env: Map[String, String] = defaultEnv): Option[String] = { + env.get("SPARK_CONF_DIR") + .orElse(env.get("SPARK_HOME").map { t => s"$t${File.separator}conf" }) + .map { t => new File(s"$t${File.separator}spark-defaults.conf")} + .filter(_.isFile) + .map(_.getAbsolutePath) + } + + /** Load properties present in the given file. */ + def getPropertiesFromFile(filename: String): Map[String, String] = { + val file = new File(filename) + require(file.exists(), s"Properties file $file does not exist") + require(file.isFile(), s"Properties file $file is not a normal file") + + val inReader = new InputStreamReader(new FileInputStream(file), "UTF-8") + try { + val properties = new Properties() + properties.load(inReader) + properties.stringPropertyNames().asScala.map( + k => (k, properties.getProperty(k).trim)).toMap + } finally { + inReader.close() + } + } +} + +object SparkUtils extends SparkUtils { + override val defaultEnv = sys.env +} diff --git a/app/org/apache/spark/SparkMetricsAggregator.java b/app/org/apache/spark/SparkMetricsAggregator.java deleted file mode 100644 index ef8566b41..000000000 --- a/app/org/apache/spark/SparkMetricsAggregator.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.spark; -import com.linkedin.drelephant.math.Statistics; -import org.apache.commons.io.FileUtils; - -import com.linkedin.drelephant.analysis.HadoopApplicationData; -import com.linkedin.drelephant.analysis.HadoopMetricsAggregator; -import com.linkedin.drelephant.analysis.HadoopAggregatedData; -import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkExecutorData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import java.util.Iterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -public class SparkMetricsAggregator implements HadoopMetricsAggregator { - - private static final Logger logger = LoggerFactory.getLogger(SparkMetricsAggregator.class); - - private AggregatorConfigurationData _aggregatorConfigurationData; - private double _storageMemWastageBuffer = 0.5; - - private static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; - private static final String STORAGE_MEM_WASTAGE_BUFFER = "storage_mem_wastage_buffer"; - - private HadoopAggregatedData _hadoopAggregatedData = new HadoopAggregatedData(); - - - public SparkMetricsAggregator(AggregatorConfigurationData _aggregatorConfigurationData) { - this._aggregatorConfigurationData = _aggregatorConfigurationData; - String configValue = _aggregatorConfigurationData.getParamMap().get(STORAGE_MEM_WASTAGE_BUFFER); - if(configValue != null) { - _storageMemWastageBuffer = Double.parseDouble(configValue); - } - } - - @Override - public void aggregate(HadoopApplicationData data) { - long resourceUsed = 0; - long resourceWasted = 0; - SparkApplicationData applicationData = (SparkApplicationData) data; - - long perExecutorMem = - MemoryFormatUtils.stringToBytes(applicationData.getEnvironmentData().getSparkProperty(SPARK_EXECUTOR_MEMORY, "0")); - - Iterator executorIds = applicationData.getExecutorData().getExecutors().iterator(); - - while(executorIds.hasNext()) { - String executorId = executorIds.next(); - SparkExecutorData.ExecutorInfo executorInfo = applicationData.getExecutorData().getExecutorInfo(executorId); - // store the resourceUsed in MBSecs - resourceUsed += (executorInfo.duration / Statistics.SECOND_IN_MS) * (perExecutorMem / FileUtils.ONE_MB); - // maxMem is the maximum available storage memory - // memUsed is how much storage memory is used. - // any difference is wasted after a buffer of 50% is wasted - long excessMemory = (long) (executorInfo.maxMem - (executorInfo.memUsed * (1.0 + _storageMemWastageBuffer))); - if( excessMemory > 0) { - resourceWasted += (executorInfo.duration / Statistics.SECOND_IN_MS) * (excessMemory / FileUtils.ONE_MB); - } - } - - _hadoopAggregatedData.setResourceUsed(resourceUsed); - _hadoopAggregatedData.setResourceWasted(resourceWasted); - // TODO: to find a way to calculate the delay - _hadoopAggregatedData.setTotalDelay(0L); - } - - @Override - public HadoopAggregatedData getResult() { - return _hadoopAggregatedData; - } -} diff --git a/app/org/apache/spark/deploy/history/SparkDataCollection.scala b/app/org/apache/spark/deploy/history/SparkDataCollection.scala deleted file mode 100644 index 4fdae9d92..000000000 --- a/app/org/apache/spark/deploy/history/SparkDataCollection.scala +++ /dev/null @@ -1,317 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.spark.deploy.history - - -import com.linkedin.drelephant.analysis.ApplicationType -import com.linkedin.drelephant.spark.data._ -import SparkExecutorData.ExecutorInfo -import SparkJobProgressData.JobInfo -import org.apache.spark.scheduler.{StageInfo, ApplicationEventListener} -import org.apache.spark.storage.{StorageStatusTrackingListener, StorageStatus, RDDInfo, StorageStatusListener} -import org.apache.spark.ui.env.EnvironmentListener -import org.apache.spark.ui.exec.ExecutorsListener -import org.apache.spark.ui.jobs.JobProgressListener -import org.apache.spark.ui.storage.StorageListener -import org.apache.spark.util.collection.OpenHashSet - -import java.util.{Set => JSet} -import java.util.{HashSet => JHashSet} -import java.util.{List => JList} -import java.util.{ArrayList => JArrayList} -import java.util.Properties - -import scala.collection.mutable - - -/** - * This class wraps the logic of collecting the data in SparkEventListeners into the - * HadoopApplicationData instances. - * - * Notice: - * This has to live in Spark's scope because ApplicationEventListener is in private[spark] scope. And it is problematic - * to compile if written in Java. - */ -class SparkDataCollection(applicationEventListener: ApplicationEventListener, - jobProgressListener: JobProgressListener, - storageStatusListener: StorageStatusListener, - environmentListener: EnvironmentListener, - executorsListener: ExecutorsListener, - storageListener: StorageListener, - storageStatusTrackingListener: StorageStatusTrackingListener) extends SparkApplicationData { - private var _applicationData: SparkGeneralData = null; - private var _jobProgressData: SparkJobProgressData = null; - private var _environmentData: SparkEnvironmentData = null; - private var _executorData: SparkExecutorData = null; - private var _storageData: SparkStorageData = null; - private var _isThrottled: Boolean = false; - - import SparkDataCollection._ - - - def throttle(): Unit = { - _isThrottled = true - } - - override def isThrottled(): Boolean = _isThrottled - - override def getApplicationType(): ApplicationType = APPLICATION_TYPE - - override def getConf(): Properties = getEnvironmentData().getSparkProperties() - - override def isEmpty(): Boolean = !isThrottled() && getExecutorData().getExecutors.isEmpty() - - override def getGeneralData(): SparkGeneralData = { - if (_applicationData == null) { - _applicationData = new SparkGeneralData() - - applicationEventListener.adminAcls match { - case Some(s: String) => { - _applicationData.setAdminAcls(stringToSet(s)) - } - case None => { - // do nothing - } - } - - applicationEventListener.viewAcls match { - case Some(s: String) => { - _applicationData.setViewAcls(stringToSet(s)) - } - case None => { - // do nothing - } - } - - applicationEventListener.appId match { - case Some(s: String) => { - _applicationData.setApplicationId(s) - } - case None => { - // do nothing - } - } - - applicationEventListener.appName match { - case Some(s: String) => { - _applicationData.setApplicationName(s) - } - case None => { - // do nothing - } - } - - applicationEventListener.sparkUser match { - case Some(s: String) => { - _applicationData.setSparkUser(s) - } - case None => { - // do nothing - } - } - - applicationEventListener.startTime match { - case Some(s: Long) => { - _applicationData.setStartTime(s) - } - case None => { - // do nothing - } - } - - applicationEventListener.endTime match { - case Some(s: Long) => { - _applicationData.setEndTime(s) - } - case None => { - // do nothing - } - } - } - _applicationData - } - - override def getEnvironmentData(): SparkEnvironmentData = { - if (_environmentData == null) { - // Notice: we ignore jvmInformation and classpathEntries, because they are less likely to be used by any analyzer. - _environmentData = new SparkEnvironmentData() - environmentListener.systemProperties.foreach { case (name, value) => - _environmentData.addSystemProperty(name, value) - } - environmentListener.sparkProperties.foreach { case (name, value) => - _environmentData.addSparkProperty(name, value) - } - } - _environmentData - } - - override def getExecutorData(): SparkExecutorData = { - if (_executorData == null) { - _executorData = new SparkExecutorData() - - for (statusId <- 0 until executorsListener.storageStatusList.size) { - val info = new ExecutorInfo() - - val status = executorsListener.storageStatusList(statusId) - - info.execId = status.blockManagerId.executorId - info.hostPort = status.blockManagerId.hostPort - info.rddBlocks = status.numBlocks - - // Use a customized listener to fetch the peak memory used, the data contained in status are - // the current used memory that is not useful in offline settings. - info.memUsed = storageStatusTrackingListener.executorIdToMaxUsedMem.getOrElse(info.execId, 0L) - info.maxMem = status.maxMem - info.diskUsed = status.diskUsed - info.activeTasks = executorsListener.executorToTasksActive.getOrElse(info.execId, 0) - info.failedTasks = executorsListener.executorToTasksFailed.getOrElse(info.execId, 0) - info.completedTasks = executorsListener.executorToTasksComplete.getOrElse(info.execId, 0) - info.totalTasks = info.activeTasks + info.failedTasks + info.completedTasks - info.duration = executorsListener.executorToDuration.getOrElse(info.execId, 0L) - info.inputBytes = executorsListener.executorToInputBytes.getOrElse(info.execId, 0L) - info.shuffleRead = executorsListener.executorToShuffleRead.getOrElse(info.execId, 0L) - info.shuffleWrite = executorsListener.executorToShuffleWrite.getOrElse(info.execId, 0L) - - _executorData.setExecutorInfo(info.execId, info) - } - } - _executorData - } - - override def getJobProgressData(): SparkJobProgressData = { - if (_jobProgressData == null) { - _jobProgressData = new SparkJobProgressData() - - // Add JobInfo - jobProgressListener.jobIdToData.foreach { case (id, data) => - val jobInfo = new JobInfo() - - jobInfo.jobId = data.jobId - jobInfo.jobGroup = data.jobGroup.getOrElse("") - jobInfo.numActiveStages = data.numActiveStages - jobInfo.numActiveTasks = data.numActiveTasks - jobInfo.numCompletedTasks = data.numCompletedTasks - jobInfo.numFailedStages = data.numFailedStages - jobInfo.numFailedTasks = data.numFailedTasks - jobInfo.numSkippedStages = data.numSkippedStages - jobInfo.numSkippedTasks = data.numSkippedTasks - jobInfo.numTasks = data.numTasks - - jobInfo.startTime = data.submissionTime.getOrElse(0) - jobInfo.endTime = data.completionTime.getOrElse(0) - - data.stageIds.foreach{ case (id: Int) => jobInfo.addStageId(id)} - addIntSetToJSet(data.completedStageIndices, jobInfo.completedStageIndices) - - _jobProgressData.addJobInfo(id, jobInfo) - } - - // Add Stage Info - jobProgressListener.stageIdToData.foreach { case (id, data) => - val stageInfo = new SparkJobProgressData.StageInfo() - val sparkStageInfo = jobProgressListener.stageIdToInfo.get(id._1) - stageInfo.name = sparkStageInfo match { - case Some(info: StageInfo) => { - info.name - } - case None => { - "" - } - } - stageInfo.description = data.description.getOrElse("") - stageInfo.diskBytesSpilled = data.diskBytesSpilled - stageInfo.executorRunTime = data.executorRunTime - stageInfo.duration = sparkStageInfo match { - case Some(info: StageInfo) => { - val submissionTime = info.submissionTime.getOrElse(0L) - info.completionTime.getOrElse(submissionTime) - submissionTime - } - case _ => 0L - } - stageInfo.inputBytes = data.inputBytes - stageInfo.memoryBytesSpilled = data.memoryBytesSpilled - stageInfo.numActiveTasks = data.numActiveTasks - stageInfo.numCompleteTasks = data.numCompleteTasks - stageInfo.numFailedTasks = data.numFailedTasks - stageInfo.outputBytes = data.outputBytes - stageInfo.shuffleReadBytes = data.shuffleReadTotalBytes - stageInfo.shuffleWriteBytes = data.shuffleWriteBytes - addIntSetToJSet(data.completedIndices, stageInfo.completedIndices) - - _jobProgressData.addStageInfo(id._1, id._2, stageInfo) - } - - // Add completed jobs - jobProgressListener.completedJobs.foreach { case (data) => _jobProgressData.addCompletedJob(data.jobId) } - // Add failed jobs - jobProgressListener.failedJobs.foreach { case (data) => _jobProgressData.addFailedJob(data.jobId) } - // Add completed stages - jobProgressListener.completedStages.foreach { case (data) => - _jobProgressData.addCompletedStages(data.stageId, data.attemptId) - } - // Add failed stages - jobProgressListener.failedStages.foreach { case (data) => - _jobProgressData.addFailedStages(data.stageId, data.attemptId) - } - } - _jobProgressData - } - - // This method returns a combined information from StorageStatusListener and StorageListener - override def getStorageData(): SparkStorageData = { - if (_storageData == null) { - _storageData = new SparkStorageData() - _storageData.setRddInfoList(toJList[RDDInfo](storageListener.rddInfoList)) - _storageData.setStorageStatusList(toJList[StorageStatus](storageStatusListener.storageStatusList)) - } - _storageData - } - - override def getAppId: String = { - getGeneralData().getApplicationId - } -} - -object SparkDataCollection { - private val APPLICATION_TYPE = new ApplicationType("SPARK") - - def stringToSet(str: String): JSet[String] = { - val set = new JHashSet[String]() - str.split(",").foreach { case t: String => set.add(t)} - set - } - - def toJList[T](seq: Seq[T]): JList[T] = { - val list = new JArrayList[T]() - seq.foreach { case (item: T) => list.add(item)} - list - } - - def addIntSetToJSet(set: OpenHashSet[Int], jset: JSet[Integer]): Unit = { - val it = set.iterator - while (it.hasNext) { - jset.add(it.next()) - } - } - - def addIntSetToJSet(set: mutable.HashSet[Int], jset: JSet[Integer]): Unit = { - val it = set.iterator - while (it.hasNext) { - jset.add(it.next()) - } - } -} diff --git a/app/org/apache/spark/deploy/history/SparkFSFetcher.scala b/app/org/apache/spark/deploy/history/SparkFSFetcher.scala deleted file mode 100644 index 48e453c8f..000000000 --- a/app/org/apache/spark/deploy/history/SparkFSFetcher.scala +++ /dev/null @@ -1,354 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.spark.deploy.history - - -import java.net.{HttpURLConnection, URL, URI} -import java.security.PrivilegedAction -import java.io.{IOException, BufferedInputStream, InputStream} -import java.{io, util} -import java.util.ArrayList -import javax.ws.rs.core.UriBuilder -import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData -import com.linkedin.drelephant.security.HadoopSecurity -import com.linkedin.drelephant.spark.data.SparkApplicationData -import com.linkedin.drelephant.util.{MemoryFormatUtils, Utils} -import com.linkedin.drelephant.analysis.{ApplicationType, AnalyticJob, ElephantFetcher} -import org.apache.commons.io.FileUtils - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.hadoop.hdfs.web.WebHdfsFileSystem -import org.apache.hadoop.security.authentication.client.{AuthenticatedURL, AuthenticationException} -import org.apache.log4j.Logger -import org.apache.spark.SparkConf -import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus, ApplicationEventListener} -import org.apache.spark.storage.{StorageStatusTrackingListener, StorageStatusListener} -import org.apache.spark.ui.env.EnvironmentListener -import org.apache.spark.ui.exec.ExecutorsListener -import org.apache.spark.ui.jobs.JobProgressListener -import org.apache.spark.ui.storage.StorageListener -import org.apache.spark.io.CompressionCodec -import org.codehaus.jackson.JsonNode -import org.codehaus.jackson.map.ObjectMapper - -import scala.collection.mutable.ArrayBuffer - - -/** - * A wrapper that replays Spark event history from files and then fill proper data objects. - */ -class SparkFSFetcher(fetcherConfData: FetcherConfigurationData) extends ElephantFetcher[SparkApplicationData] { - - import SparkFSFetcher._ - - val NAME_SERVICES = "dfs.nameservices"; - val DFS_HA_NAMENODES = "dfs.ha.namenodes"; - val DFS_NAMENODE_HTTP_ADDRESS = "dfs.namenode.http-address"; - - var confEventLogSizeInMb = defEventLogSizeInMb - if (fetcherConfData.getParamMap.get(LOG_SIZE_XML_FIELD) != null) { - val logLimitSize = Utils.getParam(fetcherConfData.getParamMap.get(LOG_SIZE_XML_FIELD), 1) - if (logLimitSize != null) { - confEventLogSizeInMb = logLimitSize(0) - } - } - logger.info("The event log limit of Spark application is set to " + confEventLogSizeInMb + " MB") - - var confEventLogDir = fetcherConfData.getParamMap.get(LOG_DIR_XML_FIELD) - if (confEventLogDir == null || confEventLogDir.isEmpty) { - confEventLogDir = defEventLogDir - } - logger.info("The event log directory of Spark application is set to " + confEventLogDir) - - private val _sparkConf = new SparkConf() - - /* Lazy loading for the log directory is very important. Hadoop Configuration() takes time to load itself to reflect - * properties in the configuration files. Triggering it too early will sometimes make the configuration object empty. - */ - private lazy val _logDir: String = { - val conf = new Configuration() - val nodeAddress = getNamenodeAddress(conf); - val hdfsAddress = if (nodeAddress == null) "" else "webhdfs://" + nodeAddress - - val uri = new URI(_sparkConf.get("spark.eventLog.dir", confEventLogDir)) - val logDir = hdfsAddress + uri.getPath - logger.info("Looking for spark logs at logDir: " + logDir) - logDir - } - - /** - * Returns the namenode address of the active nameNode - * @param conf The Hadoop configuration - * @return The namenode address of the active namenode - */ - def getNamenodeAddress(conf: Configuration): String = { - - // check if the fetcherconf has namenode addresses. There can be multiple addresses and - // we need to check the active namenode address. If a value is specified in the fetcherconf - // then the value obtained from hadoop configuration won't be used. - if (fetcherConfData.getParamMap.get(NAMENODE_ADDRESSES) != null) { - val nameNodes: Array[String] = fetcherConfData.getParamMap.get(NAMENODE_ADDRESSES).split(","); - for (nameNode <- nameNodes) { - if (checkActivation(nameNode)) { - return nameNode; - } - } - } - - // if we couldn't find the namenode address in fetcherconf, try to find it in hadoop configuration. - var isHAEnabled: Boolean = false; - if (conf.get(NAME_SERVICES) != null) { - isHAEnabled = true; - } - - // check if HA is enabled - if (isHAEnabled) { - // There can be multiple nameservices separated by ',' in case of HDFS federation. It is not supported right now. - if (conf.get(NAME_SERVICES).split(",").length > 1) { - logger.info("Multiple name services found. HDFS federation is not supported right now.") - return null; - } - val nameService: String = conf.get(NAME_SERVICES); - val nameNodeIdentifier: String = conf.get(DFS_HA_NAMENODES + "." + nameService); - if (nameNodeIdentifier != null) { - // there can be multiple namenode identifiers separated by ',' - for (nameNodeIdentifierValue <- nameNodeIdentifier.split(",")) { - val httpValue = conf.get(DFS_NAMENODE_HTTP_ADDRESS + "." + nameService + "." + nameNodeIdentifierValue); - if (httpValue != null && checkActivation(httpValue)) { - logger.info("Active namenode : " + httpValue); - return httpValue; - } - } - } - } - - // if HA is disabled, return the namenode http-address. - return conf.get(DFS_NAMENODE_HTTP_ADDRESS); - } - - /** - * Checks if the namenode specified is active or not - * @param httpValue The namenode configuration http value - * @return True if the namenode is active, otherwise false - */ - def checkActivation(httpValue: String): Boolean = { - val url: URL = new URL("http://" + httpValue + "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"); - val rootNode: JsonNode = readJsonNode(url); - val status: String = rootNode.path("beans").get(0).path("State").getValueAsText(); - if (status.equals("active")) { - return true; - } - return false; - } - - /** - * Returns the jsonNode which is read from the url - * @param url The url of the server - * @return The jsonNode parsed from the url - */ - def readJsonNode(url: URL): JsonNode = { - val _token: AuthenticatedURL.Token = new AuthenticatedURL.Token(); - val _authenticatedURL: AuthenticatedURL = new AuthenticatedURL(); - val _objectMapper: ObjectMapper = new ObjectMapper(); - val conn: HttpURLConnection = _authenticatedURL.openConnection(url, _token) - return _objectMapper.readTree(conn.getInputStream) - } - - - private val _security = new HadoopSecurity() - - private def fs: FileSystem = { - - // For test purpose, if no host presented, use the local file system. - if (new URI(_logDir).getHost == null) { - FileSystem.getLocal(new Configuration()) - } else { - val filesystem = new WebHdfsFileSystem() - filesystem.initialize(new URI(_logDir), new Configuration()) - filesystem - } - } - - def fetchData(analyticJob: AnalyticJob): SparkApplicationData = { - val appId = analyticJob.getAppId() - _security.doAs[SparkDataCollection](new PrivilegedAction[SparkDataCollection] { - override def run(): SparkDataCollection = { - /* Most of Spark logs will be in directory structure: /LOG_DIR/[application_id]. - * - * Some logs (Spark 1.3+) are in /LOG_DIR/[application_id].snappy - * - * Currently we won't be able to parse them even we manually set up the codec. There is problem - * in JsonProtocol#sparkEventFromJson that it does not handle unmatched SparkListenerEvent, which means - * it is only backward compatible but not forward. And switching the dependency to Spark 1.3 will raise more - * problems due to the fact that we are touching the internal codes. - * - * In short, this fetcher only works with Spark <=1.2, and we should switch to JSON endpoints with Spark's - * future release. - */ - val replayBus = new ReplayListenerBus() - val applicationEventListener = new ApplicationEventListener - val jobProgressListener = new JobProgressListener(new SparkConf()) - val environmentListener = new EnvironmentListener - val storageStatusListener = new StorageStatusListener - val executorsListener = new ExecutorsListener(storageStatusListener) - val storageListener = new StorageListener(storageStatusListener) - - // This is a customized listener that tracks peak used memory - // The original listener only tracks the current in use memory which is useless in offline scenario. - val storageStatusTrackingListener = new StorageStatusTrackingListener() - replayBus.addListener(storageStatusTrackingListener) - - val dataCollection = new SparkDataCollection(applicationEventListener = applicationEventListener, - jobProgressListener = jobProgressListener, - environmentListener = environmentListener, - storageStatusListener = storageStatusListener, - executorsListener = executorsListener, - storageListener = storageListener, - storageStatusTrackingListener = storageStatusTrackingListener) - - replayBus.addListener(applicationEventListener) - replayBus.addListener(jobProgressListener) - replayBus.addListener(environmentListener) - replayBus.addListener(storageStatusListener) - replayBus.addListener(executorsListener) - replayBus.addListener(storageListener) - - val logPath = new Path(_logDir, appId) - val logInput: InputStream = - if (isLegacyLogDirectory(logPath)) { - if (!shouldThrottle(logPath)) { - openLegacyEventLog(logPath) - } else { - null - } - } else { - val sparkLogExt = Option(fetcherConfData.getParamMap.get(SPARK_LOG_EXT)).getOrElse(defSparkLogExt) - val logFilePath = new Path(logPath + sparkLogExt) - if (!shouldThrottle(logFilePath)) { - EventLoggingListener.openEventLog(logFilePath, fs) - } else { - null - } - } - - if (logInput == null) { - dataCollection.throttle() - // Since the data set is empty, we need to set the application id, - // so that we could detect this is Spark job type - dataCollection.getGeneralData().setApplicationId(appId) - dataCollection.getConf().setProperty("spark.app.id", appId) - - logger.info("The event log of Spark application: " + appId + " is over the limit size of " - + confEventLogSizeInMb + " MB, the parsing process gets throttled.") - } else { - logger.info("Replaying Spark logs for application: " + appId) - - replayBus.replay(logInput, logPath.toString(), false) - - logger.info("Replay completed for application: " + appId) - } - - dataCollection - } - }) - } - - /** - * Checks if the log path stores the legacy event log. (Spark <= 1.2 store an event log in a directory) - * - * @param entry The path to check - * @return true if it is legacy log path, else false - */ - private def isLegacyLogDirectory(entry: Path): Boolean = fs.exists(entry) && fs.getFileStatus(entry).isDirectory() - - /** - * Opens a legacy log path - * - * @param dir The directory to open - * @return an InputStream - */ - private def openLegacyEventLog(dir: Path): InputStream = { - val children = fs.listStatus(dir) - var eventLogPath: Path = null - var codecName: Option[String] = None - - children.foreach { child => - child.getPath().getName() match { - case name if name.startsWith(LOG_PREFIX) => - eventLogPath = child.getPath() - case codec if codec.startsWith(COMPRESSION_CODEC_PREFIX) => - codecName = Some(codec.substring(COMPRESSION_CODEC_PREFIX.length())) - case _ => - } - } - - if (eventLogPath == null) { - throw new IllegalArgumentException(s"$dir is not a Spark application log directory.") - } - - val codec = try { - codecName.map { c => CompressionCodec.createCodec(_sparkConf, c) } - } catch { - case e: Exception => - throw new IllegalArgumentException(s"Unknown compression codec $codecName.") - } - - val in = new BufferedInputStream(fs.open(eventLogPath)) - codec.map(_.compressedInputStream(in)).getOrElse(in) - } - - /** - * Checks if the log parser should be throttled when the file is too large. - * Note: the current Spark's implementation of ReplayListenerBus will take more than 80 minutes to read a compressed - * 500 MB event log file. Allowing such reading might block the entire Dr Elephant thread pool. - * - * @param eventLogPath The event log path - * @return If the event log parsing should be throttled - */ - private def shouldThrottle(eventLogPath: Path): Boolean = { - fs.getFileStatus(eventLogPath).getLen() > (confEventLogSizeInMb * FileUtils.ONE_MB) - } - - def getEventLogSize(): Double = { - confEventLogSizeInMb - } - - def getEventLogDir(): String = { - confEventLogDir - } - -} - -private object SparkFSFetcher { - private val logger = Logger.getLogger(SparkFSFetcher.getClass) - - var defEventLogDir = "/system/spark-history" - var defEventLogSizeInMb = 100d; // 100MB - var defSparkLogExt = "_1.snappy" - - val LOG_SIZE_XML_FIELD = "event_log_size_limit_in_mb" - val LOG_DIR_XML_FIELD = "event_log_dir" - - // Constants used to parse <= Spark 1.2.0 log directories. - val LOG_PREFIX = "EVENT_LOG_" - val COMPRESSION_CODEC_PREFIX = EventLoggingListener.COMPRESSION_CODEC_KEY + "_" - - // Param map property names that allow users to configer various aspects of the fetcher - val NAMENODE_ADDRESSES = "namenode_addresses" - val SPARK_LOG_EXT = "spark_log_ext" -} diff --git a/app/org/apache/spark/storage/StorageStatusTrackingListener.scala b/app/org/apache/spark/storage/StorageStatusTrackingListener.scala deleted file mode 100644 index 5d30a2887..000000000 --- a/app/org/apache/spark/storage/StorageStatusTrackingListener.scala +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.spark.storage - - -import scala.collection.mutable - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.scheduler._ - - -/** - * :: DeveloperApi :: - * A modified version of StorageStatusListener that tracks the peak memory usage during the entire application runtime. - * - * NOTICE: this class copies StorageStatusListener's codes instead of extending from it, because the methods that - * require being overridden are all in private scope. - */ -@DeveloperApi -class StorageStatusTrackingListener 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 - - val executorIdToMaxUsedMem = mutable.Map[String, Long]() - - /** Update storage status list to reflect updated block statuses */ - private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { - executorIdToStorageStatus.get(execId).foreach { storageStatus => - updatedBlocks.foreach { case (blockId, updatedStatus) => - if (updatedStatus.storageLevel == StorageLevel.NONE) { - storageStatus.removeBlock(blockId) - } else { - storageStatus.updateBlock(blockId, updatedStatus) - } - } - } - updateUsedMem() - } - - /** Update storage status list to reflect the removal of an RDD from the cache */ - private def updateStorageStatus(unpersistedRDDId: Int): Unit = { - storageStatusList.foreach { storageStatus => - storageStatus.rddBlocksById(unpersistedRDDId).foreach { case (blockId, _) => - storageStatus.removeBlock(blockId) - } - } - updateUsedMem() - } - - private def updateUsedMem(): Unit = { - executorIdToStorageStatus.foreach { case (execId, storageStatus) => - val currentMemUsed = storageStatus.memUsed - if (currentMemUsed > executorIdToMaxUsedMem.getOrElse(execId, 0L)) { - executorIdToMaxUsedMem(execId) = currentMemUsed - } - } - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - synchronized { - val info = taskEnd.taskInfo - val metrics = taskEnd.taskMetrics - if (info != null && metrics != null) { - val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) - if (updatedBlocks.length > 0) { - updateStorageStatus(info.executorId, updatedBlocks) - } - } - } - } - - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = { - synchronized { - updateStorageStatus(unpersistRDD.rddId) - } - } - - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { - synchronized { - val blockManagerId = blockManagerAdded.blockManagerId - val executorId = blockManagerId.executorId - val maxMem = blockManagerAdded.maxMem - val storageStatus = new StorageStatus(blockManagerId, maxMem) - executorIdToStorageStatus(executorId) = storageStatus - } - } - - override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { - synchronized { - val executorId = blockManagerRemoved.blockManagerId.executorId - executorIdToStorageStatus.remove(executorId) - } - } -} diff --git a/app/views/help/spark/helpConfigurationHeuristic.scala.html b/app/views/help/spark/helpConfigurationHeuristic.scala.html new file mode 100644 index 000000000..abadd6167 --- /dev/null +++ b/app/views/help/spark/helpConfigurationHeuristic.scala.html @@ -0,0 +1,18 @@ +@* +* Copyright 2016 LinkedIn Corp. +* +* Licensed under the Apache License, Version 2.0 (the "License"); you may not +* use this file except in compliance with the License. You may obtain a copy of +* the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +* License for the specific language governing permissions and limitations under +* the License. +*@ +

The results from this heuristic primarily inform you about key app +configuration settings, including driver memory, executor cores, +executor instances, executor memory, and the serializer.

diff --git a/app/views/help/spark/helpExecutorsHeuristic.scala.html b/app/views/help/spark/helpExecutorsHeuristic.scala.html new file mode 100644 index 000000000..70084339c --- /dev/null +++ b/app/views/help/spark/helpExecutorsHeuristic.scala.html @@ -0,0 +1,36 @@ +@* +* Copyright 2016 LinkedIn Corp. +* +* Licensed under the Apache License, Version 2.0 (the "License"); you may not +* use this file except in compliance with the License. You may obtain a copy of +* the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +* License for the specific language governing permissions and limitations under +* the License. +*@ +

This heuristic concerns the distribution (min, 25p, median, 75p, +max) of key executor metrics including input bytes, shuffle read +bytes, shuffle write bytes, storage memory used, and task time. The +max-to-median ratio determines the severity of any particular metric.

+ +

Spark application get resources from YARN allocated all at once, +and don't release these until the application completes. Thus, it's +important to balance load on the executors to avoid wasting +resources.

+ +

To achieve better load balancing:

+ +
    +
  • use an appropriate number of partitions (some small multiple of + the # of executors) so that there are enough tasks handling those + partitions to keep the executors busy
  • +
  • try avoiding key skew; you should know which partitioner you are + using and what is the distribution of your keys
  • +
  • consider enabling spark.speculation, so that straggler tasks can + be re-launched
  • +
diff --git a/app/views/help/spark/helpJobsHeuristic.scala.html b/app/views/help/spark/helpJobsHeuristic.scala.html new file mode 100644 index 000000000..e72005e00 --- /dev/null +++ b/app/views/help/spark/helpJobsHeuristic.scala.html @@ -0,0 +1,20 @@ +@* +* Copyright 2016 LinkedIn Corp. +* +* Licensed under the Apache License, Version 2.0 (the "License"); you may not +* use this file except in compliance with the License. You may obtain a copy of +* the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +* License for the specific language governing permissions and limitations under +* the License. +*@ +

This heuristic reports job failures and high task failure rates for +each job.

+ +

Job/task failures can occur for a number of reasons, so it is +recommended to look at the YARN application error logs.

diff --git a/app/views/help/spark/helpStagesHeuristic.scala.html b/app/views/help/spark/helpStagesHeuristic.scala.html new file mode 100644 index 000000000..6af0162d1 --- /dev/null +++ b/app/views/help/spark/helpStagesHeuristic.scala.html @@ -0,0 +1,20 @@ +@* +* Copyright 2016 LinkedIn Corp. +* +* Licensed under the Apache License, Version 2.0 (the "License"); you may not +* use this file except in compliance with the License. You may obtain a copy of +* the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +* License for the specific language governing permissions and limitations under +* the License. +*@ +

This heuristic reports stage failures, high task failure rates for +each stage, and long average executor runtimes for each stage.

+ +

Stage/task failures can occur for a number of reasons, so it is +recommended to look at the YARN application error logs.

diff --git a/build.sbt b/build.sbt index 15d9c9ddb..071e15337 100644 --- a/build.sbt +++ b/build.sbt @@ -25,7 +25,7 @@ organization := "com.linkedin.drelephant" javacOptions in Compile ++= Seq("-source", "1.6", "-target", "1.6") -libraryDependencies ++= dependencies +libraryDependencies ++= dependencies map { _.excludeAll(exclusionRules: _*) } // Create a new custom configuration called compileonly ivyConfigurations += config("compileonly").hide @@ -34,3 +34,5 @@ ivyConfigurations += config("compileonly").hide unmanagedClasspath in Compile ++= update.value.select(configurationFilter("compileonly")) playJavaSettings + +scalaVersion := "2.10.4" diff --git a/project/Dependencies.scala b/project/Dependencies.scala index a0b6206e0..b53e2de17 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -25,6 +25,8 @@ object Dependencies { lazy val gsonVersion = "2.2.4" lazy val guavaVersion = "18.0" // Hadoop defaultly are using guava 11.0, might raise NoSuchMethodException lazy val jacksonMapperAslVersion = "1.7.3" + lazy val jacksonVersion = "2.5.3" + lazy val jerseyVersion = "2.24" lazy val jsoupVersion = "1.7.3" lazy val mysqlConnectorVersion = "5.1.36" lazy val oozieClientVersion = "4.2.0" @@ -61,8 +63,10 @@ object Dependencies { var requiredDep = Seq( "com.google.code.gson" % "gson" % gsonVersion, "com.google.guava" % "guava" % guavaVersion, + "com.jsuereth" %% "scala-arm" % "1.4", "commons-codec" % "commons-codec" % commonsCodecVersion, "commons-io" % "commons-io" % commonsIoVersion, + "javax.ws.rs" % "javax.ws.rs-api" % "2.0.1", "mysql" % "mysql-connector-java" % mysqlConnectorVersion, "org.apache.hadoop" % "hadoop-auth" % hadoopVersion % "compileonly", "org.apache.hadoop" % "hadoop-mapreduce-client-core" % hadoopVersion % "compileonly", @@ -74,12 +78,26 @@ object Dependencies { "org.apache.oozie" % "oozie-client" % oozieClientVersion excludeAll( ExclusionRule(organization = "org.apache.hadoop") ), + "org.glassfish.jersey.core" % "jersey-client" % jerseyVersion, + "org.glassfish.jersey.core" % "jersey-common" % jerseyVersion, + "org.glassfish.jersey.media" % "jersey-media-json-jackson" % jerseyVersion % Test, + "org.glassfish.jersey.test-framework" % "jersey-test-framework-core" % jerseyVersion % Test, + "org.glassfish.jersey.test-framework.providers" % "jersey-test-framework-provider-grizzly2" % jerseyVersion % Test, + "com.fasterxml.jackson.core" % "jackson-databind" % jacksonVersion, + "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion, "io.dropwizard.metrics" % "metrics-core" % "3.1.2", "io.dropwizard.metrics" % "metrics-healthchecks" % "3.1.2", "org.mockito" % "mockito-core" % "1.10.19" exclude ("org.hamcrest", "hamcrest-core"), - "org.jmockit" % "jmockit" % "1.23" % Test + "org.jmockit" % "jmockit" % "1.23" % Test, + "org.scala-lang.modules" %% "scala-async" % "0.9.5", + "org.scalatest" %% "scalatest" % "3.0.0" % Test ) :+ sparkExclusion var dependencies = Seq(javaJdbc, javaEbean, cache) dependencies ++= requiredDep + + val exclusionRules = Seq( + ExclusionRule(organization = "com.sun.jersey", name = "jersey-core"), + ExclusionRule(organization = "com.sun.jersey", name = "jersey-server") + ) } diff --git a/project/build.properties b/project/build.properties index 58add1433..bb96499e0 100644 --- a/project/build.properties +++ b/project/build.properties @@ -14,4 +14,4 @@ # the License. # -sbt.version=0.13.0 +sbt.version=0.13.2 \ No newline at end of file diff --git a/test/com/linkedin/drelephant/analysis/SeverityThresholdsTest.scala b/test/com/linkedin/drelephant/analysis/SeverityThresholdsTest.scala new file mode 100644 index 000000000..488118e01 --- /dev/null +++ b/test/com/linkedin/drelephant/analysis/SeverityThresholdsTest.scala @@ -0,0 +1,73 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.analysis + +import org.scalatest.{FunSpec, Matchers} + +class SeverityThresholdsTest extends FunSpec with Matchers { + describe("SeverityThresholds") { + it("can be used to represent thresholds considered in ascending order") { + val thresholds = SeverityThresholds(low = 0.2D, moderate = 0.4D, severe = 0.6D, critical = 0.8D, ascending = true) + thresholds.severityOf(0.1D) should be(Severity.NONE) + thresholds.severityOf(0.2D) should be(Severity.LOW) + thresholds.severityOf(0.3D) should be(Severity.LOW) + thresholds.severityOf(0.4D) should be(Severity.MODERATE) + thresholds.severityOf(0.5D) should be(Severity.MODERATE) + thresholds.severityOf(0.6D) should be(Severity.SEVERE) + thresholds.severityOf(0.7D) should be(Severity.SEVERE) + thresholds.severityOf(0.8D) should be(Severity.CRITICAL) + thresholds.severityOf(0.9D) should be(Severity.CRITICAL) + } + + it("can be used to represent thresholds considered in descending order") { + val thresholds = SeverityThresholds(low = 0.8D, moderate = 0.6D, severe = 0.4D, critical = 0.2D, ascending = false) + thresholds.severityOf(0.1D) should be(Severity.CRITICAL) + thresholds.severityOf(0.2D) should be(Severity.CRITICAL) + thresholds.severityOf(0.3D) should be(Severity.SEVERE) + thresholds.severityOf(0.4D) should be(Severity.SEVERE) + thresholds.severityOf(0.5D) should be(Severity.MODERATE) + thresholds.severityOf(0.6D) should be(Severity.MODERATE) + thresholds.severityOf(0.7D) should be(Severity.LOW) + thresholds.severityOf(0.8D) should be(Severity.LOW) + thresholds.severityOf(0.9D) should be(Severity.NONE) + } + + it("can be parsed as ascending thresholds from a string that can be processed by Utils.getParam") { + SeverityThresholds.parse("0.2,0.4,0.6,0.8", ascending = true) should be( + Some(SeverityThresholds(low = 0.2D, moderate = 0.4D, severe = 0.6D, critical = 0.8D, ascending = true)) + ) + } + + it("can be parsed as descending thresholds from a string that can be processed by Utils.getParam") { + SeverityThresholds.parse("0.8,0.6,0.4,0.2", ascending = false) should be( + Some(SeverityThresholds(low = 0.8D, moderate = 0.6D, severe = 0.4D, critical = 0.2D, ascending = false)) + ) + } + + it("cannot be created as ascending thresholds with unordered values") { + an[IllegalArgumentException] should be thrownBy( + SeverityThresholds(low = 0.8D, moderate = 0.6D, severe = 0.4D, critical = 0.2D, ascending = true) + ) + } + + it("cannot be created as descending thresholds with unordered values") { + an[IllegalArgumentException] should be thrownBy( + SeverityThresholds(low = 0.2D, moderate = 0.4D, severe = 0.6D, critical = 0.8D, ascending = false) + ) + } + } +} diff --git a/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java b/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java index be3f6bbcd..2db6431c4 100644 --- a/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java +++ b/test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java @@ -127,10 +127,6 @@ public void testParseFetcherConf5() { FetcherConfiguration fetcherConf = new FetcherConfiguration(document5.getDocumentElement()); assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1); assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark); - assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getParamMap().size(), 3); - assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getParamMap().get(logSizeField), logSizeValue); - assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getParamMap().get(logDirField), logDirValue); } } - diff --git a/test/com/linkedin/drelephant/math/StatisticsTest.java b/test/com/linkedin/drelephant/math/StatisticsTest.java index 3800b2f86..9c28bf56b 100644 --- a/test/com/linkedin/drelephant/math/StatisticsTest.java +++ b/test/com/linkedin/drelephant/math/StatisticsTest.java @@ -98,6 +98,14 @@ public void testPercentile() { assertEquals(10, Statistics.percentile(finishTimes,10)); assertEquals(10, Statistics.percentile(finishTimes,4)); + List oddLengthValues = new ArrayList(); + oddLengthValues.add(1L); + oddLengthValues.add(2L); + oddLengthValues.add(3L); + oddLengthValues.add(4L); + oddLengthValues.add(5L); + assertEquals(3L, Statistics.percentile(oddLengthValues, 50)); + List finishTimeSingle = new ArrayList(); finishTimeSingle.add(10L); assertEquals(10,Statistics.percentile(finishTimeSingle,100)); diff --git a/test/com/linkedin/drelephant/spark/MockSparkApplicationData.java b/test/com/linkedin/drelephant/spark/MockSparkApplicationData.java deleted file mode 100644 index 40aeacd05..000000000 --- a/test/com/linkedin/drelephant/spark/MockSparkApplicationData.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark; - -import com.linkedin.drelephant.analysis.ApplicationType; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkEnvironmentData; -import com.linkedin.drelephant.spark.data.SparkExecutorData; -import com.linkedin.drelephant.spark.data.SparkGeneralData; -import com.linkedin.drelephant.spark.data.SparkJobProgressData; -import com.linkedin.drelephant.spark.data.SparkStorageData; -import java.util.Properties; - - -/** - * This is a pseudo local implementation of SparkApplicationData interface, supposed to be used for test purpose. - */ -public class MockSparkApplicationData implements SparkApplicationData { - private static final ApplicationType APPLICATION_TYPE = new ApplicationType("SPARK"); - - private final SparkGeneralData _sparkGeneralData; - private final SparkEnvironmentData _sparkEnvironmentData; - private final SparkExecutorData _sparkExecutorData; - private final SparkJobProgressData _sparkJobProgressData; - private final SparkStorageData _sparkStorageData; - - public MockSparkApplicationData() { - _sparkGeneralData = new SparkGeneralData(); - _sparkEnvironmentData = new SparkEnvironmentData(); - _sparkExecutorData = new SparkExecutorData(); - _sparkJobProgressData = new SparkJobProgressData(); - _sparkStorageData = new SparkStorageData(); - } - - @Override - public boolean isThrottled() { - return false; - } - - @Override - public SparkGeneralData getGeneralData() { - return _sparkGeneralData; - } - - @Override - public SparkEnvironmentData getEnvironmentData() { - return _sparkEnvironmentData; - } - - @Override - public SparkExecutorData getExecutorData() { - return _sparkExecutorData; - } - - @Override - public SparkJobProgressData getJobProgressData() { - return _sparkJobProgressData; - } - - @Override - public SparkStorageData getStorageData() { - return _sparkStorageData; - } - - @Override - public Properties getConf() { - return getEnvironmentData().getSparkProperties(); - } - - @Override - public String getAppId() { - return getGeneralData().getApplicationId(); - } - - @Override - public ApplicationType getApplicationType() { - return APPLICATION_TYPE; - } - - @Override - public boolean isEmpty() { - return getExecutorData().getExecutors().isEmpty(); - } -} diff --git a/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala new file mode 100644 index 000000000..035842107 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala @@ -0,0 +1,169 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark + +import java.util.Date + +import scala.collection.JavaConverters + +import com.linkedin.drelephant.analysis.ApplicationType +import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData +import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo, ExecutorSummary} +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate +import org.scalatest.{FunSpec, Matchers} + +class SparkMetricsAggregatorTest extends FunSpec with Matchers { + import SparkMetricsAggregatorTest._ + + describe("SparkMetricsAggregator") { + val aggregatorConfigurationData = newFakeAggregatorConfigurationData( + Map("allocated_memory_waste_buffer_percentage" -> "0.5") + ) + + val appId = "application_1" + + val applicationInfo = { + val applicationAttemptInfo = { + val now = System.currentTimeMillis + val duration = 8000000L + newFakeApplicationAttemptInfo(Some("1"), startTime = new Date(now - duration), endTime = new Date(now)) + } + new ApplicationInfo(appId, name = "app", Seq(applicationAttemptInfo)) + } + + val restDerivedData = { + val executorSummaries = Seq( + newFakeExecutorSummary(id = "1", totalDuration = 1000000L), + newFakeExecutorSummary(id = "2", totalDuration = 3000000L) + ) + SparkRestDerivedData( + applicationInfo, + jobDatas = Seq.empty, + stageDatas = Seq.empty, + executorSummaries + ) + } + + describe("when it has log-derived data") { + val logDerivedData = { + val environmentUpdate = newFakeSparkListenerEnvironmentUpdate( + Map( + "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer", + "spark.storage.memoryFraction" -> "0.3", + "spark.driver.memory" -> "2G", + "spark.executor.instances" -> "2", + "spark.executor.memory" -> "4g", + "spark.shuffle.memoryFraction" -> "0.5" + ) + ) + SparkLogDerivedData(environmentUpdate) + } + + val data = SparkApplicationData(appId, restDerivedData, Some(logDerivedData)) + + val aggregator = new SparkMetricsAggregator(aggregatorConfigurationData) + aggregator.aggregate(data) + + val result = aggregator.getResult + + it("calculates resources used") { + val executorMemoryMb = 4096 + val totalExecutorTaskTimeSeconds = 1000 + 3000 + result.getResourceUsed should be(executorMemoryMb * totalExecutorTaskTimeSeconds) + } + + it("calculates resources wasted") { + val totalExecutorMemoryMb = 2 * 4096 + val applicationDurationSeconds = 8000 + + val executorMemoryMb = 4096 + val totalExecutorTaskTimeSeconds = 1000 + 3000 + + result.getResourceWasted should be(4096 * 4000) + } + + it("doesn't calculate total delay") { + result.getTotalDelay should be(0L) + } + } + + describe("when it doesn't have log-derived data") { + val data = SparkApplicationData(appId, restDerivedData, logDerivedData = None) + + val aggregator = new SparkMetricsAggregator(aggregatorConfigurationData) + aggregator.aggregate(data) + + val result = aggregator.getResult + + it("doesn't calculate resources used") { + result.getResourceUsed should be(0L) + } + + it("doesn't calculate resources wasted") { + result.getResourceWasted should be(0L) + } + + it("doesn't calculate total delay") { + result.getTotalDelay should be(0L) + } + } + } +} + +object SparkMetricsAggregatorTest { + import JavaConverters._ + + def newFakeAggregatorConfigurationData(params: Map[String, String] = Map.empty): AggregatorConfigurationData = + new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", new ApplicationType("SPARK"), params.asJava) + + def newFakeSparkListenerEnvironmentUpdate(appConfigurationProperties: Map[String, String]): SparkListenerEnvironmentUpdate = + SparkListenerEnvironmentUpdate(Map("Spark Properties" -> appConfigurationProperties.toSeq)) + + def newFakeApplicationAttemptInfo( + attemptId: Option[String], + startTime: Date, + endTime: Date + ): ApplicationAttemptInfo = new ApplicationAttemptInfo( + attemptId, + startTime, + endTime, + sparkUser = "foo", + completed = true + ) + + def newFakeExecutorSummary( + id: String, + totalDuration: Long + ): ExecutorSummary = new ExecutorSummary( + id, + hostPort = "", + rddBlocks = 0, + memoryUsed = 0, + diskUsed = 0, + activeTasks = 0, + failedTasks = 0, + completedTasks = 0, + totalTasks = 0, + totalDuration, + totalInputBytes = 0, + totalShuffleRead = 0, + totalShuffleWrite = 0, + maxMemory = 0, + executorLogs = Map.empty + ) +} diff --git a/test/com/linkedin/drelephant/spark/TestSparkAggregatedMetrics.java b/test/com/linkedin/drelephant/spark/TestSparkAggregatedMetrics.java deleted file mode 100644 index ae4616d86..000000000 --- a/test/com/linkedin/drelephant/spark/TestSparkAggregatedMetrics.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark; - -import com.linkedin.drelephant.analysis.ApplicationType; -import com.linkedin.drelephant.analysis.HadoopApplicationData; -import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData; -import com.linkedin.drelephant.spark.data.SparkExecutorData; -import org.apache.commons.io.FileUtils; -import org.junit.Assert; -import org.apache.spark.SparkMetricsAggregator; -import org.apache.spark.deploy.history.SparkDataCollection; -import org.junit.Test; - - -public class TestSparkAggregatedMetrics { - private static final String SPARK_EXECUTOR_MEMORY = "spark.executor.memory"; - - private SparkExecutorData.ExecutorInfo mockExecutorInfo(long maxMem, long memUsed, long duration) { - SparkExecutorData.ExecutorInfo executorInfo = new SparkExecutorData.ExecutorInfo(); - executorInfo.maxMem = maxMem; - executorInfo.memUsed = memUsed; - executorInfo.duration = duration; - - return executorInfo; - } - @Test - public void TestNullExecutors() { - ApplicationType appType = new ApplicationType("SPARK"); - AggregatorConfigurationData conf = - new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); - SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); - - MockSparkApplicationData appData = new MockSparkApplicationData(); - - metrics.aggregate(appData); - - Assert.assertEquals(metrics.getResult().getResourceUsed() , 0L); - Assert.assertEquals(metrics.getResult().getResourceWasted() , 0L); - Assert.assertEquals(metrics.getResult().getTotalDelay() , 0L); - } - @Test - public void TestValidExecutorsWithNoEnvironmentData() { - ApplicationType appType = new ApplicationType("SPARK"); - AggregatorConfigurationData conf = - new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); - SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); - - MockSparkApplicationData appData = new MockSparkApplicationData(); - appData.getExecutorData().setExecutorInfo("1", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - appData.getExecutorData().setExecutorInfo("2", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - - metrics.aggregate(appData); - - Assert.assertEquals(0L, metrics.getResult().getResourceUsed()); - Assert.assertEquals(20L, metrics.getResult().getResourceWasted()); - Assert.assertEquals(0L, metrics.getResult().getTotalDelay()); - } - - @Test - public void TestValidExecutorsAndValidEnvironmentData() { - ApplicationType appType = new ApplicationType("SPARK"); - AggregatorConfigurationData conf = - new AggregatorConfigurationData("org.apache.spark.SparkMetricsAggregator", appType, null); - SparkMetricsAggregator metrics = new SparkMetricsAggregator(conf); - - MockSparkApplicationData appData = new MockSparkApplicationData(); - appData.getExecutorData().setExecutorInfo("1", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - appData.getExecutorData().setExecutorInfo("2", mockExecutorInfo(100 * FileUtils.ONE_MB, 60 * FileUtils.ONE_MB, 1000)); - - appData.getEnvironmentData().addSparkProperty(SPARK_EXECUTOR_MEMORY, "1048576000"); - - metrics.aggregate(appData); - - Assert.assertEquals(2000L, metrics.getResult().getResourceUsed()); - Assert.assertEquals(20L, metrics.getResult().getResourceWasted()); - Assert.assertEquals(0L, metrics.getResult().getTotalDelay()); - } - -} diff --git a/test/com/linkedin/drelephant/spark/data/SparkApplicationDataTest.scala b/test/com/linkedin/drelephant/spark/data/SparkApplicationDataTest.scala new file mode 100644 index 000000000..5cd686bb5 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/data/SparkApplicationDataTest.scala @@ -0,0 +1,82 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.data + +import java.util.Date + +import scala.collection.JavaConverters + +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo} +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate +import org.scalatest.{FunSpec, Matchers} + +class SparkApplicationDataTest extends FunSpec with Matchers { + import SparkApplicationDataTest._ + import JavaConverters._ + + describe("SparkApplicationData") { + val appId = "application_1" + val attemptId = Some("1") + + val applicationAttemptInfo = { + val now = System.currentTimeMillis + val duration = 8000000L + newFakeApplicationAttemptInfo(attemptId, startTime = new Date(now - duration), endTime = new Date(now)) + } + + val restDerivedData = SparkRestDerivedData( + new ApplicationInfo(appId, "app", Seq(applicationAttemptInfo)), + jobDatas = Seq.empty, + stageDatas = Seq.empty, + executorSummaries = Seq.empty + ) + + val configurationProperties = Map( + "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer", + "spark.storage.memoryFraction" -> "0.3", + "spark.driver.memory" -> "2G", + "spark.executor.instances" -> "900", + "spark.executor.memory" -> "1g", + "spark.shuffle.memoryFraction" -> "0.5" + ) + + val logDerivedData = SparkLogDerivedData( + SparkListenerEnvironmentUpdate(Map("Spark Properties" -> configurationProperties.toSeq)) + ) + + describe(".getConf") { + it("returns the Spark properties") { + val data = SparkApplicationData(appId, restDerivedData, Some(logDerivedData)) + data.getConf.asScala should contain theSameElementsAs(configurationProperties) + } + } + } +} + +object SparkApplicationDataTest { + def newFakeApplicationAttemptInfo( + attemptId: Option[String], + startTime: Date, + endTime: Date + ): ApplicationAttemptInfo = new ApplicationAttemptInfo( + attemptId, + startTime, + endTime, + sparkUser = "foo", + completed = true + ) +} diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala new file mode 100644 index 000000000..7d37168a6 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala @@ -0,0 +1,215 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.fetchers + +import java.io.{File, FileOutputStream, InputStream, OutputStream} +import java.util.Date + +import scala.collection.JavaConverters +import scala.concurrent.{ExecutionContext, Future} + +import com.google.common.io.Files +import com.linkedin.drelephant.analysis.{AnalyticJob, ApplicationType} +import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData +import com.linkedin.drelephant.spark.data.{SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo} +import com.linkedin.drelephant.util.SparkUtils +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate +import org.mockito.Mockito +import org.scalatest.{FunSpec, Matchers} + +class SparkFetcherTest extends FunSpec with Matchers { + import SparkFetcherTest._ + + describe("SparkFetcher") { + import ExecutionContext.Implicits.global + + val fetcherConfigurationData = newFakeFetcherConfigurationData() + + val appId = "application_1" + + val t2 = System.currentTimeMillis + val t1 = t2 - 1 + val duration = 8000000L + + val restDerivedData = SparkRestDerivedData( + new ApplicationInfo( + appId, + "app", + Seq( + newFakeApplicationAttemptInfo(Some("2"), startTime = new Date(t2 - duration), endTime = new Date(t2)), + newFakeApplicationAttemptInfo(Some("1"), startTime = new Date(t1 - duration), endTime = new Date(t1)) + ) + ), + jobDatas = Seq.empty, + stageDatas = Seq.empty, + executorSummaries = Seq.empty + ) + + val logDerivedData = SparkLogDerivedData(SparkListenerEnvironmentUpdate(Map.empty)) + + val analyticJob = new AnalyticJob().setAppId(appId) + + it("returns data") { + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf = new SparkConf() + override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future(restDerivedData)) + override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future(logDerivedData))) + } + val data = sparkFetcher.fetchData(analyticJob) + data.appId should be(appId) + } + + it("throws an exception if the REST client fails") { + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf = new SparkConf() + override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future { throw new Exception() }) + override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future(logDerivedData))) + } + + an[Exception] should be thrownBy { sparkFetcher.fetchData(analyticJob) } + } + + it("throws an exception if the log client fails") { + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkConf = new SparkConf() + override lazy val sparkRestClient = newFakeSparkRestClient(appId, Future(restDerivedData)) + override lazy val sparkLogClient = Some(newFakeSparkLogClient(appId, Some("2"), Future { throw new Exception() })) + } + + an[Exception] should be thrownBy { sparkFetcher.fetchData(analyticJob) } + } + + it("gets its SparkConf when SPARK_CONF_DIR is set") { + val tempDir = Files.createTempDir() + + val testResourceIn = getClass.getClassLoader.getResourceAsStream("spark-defaults.conf") + val testResourceFile = new File(tempDir, "spark-defaults.conf") + val testResourceOut = new FileOutputStream(testResourceFile) + managedCopyInputStreamToOutputStream(testResourceIn, testResourceOut) + + val fetcherConfigurationData = newFakeFetcherConfigurationData() + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkUtils = new SparkUtils() { + override val defaultEnv = Map("SPARK_CONF_DIR" -> tempDir.toString) + } + } + val sparkConf = sparkFetcher.sparkConf + + tempDir.delete() + + sparkConf.get("spark.yarn.historyServer.address") should be("jh1.grid.example.com:18080") + sparkConf.get("spark.eventLog.enabled") should be("true") + sparkConf.get("spark.eventLog.compress") should be("true") + sparkConf.get("spark.eventLog.dir") should be("hdfs://nn1.grid.example.com:9000/logs/spark") + } + + it("gets its SparkConf when SPARK_HOME is set") { + val tempDir = Files.createTempDir() + val tempConfDir = new File(tempDir, "conf") + tempConfDir.mkdir() + + val testResourceIn = getClass.getClassLoader.getResourceAsStream("spark-defaults.conf") + val testResourceFile = new File(tempConfDir, "spark-defaults.conf") + val testResourceOut = new FileOutputStream(testResourceFile) + managedCopyInputStreamToOutputStream(testResourceIn, testResourceOut) + + val fetcherConfigurationData = newFakeFetcherConfigurationData() + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkUtils = new SparkUtils() { + override val defaultEnv = Map("SPARK_HOME" -> tempDir.toString) + } + } + val sparkConf = sparkFetcher.sparkConf + + tempDir.delete() + + sparkConf.get("spark.yarn.historyServer.address") should be("jh1.grid.example.com:18080") + sparkConf.get("spark.eventLog.enabled") should be("true") + sparkConf.get("spark.eventLog.compress") should be("true") + sparkConf.get("spark.eventLog.dir") should be("hdfs://nn1.grid.example.com:9000/logs/spark") + } + + it("throws an exception if neither SPARK_CONF_DIR nor SPARK_HOME are set") { + val fetcherConfigurationData = newFakeFetcherConfigurationData() + val sparkFetcher = new SparkFetcher(fetcherConfigurationData) { + override lazy val sparkUtils = new SparkUtils() { override val defaultEnv = Map.empty[String, String] } + } + an[IllegalStateException] should be thrownBy { sparkFetcher.sparkConf } + } + } +} + +object SparkFetcherTest { + import JavaConverters._ + + def newFakeFetcherConfigurationData(): FetcherConfigurationData = + new FetcherConfigurationData(classOf[SparkFetcher].getName, new ApplicationType("SPARK"), Map.empty.asJava) + + def newFakeApplicationAttemptInfo( + attemptId: Option[String], + startTime: Date, + endTime: Date + ): ApplicationAttemptInfo = new ApplicationAttemptInfo( + attemptId, + startTime, + endTime, + sparkUser = "foo", + completed = true + ) + + def newFakeSparkRestClient( + appId: String, + restDerivedData: Future[SparkRestDerivedData] + )( + implicit ec: ExecutionContext + ): SparkRestClient = { + val sparkRestClient = Mockito.mock(classOf[SparkRestClient]) + Mockito.when(sparkRestClient.fetchData(appId)).thenReturn(restDerivedData) + sparkRestClient + } + + def newFakeSparkLogClient( + appId: String, + attemptId: Option[String], + logDerivedData: Future[SparkLogDerivedData] + )( + implicit ec: ExecutionContext + ): SparkLogClient = { + val sparkLogClient = Mockito.mock(classOf[SparkLogClient]) + Mockito.when(sparkLogClient.fetchData(appId, attemptId)).thenReturn(logDerivedData) + sparkLogClient + } + + def managedCopyInputStreamToOutputStream(in: => InputStream, out: => OutputStream): Unit = { + for { + input <- resource.managed(in) + output <- resource.managed(out) + } { + val buffer = new Array[Byte](512) + def read(): Unit = input.read(buffer) match { + case -1 => () + case bytesRead => { + output.write(buffer, 0, bytesRead) + read() + } + } + read() + } + } +} diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala new file mode 100644 index 000000000..d5fd38927 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala @@ -0,0 +1,130 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.fetchers + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream} +import java.net.URI + +import scala.concurrent.ExecutionContext + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FSDataInputStream, FileSystem, Path, PositionedReadable} +import org.apache.hadoop.io.compress.CompressionInputStream +import org.apache.spark.SparkConf +import org.mockito.BDDMockito +import org.scalatest.{AsyncFunSpec, Matchers} +import org.scalatest.mockito.MockitoSugar +import org.xerial.snappy.SnappyOutputStream + +class SparkLogClientTest extends AsyncFunSpec with Matchers with MockitoSugar { + import SparkLogClientTest._ + + describe("SparkLogClient") { + it("throws an exception if spark.eventLog.dir is missing") { + an[IllegalArgumentException] should be thrownBy { new SparkLogClient(new Configuration(), new SparkConf()) } + } + + it("uses spark.eventLog.dir if it is already an webhdfs URI") { + val hadoopConfiguration = new Configuration() + val sparkConf = new SparkConf().set("spark.eventLog.dir", "webhdfs://nn1.grid.example.com:50070/logs/spark") + val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf) + sparkLogClient.webhdfsEventLogUri should be(new URI("webhdfs://nn1.grid.example.com:50070/logs/spark")) + } + + it("uses a webhdfs URI constructed from spark.eventLog.dir and dfs.namenode.http-address if spark.eventLog.dir is an hdfs URI") { + val hadoopConfiguration = new Configuration() + hadoopConfiguration.set("dfs.namenode.http-address", "0.0.0.0:50070") + val sparkConf = new SparkConf().set("spark.eventLog.dir", "hdfs://nn1.grid.example.com:9000/logs/spark") + val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf) + sparkLogClient.webhdfsEventLogUri should be(new URI("webhdfs://nn1.grid.example.com:50070/logs/spark")) + } + + it("returns the desired data from the Spark event logs") { + import ExecutionContext.Implicits.global + + val hadoopConfiguration = new Configuration() + hadoopConfiguration.set("dfs.namenode.http-address", "0.0.0.0:50070") + + val sparkConf = + new SparkConf() + .set("spark.eventLog.dir", "hdfs://nn1.grid.example.com:9000/logs/spark") + .set("spark.eventLog.compress", "true") + .set("spark.io.compression.codec", "snappy") + + val appId = "application_1" + val attemptId = Some("1") + + val testResourceIn = getClass.getClassLoader.getResourceAsStream("spark_event_logs/event_log_2") + val byteOut = new ByteArrayOutputStream() + val snappyOut = new SnappyOutputStream(byteOut) + managedCopyInputStreamToOutputStream(testResourceIn, snappyOut) + + val sparkLogClient = new SparkLogClient(hadoopConfiguration, sparkConf) { + override lazy val fs: FileSystem = { + val fs = mock[FileSystem] + val expectedPath = new Path("webhdfs://nn1.grid.example.com:50070/logs/spark/application_1_1.snappy") + BDDMockito.given(fs.exists(expectedPath)).willReturn(true) + BDDMockito.given(fs.open(expectedPath)).willReturn( + new FSDataInputStream(new FakeCompressionInputStream(new ByteArrayInputStream(byteOut.toByteArray))) + ) + fs + } + } + + sparkLogClient.fetchData(appId, attemptId).map { logDerivedData => + val expectedProperties = Map( + "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer", + "spark.storage.memoryFraction" -> "0.3", + "spark.driver.memory" -> "2G", + "spark.executor.instances" -> "900", + "spark.executor.memory" -> "1g", + "spark.shuffle.memoryFraction" -> "0.5" + ) + val actualProperties = logDerivedData.appConfigurationProperties + actualProperties should be(expectedProperties) + } + } + } +} + +object SparkLogClientTest { + class FakeCompressionInputStream(in: InputStream) extends CompressionInputStream(in) with PositionedReadable { + override def read(): Int = in.read() + override def read(b: Array[Byte], off: Int, len: Int): Int = in.read(b, off, len) + override def read(pos: Long, buffer: Array[Byte], off: Int, len: Int): Int = ??? + override def readFully(pos: Long, buffer: Array[Byte], off: Int, len: Int): Unit = ??? + override def readFully(pos: Long, buffer: Array[Byte]): Unit = ??? + override def resetState(): Unit = ??? + } + + def managedCopyInputStreamToOutputStream(in: => InputStream, out: => OutputStream): Unit = { + for { + input <- resource.managed(in) + output <- resource.managed(out) + } { + val buffer = new Array[Byte](512) + def read(): Unit = input.read(buffer) match { + case -1 => () + case bytesRead => { + output.write(buffer, 0, bytesRead) + read() + } + } + read() + } + } +} diff --git a/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala b/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala new file mode 100644 index 000000000..7f325739d --- /dev/null +++ b/test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala @@ -0,0 +1,188 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.fetchers + +import java.net.URI +import java.text.SimpleDateFormat +import java.util.{Calendar, Date, SimpleTimeZone} + +import scala.concurrent.ExecutionContext +import scala.util.Try + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationAttemptInfo, ApplicationInfo, ExecutorSummary, JobData, StageData} +import javax.ws.rs.{GET, Path, PathParam, Produces} +import javax.ws.rs.client.WebTarget +import javax.ws.rs.core.{Application, MediaType} +import javax.ws.rs.ext.ContextResolver +import org.apache.spark.SparkConf +import org.glassfish.jersey.client.ClientConfig +import org.glassfish.jersey.server.ResourceConfig +import org.glassfish.jersey.test.{JerseyTest, TestProperties} +import org.scalatest.{AsyncFunSpec, Matchers} +import org.scalatest.compatible.Assertion + +class SparkRestClientTest extends AsyncFunSpec with Matchers { + import SparkRestClientTest._ + + describe("SparkRestClient") { + it("throws an exception if spark.eventLog.dir is missing") { + an[IllegalArgumentException] should be thrownBy(new SparkRestClient(new SparkConf())) + } + + it("returns the desired data from the Spark REST API") { + import ExecutionContext.Implicits.global + val fakeJerseyServer = new FakeJerseyServer() { + override def configure(): Application = super.configure() match { + case resourceConfig: ResourceConfig => + resourceConfig + .register(classOf[FetchDataFixtures.ApiResource]) + .register(classOf[FetchDataFixtures.ApplicationResource]) + .register(classOf[FetchDataFixtures.JobsResource]) + .register(classOf[FetchDataFixtures.StagesResource]) + .register(classOf[FetchDataFixtures.ExecutorsResource]) + case config => config + } + } + + fakeJerseyServer.setUp() + + val historyServerUri = fakeJerseyServer.target.getUri + + val sparkConf = new SparkConf().set("spark.yarn.historyServer.address", s"${historyServerUri.getHost}:${historyServerUri.getPort}") + val sparkRestClient = new SparkRestClient(sparkConf) + + sparkRestClient.fetchData(FetchDataFixtures.APP_ID) map { restDerivedData => + restDerivedData.applicationInfo.id should be(FetchDataFixtures.APP_ID) + restDerivedData.applicationInfo.name should be(FetchDataFixtures.APP_NAME) + restDerivedData.jobDatas should not be(None) + restDerivedData.stageDatas should not be(None) + restDerivedData.executorSummaries should not be(None) + } andThen { case assertion: Try[Assertion] => + fakeJerseyServer.tearDown() + assertion + } + } + } +} + +object SparkRestClientTest { + class FakeJerseyServer extends JerseyTest { + override def configure(): Application = { + forceSet(TestProperties.CONTAINER_PORT, "0") + enable(TestProperties.LOG_TRAFFIC) + enable(TestProperties.DUMP_ENTITY) + + new ResourceConfig() + .register(classOf[FakeJerseyObjectMapperProvider]) + } + + override def configureClient(clientConfig: ClientConfig): Unit = { + clientConfig.register(classOf[FakeJerseyObjectMapperProvider]) + } + } + + class FakeJerseyObjectMapperProvider extends ContextResolver[ObjectMapper] { + lazy val objectMapper = { + val objectMapper = new ObjectMapper() + objectMapper.registerModule(DefaultScalaModule) + objectMapper.setDateFormat(dateFormat) + objectMapper + } + + lazy val dateFormat = { + val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'") + val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT")) + iso8601.setCalendar(cal) + iso8601 + } + + override def getContext(cls: Class[_]): ObjectMapper = objectMapper + } + + object FetchDataFixtures { + val APP_ID = "application_1" + val APP_NAME = "app" + + @Path("/api/v1") + class ApiResource { + @Path("applications/{appId}") + def getApplication(): ApplicationResource = new ApplicationResource() + + @Path("applications/{appId}/{attemptId}/jobs") + def getJobs(): JobsResource = new JobsResource() + + @Path("applications/{appId}/{attemptId}/stages") + def getStages(): StagesResource = new StagesResource() + + @Path("applications/{appId}/{attemptId}/executors") + def getExecutors(): ExecutorsResource = new ExecutorsResource() + } + + @Produces(Array(MediaType.APPLICATION_JSON)) + class ApplicationResource { + @GET + def getApplication(@PathParam("appId") appId: String): ApplicationInfo = { + val t2 = System.currentTimeMillis + val t1 = t2 - 1 + val duration = 8000000L + new ApplicationInfo( + APP_ID, + APP_NAME, + Seq( + newFakeApplicationAttemptInfo(Some("2"), startTime = new Date(t2 - duration), endTime = new Date(t2)), + newFakeApplicationAttemptInfo(Some("1"), startTime = new Date(t1 - duration), endTime = new Date(t1)) + ) + ) + } + } + + @Produces(Array(MediaType.APPLICATION_JSON)) + class JobsResource { + @GET + def getJobs(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Seq[JobData] = + if (attemptId == "2") Seq.empty else throw new Exception() + } + + @Produces(Array(MediaType.APPLICATION_JSON)) + class StagesResource { + @GET + def getStages(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Seq[StageData] = + if (attemptId == "2") Seq.empty else throw new Exception() + } + + @Produces(Array(MediaType.APPLICATION_JSON)) + class ExecutorsResource { + @GET + def getExecutors(@PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): Seq[ExecutorSummary] = + if (attemptId == "2") Seq.empty else throw new Exception() + } + } + + def newFakeApplicationAttemptInfo( + attemptId: Option[String], + startTime: Date, + endTime: Date + ): ApplicationAttemptInfo = new ApplicationAttemptInfo( + attemptId, + startTime, + endTime, + sparkUser = "foo", + completed = true + ) +} diff --git a/test/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristicTest.java b/test/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristicTest.java deleted file mode 100644 index 2bda9ba15..000000000 --- a/test/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristicTest.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.ApplicationType; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.MockSparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; -import junit.framework.TestCase; - -import static com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic.SPARK_DRIVER_MEMORY; -import static com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic.SPARK_EXECUTOR_CORES; -import static com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic.SPARK_SERIALIZER; -import static com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic.SPARK_SHUFFLE_MANAGER; - - -/** - * This class test the BestPropertiesConventionHeuristic - * - */ -public class BestPropertiesConventionHeuristicTest extends TestCase { - public void testPropertiesCheck() { - assertEquals(analyzeJob(getDefaultGoodProperteis()), Severity.NONE); - - assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "7G"))); - assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "8G"))); - assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_DRIVER_MEMORY, "9G"))); - - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "1"))); - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "2"))); - assertEquals(Severity.CRITICAL, analyzeJob(getPropertiesAndOverideOne(SPARK_EXECUTOR_CORES, "4"))); - - - assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_SERIALIZER, "foo"))); - assertEquals(Severity.MODERATE, analyzeJob(getPropertiesAndOverideOne(SPARK_SHUFFLE_MANAGER, "hash"))); - } - - public void testNullSettings() { - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_SERIALIZER))); - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_SHUFFLE_MANAGER))); - assertEquals(Severity.NONE, analyzeJob(getPropertiesAndRemove(SPARK_EXECUTOR_CORES))); - } - - private static Properties getDefaultGoodProperteis() { - Properties properties = new Properties(); - properties.put(SPARK_DRIVER_MEMORY, "1G"); - properties.put(SPARK_EXECUTOR_CORES, "1"); - properties.put(SPARK_SERIALIZER, "org.apache.spark.serializer.KryoSerializer"); - properties.put(SPARK_SHUFFLE_MANAGER, "sort"); - - return properties; - } - - private static Properties getPropertiesAndOverideOne(String key, String value) { - Properties properties = getDefaultGoodProperteis(); - properties.put(key, value); - return properties; - } - - private static Properties getPropertiesAndRemove(String key) { - Properties properties = getDefaultGoodProperteis(); - properties.remove(key); - return properties; - } - - private Severity analyzeJob(Properties sparkProperties) { - SparkApplicationData data = new MockSparkApplicationData(); - for (String key : sparkProperties.stringPropertyNames()) { - data.getEnvironmentData().addSparkProperty(key, sparkProperties.getProperty(key)); - } - Map paramsMap = new HashMap(); - return new BestPropertiesConventionHeuristic(new HeuristicConfigurationData("test_heuristic", "test_class", - "test_view", new ApplicationType("test_apptype"), paramsMap)).apply(data).getSeverity(); - } -} diff --git a/test/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristicTest.scala new file mode 100644 index 000000000..f1e44602f --- /dev/null +++ b/test/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristicTest.scala @@ -0,0 +1,198 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import com.linkedin.drelephant.spark.data.SparkRestDerivedData +import com.linkedin.drelephant.spark.fetchers.statusapiv1.ApplicationInfo +import scala.collection.JavaConverters + +import com.linkedin.drelephant.analysis.{ApplicationType, Severity} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData} +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate +import org.scalatest.{FunSpec, Matchers} + + +class ConfigurationHeuristicTest extends FunSpec with Matchers { + import ConfigurationHeuristicTest._ + + describe("ConfigurationHeuristic") { + val heuristicConfigurationData = newFakeHeuristicConfigurationData( + Map( + "serializer_if_non_null_recommendation" -> "org.apache.spark.serializer.KryoSerializer", + "shuffle_manager_if_non_null_recommendation" -> "sort" + ) + ) + + val configurationHeuristic = new ConfigurationHeuristic(heuristicConfigurationData) + + describe(".apply") { + val configurationProperties = Map( + "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer", + "spark.storage.memoryFraction" -> "0.3", + "spark.driver.memory" -> "2G", + "spark.executor.instances" -> "900", + "spark.executor.memory" -> "1g", + "spark.shuffle.memoryFraction" -> "0.5" + ) + + val data = newFakeSparkApplicationData(configurationProperties) + val heuristicResult = configurationHeuristic.apply(data) + val heuristicResultDetails = heuristicResult.getHeuristicResultDetails + + it("returns the severity") { + heuristicResult.getSeverity should be(Severity.NONE) + } + + it("returns the driver memory") { + val details = heuristicResultDetails.get(0) + details.getName should include("spark.driver.memory") + details.getValue should be("2 GB") + } + + it("returns the executor memory") { + val details = heuristicResultDetails.get(1) + details.getName should include("spark.executor.memory") + details.getValue should be("1 GB") + } + + it("returns the executor instances") { + val details = heuristicResultDetails.get(2) + details.getName should include("spark.executor.instances") + details.getValue should be("900") + } + + it("returns the executor cores") { + val details = heuristicResultDetails.get(3) + details.getName should include("spark.executor.cores") + details.getValue should include("default") + } + + it("returns the serializer") { + val details = heuristicResultDetails.get(4) + details.getName should include("spark.serializer") + details.getValue should be("org.apache.spark.serializer.KryoSerializer") + } + } + + describe(".Evaluator") { + import ConfigurationHeuristic.Evaluator + + def newEvaluatorWithConfigurationProperties(configurationProperties: Map[String, String]): Evaluator = { + new Evaluator(configurationHeuristic, newFakeSparkApplicationData(configurationProperties)) + } + + it("has the driver memory bytes when they're present") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.driver.memory" -> "2G")) + evaluator.driverMemoryBytes should be(Some(2L * 1024 * 1024 * 1024)) + } + + it("has no driver memory bytes when they're absent") { + val evaluator = newEvaluatorWithConfigurationProperties(Map.empty) + evaluator.driverMemoryBytes should be(None) + } + + it("has the executor memory bytes when they're present") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.executor.memory" -> "1g")) + evaluator.executorMemoryBytes should be(Some(1L * 1024 * 1024 * 1024)) + } + + it("has no executor memory bytes when they're absent") { + val evaluator = newEvaluatorWithConfigurationProperties(Map.empty) + evaluator.executorMemoryBytes should be(None) + } + + it("has the executor instances when they're present") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.executor.instances" -> "900")) + evaluator.executorInstances should be(Some(900)) + } + + it("has no executor instances when they're absent") { + val evaluator = newEvaluatorWithConfigurationProperties(Map.empty) + evaluator.executorInstances should be(None) + } + + it("has the executor cores when they're present") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.executor.cores" -> "2")) + evaluator.executorCores should be(Some(2)) + } + + it("has no executor cores when they're absent") { + val evaluator = newEvaluatorWithConfigurationProperties(Map.empty) + evaluator.executorCores should be(None) + } + + it("has the serializer when it's present") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.KryoSerializer")) + evaluator.serializer should be(Some("org.apache.spark.serializer.KryoSerializer")) + } + + it("has no serializer when it's absent") { + val evaluator = newEvaluatorWithConfigurationProperties(Map.empty) + evaluator.serializer should be(None) + } + + it("has the severity of the serializer setting when it matches our recommendation") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.KryoSerializer")) + evaluator.serializerSeverity should be(Severity.NONE) + } + + it("has the severity of the serializer setting when it doesn't match our recommendation and is non-null") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.FooSerializer")) + evaluator.serializerSeverity should be(Severity.MODERATE) + } + + it("has the severity of the serializer setting when it is null") { + val evaluator = newEvaluatorWithConfigurationProperties(Map.empty) + evaluator.serializerSeverity should be(Severity.NONE) + } + + it("computes the overall severity when there are some issues") { + val evaluator = newEvaluatorWithConfigurationProperties(Map("spark.serializer" -> "org.apache.spark.serializer.FooSerializer")) + evaluator.severity should be(Severity.MODERATE) + } + + it("computes the overall severity when there are no issues") { + val evaluator = newEvaluatorWithConfigurationProperties(Map.empty) + evaluator.severity should be(Severity.NONE) + } + } + } +} + +object ConfigurationHeuristicTest { + import JavaConverters._ + + def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData = + new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava) + + def newFakeSparkApplicationData(appConfigurationProperties: Map[String, String]): SparkApplicationData = { + val logDerivedData = SparkLogDerivedData( + SparkListenerEnvironmentUpdate(Map("Spark Properties" -> appConfigurationProperties.toSeq)) + ) + + val appId = "application_1" + val restDerivedData = SparkRestDerivedData( + new ApplicationInfo(appId, name = "app", Seq.empty), + jobDatas = Seq.empty, + stageDatas = Seq.empty, + executorSummaries = Seq.empty + ) + + SparkApplicationData(appId, restDerivedData, Some(logDerivedData)) + } +} diff --git a/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala new file mode 100644 index 000000000..5189ea7f2 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala @@ -0,0 +1,261 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import scala.collection.JavaConverters + +import com.linkedin.drelephant.analysis.{ApplicationType, Severity, SeverityThresholds} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, ExecutorSummary} +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate +import org.scalatest.{FunSpec, Matchers} + + +class ExecutorsHeuristicTest extends FunSpec with Matchers { + import ExecutorsHeuristicTest._ + + describe("ExecutorsHeuristic") { + val heuristicConfigurationData = newFakeHeuristicConfigurationData( + Map( + "max_to_median_ratio_severity_thresholds" -> "1.414,2,4,16", + "ignore_max_bytes_less_than_threshold" -> "4000000", + "ignore_max_millis_less_than_threshold" -> "4000001" + ) + ) + val executorsHeuristic = new ExecutorsHeuristic(heuristicConfigurationData) + + val maxMemory = 5000000L + + val executorSummaries = Seq( + newFakeExecutorSummary( + id = "1", + memoryUsed = 1000000L, + totalDuration = 1000001L, + totalInputBytes = 1000002L, + totalShuffleRead = 1000003L, + totalShuffleWrite = 1000004L, + maxMemory + ), + newFakeExecutorSummary( + id = "2", + memoryUsed = 2000000L, + totalDuration = 2000001L, + totalInputBytes = 2000002L, + totalShuffleRead = 2000003L, + totalShuffleWrite = 2000004L, + maxMemory + ), + newFakeExecutorSummary( + id = "3", + memoryUsed = 3000000L, + totalDuration = 3000001L, + totalInputBytes = 3000002L, + totalShuffleRead = 3000003L, + totalShuffleWrite = 3000004L, + maxMemory + ), + newFakeExecutorSummary( + id = "4", + memoryUsed = 4000000L, + totalDuration = 4000001L, + totalInputBytes = 4000002L, + totalShuffleRead = 4000003L, + totalShuffleWrite = 4000004L, + maxMemory + ) + ) + + describe(".apply") { + val data = newFakeSparkApplicationData(executorSummaries) + val heuristicResult = executorsHeuristic.apply(data) + val heuristicResultDetails = heuristicResult.getHeuristicResultDetails + + it("returns the severity") { + heuristicResult.getSeverity should be(Severity.LOW) + } + + it("returns the total storage memory allocated") { + val details = heuristicResultDetails.get(0) + details.getName should include("storage memory allocated") + details.getValue should be("19.07 MB") + } + + it("returns the total storage memory used") { + val details = heuristicResultDetails.get(1) + details.getName should include("storage memory used") + details.getValue should be("9.54 MB") + } + + it("returns the storage memory utilization rate") { + val details = heuristicResultDetails.get(2) + details.getName should include("storage memory utilization rate") + details.getValue should be("0.500") + } + + it("returns the distribution of storage memory used among executors") { + val details = heuristicResultDetails.get(3) + details.getName should include("storage memory used") + details.getValue should include regex("976.56 KB.*976.56 KB.*2.38 MB.*2.86 MB.*3.81 MB") + } + + it("returns the distribution of task time among executors") { + val details = heuristicResultDetails.get(4) + details.getName should include("task time") + details.getValue should include regex("16 min 40 sec.*16 min 40 sec.*41 min 40 sec.*50 min.*1 hr 6 min 40 sec") + } + + it("returns the distribution of input bytes among executors") { + val details = heuristicResultDetails.get(5) + details.getName should include("input bytes") + details.getValue should include regex("976.56 KB.*976.56 KB.*2.38 MB.*2.86 MB.*3.81 MB") + } + + it("returns the distribution of shuffle read bytes among executors") { + val details = heuristicResultDetails.get(6) + details.getName should include("shuffle read bytes") + details.getValue should include regex("976.57 KB.*976.57 KB.*2.38 MB.*2.86 MB.*3.81 MB") + } + + it("returns the distribution of shuffle write bytes among executors") { + val details = heuristicResultDetails.get(7) + details.getName should include("shuffle write bytes") + details.getValue should include regex("976.57 KB.*976.57 KB.*2.38 MB.*2.86 MB.*3.81 MB") + } + } + + describe(".Evaluator") { + import ExecutorsHeuristic.Evaluator + import ExecutorsHeuristic.Distribution + + val data = newFakeSparkApplicationData(executorSummaries) + val evaluator = new Evaluator(executorsHeuristic, data) + + it("has the total storage memory allocated") { + evaluator.totalStorageMemoryAllocated should be(20000000L) + } + + it("has the total storage memory used") { + evaluator.totalStorageMemoryUsed should be(10000000L) + } + + it("has the storage memory utilization rate") { + evaluator.storageMemoryUtilizationRate should be(0.5D) + } + + it("has the distribution of storage memory used among executors") { + evaluator.storageMemoryUsedDistribution should be( + Distribution(1000000L, 1000000L, 2500000L, 3000000L, 4000000L) + ) + } + + it("has the distribution of task time among executors") { + evaluator.taskTimeDistribution should be( + Distribution(1000001L, 1000001L, 2500001L, 3000001L, 4000001L) + ) + } + + it("has the distribution of input bytes among executors") { + evaluator.inputBytesDistribution should be( + Distribution(1000002L, 1000002L, 2500002L, 3000002L, 4000002L) + ) + } + + it("has the distribution of shuffle read among executors") { + evaluator.shuffleReadBytesDistribution should be( + Distribution(1000003L, 1000003L, 2500003L, 3000003L, 4000003L) + ) + } + + it("has the distribution of shuffle write among executors") { + evaluator.shuffleWriteBytesDistribution should be( + Distribution(1000004L, 1000004L, 2500004L, 3000004L, 4000004L) + ) + } + + it("computes the overall severity") { + evaluator.severity should be(Severity.LOW) + } + + it("computes the severity of a given distribution, when the max is large enough") { + val distribution = Distribution(min = 0L, p25 = 1000L, median = 1000L, p75 = 1000L, max = 16000L) + evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16000L) should be(Severity.CRITICAL) + } + + it("computes the severity of a given distribution, when the max is not large enough") { + val distribution = Distribution(min = 0L, p25 = 1000L, median = 1000L, p75 = 1000L, max = 16000L) + evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16001L) should be(Severity.NONE) + } + + it("computes the severity of a given distribution, when the median is zero and the max is large enough") { + val distribution = Distribution(min = 0L, p25 = 0L, median = 0L, p75 = 0L, max = 16000L) + evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16000L) should be(Severity.CRITICAL) + } + + it("computes the severity of a given distribution, when the median is zero and the max is not large enough") { + val distribution = Distribution(min = 0L, p25 = 0L, median = 0L, p75 = 0L, max = 16000L) + evaluator.severityOfDistribution(distribution, ignoreMaxLessThanThreshold = 16001L) should be(Severity.NONE) + } + } + } +} + +object ExecutorsHeuristicTest { + import JavaConverters._ + + def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData = + new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava) + + def newFakeExecutorSummary( + id: String, + memoryUsed: Long, + totalDuration: Long, + totalInputBytes: Long, + totalShuffleRead: Long, + totalShuffleWrite: Long, + maxMemory: Long + ): ExecutorSummary = new ExecutorSummary( + id, + hostPort = "", + rddBlocks = 0, + memoryUsed, + diskUsed = 0, + activeTasks = 0, + failedTasks = 0, + completedTasks = 0, + totalTasks = 0, + totalDuration, + totalInputBytes, + totalShuffleRead, + totalShuffleWrite, + maxMemory, + executorLogs = Map.empty + ) + + def newFakeSparkApplicationData(executorSummaries: Seq[ExecutorSummary]): SparkApplicationData = { + val appId = "application_1" + + val restDerivedData = SparkRestDerivedData( + new ApplicationInfo(appId, name = "app", Seq.empty), + jobDatas = Seq.empty, + stageDatas = Seq.empty, + executorSummaries + ) + + SparkApplicationData(appId, restDerivedData, logDerivedData = None) + } +} diff --git a/test/com/linkedin/drelephant/spark/heuristics/JobsHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/JobsHeuristicTest.scala new file mode 100644 index 000000000..2a992576f --- /dev/null +++ b/test/com/linkedin/drelephant/spark/heuristics/JobsHeuristicTest.scala @@ -0,0 +1,165 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import scala.collection.JavaConverters + +import com.linkedin.drelephant.analysis.{ApplicationType, Severity} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, JobData} +import org.apache.spark.JobExecutionStatus +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate +import org.scalatest.{FunSpec, Matchers} + + +class JobsHeuristicTest extends FunSpec with Matchers { + import JobsHeuristicTest._ + + describe("JobsHeuristic") { + val heuristicConfigurationData = newFakeHeuristicConfigurationData( + Map( + "job_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8", + "job_task_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8" + ) + ) + val jobsHeuristic = new JobsHeuristic(heuristicConfigurationData) + val jobDatas = Seq( + newFakeJobData(0, "foo", JobExecutionStatus.SUCCEEDED, numCompleteTasks = 10, numFailedTasks = 0), + newFakeJobData(1, "bar", JobExecutionStatus.SUCCEEDED, numCompleteTasks = 8, numFailedTasks = 2), + newFakeJobData(2, "baz", JobExecutionStatus.SUCCEEDED, numCompleteTasks = 6, numFailedTasks = 4), + newFakeJobData(3, "aaa", JobExecutionStatus.FAILED, numCompleteTasks = 4, numFailedTasks = 6), + newFakeJobData(4, "zzz", JobExecutionStatus.FAILED, numCompleteTasks = 2, numFailedTasks = 8) + ) + + describe(".apply") { + val data = newFakeSparkApplicationData(jobDatas) + val heuristicResult = jobsHeuristic.apply(data) + val heuristicResultDetails = heuristicResult.getHeuristicResultDetails + + it("returns the severity") { + heuristicResult.getSeverity should be(Severity.CRITICAL) + } + + it("returns the number of completed jobs") { + heuristicResultDetails.get(0).getValue should be("3") + } + + it("returns the number of failed jobs") { + heuristicResultDetails.get(1).getValue should be("2") + } + + it("returns the list of failed jobs") { + heuristicResultDetails.get(2).getValue should be( + s"""|job 3, aaa + |job 4, zzz""".stripMargin + ) + } + + it("returns the job failure rate") { + heuristicResultDetails.get(3).getValue should be("0.400") + } + + it("returns the list of jobs with high task failure rates") { + heuristicResultDetails.get(4).getValue should be( + s"""|job 3, aaa (task failure rate: 0.600) + |job 4, zzz (task failure rate: 0.800)""".stripMargin + ) + } + } + + describe(".Evaluator") { + import JobsHeuristic.Evaluator + + val data = newFakeSparkApplicationData(jobDatas) + val evaluator = new Evaluator(jobsHeuristic, data) + + it("has the number of completed jobs") { + evaluator.numCompletedJobs should be(3) + } + + it("has the number of failed jobs") { + evaluator.numFailedJobs should be(2) + } + + it("has the list of failed jobs") { + val jobIds = evaluator.failedJobs.map { _.jobId } + jobIds should contain theSameElementsInOrderAs(Seq(3, 4)) + } + + it("has the job failure rate") { + evaluator.jobFailureRate should be(Some(0.4D)) + } + + it("has the list of jobs with high task failure rates") { + val jobIdsAndTaskFailureRates = + evaluator.jobsWithHighTaskFailureRates.map { case (jobData, taskFailureRate) => (jobData.jobId, taskFailureRate) } + jobIdsAndTaskFailureRates should contain theSameElementsInOrderAs(Seq((3, 0.6D), (4, 0.8D))) + } + + it("computes the overall severity") { + evaluator.severity should be(Severity.CRITICAL) + } + } + } +} + +object JobsHeuristicTest { + import JavaConverters._ + + def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData = + new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava) + + def newFakeJobData( + jobId: Int, + name: String, + status: JobExecutionStatus, + numCompleteTasks: Int, + numFailedTasks: Int + ): JobData = new JobData( + jobId, + name, + description = None, + submissionTime = None, + completionTime = None, + stageIds = Seq.empty, + jobGroup = None, + status, + numTasks = numCompleteTasks + numFailedTasks, + numActiveTasks = 0, + numCompleteTasks, + numSkippedTasks = 0, + numFailedTasks, + numActiveStages = 0, + numCompletedStages = 0, + numSkippedStages = 0, + numFailedStages = 0 + ) + + def newFakeSparkApplicationData(jobDatas: Seq[JobData]): SparkApplicationData = { + val appId = "application_1" + + val restDerivedData = SparkRestDerivedData( + new ApplicationInfo(appId, name = "app", Seq.empty), + jobDatas, + stageDatas = Seq.empty, + executorSummaries = Seq.empty + ) + + SparkApplicationData(appId, restDerivedData, logDerivedData = None) + } +} diff --git a/test/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristicTest.java b/test/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristicTest.java deleted file mode 100644 index c73ed33ab..000000000 --- a/test/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristicTest.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.linkedin.drelephant.spark.heuristics; - -import com.linkedin.drelephant.analysis.ApplicationType; -import com.linkedin.drelephant.analysis.HeuristicResult; -import com.linkedin.drelephant.analysis.HeuristicResultDetails; -import com.linkedin.drelephant.analysis.Severity; -import com.linkedin.drelephant.spark.MockSparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkApplicationData; -import com.linkedin.drelephant.spark.data.SparkEnvironmentData; -import com.linkedin.drelephant.spark.data.SparkExecutorData; -import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; -import com.linkedin.drelephant.util.MemoryFormatUtils; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import junit.framework.TestCase; - -import static com.linkedin.drelephant.spark.data.SparkExecutorData.EXECUTOR_DRIVER_NAME; -import static com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic.SPARK_DRIVER_MEMORY; -import static com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_INSTANCES; -import static com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic.SPARK_EXECUTOR_MEMORY; -import static com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic.DEFAULT_SPARK_STORAGE_MEMORY_FRACTION; - - -/** - * This class tests the heuristic rule: MemoryLimitHeuristic - */ -public class MemoryLimitHeuristicTest extends TestCase { - public void testTotalMemoryRule() { - // Test if the total memory limit is working, set all peak memory to arbirarity 100% - assertEquals(Severity.NONE, analyzeJob(1, "1G", "1G", "1G")); - assertEquals(Severity.NONE, analyzeJob(100, "1G", "1G", "100G")); - assertEquals(Severity.NONE, analyzeJob(10, "10G", "1G", "100G")); - assertEquals(Severity.LOW, analyzeJob(600, "1G", "1G", "600G")); - assertEquals(Severity.MODERATE, analyzeJob(2400, "512M", "1G", "1.2T")); - assertEquals(Severity.SEVERE, analyzeJob(1600, "1G", "1G", "1.6T")); - assertEquals(Severity.CRITICAL, analyzeJob(4200, "512M", "1G", "2.1T")); - } - - public void testMemoryUtilizationRule() { - // Test if the total memory utilization is working - - // When the total memory is too low, ignore the ratio calculation - assertEquals(Severity.NONE, analyzeJob(1, "1G", "1G", "0B")); - // When we barely pass the safe zone - assertEquals(Severity.CRITICAL, analyzeJob(1000, "1G", "1G", "0B")); - - // Normal situations - assertEquals(Severity.LOW, analyzeJob(1000, "1G", "1G", getPeakMemory(0.7d, 1000, "1G"))); - assertEquals(Severity.MODERATE, analyzeJob(1000, "1G", "1G", getPeakMemory(0.5d, 1000, "1G"))); - assertEquals(Severity.SEVERE, analyzeJob(1000, "1G", "1G", getPeakMemory(0.3d, 1000, "1G"))); - assertEquals(Severity.CRITICAL, analyzeJob(1000, "1G", "1G", getPeakMemory(0.1d, 1000, "1G"))); - } - - public void testCombinedRules() { - // Mix multiple rules together, majorly check the combined logic - assertEquals(Severity.CRITICAL, analyzeJob(1, "1G", "10T", "0B")); - assertEquals(Severity.CRITICAL, analyzeJob(2500, "1G", "1G", getPeakMemory(0.7d, 2500, "1G"))); - } - - public void testMissingSparkDriverMemoryProperty() { - HeuristicResult result = getJobresult(100, "1G", "700M", getPeakMemory(1.0d, 100, "1G"), SPARK_DRIVER_MEMORY); - for (HeuristicResultDetails detail : result.getHeuristicResultDetails()) { - if (detail.getName().startsWith("\"Total driver memory allocated")) { - assertEquals("\"Total driver memory allocated\",\"700 MB\"", detail.getName()); - } - } - } - - private static String getPeakMemory(double utilRatio, int executors, String memPerExecutor) { - long totalMem = - (long) (executors * MemoryFormatUtils.stringToBytes(memPerExecutor) * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); - return MemoryFormatUtils.bytesToString((long) (utilRatio * totalMem)); - } - - private Severity analyzeJob(int executors, String memPerExecutor, String driverMem, String peakTotalMem, - String... skippedProperties) { - return getJobresult(executors, memPerExecutor, driverMem, peakTotalMem, skippedProperties).getSeverity(); - } - - private HeuristicResult getJobresult(int executors, String memPerExecutor, String driverMem, String peakTotalMem, - String... skippedProperties) { - Set filters = new HashSet(); - for (int i = 0; i < skippedProperties.length; i++) { - filters.add(skippedProperties[i]); - } - - SparkApplicationData data = new MockSparkApplicationData(); - SparkEnvironmentData env = data.getEnvironmentData(); - if (!filters.contains(SPARK_EXECUTOR_INSTANCES)) { - env.addSparkProperty(SPARK_EXECUTOR_INSTANCES, String.valueOf(executors)); - } - if (!filters.contains(SPARK_EXECUTOR_MEMORY)) { - env.addSparkProperty(SPARK_EXECUTOR_MEMORY, memPerExecutor); - } - if (!filters.contains(SPARK_DRIVER_MEMORY)) { - env.addSparkProperty(SPARK_DRIVER_MEMORY, driverMem); - } - - SparkExecutorData exe = data.getExecutorData(); - SparkExecutorData.ExecutorInfo driverInfo = new SparkExecutorData.ExecutorInfo(); - driverInfo.maxMem = (long) (MemoryFormatUtils.stringToBytes(driverMem) * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); - exe.setExecutorInfo(EXECUTOR_DRIVER_NAME, driverInfo); - - long bytesPerExecutor = MemoryFormatUtils.stringToBytes(memPerExecutor); - - /* Assign evenly the peak memory to each executor, in practical cases, we might observe the executor peak memory - * can vary a bit due to data skewness and imperfect partitioning. - */ - long peakMemToGenerate = MemoryFormatUtils.stringToBytes(peakTotalMem) / executors; - for (int i = 0; i < executors; i++) { - SparkExecutorData.ExecutorInfo info = new SparkExecutorData.ExecutorInfo(); - info.maxMem = (long) (bytesPerExecutor * DEFAULT_SPARK_STORAGE_MEMORY_FRACTION); - info.memUsed = peakMemToGenerate; - exe.setExecutorInfo(String.valueOf(i), info); - } - Map paramsMap = new HashMap(); - return new MemoryLimitHeuristic(new HeuristicConfigurationData("test_heuristic", "test_class", "test_view", - new ApplicationType("test_apptype"), paramsMap)).apply(data); - } -} diff --git a/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala b/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala new file mode 100644 index 000000000..6794df530 --- /dev/null +++ b/test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala @@ -0,0 +1,191 @@ +/* + * Copyright 2016 LinkedIn Corp. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.linkedin.drelephant.spark.heuristics + +import scala.collection.JavaConverters +import scala.concurrent.duration.Duration + +import com.linkedin.drelephant.analysis.{ApplicationType, Severity} +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData +import com.linkedin.drelephant.spark.data.{SparkApplicationData, SparkLogDerivedData, SparkRestDerivedData} +import com.linkedin.drelephant.spark.fetchers.statusapiv1.{ApplicationInfo, JobData, StageData} +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate +import org.apache.spark.status.api.v1.StageStatus +import org.scalatest.{FunSpec, Matchers} + + +class StagesHeuristicTest extends FunSpec with Matchers { + import StagesHeuristicTest._ + + describe("StagesHeuristic") { + val heuristicConfigurationData = newFakeHeuristicConfigurationData( + Map( + "stage_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8", + "stage_task_failure_rate_severity_thresholds" -> "0.2,0.4,0.6,0.8", + "stage_runtime_minutes_severity_thresholds" -> "15,30,45,60" + ) + ) + val stagesHeuristic = new StagesHeuristic(heuristicConfigurationData) + val stageDatas = Seq( + newFakeStageData(StageStatus.COMPLETE, 0, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("2min").toMillis, "foo"), + newFakeStageData(StageStatus.COMPLETE, 1, numCompleteTasks = 8, numFailedTasks = 2, executorRunTime = Duration("2min").toMillis, "bar"), + newFakeStageData(StageStatus.COMPLETE, 2, numCompleteTasks = 6, numFailedTasks = 4, executorRunTime = Duration("2min").toMillis, "baz"), + newFakeStageData(StageStatus.FAILED, 3, numCompleteTasks = 4, numFailedTasks = 6, executorRunTime = Duration("2min").toMillis, "aaa"), + newFakeStageData(StageStatus.FAILED, 4, numCompleteTasks = 2, numFailedTasks = 8, executorRunTime = Duration("2min").toMillis, "zzz"), + newFakeStageData(StageStatus.COMPLETE, 5, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("0min").toMillis, "bbb"), + newFakeStageData(StageStatus.COMPLETE, 6, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("30min").toMillis, "ccc"), + newFakeStageData(StageStatus.COMPLETE, 7, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("60min").toMillis, "ddd"), + newFakeStageData(StageStatus.COMPLETE, 8, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("90min").toMillis, "eee"), + newFakeStageData(StageStatus.COMPLETE, 9, numCompleteTasks = 10, numFailedTasks = 0, executorRunTime = Duration("120min").toMillis, "fff") + ) + + val appConfigurationProperties = Map("spark.executor.instances" -> "2") + + describe(".apply") { + val data = newFakeSparkApplicationData(stageDatas, appConfigurationProperties) + val heuristicResult = stagesHeuristic.apply(data) + val heuristicResultDetails = heuristicResult.getHeuristicResultDetails + + it("returns the severity") { + heuristicResult.getSeverity should be(Severity.CRITICAL) + } + + it("returns the number of completed stages") { + heuristicResultDetails.get(0).getValue should be("8") + } + + it("returns the number of failed stages") { + heuristicResultDetails.get(1).getValue should be("2") + } + + it("returns the stage failure rate") { + heuristicResultDetails.get(2).getValue should be("0.200") + } + + it("returns the list of stages with high task failure rates") { + heuristicResultDetails.get(3).getValue should be( + s"""|stage 3, attempt 0 (task failure rate: 0.600) + |stage 4, attempt 0 (task failure rate: 0.800)""".stripMargin + ) + } + + it("returns the list of stages with long runtimes") { + heuristicResultDetails.get(4).getValue should be( + s"""|stage 8, attempt 0 (runtime: 45 min) + |stage 9, attempt 0 (runtime: 1 hr)""".stripMargin + ) + } + } + + describe(".Evaluator") { + import StagesHeuristic.Evaluator + + val data = newFakeSparkApplicationData(stageDatas, appConfigurationProperties) + val evaluator = new Evaluator(stagesHeuristic, data) + + it("has the number of completed stages") { + evaluator.numCompletedStages should be(8) + } + + it("has the number of failed stages") { + evaluator.numFailedStages should be(2) + } + + it("has the stage failure rate") { + evaluator.stageFailureRate should be(Some(0.2D)) + } + + it("has the list of stages with high task failure rates") { + val stageIdsAndTaskFailureRates = + evaluator.stagesWithHighTaskFailureRates.map { case (stageData, taskFailureRate) => (stageData.stageId, taskFailureRate) } + stageIdsAndTaskFailureRates should contain theSameElementsInOrderAs(Seq((3, 0.6D), (4, 0.8D))) + } + + it("has the list of stages with long average executor runtimes") { + val stageIdsAndRuntimes = + evaluator.stagesWithLongAverageExecutorRuntimes.map { case (stageData, runtime) => (stageData.stageId, runtime) } + stageIdsAndRuntimes should contain theSameElementsInOrderAs( + Seq((8, Duration("45min").toMillis), (9, Duration("60min").toMillis)) + ) + } + + it("computes the overall severity") { + evaluator.severity should be(Severity.CRITICAL) + } + } + } +} + +object StagesHeuristicTest { + import JavaConverters._ + + def newFakeHeuristicConfigurationData(params: Map[String, String] = Map.empty): HeuristicConfigurationData = + new HeuristicConfigurationData("heuristic", "class", "view", new ApplicationType("type"), params.asJava) + + def newFakeStageData( + status: StageStatus, + stageId: Int, + numCompleteTasks: Int, + numFailedTasks: Int, + executorRunTime: Long, + name: String + ): StageData = new StageData( + status, + stageId, + attemptId = 0, + numActiveTasks = numCompleteTasks + numFailedTasks, + numCompleteTasks, + numFailedTasks, + executorRunTime, + inputBytes = 0, + inputRecords = 0, + outputBytes = 0, + outputRecords = 0, + shuffleReadBytes = 0, + shuffleReadRecords = 0, + shuffleWriteBytes = 0, + shuffleWriteRecords = 0, + memoryBytesSpilled = 0, + diskBytesSpilled = 0, + name, + details = "", + schedulingPool = "", + accumulatorUpdates = Seq.empty, + tasks = None, + executorSummary = None + ) + + def newFakeSparkApplicationData( + stageDatas: Seq[StageData], + appConfigurationProperties: Map[String, String] + ): SparkApplicationData = { + val appId = "application_1" + + val restDerivedData = SparkRestDerivedData( + new ApplicationInfo(appId, name = "app", Seq.empty), + jobDatas = Seq.empty, + stageDatas, + executorSummaries = Seq.empty + ) + + val logDerivedData = SparkLogDerivedData( + SparkListenerEnvironmentUpdate(Map("Spark Properties" -> appConfigurationProperties.toSeq)) + ) + + SparkApplicationData(appId, restDerivedData, Some(logDerivedData)) + } +} diff --git a/test/com/linkedin/drelephant/util/InfoExtractorTest.java b/test/com/linkedin/drelephant/util/InfoExtractorTest.java index 87da397d8..6efe52333 100644 --- a/test/com/linkedin/drelephant/util/InfoExtractorTest.java +++ b/test/com/linkedin/drelephant/util/InfoExtractorTest.java @@ -26,9 +26,11 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; import mockit.Expectations; import mockit.Mocked; +import mockit.integration.junit4.JMockit; import org.apache.oozie.client.OozieClient; import org.apache.oozie.client.WorkflowJob; @@ -38,6 +40,7 @@ import static org.junit.Assert.assertEquals; +@RunWith(JMockit.class) public class InfoExtractorTest { @Mocked OozieClient oozieClient; diff --git a/test/org/apache/spark/deploy/history/DummySparkFSFetcher.scala b/test/org/apache/spark/deploy/history/DummySparkFSFetcher.scala deleted file mode 100644 index aae10dc3f..000000000 --- a/test/org/apache/spark/deploy/history/DummySparkFSFetcher.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.spark.deploy.history - -import java.net.URL; -import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData -import org.codehaus.jackson.JsonNode -import org.codehaus.jackson.node.{ObjectNode, JsonNodeFactory} - -class DummySparkFSFetcher(val fetcherConfData: FetcherConfigurationData) extends SparkFSFetcher(fetcherConfData) { - - override def readJsonNode(url: URL): JsonNode = { - - // Create json object of the form: - // { - // "beans" : [ { - // "name" : "Hadoop:service=NameNode,name=NameNodeStatus", - // "modelerType" : "org.apache.hadoop.hdfs.server.namenode.NameNode", - // "NNRole" : "NameNode", - // "HostAndPort" : "sample-sample01-ha2.grid.company.com:9000", - // "SecurityEnabled" : true, - // "State" : "active" - // } ] - // } - val nodeFactory: JsonNodeFactory = JsonNodeFactory.instance; - val node: ObjectNode = nodeFactory.objectNode(); - val child: ObjectNode = nodeFactory.objectNode(); - child.put("name", "Hadoop:service=NameNode, name=NameNodeStatus"); - child.put("modelerType", "org.apache.hadoop.hdfs.server.namenode.NameNode"); - child.put("NNRole", "NameNode"); - child.put("HostAndPort", "sample-sample01-ha2.grid.company.com:9000"); - child.put("SecurityEnabled", "true"); - - val activeNameNodeUrls = Array( - "http://sample-ha2.grid.company.com:50070/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus", - "http://sample-ha4.grid.company.com:50070/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus" - ); - // make ha2,ha4 active and other standby - if(url.toString().equals(activeNameNodeUrls(0)) || url.toString.equals(activeNameNodeUrls(1))) { - child.put("State","active") - } else { - child.put("State", "standby"); - } - node.putArray("beans").add(child); - return node; - } - -} diff --git a/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java b/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java deleted file mode 100644 index 0073b868f..000000000 --- a/test/org/apache/spark/deploy/history/SparkDataCollectionTest.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.spark.deploy.history; - -import com.linkedin.drelephant.spark.data.SparkJobProgressData; -import org.apache.spark.SparkConf; -import org.apache.spark.scheduler.ApplicationEventListener; -import org.apache.spark.scheduler.ReplayListenerBus; -import org.apache.spark.storage.StorageStatusListener; -import org.apache.spark.storage.StorageStatusTrackingListener; -import org.apache.spark.ui.env.EnvironmentListener; -import org.apache.spark.ui.exec.ExecutorsListener; -import org.apache.spark.ui.jobs.JobProgressListener; -import org.apache.spark.ui.storage.StorageListener; -import org.junit.BeforeClass; -import org.junit.Test; - -import java.io.BufferedInputStream; -import java.io.InputStream; - -import static org.junit.Assert.assertNotNull; - -public class SparkDataCollectionTest { - - private static final String event_log_dir = "spark_event_logs/"; - - @Test - public void testCollectJobProgressData() { - ReplayListenerBus replayBus = new ReplayListenerBus(); - JobProgressListener jobProgressListener = new JobProgressListener(new SparkConf()); - - replayBus.addListener(jobProgressListener); - - SparkDataCollection dataCollection = new SparkDataCollection(null, jobProgressListener, - null, null, null, null, null); - - InputStream in = new BufferedInputStream( - SparkDataCollectionTest.class.getClassLoader().getResourceAsStream(event_log_dir + "event_log_1")); - replayBus.replay(in, in.toString(), false); - - SparkJobProgressData jobProgressData = dataCollection.getJobProgressData(); - assertNotNull("can't get job progress data", jobProgressData); - } - -} diff --git a/test/org/apache/spark/deploy/history/SparkFsFetcherTest.java b/test/org/apache/spark/deploy/history/SparkFsFetcherTest.java deleted file mode 100644 index ec7a8103b..000000000 --- a/test/org/apache/spark/deploy/history/SparkFsFetcherTest.java +++ /dev/null @@ -1,219 +0,0 @@ -/* - * Copyright 2016 LinkedIn Corp. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package org.apache.spark.deploy.history; - -import com.linkedin.drelephant.analysis.ElephantFetcher; -import com.linkedin.drelephant.configurations.fetcher.FetcherConfiguration; -import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData; -import org.apache.hadoop.fs.Path; -import org.junit.BeforeClass; -import org.junit.Test; -import org.w3c.dom.Document; -import org.xml.sax.SAXException; -import org.apache.spark.deploy.history.DummySparkFSFetcher; - -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; - -import org.apache.hadoop.conf.Configuration; - -import static org.junit.Assert.assertEquals; - -public class SparkFsFetcherTest { - - private static Document document1 = null; - private static Document document2 = null; - private static Document document3 = null; - private static Document document4 = null; - - private static final String spark = "SPARK"; - private static final String defEventLogDir = "/system/spark-history"; - private static final String confEventLogDir = "/custom/configured"; - private static final double defEventLogSize = 100; - private static final double confEventLogSize = 50; - - @BeforeClass - public static void runBeforeClass() { - try { - DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance(); - DocumentBuilder builder = factory.newDocumentBuilder(); - document1 = builder.parse( - SparkFsFetcherTest.class.getClassLoader().getResourceAsStream( - "configurations/fetcher/FetcherConfTest5.xml")); - document2 = builder.parse( - SparkFsFetcherTest.class.getClassLoader().getResourceAsStream( - "configurations/fetcher/FetcherConfTest6.xml")); - document3 = builder.parse( - SparkFsFetcherTest.class.getClassLoader().getResourceAsStream( - "configurations/fetcher/FetcherConfTest7.xml")); - document4 = builder.parse( - SparkFsFetcherTest.class.getClassLoader().getResourceAsStream( - "configurations/fetcher/FetcherConfTest8.xml")); - } catch (ParserConfigurationException e) { - throw new RuntimeException("XML Parser could not be created.", e); - } catch (SAXException e) { - throw new RuntimeException("Test files are not properly formed", e); - } catch (IOException e) { - throw new RuntimeException("Unable to read test files ", e); - } - } - - /** - * Test for verifying the configured event log directory and log size - * - * - * 50 - * /custom/configured - * - */ - @Test - public void testSparkFetcherConfig() { - FetcherConfiguration fetcherConf = new FetcherConfiguration(document1.getDocumentElement()); - assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1); - assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark); - - Class fetcherClass = null; - FetcherConfigurationData data = fetcherConf.getFetchersConfigurationData().get(0); - try { - fetcherClass = SparkFsFetcherTest.class.getClassLoader().loadClass(data.getClassName()); - Object sparkFetcherInstance = fetcherClass.getConstructor(FetcherConfigurationData.class).newInstance(data); - if (!(sparkFetcherInstance instanceof ElephantFetcher)) { - throw new IllegalArgumentException( - "Class " + fetcherClass.getName() + " is not an implementation of " + ElephantFetcher.class.getName()); - } - - // Check if the configurations are picked up correctly - assertEquals(confEventLogSize, ((SparkFSFetcher) sparkFetcherInstance).getEventLogSize(), 0); - assertEquals(confEventLogDir, ((SparkFSFetcher) sparkFetcherInstance).getEventLogDir()); - - } catch (InstantiationException e) { - throw new RuntimeException("Could not instantiate class " + data.getClassName(), e); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Could not find class " + data.getClassName(), e); - } catch (IllegalAccessException e) { - throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e); - } catch (InvocationTargetException e) { - throw new RuntimeException("Could not invoke class " + data.getClassName(), e); - } catch (NoSuchMethodException e) { - throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e); - } - } - - /** - * Test for verifying unspecified log directory and log size configs - * - * - * - */ - @Test - public void testSparkFetcherUnspecifiedConfig() { - FetcherConfiguration fetcherConf = new FetcherConfiguration(document3.getDocumentElement()); - assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1); - assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark); - - Class fetcherClass = null; - FetcherConfigurationData data = fetcherConf.getFetchersConfigurationData().get(0); - try { - fetcherClass = SparkFsFetcherTest.class.getClassLoader().loadClass(data.getClassName()); - Object sparkFetcherInstance = fetcherClass.getConstructor(FetcherConfigurationData.class).newInstance(data); - if (!(sparkFetcherInstance instanceof ElephantFetcher)) { - throw new IllegalArgumentException( - "Class " + fetcherClass.getName() + " is not an implementation of " + ElephantFetcher.class.getName()); - } - - // Check if the default values are used - assertEquals(defEventLogSize, ((SparkFSFetcher) sparkFetcherInstance).getEventLogSize(), 0); - assertEquals(defEventLogDir, ((SparkFSFetcher) sparkFetcherInstance).getEventLogDir()); - - } catch (InstantiationException e) { - throw new RuntimeException("Could not instantiate class " + data.getClassName(), e); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Could not find class " + data.getClassName(), e); - } catch (IllegalAccessException e) { - throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e); - } catch (InvocationTargetException e) { - throw new RuntimeException("Could not invoke class " + data.getClassName(), e); - } catch (NoSuchMethodException e) { - throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e); - } - } - - /** - * Test for verifying empty log directory and log size configs - * - * - * - * /system/spark-history - * - */ - @Test - public void testSparkFetcherEmptyConfig() { - FetcherConfiguration fetcherConf = new FetcherConfiguration(document2.getDocumentElement()); - assertEquals(fetcherConf.getFetchersConfigurationData().size(), 1); - assertEquals(fetcherConf.getFetchersConfigurationData().get(0).getAppType().getName(), spark); - - Class fetcherClass = null; - FetcherConfigurationData data = fetcherConf.getFetchersConfigurationData().get(0); - try { - fetcherClass = SparkFsFetcherTest.class.getClassLoader().loadClass(data.getClassName()); - Object sparkFetcherInstance = fetcherClass.getConstructor(FetcherConfigurationData.class).newInstance(data); - if (!(sparkFetcherInstance instanceof ElephantFetcher)) { - throw new IllegalArgumentException( - "Class " + fetcherClass.getName() + " is not an implementation of " + ElephantFetcher.class.getName()); - } - - // Check if the default values are used - assertEquals(defEventLogSize, ((SparkFSFetcher) sparkFetcherInstance).getEventLogSize(), 0); - assertEquals(defEventLogDir, ((SparkFSFetcher) sparkFetcherInstance).getEventLogDir()); - - } catch (InstantiationException e) { - throw new RuntimeException("Could not instantiate class " + data.getClassName(), e); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Could not find class " + data.getClassName(), e); - } catch (IllegalAccessException e) { - throw new RuntimeException("Could not access constructor for class" + data.getClassName(), e); - } catch (InvocationTargetException e) { - throw new RuntimeException("Could not invoke class " + data.getClassName(), e); - } catch (NoSuchMethodException e) { - throw new RuntimeException("Could not find constructor for class " + data.getClassName(), e); - } - } - - // checks the namenode address from the hadoopconf - @Test - public void testGetNamenodeAddressFromHadoopConf() { - FetcherConfiguration fetcherConf = new FetcherConfiguration(document2.getDocumentElement()); - DummySparkFSFetcher fetcher = new DummySparkFSFetcher(fetcherConf.getFetchersConfigurationData().get(0)); - Configuration conf = new Configuration(); - String nameNode = fetcher.getNamenodeAddress(conf); - assertEquals(nameNode,"sample-ha2.grid.company.com:50070"); - } - - // checks the namenode address from fetcherConf - @Test - public void testGetNamenodeAddressFromFetcherConf() { - FetcherConfiguration fetcherConf = new FetcherConfiguration(document4.getDocumentElement()); - DummySparkFSFetcher fetcher = new DummySparkFSFetcher(fetcherConf.getFetchersConfigurationData().get(0)); - Configuration conf = new Configuration(); - String nameNode = fetcher.getNamenodeAddress(conf); - assertEquals(nameNode,"sample-ha4.grid.company.com:50070"); - } -} - diff --git a/test/resources/configurations/fetcher/FetcherConfTest5.xml b/test/resources/configurations/fetcher/FetcherConfTest5.xml index 4ad18425b..2372f0828 100644 --- a/test/resources/configurations/fetcher/FetcherConfTest5.xml +++ b/test/resources/configurations/fetcher/FetcherConfTest5.xml @@ -18,10 +18,8 @@ spark - org.apache.spark.deploy.history.SparkFSFetcher + com.linkedin.drelephant.spark.fetchers.SparkFetcher - 50 - /custom/configured diff --git a/test/resources/configurations/fetcher/FetcherConfTest6.xml b/test/resources/configurations/fetcher/FetcherConfTest6.xml deleted file mode 100644 index 0adb493f0..000000000 --- a/test/resources/configurations/fetcher/FetcherConfTest6.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - - spark - org.apache.spark.deploy.history.SparkFSFetcher - - - - - - diff --git a/test/resources/configurations/fetcher/FetcherConfTest7.xml b/test/resources/configurations/fetcher/FetcherConfTest7.xml deleted file mode 100644 index 1564cb313..000000000 --- a/test/resources/configurations/fetcher/FetcherConfTest7.xml +++ /dev/null @@ -1,25 +0,0 @@ - - - - - - spark - org.apache.spark.deploy.history.SparkFSFetcher - - - - diff --git a/test/resources/configurations/fetcher/FetcherConfTest8.xml b/test/resources/configurations/fetcher/FetcherConfTest8.xml deleted file mode 100644 index f49103dae..000000000 --- a/test/resources/configurations/fetcher/FetcherConfTest8.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - - - spark - org.apache.spark.deploy.history.SparkFSFetcher - - sample-ha3.grid.company.com:50070,sample-ha4.grid.company.com:50070 - - - \ No newline at end of file diff --git a/test/resources/spark-defaults.conf b/test/resources/spark-defaults.conf new file mode 100644 index 000000000..de9cc375a --- /dev/null +++ b/test/resources/spark-defaults.conf @@ -0,0 +1,4 @@ +spark.yarn.historyServer.address = jh1.grid.example.com:18080 +spark.eventLog.enabled = true +spark.eventLog.compress = true +spark.eventLog.dir = hdfs://nn1.grid.example.com:9000/logs/spark diff --git a/test/resources/spark_event_logs/event_log_1 b/test/resources/spark_event_logs/event_log_1 deleted file mode 100644 index 0ed97efaf..000000000 --- a/test/resources/spark_event_logs/event_log_1 +++ /dev/null @@ -1,32 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"1.5.1"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"10.20.0.71","Port":58838},"Maximum Memory":1111794647,"Timestamp":1458126388757} -{"Event":"SparkListenerApplicationStart","App Name":"PythonPi","App ID":"application_1457600942802_0093","Timestamp":1458126354336,"User":"hdfs"} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1458126390170,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at pi.py:39","Number of Tasks":10,"RDD Info":[{"RDD ID":1,"Name":"PythonRDD","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"collect\"}","callSite.short":"reduce at pi.py:39"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at pi.py:39","Number of Tasks":10,"RDD Info":[{"RDD ID":1,"Name":"PythonRDD","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"","Submission Time":1458126390256,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"1\",\"name\":\"collect\"}","callSite.short":"reduce at pi.py:39"}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1458126397624,"Executor ID":"2","Executor Info":{"Host":".hello.com","Total Cores":2,"Log Urls":{"stdout":"http://hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000003/hdfs/stdout?start=-4096","stderr":"http://hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000003/hdfs/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":".hello.com","Port":36478},"Maximum Memory":2223023063,"Timestamp":1458126398028} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1458126398712,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1458126398726,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1458126398962,"Executor ID":"1","Executor Info":{"Host":".hello.com","Total Cores":2,"Log Urls":{"stdout":"http://.hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000002/hdfs/stdout?start=-4096","stderr":"http://.hello.com:8042/node/containerlogs/container_e38_1457600942802_0093_01_000002/hdfs/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1458126398970,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1458126398973,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":".hello.com","Port":38464},"Maximum Memory":2223023063,"Timestamp":1458126399357} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1458126403532,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1458126398726,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126403558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":1091,"Executor Run Time":408,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1458126403563,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1458126398712,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126403565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":2605,"Executor Run Time":411,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1458126404784,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1458126398973,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126404787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":1375,"Executor Run Time":473,"Result Size":1018,"JVM GC Time":48,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1458126404791,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1458126398970,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126404793,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":1385,"Executor Run Time":473,"Result Size":1018,"JVM GC Time":48,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1458126405270,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1458126403532,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126405273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":16,"Executor Run Time":115,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1458126406523,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1458126404784,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126406526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":14,"Executor Run Time":115,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1458126403563,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126406809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":14,"Executor Run Time":125,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1458126404791,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126408068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":15,"Executor Run Time":118,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1458126405270,"Executor ID":"2","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126408323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"hello.com","Executor Deserialize Time":13,"Executor Run Time":116,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1458126406523,"Executor ID":"1","Host":".hello.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1458126409598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":".hello.com","Executor Deserialize Time":12,"Executor Run Time":116,"Result Size":1018,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at pi.py:39","Number of Tasks":10,"RDD Info":[{"RDD ID":1,"Name":"PythonRDD","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Use ExternalBlockStore":false,"Deserialized":false,"Replication":1},"Number of Partitions":10,"Number of Cached Partitions":0,"Memory Size":0,"ExternalBlockStore Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"","Submission Time":1458126390256,"Completion Time":1458126409599,"Accumulables":[]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1458126409602,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1458126409609} diff --git a/test/resources/spark_event_logs/event_log_2 b/test/resources/spark_event_logs/event_log_2 new file mode 100644 index 000000000..35101daa4 --- /dev/null +++ b/test/resources/spark_event_logs/event_log_2 @@ -0,0 +1,4 @@ +{"Event":"SparkListenerLogStart","Spark Version":"1.4.1"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"127.0.0.1","Port":54157},"Maximum Memory":515411804,"Timestamp":1475761114342} +{"Event":"SparkListenerEnvironmentUpdate","Spark Properties":{"spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.storage.memoryFraction":"0.3","spark.driver.memory":"2G","spark.executor.instances":"900","spark.executor.memory":"1g","spark.shuffle.memoryFraction":"0.5"},"JVM Information":{},"System Properties":{},"Classpath Entries":{}} +{"Event":"SparkListenerApplicationStart","App Name":"app","App ID":"application_1","Timestamp":1475761112578,"User":"foo","App Attempt ID":"1"}