Permalink
Browse files

Rewrite Spark fetcher/heuristics. (#162)

* Rewrite Spark fetcher/heuristics.

The purpose of this update is to:

- rewrite the Spark data fetcher to use Spark event logs minimally, since it can be expensive to download and process these fully as done before
- rewrite the Spark data fetcher to use the [Spark monitoring REST API](https://spark.apache.org/docs/1.4.1/monitoring.html#rest-api), which provides almost all of the information Spark heuristics need
- update the Spark heuristics to provide hopefully more useful information and avoid being arbitrarily restrictive

The new Spark-related code is provided in parallel to the old Spark-related code. To enable it:

- Uncomment and swap in the appropriate fragments in `AggregatorConf.xml`, `FetcherConf.xml`, and `HeuristicConf.xml`.
- Set `SPARK_CONF_DIR` (or `SPARK_HOME`) to an appropriate location so that Dr. Elephant can find `spark-defaults.conf`.

Heuristics added:

- "Executor shuffle read bytes distribution": We now provide a distribution with min/25p/median/75p/max, with severity based on a max-to-median ratio.
- "Executor shuffle write bytes distribution": We now provide a distribution with min/25p/median/75p/max, with severity based on a max-to-median ratio.

Heuristics changed:

- "Average input size" -> "Executor input bytes distribution": Instead of providing an average along with min/max, we now provide a distribution with min/25p/median/75p/max, with severity based on a max-to-median ratio.
- "Average peak storage memory" -> "Executor storage memory used distribution": Instead of providing an average along with min/max, we now provide a distribution with min/25p/median/75p/max, with severity based on a max-to-median ratio.
- "Average runtime" -> "Executor task time distribution": Instead of providing an average along with min/max, we now provide a distribution with min/25p/median/75p/max, with severity based on a max-to-median ratio.
- "Memory utilization rate" -> "Executor storage memory utilization rate": This seemed to imply total memory but it is just the utilization rate for storage memory, so has been relabeled to indicate that. Shuffle memory is important too (but we don't seem to have access to shuffle memory utilization metrics).
- "Total memory used at peak" -> "Total executor storage memory used": This also refers to storage memory. It has been relabeled to indicate that.
- "Spark problematic stages" -> ("Spark stages with high task failure rates", "Spark stages with long average executor runtimes"): This was a combination of stages with high task failure rates and those with long runtimes. Those have been separated.

Heuristics removed:

- spark.executor.cores: I think this is somewhat discretionary. At the very least, our internal recommendation stopped matching the one in Dr. Elephant.
- spark.shuffle.manager: This was changed to "sort" by default as of Spark 1.2, so there is no current use for checking this setting.
- "Average output size": Metrics related to output size appear to be deprecated or non-existent, so there is no current use for checking this setting.

Finally, overall waste metrics are calculated based on allocation [app runtime * # of executors * executor memory] vs. usage [total executor run time * executor memory]. They were previously calculated based only on storage memory and some 50% buffer, which I didn't understand.

Added unit tests and also tested against our internal cluster as much as practically I could. Will need help to fully validate.
  • Loading branch information...
rayortigas authored and akshayrai committed Dec 13, 2016
1 parent 134bab1 commit 28f4025bbade1be0fc93111ee439859c530a8747
Showing with 3,622 additions and 3,353 deletions.
  1. +2 −4 app-conf/AggregatorConf.xml
  2. +2 −11 app-conf/FetcherConf.xml
  3. +12 −50 app-conf/HeuristicConf.xml
  4. +0 −1 app/com/linkedin/drelephant/ElephantContext.java
  5. +8 −8 app/com/linkedin/drelephant/analysis/HadoopAggregatedData.java
  6. +4 −0 app/com/linkedin/drelephant/analysis/HeuristicResultDetails.java
  7. +53 −0 app/com/linkedin/drelephant/analysis/SeverityThresholds.scala
  8. +4 −1 app/com/linkedin/drelephant/math/Statistics.java
  9. +120 −0 app/com/linkedin/drelephant/spark/SparkMetricsAggregator.scala
  10. +70 −0 app/com/linkedin/drelephant/spark/data/SparkApplicationData.scala
  11. +0 −70 app/com/linkedin/drelephant/spark/data/SparkEnvironmentData.java
  12. +0 −70 app/com/linkedin/drelephant/spark/data/SparkExecutorData.java
  13. +0 −89 app/com/linkedin/drelephant/spark/data/SparkGeneralData.java
  14. +0 −265 app/com/linkedin/drelephant/spark/data/SparkJobProgressData.java
  15. +4 −10 ...s/JobRuntimeHeuristicTest.java → app/com/linkedin/drelephant/spark/data/SparkLogDerivedData.scala
  16. +8 −19 app/com/linkedin/drelephant/spark/data/{SparkApplicationData.java → SparkRestDerivedData.scala}
  17. +0 −46 app/com/linkedin/drelephant/spark/data/SparkStorageData.java
  18. +101 −0 app/com/linkedin/drelephant/spark/fetchers/SparkFetcher.scala
  19. +237 −0 app/com/linkedin/drelephant/spark/fetchers/SparkLogClient.scala
  20. +158 −0 app/com/linkedin/drelephant/spark/fetchers/SparkRestClient.scala
  21. +250 −0 app/com/linkedin/drelephant/spark/fetchers/statusapiv1/statusapiv1.scala
  22. +0 −146 app/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristic.java
  23. +132 −0 app/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristic.scala
  24. +0 −61 app/com/linkedin/drelephant/spark/heuristics/EventLogLimitHeuristic.java
  25. +0 −217 app/com/linkedin/drelephant/spark/heuristics/ExecutorLoadHeuristic.java
  26. +228 −0 app/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristic.scala
  27. +0 −135 app/com/linkedin/drelephant/spark/heuristics/JobRuntimeHeuristic.java
  28. +148 −0 app/com/linkedin/drelephant/spark/heuristics/JobsHeuristic.scala
  29. +0 −207 app/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristic.java
  30. +0 −160 app/com/linkedin/drelephant/spark/heuristics/StageRuntimeHeuristic.java
  31. +210 −0 app/com/linkedin/drelephant/spark/heuristics/StagesHeuristic.scala
  32. +0 −30 app/com/linkedin/drelephant/util/InfoExtractor.java
  33. +62 −0 app/com/linkedin/drelephant/util/SparkUtils.scala
  34. +0 −89 app/org/apache/spark/SparkMetricsAggregator.java
  35. +0 −317 app/org/apache/spark/deploy/history/SparkDataCollection.scala
  36. +0 −354 app/org/apache/spark/deploy/history/SparkFSFetcher.scala
  37. +0 −110 app/org/apache/spark/storage/StorageStatusTrackingListener.scala
  38. +18 −0 app/views/help/spark/helpConfigurationHeuristic.scala.html
  39. +36 −0 app/views/help/spark/helpExecutorsHeuristic.scala.html
  40. +20 −0 app/views/help/spark/helpJobsHeuristic.scala.html
  41. +20 −0 app/views/help/spark/helpStagesHeuristic.scala.html
  42. +3 −1 build.sbt
  43. +19 −1 project/Dependencies.scala
  44. +1 −1 project/build.properties
  45. +73 −0 test/com/linkedin/drelephant/analysis/SeverityThresholdsTest.scala
  46. +0 −4 test/com/linkedin/drelephant/configurations/fetcher/FetcherConfigurationTest.java
  47. +8 −0 test/com/linkedin/drelephant/math/StatisticsTest.java
  48. +0 −98 test/com/linkedin/drelephant/spark/MockSparkApplicationData.java
  49. +169 −0 test/com/linkedin/drelephant/spark/SparkMetricsAggregatorTest.scala
  50. +0 −94 test/com/linkedin/drelephant/spark/TestSparkAggregatedMetrics.java
  51. +82 −0 test/com/linkedin/drelephant/spark/data/SparkApplicationDataTest.scala
  52. +215 −0 test/com/linkedin/drelephant/spark/fetchers/SparkFetcherTest.scala
  53. +130 −0 test/com/linkedin/drelephant/spark/fetchers/SparkLogClientTest.scala
  54. +188 −0 test/com/linkedin/drelephant/spark/fetchers/SparkRestClientTest.scala
  55. +0 −93 test/com/linkedin/drelephant/spark/heuristics/BestPropertiesConventionHeuristicTest.java
  56. +198 −0 test/com/linkedin/drelephant/spark/heuristics/ConfigurationHeuristicTest.scala
  57. +261 −0 test/com/linkedin/drelephant/spark/heuristics/ExecutorsHeuristicTest.scala
  58. +165 −0 test/com/linkedin/drelephant/spark/heuristics/JobsHeuristicTest.scala
  59. +0 −138 test/com/linkedin/drelephant/spark/heuristics/MemoryLimitHeuristicTest.java
  60. +191 −0 test/com/linkedin/drelephant/spark/heuristics/StagesHeuristicTest.scala
  61. +3 −0 test/com/linkedin/drelephant/util/InfoExtractorTest.java
  62. +0 −62 test/org/apache/spark/deploy/history/DummySparkFSFetcher.scala
  63. +0 −59 test/org/apache/spark/deploy/history/SparkDataCollectionTest.java
  64. +0 −219 test/org/apache/spark/deploy/history/SparkFsFetcherTest.java
  65. +1 −3 test/resources/configurations/fetcher/FetcherConfTest5.xml
  66. +0 −27 test/resources/configurations/fetcher/FetcherConfTest6.xml
  67. +0 −25 test/resources/configurations/fetcher/FetcherConfTest7.xml
  68. +0 −26 test/resources/configurations/fetcher/FetcherConfTest8.xml
  69. +4 −0 test/resources/spark-defaults.conf
  70. +0 −32 test/resources/spark_event_logs/event_log_1
  71. +4 −0 test/resources/spark_event_logs/event_log_2
@@ -35,11 +35,9 @@
</aggregator>
<aggregator>
<applicationtype>spark</applicationtype>
<classname>org.apache.spark.SparkMetricsAggregator</classname>
<!--
<classname>com.linkedin.drelephant.spark.SparkMetricsAggregator</classname>
<params>
<storage_mem_wastage_buffer>0.5</storage_mem_wastage_buffer>
<allocated_memory_waste_buffer_percentage>0.5</allocated_memory_waste_buffer_percentage>
</params>
-->
</aggregator>
</aggregators>
@@ -56,18 +56,9 @@
</params>
</fetcher>
-->

<fetcher>
<applicationtype>spark</applicationtype>
<classname>org.apache.spark.deploy.history.SparkFSFetcher</classname>
<!--
<params>
<event_log_size_limit_in_mb>100</event_log_size_limit_in_mb>
<event_log_dir>/system/spark-history</event_log_dir>
<spark_log_ext>_1.snappy</spark_log_ext>
#the values specified in namenode_addresses will be used for obtaining spark logs. The cluster configuration will be ignored.
<namenode_addresses>address1,address2</namenode_addresses>
</params>
-->
<classname>com.linkedin.drelephant.spark.fetchers.SparkFetcher</classname>
</fetcher>
</fetchers>
@@ -161,65 +161,27 @@

<heuristic>
<applicationtype>spark</applicationtype>
<heuristicname>Spark Configuration Best Practice</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.BestPropertiesConventionHeuristic</classname>
<viewname>views.html.help.spark.helpBestProperties</viewname>
<!--<params>
<driver_memory_severity_in_gb>4, 4, 8, 8</driver_memory_severity_in_gb>
<num_core_severity>2</num_core_severity>
</params>-->
<heuristicname>Spark Configuration</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.ConfigurationHeuristic</classname>
<viewname>views.html.help.spark.helpConfigurationHeuristic</viewname>
</heuristic>

<heuristic>
<applicationtype>spark</applicationtype>
<heuristicname>Spark Memory Limit</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.MemoryLimitHeuristic</classname>
<viewname>views.html.help.spark.helpMemoryLimit</viewname>
<!--<params>
<total_mem_severity_in_tb>0.5, 1, 1.5, 2</total_mem_severity_in_tb>
<mem_util_severity>0.8, 0.6, 0.4, 0.2</mem_util_severity>
</params>-->
<heuristicname>Spark Executor Metrics</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.ExecutorsHeuristic</classname>
<viewname>views.html.help.spark.helpExecutorsHeuristic</viewname>
</heuristic>

<heuristic>
<applicationtype>spark</applicationtype>
<heuristicname>Spark Stage Runtime</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.StageRuntimeHeuristic</classname>
<viewname>views.html.help.spark.helpStageRuntime</viewname>
<!--<params>
<stage_runtime_severity_in_min>15, 30, 60, 60</stage_runtime_severity_in_min>
<stage_failure_rate_severity>0.3, 0.3, 0.5, 0.5</stage_failure_rate_severity>
<single_stage_tasks_failure_rate_severity>0.0, 0.3, 0.5, 0.5</single_stage_tasks_failure_rate_severity>
</params>-->
<heuristicname>Spark Job Metrics</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.JobsHeuristic</classname>
<viewname>views.html.help.spark.helpJobsHeuristic</viewname>
</heuristic>

<heuristic>
<applicationtype>spark</applicationtype>
<heuristicname>Spark Job Runtime</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.JobRuntimeHeuristic</classname>
<viewname>views.html.help.spark.helpJobRuntime</viewname>
<!--<params>
<avg_job_failure_rate_severity>0.1, 0.3, 0.5, 0.5</avg_job_failure_rate_severity>
<single_job_failure_rate_severity>0.0, 0.3, 0.5, 0.5</single_job_failure_rate_severity>
</params>-->
</heuristic>

<heuristic>
<applicationtype>spark</applicationtype>
<heuristicname>Spark Executor Load Balance</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.ExecutorLoadHeuristic</classname>
<viewname>views.html.help.spark.helpExecutorLoad</viewname>
<!--<params>
<looser_metric_deviation_severity>0.8, 1, 1.2, 1.4</looser_metric_deviation_severity>
<metric_deviation_severity>0.4, 0.6, 0.8, 1.0</metric_deviation_severity>
</params>-->
</heuristic>

<heuristic>
<applicationtype>spark</applicationtype>
<heuristicname>Spark Event Log Limit</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.EventLogLimitHeuristic</classname>
<viewname>views.html.help.spark.helpEventLogLimit</viewname>
<heuristicname>Spark Stage Metrics</heuristicname>
<classname>com.linkedin.drelephant.spark.heuristics.StagesHeuristic</classname>
<viewname>views.html.help.spark.helpStagesHeuristic</viewname>
</heuristic>

</heuristics>
@@ -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;
@@ -26,32 +26,32 @@
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;
@@ -29,6 +29,10 @@
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;
@@ -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)
}
}
@@ -98,7 +98,10 @@ public static long percentile(List<Long> 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) {
@@ -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"
}
Oops, something went wrong.

0 comments on commit 28f4025

Please sign in to comment.