(spark) branch branch-3.5 updated: [SPARK-47398][SQL] Extract a trait for InMemoryTableScanExec to allow for extending functionality
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new 1fe396213b57 [SPARK-47398][SQL] Extract a trait for InMemoryTableScanExec to allow for extending functionality 1fe396213b57 is described below commit 1fe396213b57e4697145f5fa1b9f0d24a35399df Author: Raza Jafri AuthorDate: Thu Mar 21 14:45:56 2024 -0500 [SPARK-47398][SQL] Extract a trait for InMemoryTableScanExec to allow for extending functionality ### What changes were proposed in this pull request? We are proposing to allow the users to have a custom `InMemoryTableScanExec`. To accomplish this we can follow the same convention we followed for `ShuffleExchangeLike` and `BroadcastExchangeLike` ### Why are the changes needed? In the PR added by ulysses-you, we are wrapping `InMemoryTableScanExec` inside `TableCacheQueryStageExec`. This could potentially cause problems, especially in the RAPIDS Accelerator for Apache Spark, where we replace `InMemoryTableScanExec` with a customized version that has optimizations needed by us. This situation could lead to the loss of benefits from [SPARK-42101](https://issues.apache.org/jira/browse/SPARK-42101) or even result in Spark throwing an Exception. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Ran the existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #45525 from razajafri/extract-inmem-trait. Authored-by: Raza Jafri Signed-off-by: Thomas Graves (cherry picked from commit 6a27789ad7d59cd133653a49be0bb49729542abe) Signed-off-by: Thomas Graves --- .../execution/adaptive/AdaptiveSparkPlanExec.scala | 16 +-- .../sql/execution/adaptive/QueryStageExec.scala| 8 +++--- .../execution/columnar/InMemoryTableScanExec.scala | 33 -- .../adaptive/AdaptiveQueryExecSuite.scala | 6 ++-- 4 files changed, 45 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 96b83a91cc73..d2e879e3eddb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._ import org.apache.spark.sql.execution.bucketing.{CoalesceBucketsInJoin, DisableUnnecessaryBucketedScan} -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.columnar.InMemoryTableScanLike import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLAdaptiveSQLMetricUpdates, SQLPlanMetric} import org.apache.spark.sql.internal.SQLConf @@ -254,7 +254,7 @@ case class AdaptiveSparkPlanExec( //and display SQL metrics correctly. // 2. If the `QueryExecution` does not match the current execution ID, it means the execution //ID belongs to another (parent) query, and we should not call update UI in this query. -//e.g., a nested `AdaptiveSparkPlanExec` in `InMemoryTableScanExec`. +//e.g., a nested `AdaptiveSparkPlanExec` in `InMemoryTableScanLike`. // // That means only the root `AdaptiveSparkPlanExec` of the main query that triggers this // query execution need to do a plan update for the UI. @@ -557,9 +557,9 @@ case class AdaptiveSparkPlanExec( } } -case i: InMemoryTableScanExec => - // There is no reuse for `InMemoryTableScanExec`, which is different from `Exchange`. If we - // hit it the first time, we should always create a new query stage. +case i: InMemoryTableScanLike => + // There is no reuse for `InMemoryTableScanLike`, which is different from `Exchange`. + // If we hit it the first time, we should always create a new query stage. val newStage = newQueryStage(i) CreateStageResult( newPlan = newStage, @@ -604,12 +604,12 @@ case class AdaptiveSparkPlanExec( } BroadcastQueryStageExec(currentStageId, newPlan, e.canonicalized) } - case i: InMemoryTableScanExec => + case i: InMemoryTableScanLike => // Apply `queryStageOptimizerRules` so that we can reuse subquery. -// No need to apply `postStageCreationRules` for `InMemoryTableScanExec` +// No need to apply `postStageCreati
(spark) branch master updated: [SPARK-47398][SQL] Extract a trait for InMemoryTableScanExec to allow for extending functionality
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6a27789ad7d5 [SPARK-47398][SQL] Extract a trait for InMemoryTableScanExec to allow for extending functionality 6a27789ad7d5 is described below commit 6a27789ad7d59cd133653a49be0bb49729542abe Author: Raza Jafri AuthorDate: Thu Mar 21 14:45:56 2024 -0500 [SPARK-47398][SQL] Extract a trait for InMemoryTableScanExec to allow for extending functionality ### What changes were proposed in this pull request? We are proposing to allow the users to have a custom `InMemoryTableScanExec`. To accomplish this we can follow the same convention we followed for `ShuffleExchangeLike` and `BroadcastExchangeLike` ### Why are the changes needed? In the PR added by ulysses-you, we are wrapping `InMemoryTableScanExec` inside `TableCacheQueryStageExec`. This could potentially cause problems, especially in the RAPIDS Accelerator for Apache Spark, where we replace `InMemoryTableScanExec` with a customized version that has optimizations needed by us. This situation could lead to the loss of benefits from [SPARK-42101](https://issues.apache.org/jira/browse/SPARK-42101) or even result in Spark throwing an Exception. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Ran the existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #45525 from razajafri/extract-inmem-trait. Authored-by: Raza Jafri Signed-off-by: Thomas Graves --- .../execution/adaptive/AdaptiveSparkPlanExec.scala | 16 +-- .../sql/execution/adaptive/QueryStageExec.scala| 8 +++--- .../execution/columnar/InMemoryTableScanExec.scala | 33 -- .../adaptive/AdaptiveQueryExecSuite.scala | 6 ++-- 4 files changed, 45 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 2879aaca7215..a5e681535cb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._ import org.apache.spark.sql.execution.bucketing.{CoalesceBucketsInJoin, DisableUnnecessaryBucketedScan} -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.columnar.InMemoryTableScanLike import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLAdaptiveSQLMetricUpdates, SQLPlanMetric} import org.apache.spark.sql.internal.SQLConf @@ -255,7 +255,7 @@ case class AdaptiveSparkPlanExec( //and display SQL metrics correctly. // 2. If the `QueryExecution` does not match the current execution ID, it means the execution //ID belongs to another (parent) query, and we should not call update UI in this query. -//e.g., a nested `AdaptiveSparkPlanExec` in `InMemoryTableScanExec`. +//e.g., a nested `AdaptiveSparkPlanExec` in `InMemoryTableScanLike`. // // That means only the root `AdaptiveSparkPlanExec` of the main query that triggers this // query execution need to do a plan update for the UI. @@ -558,9 +558,9 @@ case class AdaptiveSparkPlanExec( } } -case i: InMemoryTableScanExec => - // There is no reuse for `InMemoryTableScanExec`, which is different from `Exchange`. If we - // hit it the first time, we should always create a new query stage. +case i: InMemoryTableScanLike => + // There is no reuse for `InMemoryTableScanLike`, which is different from `Exchange`. + // If we hit it the first time, we should always create a new query stage. val newStage = newQueryStage(i) CreateStageResult( newPlan = newStage, @@ -605,12 +605,12 @@ case class AdaptiveSparkPlanExec( } BroadcastQueryStageExec(currentStageId, newPlan, e.canonicalized) } - case i: InMemoryTableScanExec => + case i: InMemoryTableScanLike => // Apply `queryStageOptimizerRules` so that we can reuse subquery. -// No need to apply `postStageCreationRules` for `InMemoryTableScanExec` +// No need to apply `postStageCreationRules` for `InMemoryTableScanLike` // as it's a leaf node. val newPlan = optimizeQueryStage(i, isF
(spark) branch master updated: [SPARK-47458][CORE] Fix the problem with calculating the maximum concurrent tasks for the barrier stage
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 90560dce85b0 [SPARK-47458][CORE] Fix the problem with calculating the maximum concurrent tasks for the barrier stage 90560dce85b0 is described below commit 90560dce85b06041e90699f168f67ed2af5a5ca2 Author: Bobby Wang AuthorDate: Tue Mar 19 10:07:43 2024 -0500 [SPARK-47458][CORE] Fix the problem with calculating the maximum concurrent tasks for the barrier stage ### What changes were proposed in this pull request? This PR addresses the problem of calculating the maximum concurrent tasks while evaluating the number of slots for barrier stages, specifically for the case when the task resource amount is greater than 1. ### Why are the changes needed? ``` scala test("problem of calculating the maximum concurrent task") { withTempDir { dir => val discoveryScript = createTempScriptWithExpectedOutput( dir, "gpuDiscoveryScript", """{"name": "gpu","addresses":["0", "1", "2", "3"]}""") val conf = new SparkConf() // Setup a local cluster which would only has one executor with 2 CPUs and 1 GPU. .setMaster("local-cluster[1, 6, 1024]") .setAppName("test-cluster") .set(WORKER_GPU_ID.amountConf, "4") .set(WORKER_GPU_ID.discoveryScriptConf, discoveryScript) .set(EXECUTOR_GPU_ID.amountConf, "4") .set(TASK_GPU_ID.amountConf, "2") // disable barrier stage retry to fail the application as soon as possible .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES, 1) sc = new SparkContext(conf) TestUtils.waitUntilExecutorsUp(sc, 1, 6) // Setup a barrier stage which contains 2 tasks and each task requires 1 CPU and 1 GPU. // Therefore, the total resources requirement (2 CPUs and 2 GPUs) of this barrier stage // can not be satisfied since the cluster only has 2 CPUs and 1 GPU in total. assert(sc.parallelize(Range(1, 10), 2) .barrier() .mapPartitions { iter => iter } .collect() sameElements Range(1, 10).toArray[Int]) } } ``` In the described test scenario, the executor has 6 CPU cores and 4 GPUs, and each task requires 1 CPU core and 2 GPUs. Consequently, the maximum number of concurrent tasks should be 2. However, the issue arises when attempting to launch the subsequent 2 barrier tasks, as the 'checkBarrierStageWithNumSlots' function gets the incorrect concurrent task limit that is 1 instead of 2. The bug needs to be fixed. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The existing and newly added unit tests should pass ### Was this patch authored or co-authored using generative AI tooling? No Closes #45528 from wbo4958/2-gpu. Authored-by: Bobby Wang Signed-off-by: Thomas Graves --- .../spark/scheduler/ExecutorResourceInfo.scala | 15 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 6 ++- .../cluster/CoarseGrainedSchedulerBackend.scala| 3 +- .../CoarseGrainedSchedulerBackendSuite.scala | 44 - .../spark/scheduler/TaskSchedulerImplSuite.scala | 56 +- 5 files changed, 105 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index d9fbd23f3aa4..c0f4475d20cc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -33,19 +33,6 @@ private[spark] class ExecutorResourceInfo( override protected def resourceName = this.name override protected def resourceAddresses = this.addresses - - /** - * Calculate how many parts the executor can offer according to the task resource amount - * @param taskAmount how many resource amount the task required - * @return the total parts - */ - def totalParts(taskAmount: Double): Int = { -assert(taskAmount > 0.0) -if (taskAmount >= 1.0) { - addresses.length / taskAmount.ceil.toInt -} else { - addresses.length * Math.floor(1.0 / taskAmount).toInt -} - } + def totalAddressesAmount: Int = this.addresses.length } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/T
(spark) branch master updated: [SPARK-47208][CORE] Allow overriding base overhead memory
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4b073fd607f1 [SPARK-47208][CORE] Allow overriding base overhead memory 4b073fd607f1 is described below commit 4b073fd607f14153ca511da08dbc28c7340b287a Author: jpcorreia99 AuthorDate: Thu Mar 14 08:23:23 2024 -0500 [SPARK-47208][CORE] Allow overriding base overhead memory ### What changes were proposed in this pull request? We can already select the desired overhead memory directly via the `spark.driver/executor.memoryOverhead` flags, however, if that flag is not present the overhead memory calculation goes as follows: ``` overhead_memory = Max(384, 'spark.driver/executor.memory' * 'spark.driver/executor.memoryOverheadFactor') [where the 'memoryOverheadFactor' flag defaults to 0.1] ``` This PR adds two new spark configs: `spark.driver.minMemoryOverhead` and `spark.executor.minMemoryOverhead`, which can be used to override the 384Mib minimum value. The memory overhead calculation will now be : ``` min_memory = sparkConf.get('spark.driver/executor.minMemoryOverhead').getOrElse(384) overhead_memory = Max(min_memory, 'spark.driver/executor.memory' * 'spark.driver/executor.memoryOverheadFactor') ``` ### Why are the changes needed? There are certain times where being able to override the 384Mb minimum directly can be beneficial. We may have a scenario where a lot of off-heap operations are performed (ex: using package managers/native compression/decompression) where we don't have a need for a large JVM heap but we may still need a signficant amount of memory in the spark node. Using the `memoryOverheadFactor` config flag may not prove appropriate, since we may not want the overhead allocation to directly scale with JVM memory, as a cost saving/resource limitation problem. ### Does this PR introduce _any_ user-facing change? Yes, as described above, two new flags have been added to the spark config. No break of existing behaviours. ### How was this patch tested? Added tests for 3 cases: - If `spark.driver/executor.memoryOverhead` is set, then the new changes have no effect. - If `spark.driver/executor.minMemoryOverhead` is set and its value is higher than 'spark.driver/executor.memory' * 'spark.driver/executor.memoryOverheadFactor', the total memory will be the allocated JVM memory + `spark.driver/executor.minMemoryOverhead` - If `spark.driver/executor.minMemoryOverhead` but its value is lower than 'spark.driver/executor.memory' * 'spark.driver/executor.memoryOverheadFactor', the total memory will be the allocated JVM memory + 'spark.driver/executor.memory' * 'spark.driver/executor.memoryOverheadFactor'. ### Was this patch authored or co-authored using generative AI tooling? No Closes #45240 from jpcorreia99/jcorrreia/MinOverheadMemoryOverride. Authored-by: jpcorreia99 Signed-off-by: Thomas Graves --- .../org/apache/spark/internal/config/package.scala | 17 + .../apache/spark/resource/ResourceProfile.scala| 8 +- .../spark/resource/ResourceProfileSuite.scala | 2 +- docs/configuration.md | 22 +- .../k8s/features/BasicDriverFeatureStep.scala | 7 +- .../k8s/features/BasicExecutorFeatureStep.scala| 2 + .../k8s/features/BasicDriverFeatureStepSuite.scala | 85 -- .../features/BasicExecutorFeatureStepSuite.scala | 43 ++- .../org/apache/spark/deploy/yarn/Client.scala | 16 +++- .../apache/spark/deploy/yarn/YarnAllocator.scala | 4 +- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 33 + .../spark/deploy/yarn/YarnAllocatorSuite.scala | 36 - 12 files changed, 252 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 1fcf75b02503..aa240b5cc5b5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -117,6 +117,14 @@ package object config { .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val DRIVER_MIN_MEMORY_OVERHEAD = ConfigBuilder("spark.driver.minMemoryOverhead") +.doc("The minimum amount of non-heap memory to be allocated per driver in cluster mode, " + + "in MiB unless otherwise specified. This value is ignored if " + + "spark.driver.memoryOverhead is set directly.") +.version("4.0.0") +
(spark) branch master updated: [SPARK-45527][CORE] Use fraction to do the resource calculation
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ae2e00e8f8e9 [SPARK-45527][CORE] Use fraction to do the resource calculation ae2e00e8f8e9 is described below commit ae2e00e8f8e96fe85ecd5539aac0803216b8e66f Author: Bobby Wang AuthorDate: Thu Jan 4 09:19:22 2024 -0600 [SPARK-45527][CORE] Use fraction to do the resource calculation ### What changes were proposed in this pull request? This (PR) introduces the utilization of fractions instead of slots, which is similar to the CPU strategy, for determining whether a worker offer can provide the necessary resources to tasks. For instance, when an executor reports to the driver with [gpu, ["1,", "2"]], the driver constructs an executor data map. The keys in the map represent the GPU addresses, and their default values are set to 1.0, indicating one whole GPU. Consequently, the available resource amounts for the executor are as follows: { "1" -> 1.0f, "2" -> 1.0f }. When offering resources to a task that requires 1 CPU and 0.08 GPU, the worker offer examines the available resource amounts. It identifies that the capacity of GPU address "1.0" is greater than the task's GPU requirement (1.0 >= 0.08). Therefore, Spark assigns the GPU address "1" to this task. After the assignment, the available resource amounts for this executor are updated to { "1" -> 0.92, "2" -> 1.0}, ensuring that the remaining resources can be allocated to other tasks. In scenarios where other tasks, using different task resource profiles, request varying GPU amounts when dynamic allocation is disabled, Spark applies the same comparison approach. It compares the task's GPU requirement with the available resource amounts to determine if the resources can be assigned to the task. ### Why are the changes needed? The existing resources offering including gpu, fpga is based on "slots per address", which is defined by the default resource profile. and it's a fixed number for all different resource profiles when dynamic allcation is disabled. Consider the below test case, ``` scala withTempDir { dir => val scriptPath = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", """{"name": "gpu","addresses":["0"]}""") val conf = new SparkConf() .setAppName("test") .setMaster("local-cluster[1, 12, 1024]") .set("spark.executor.cores", "12") conf.set("spark.worker.resource.gpu.amount", "1") conf.set("spark.worker.resource.gpu.discoveryScript", scriptPath) conf.set("spark.executor.resource.gpu.amount", "1") conf.set("spark.task.resource.gpu.amount", "0.08") sc = new SparkContext(conf) val rdd = sc.range(0, 100, 1, 4) var rdd1 = rdd.repartition(3) val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 1.0) val rp = new ResourceProfileBuilder().require(treqs).build rdd1 = rdd1.withResources(rp) assert(rdd1.collect().size === 100) } ``` During the initial stages, Spark generates a default resource profile based on the configurations. The calculation for determining the slots per GPU address is performed as "spark.executor.resource.gpu.amount / spark.task.resource.gpu.amount", resulting in a value of 12 (1/0.08 = 12). This means that Spark can accommodate up to 12 tasks running on each GPU address simultaneously. The job is then divided into two stages. The first stage, which consists of 4 tasks, runs concurrently based on the default resource profile. However, the second stage, comprising 3 tasks, runs sequentially using a new task resource profile. This new profile specifies that each task requires 1 CPU and 1.0 full GPU. In reality, the tasks in the second stage are running in parallel, which is the underlying issue. The problem lies in the line `new TaskResourceRequests().cpus(1).resource("gpu", 1.0)`. The value of 1.0 for the GPU, or any value below 1.0 (specifically, (0, 0.5] which is rounded up to 1.0, spark throws an exception if the value is in (0.5, 1)), is merely requesting the number of slots. In this case, it is requesting only 1 slot. Consequently, each task necessitates 1 CPU core and 1 GPU slot, resulting in all tasks running simultaneously. ### Does t
[spark] branch branch-3.5 updated: [SPARK-45495][CORE] Support stage level task resource profile for k8s cluster when dynamic allocation disabled
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new e8d7497abad [SPARK-45495][CORE] Support stage level task resource profile for k8s cluster when dynamic allocation disabled e8d7497abad is described below commit e8d7497abadd0bccb4bd8e615aadc77fc2038566 Author: Bobby Wang AuthorDate: Fri Oct 13 10:50:18 2023 -0500 [SPARK-45495][CORE] Support stage level task resource profile for k8s cluster when dynamic allocation disabled ### What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/37268 which supports stage-level task resource profile for standalone cluster when dynamic allocation is disabled. This PR enables stage-level task resource profile for the Kubernetes cluster. ### Why are the changes needed? Users who work on spark ML/DL cases running on Kubernetes would expect stage-level task resource profile feature. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The current tests of https://github.com/apache/spark/pull/37268 can also cover this PR since both Kubernetes and standalone cluster share the same TaskSchedulerImpl class which implements this feature. Apart from that, modifying the existing test to cover the Kubernetes cluster. Apart from that, I also performed some manual tests which have been updated in the comments. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43323 from wbo4958/k8s-stage-level. Authored-by: Bobby Wang Signed-off-by: Thomas Graves (cherry picked from commit 632eabdb6dfb78c0a5dc84c01806548e1dc6dd0a) Signed-off-by: Thomas Graves --- .../scala/org/apache/spark/resource/ResourceProfileManager.scala | 7 --- .../org/apache/spark/resource/ResourceProfileManagerSuite.scala | 9 +++-- docs/configuration.md| 2 +- docs/running-on-kubernetes.md| 4 +++- 4 files changed, 15 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index cd7124a5724..afbacb80136 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -67,9 +67,10 @@ private[spark] class ResourceProfileManager(sparkConf: SparkConf, */ private[spark] def isSupported(rp: ResourceProfile): Boolean = { if (rp.isInstanceOf[TaskResourceProfile] && !dynamicEnabled) { - if ((notRunningUnitTests || testExceptionThrown) && !(isStandaloneOrLocalCluster || isYarn)) { -throw new SparkException("TaskResourceProfiles are only supported for Standalone and " + - "Yarn cluster for now when dynamic allocation is disabled.") + if ((notRunningUnitTests || testExceptionThrown) && +!(isStandaloneOrLocalCluster || isYarn || isK8s)) { +throw new SparkException("TaskResourceProfiles are only supported for Standalone, " + + "Yarn and Kubernetes cluster for now when dynamic allocation is disabled.") } } else { val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala index 77dc7bcb4c5..7149267583b 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -137,8 +137,8 @@ class ResourceProfileManagerSuite extends SparkFunSuite { val error = intercept[SparkException] { rpmanager.isSupported(taskProf) }.getMessage -assert(error === "TaskResourceProfiles are only supported for Standalone " + - "and Yarn cluster for now when dynamic allocation is disabled.") +assert(error === "TaskResourceProfiles are only supported for Standalone, " + + "Yarn and Kubernetes cluster for now when dynamic allocation is disabled.") // Local cluster: supports task resource profile. conf.setMaster("local-cluster[1, 1, 1024]") @@ -149,6 +149,11 @@ class ResourceProfileManagerSuite extends SparkFunSuite { conf.setMaster("yarn") rpmanager = new ResourceProfileManager(conf, listenerBus) assert(rpmanager.isSupported(taskProf)) + +// K8s: supports task resource profile. +conf.setM
[spark] branch master updated: [SPARK-45495][CORE] Support stage level task resource profile for k8s cluster when dynamic allocation disabled
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 632eabdb6df [SPARK-45495][CORE] Support stage level task resource profile for k8s cluster when dynamic allocation disabled 632eabdb6df is described below commit 632eabdb6dfb78c0a5dc84c01806548e1dc6dd0a Author: Bobby Wang AuthorDate: Fri Oct 13 10:50:18 2023 -0500 [SPARK-45495][CORE] Support stage level task resource profile for k8s cluster when dynamic allocation disabled ### What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/37268 which supports stage-level task resource profile for standalone cluster when dynamic allocation is disabled. This PR enables stage-level task resource profile for the Kubernetes cluster. ### Why are the changes needed? Users who work on spark ML/DL cases running on Kubernetes would expect stage-level task resource profile feature. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The current tests of https://github.com/apache/spark/pull/37268 can also cover this PR since both Kubernetes and standalone cluster share the same TaskSchedulerImpl class which implements this feature. Apart from that, modifying the existing test to cover the Kubernetes cluster. Apart from that, I also performed some manual tests which have been updated in the comments. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43323 from wbo4958/k8s-stage-level. Authored-by: Bobby Wang Signed-off-by: Thomas Graves --- .../scala/org/apache/spark/resource/ResourceProfileManager.scala | 7 --- .../org/apache/spark/resource/ResourceProfileManagerSuite.scala | 9 +++-- docs/configuration.md| 2 +- docs/running-on-kubernetes.md| 4 +++- 4 files changed, 15 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index cd7124a5724..afbacb80136 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -67,9 +67,10 @@ private[spark] class ResourceProfileManager(sparkConf: SparkConf, */ private[spark] def isSupported(rp: ResourceProfile): Boolean = { if (rp.isInstanceOf[TaskResourceProfile] && !dynamicEnabled) { - if ((notRunningUnitTests || testExceptionThrown) && !(isStandaloneOrLocalCluster || isYarn)) { -throw new SparkException("TaskResourceProfiles are only supported for Standalone and " + - "Yarn cluster for now when dynamic allocation is disabled.") + if ((notRunningUnitTests || testExceptionThrown) && +!(isStandaloneOrLocalCluster || isYarn || isK8s)) { +throw new SparkException("TaskResourceProfiles are only supported for Standalone, " + + "Yarn and Kubernetes cluster for now when dynamic allocation is disabled.") } } else { val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala index 77dc7bcb4c5..7149267583b 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -137,8 +137,8 @@ class ResourceProfileManagerSuite extends SparkFunSuite { val error = intercept[SparkException] { rpmanager.isSupported(taskProf) }.getMessage -assert(error === "TaskResourceProfiles are only supported for Standalone " + - "and Yarn cluster for now when dynamic allocation is disabled.") +assert(error === "TaskResourceProfiles are only supported for Standalone, " + + "Yarn and Kubernetes cluster for now when dynamic allocation is disabled.") // Local cluster: supports task resource profile. conf.setMaster("local-cluster[1, 1, 1024]") @@ -149,6 +149,11 @@ class ResourceProfileManagerSuite extends SparkFunSuite { conf.setMaster("yarn") rpmanager = new ResourceProfileManager(conf, listenerBus) assert(rpmanager.isSupported(taskProf)) + +// K8s: supports task resource profile. +conf.setMaster("k8s://foo") +rpmanager = new ResourceProfileManager(conf, listenerBus) +assert(rpmanager.isSupporte
[spark] branch branch-3.5 updated: [SPARK-45250][CORE] Support stage level task resource profile for yarn cluster when dynamic allocation disabled
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.5 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.5 by this push: new c50e371c2d3 [SPARK-45250][CORE] Support stage level task resource profile for yarn cluster when dynamic allocation disabled c50e371c2d3 is described below commit c50e371c2d3ccba9340bc8980add0753f2d7a86b Author: Bobby Wang AuthorDate: Mon Oct 2 23:00:56 2023 -0500 [SPARK-45250][CORE] Support stage level task resource profile for yarn cluster when dynamic allocation disabled ### What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/37268 which supports stage level task resource profile for standalone cluster when dynamic allocation disabled. This PR enables stage-level task resource profile for yarn cluster. ### Why are the changes needed? Users who work on spark ML/DL cases running on Yarn would expect stage-level task resource profile feature. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The current tests of https://github.com/apache/spark/pull/37268 can also cover this PR since both yarn and standalone cluster share the same TaskSchedulerImpl class which implements this feature. Apart from that, modifying the existing test to cover yarn cluster. Apart from that, I also performed some manual tests which have been updated in the comments. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43030 from wbo4958/yarn-task-resoure-profile. Authored-by: Bobby Wang Signed-off-by: Mridul Muralidharan gmail.com> (cherry picked from commit 5b80639e643b6dd09dd64c3f43ec039b2ef2f9fd) Signed-off-by: Thomas Graves --- .../apache/spark/resource/ResourceProfileManager.scala| 6 +++--- .../spark/resource/ResourceProfileManagerSuite.scala | 15 +-- docs/configuration.md | 2 +- docs/running-on-yarn.md | 6 +- 4 files changed, 22 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index 9f98d4d9c9c..cd7124a5724 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -67,9 +67,9 @@ private[spark] class ResourceProfileManager(sparkConf: SparkConf, */ private[spark] def isSupported(rp: ResourceProfile): Boolean = { if (rp.isInstanceOf[TaskResourceProfile] && !dynamicEnabled) { - if ((notRunningUnitTests || testExceptionThrown) && !isStandaloneOrLocalCluster) { -throw new SparkException("TaskResourceProfiles are only supported for Standalone " + - "cluster for now when dynamic allocation is disabled.") + if ((notRunningUnitTests || testExceptionThrown) && !(isStandaloneOrLocalCluster || isYarn)) { +throw new SparkException("TaskResourceProfiles are only supported for Standalone and " + + "Yarn cluster for now when dynamic allocation is disabled.") } } else { val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala index e97d5c7883a..77dc7bcb4c5 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -126,18 +126,29 @@ class ResourceProfileManagerSuite extends SparkFunSuite { val defaultProf = rpmanager.defaultResourceProfile assert(rpmanager.isSupported(defaultProf)) -// task resource profile. +// Standalone: supports task resource profile. val gpuTaskReq = new TaskResourceRequests().resource("gpu", 1) val taskProf = new TaskResourceProfile(gpuTaskReq.requests) assert(rpmanager.isSupported(taskProf)) +// Local: doesn't support task resource profile. conf.setMaster("local") rpmanager = new ResourceProfileManager(conf, listenerBus) val error = intercept[SparkException] { rpmanager.isSupported(taskProf) }.getMessage assert(error === "TaskResourceProfiles are only supported for Standalone " + - "cluster for now when dynamic allocation is disabled.") + "and Yarn cluster for now when dynamic allocation is disabled.") + +// Local cluster: supports task resource profile. +
[spark] branch branch-3.4 updated: [SPARK-43510][YARN] Fix YarnAllocator internal state when adding running executor after processing completed containers
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.4 by this push: new 63d59956024 [SPARK-43510][YARN] Fix YarnAllocator internal state when adding running executor after processing completed containers 63d59956024 is described below commit 63d59956024781b062791dda9990a6043b6a10c1 Author: manuzhang AuthorDate: Tue Jun 6 08:28:52 2023 -0500 [SPARK-43510][YARN] Fix YarnAllocator internal state when adding running executor after processing completed containers ### What changes were proposed in this pull request? Keep track of completed container ids in YarnAllocator and don't update internal state of a container if it's already completed. ### Why are the changes needed? YarnAllocator updates internal state adding running executors after executor launch in a separate thread. That can happen after the containers are already completed (e.g. preempted) and processed by YarnAllocator. Then YarnAllocator mistakenly thinks there are still running executors which are already lost. As a result, application hangs without any running executors. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added UT. Closes #41173 from manuzhang/spark-43510. Authored-by: manuzhang Signed-off-by: Thomas Graves (cherry picked from commit 89d44d092af4ae53fec296ca6569e240ad4c2bc5) Signed-off-by: Thomas Graves --- .../apache/spark/deploy/yarn/YarnAllocator.scala | 42 ++ .../spark/deploy/yarn/YarnAllocatorSuite.scala | 28 ++- 2 files changed, 54 insertions(+), 16 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 313b19f919d..dede5501a39 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -90,6 +90,9 @@ private[yarn] class YarnAllocator( @GuardedBy("this") private val releasedContainers = collection.mutable.HashSet[ContainerId]() + @GuardedBy("this") + private val launchingExecutorContainerIds = collection.mutable.HashSet[ContainerId]() + @GuardedBy("this") private val runningExecutorsPerResourceProfileId = new HashMap[Int, mutable.Set[String]]() @@ -742,19 +745,6 @@ private[yarn] class YarnAllocator( logInfo(s"Launching container $containerId on host $executorHostname " + s"for executor with ID $executorId for ResourceProfile Id $rpId") - def updateInternalState(): Unit = synchronized { -getOrUpdateRunningExecutorForRPId(rpId).add(executorId) -getOrUpdateNumExecutorsStartingForRPId(rpId).decrementAndGet() -executorIdToContainer(executorId) = container -containerIdToExecutorIdAndResourceProfileId(container.getId) = (executorId, rpId) - -val localallocatedHostToContainersMap = getOrUpdateAllocatedHostToContainersMapForRPId(rpId) -val containerSet = localallocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]) -containerSet += containerId -allocatedContainerToHostMap.put(containerId, executorHostname) - } - val rp = rpIdToResourceProfile(rpId) val defaultResources = ResourceProfile.getDefaultProfileExecutorResources(sparkConf) val containerMem = rp.executorResources.get(ResourceProfile.MEMORY). @@ -767,6 +757,7 @@ private[yarn] class YarnAllocator( val rpRunningExecs = getOrUpdateRunningExecutorForRPId(rpId).size if (rpRunningExecs < getOrUpdateTargetNumExecutorsForRPId(rpId)) { getOrUpdateNumExecutorsStartingForRPId(rpId).incrementAndGet() +launchingExecutorContainerIds.add(containerId) if (launchContainers) { launcherPool.execute(() => { try { @@ -784,10 +775,11 @@ private[yarn] class YarnAllocator( localResources, rp.id ).run() - updateInternalState() + updateInternalState(rpId, executorId, container) } catch { case e: Throwable => getOrUpdateNumExecutorsStartingForRPId(rpId).decrementAndGet() +launchingExecutorContainerIds.remove(containerId) if (NonFatal(e)) { logError(s"Failed to launch executor $executorId on container $containerId", e) // Assigned container should be released immediately @@ -800,7 +792,7 @@ private[yarn] class YarnAllocator(
[spark] branch master updated: [SPARK-43510][YARN] Fix YarnAllocator internal state when adding running executor after processing completed containers
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 89d44d092af [SPARK-43510][YARN] Fix YarnAllocator internal state when adding running executor after processing completed containers 89d44d092af is described below commit 89d44d092af4ae53fec296ca6569e240ad4c2bc5 Author: manuzhang AuthorDate: Tue Jun 6 08:28:52 2023 -0500 [SPARK-43510][YARN] Fix YarnAllocator internal state when adding running executor after processing completed containers ### What changes were proposed in this pull request? Keep track of completed container ids in YarnAllocator and don't update internal state of a container if it's already completed. ### Why are the changes needed? YarnAllocator updates internal state adding running executors after executor launch in a separate thread. That can happen after the containers are already completed (e.g. preempted) and processed by YarnAllocator. Then YarnAllocator mistakenly thinks there are still running executors which are already lost. As a result, application hangs without any running executors. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added UT. Closes #41173 from manuzhang/spark-43510. Authored-by: manuzhang Signed-off-by: Thomas Graves --- .../apache/spark/deploy/yarn/YarnAllocator.scala | 42 ++ .../spark/deploy/yarn/YarnAllocatorSuite.scala | 28 ++- 2 files changed, 54 insertions(+), 16 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index b6ee21ed817..19c06f95731 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -91,6 +91,9 @@ private[yarn] class YarnAllocator( @GuardedBy("this") private val releasedContainers = collection.mutable.HashSet[ContainerId]() + @GuardedBy("this") + private val launchingExecutorContainerIds = collection.mutable.HashSet[ContainerId]() + @GuardedBy("this") private val runningExecutorsPerResourceProfileId = new HashMap[Int, mutable.Set[String]]() @@ -738,19 +741,6 @@ private[yarn] class YarnAllocator( logInfo(s"Launching container $containerId on host $executorHostname " + s"for executor with ID $executorId for ResourceProfile Id $rpId") - def updateInternalState(): Unit = synchronized { -getOrUpdateRunningExecutorForRPId(rpId).add(executorId) -getOrUpdateNumExecutorsStartingForRPId(rpId).decrementAndGet() -executorIdToContainer(executorId) = container -containerIdToExecutorIdAndResourceProfileId(container.getId) = (executorId, rpId) - -val localallocatedHostToContainersMap = getOrUpdateAllocatedHostToContainersMapForRPId(rpId) -val containerSet = localallocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]) -containerSet += containerId -allocatedContainerToHostMap.put(containerId, executorHostname) - } - val rp = rpIdToResourceProfile(rpId) val defaultResources = ResourceProfile.getDefaultProfileExecutorResources(sparkConf) val containerMem = rp.executorResources.get(ResourceProfile.MEMORY). @@ -763,6 +753,7 @@ private[yarn] class YarnAllocator( val rpRunningExecs = getOrUpdateRunningExecutorForRPId(rpId).size if (rpRunningExecs < getOrUpdateTargetNumExecutorsForRPId(rpId)) { getOrUpdateNumExecutorsStartingForRPId(rpId).incrementAndGet() +launchingExecutorContainerIds.add(containerId) if (launchContainers) { launcherPool.execute(() => { try { @@ -780,10 +771,11 @@ private[yarn] class YarnAllocator( localResources, rp.id ).run() - updateInternalState() + updateInternalState(rpId, executorId, container) } catch { case e: Throwable => getOrUpdateNumExecutorsStartingForRPId(rpId).decrementAndGet() +launchingExecutorContainerIds.remove(containerId) if (NonFatal(e)) { logError(s"Failed to launch executor $executorId on container $containerId", e) // Assigned container should be released immediately @@ -796,7 +788,7 @@ private[yarn] class YarnAllocator( }) } else { // For test only - updateInternalState() + updateInternalState(rpId, exec
[spark] branch master updated: [SPARK-41585][YARN] The Spark exclude node functionality for YARN should work independently of dynamic allocation
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f76d7cc31e7 [SPARK-41585][YARN] The Spark exclude node functionality for YARN should work independently of dynamic allocation f76d7cc31e7 is described below commit f76d7cc31e72778b62647264e5b90059de888c20 Author: Luca Canali AuthorDate: Mon Mar 20 14:01:53 2023 -0500 [SPARK-41585][YARN] The Spark exclude node functionality for YARN should work independently of dynamic allocation ### What changes were proposed in this pull request? The Spark exclude node functionality for Spark on YARN, introduced in [SPARK-26688](https://issues.apache.org/jira/browse/SPARK-26688), allows users to specify a list of node names that are excluded from resource allocation. This is done using the configuration parameter: `spark.yarn.exclude.nodes` The feature currently works only for executors allocated via dynamic allocation. To use the feature on Spark 3.3.1, for example, one may need also to configure `spark.dynamicAllocation.minExecutors=0` and `spark.executor.instances=0`, therefore relying on executor resource allocation only via dynamic allocation. ### Why are the changes needed? This proposes to extend the use of Spark exclude node functionality for YARN beyond dynamic allocation, which I believe makes it more consistent also with what the documentation reports for this feature/configuration parameter. ### Does this PR introduce _any_ user-facing change? Yes, this allows using the executor exclude nodes feature for Spark on YARN also when not using dynamical allocation. ### How was this patch tested? A unit test has been added for this + manual tests on a YARN cluster. Closes #39127 from LucaCanali/excludeNodesBesidesDynamicAllocation. Authored-by: Luca Canali Signed-off-by: Thomas Graves --- .../spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala | 2 ++ .../spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala| 10 ++ 2 files changed, 12 insertions(+) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala index bfe0face8c2..2f5297107e6 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala @@ -144,6 +144,8 @@ private[spark] class YarnAllocatorNodeHealthTracker( val now = failureTracker.clock.getTimeMillis() allocatorExcludedNodeList.retain { (_, expiryTime) => expiryTime > now } } + + refreshExcludedNodes } /** diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala index c2fd5ff3165..c722b565549 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala @@ -97,6 +97,16 @@ class YarnAllocatorHealthTrackerSuite extends SparkFunSuite with Matchers verify(amClientMock, times(0)).updateBlacklist(Collections.emptyList(), Collections.emptyList()) } + test("SPARK-41585 YARN Exclude Nodes should work independently of dynamic allocation") { +sparkConf.set(YARN_EXCLUDE_NODES, Seq("host1", "host2")) +val yarnHealthTracker = createYarnAllocatorHealthTracker(sparkConf) + +// Check that host1 and host2 are in the exclude list +// Note, this covers also non-dynamic allocation +verify(amClientMock) + .updateBlacklist(Arrays.asList("host1", "host2"), Collections.emptyList()) + } + test("combining scheduler and allocation excluded node list") { sparkConf.set(YARN_EXCLUDE_NODES, Seq("initial1", "initial2")) val yarnHealthTracker = createYarnAllocatorHealthTracker(sparkConf) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-39601][YARN] AllocationFailure should not be treated as exitCausedByApp when driver is shutting down
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e857b7ad1c7 [SPARK-39601][YARN] AllocationFailure should not be treated as exitCausedByApp when driver is shutting down e857b7ad1c7 is described below commit e857b7ad1c78c57d06436e387473d83e61293c7c Author: Cheng Pan AuthorDate: Tue Dec 13 08:18:08 2022 -0600 [SPARK-39601][YARN] AllocationFailure should not be treated as exitCausedByApp when driver is shutting down ### What changes were proposed in this pull request? Treating container `AllocationFailure` as not "exitCausedByApp" when driver is shutting down. The approach is suggested at https://github.com/apache/spark/pull/36991#discussion_r915948343 ### Why are the changes needed? I observed some Spark Applications successfully completed all jobs but failed during the shutting down phase w/ reason: Max number of executor failures (16) reached, the timeline is Driver - Job success, Spark starts shutting down procedure. ``` 2022-06-23 19:50:55 CST AbstractConnector INFO - Stopped Spark74e9431b{HTTP/1.1, (http/1.1)}{0.0.0.0:0} 2022-06-23 19:50:55 CST SparkUI INFO - Stopped Spark web UI at http://hadoop2627.xxx.org:28446 2022-06-23 19:50:55 CST YarnClusterSchedulerBackend INFO - Shutting down all executors ``` Driver - A container allocate successful during shutting down phase. ``` 2022-06-23 19:52:21 CST YarnAllocator INFO - Launching container container_e94_1649986670278_7743380_02_25 on host hadoop4388.xxx.org for executor with ID 24 for ResourceProfile Id 0 ``` Executor - The executor can not connect to driver endpoint because driver already stopped the endpoint. ``` Exception in thread "main" java.lang.reflect.UndeclaredThrowableException at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1911) at org.apache.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:61) at org.apache.spark.executor.CoarseGrainedExecutorBackend$.run(CoarseGrainedExecutorBackend.scala:393) at org.apache.spark.executor.YarnCoarseGrainedExecutorBackend$.main(YarnCoarseGrainedExecutorBackend.scala:81) at org.apache.spark.executor.YarnCoarseGrainedExecutorBackend.main(YarnCoarseGrainedExecutorBackend.scala) Caused by: org.apache.spark.SparkException: Exception thrown in awaitResult: at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:301) at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75) at org.apache.spark.rpc.RpcEnv.setupEndpointRefByURI(RpcEnv.scala:101) at org.apache.spark.executor.CoarseGrainedExecutorBackend$.$anonfun$run$9(CoarseGrainedExecutorBackend.scala:413) at scala.runtime.java8.JFunction1$mcVI$sp.apply(JFunction1$mcVI$sp.java:23) at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877) at scala.collection.immutable.Range.foreach(Range.scala:158) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:876) at org.apache.spark.executor.CoarseGrainedExecutorBackend$.$anonfun$run$7(CoarseGrainedExecutorBackend.scala:411) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:62) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:61) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1893) ... 4 more Caused by: org.apache.spark.rpc.RpcEndpointNotFoundException: Cannot find endpoint: spark://CoarseGrainedSchedulerhadoop2627.xxx.org:21956 at org.apache.spark.rpc.netty.NettyRpcEnv.$anonfun$asyncSetupEndpointRefByURI$1(NettyRpcEnv.scala:148) at org.apache.spark.rpc.netty.NettyRpcEnv.$anonfun$asyncSetupEndpointRefByURI$1$adapted(NettyRpcEnv.scala:144) at scala.concurrent.Future.$anonfun$flatMap$1(Future.scala:307) at scala.concurrent.impl.Promise.$anonfun$transformWith$1(Promise.scala:41) at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64) at org.apache.spark.util.ThreadUtils$$anon$1.execute(ThreadUtils.scala:99) at scala.concurrent.impl.ExecutionContextImpl$$anon$4.execute(ExecutionContextImpl.scala:138) at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:72) at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:288) at scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:288) at scala.concurrent.impl.Promise$DefaultPromise.tryComple
[spark] branch master updated: [SPARK-40490][YARN][TESTS] Ensure `YarnShuffleIntegrationSuite` tests registeredExecFile reload scenarios
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e6699570bec [SPARK-40490][YARN][TESTS] Ensure `YarnShuffleIntegrationSuite` tests registeredExecFile reload scenarios e6699570bec is described below commit e6699570becadb91695572bca5adc1605dc1b2a8 Author: yangjie01 AuthorDate: Wed Sep 21 08:05:17 2022 -0500 [SPARK-40490][YARN][TESTS] Ensure `YarnShuffleIntegrationSuite` tests registeredExecFile reload scenarios ### What changes were proposed in this pull request? After SPARK-17321, `YarnShuffleService` will persist data to local shuffle state db/reload data from local shuffle state db only when Yarn NodeManager start with `YarnConfiguration#NM_RECOVERY_ENABLED = true`. `YarnShuffleIntegrationSuite` not set `YarnConfiguration#NM_RECOVERY_ENABLED` and the default value of the configuration is false, so `YarnShuffleIntegrationSuite` will neither trigger data persistence to the db nor verify the reload of data. This pr aims to let `YarnShuffleIntegrationSuite` restart the verification of registeredExecFile reload scenarios, to achieve this goal, this pr make the following changes: 1. Add a new un-document configuration `spark.yarn.shuffle.testing` to `YarnShuffleService`, and Initialize `_recoveryPath` when `_recoveryPath == null && spark.yarn.shuffle.testing == true`. 2. Only set `spark.yarn.shuffle.testing = true` in `YarnShuffleIntegrationSuite`, and add assertions to check `registeredExecFile` is not null to ensure that registeredExecFile reload scenarios will be verified. ### Why are the changes needed? Fix registeredExecFile reload test scenarios. Why not test by configuring `YarnConfiguration#NM_RECOVERY_ENABLED` as true? This configuration has been tried **Hadoop 3.3.4** ``` build/mvn clean install -pl resource-managers/yarn -Pyarn -Dtest=none -DwildcardSuites=org.apache.spark.deploy.yarn.YarnShuffleIntegrationWithLevelDBBackendSuite -Phadoop-3 ``` ``` 2022-09-10T11:44:42.1710230Z Cause: java.lang.ClassNotFoundException: org.apache.hadoop.shaded.org.iq80.leveldb.DBException 2022-09-10T11:44:42.1715234Z at java.net.URLClassLoader.findClass(URLClassLoader.java:387) 2022-09-10T11:44:42.1719347Z at java.lang.ClassLoader.loadClass(ClassLoader.java:419) 2022-09-10T11:44:42.1723090Z at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352) 2022-09-10T11:44:42.1726759Z at java.lang.ClassLoader.loadClass(ClassLoader.java:352) 2022-09-10T11:44:42.1731028Z at org.apache.hadoop.yarn.server.nodemanager.NodeManager.initAndStartRecoveryStore(NodeManager.java:313) 2022-09-10T11:44:42.1735424Z at org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceInit(NodeManager.java:370) 2022-09-10T11:44:42.1740303Z at org.apache.hadoop.service.AbstractService.init(AbstractService.java:164) 2022-09-10T11:44:42.1745576Z at org.apache.hadoop.yarn.server.MiniYARNCluster$NodeManagerWrapper.serviceInit(MiniYARNCluster.java:597) 2022-09-10T11:44:42.1828858Z at org.apache.hadoop.service.AbstractService.init(AbstractService.java:164) 2022-09-10T11:44:42.1829712Z at org.apache.hadoop.service.CompositeService.serviceInit(CompositeService.java:109) 2022-09-10T11:44:42.1830633Z at org.apache.hadoop.yarn.server.MiniYARNCluster.serviceInit(MiniYARNCluster.java:327) 2022-09-10T11:44:42.1831431Z at org.apache.hadoop.service.AbstractService.init(AbstractService.java:164) 2022-09-10T11:44:42.1832279Z at org.apache.spark.deploy.yarn.BaseYarnClusterSuite.beforeAll(BaseYarnClusterSuite.scala:112) ``` **Hadoop 2.7.4** ``` build/mvn clean install -pl resource-managers/yarn -Pyarn -Dtest=none -DwildcardSuites=org.apache.spark.deploy.yarn.YarnShuffleIntegrationWithLevelDBBackendSuite -Phadoop-2 ``` ``` YarnShuffleIntegrationWithLevelDBBackendSuite: org.apache.spark.deploy.yarn.YarnShuffleIntegrationWithLevelDBBackendSuite *** ABORTED *** java.lang.IllegalArgumentException: Cannot support recovery with an ephemeral server port. Check the setting of yarn.nodemanager.address at org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl.serviceStart(ContainerManagerImpl.java:395) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.service.CompositeService.serviceStart(CompositeService.java:120) at org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceStart(NodeManager.java:272) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.yarn.server.MiniYARNCluster$NodeManagerWrapper.serviceStart(MiniYARNCluster.java:560)
[spark] branch branch-3.2 updated: [SPARK-40280][SQL] Add support for parquet push down for annotated int and long
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 24818bf4c9d [SPARK-40280][SQL] Add support for parquet push down for annotated int and long 24818bf4c9d is described below commit 24818bf4c9d410259fab9ab63c407bedb7a932d1 Author: Robert (Bobby) Evans AuthorDate: Thu Sep 8 08:54:28 2022 -0500 [SPARK-40280][SQL] Add support for parquet push down for annotated int and long ### What changes were proposed in this pull request? This fixes SPARK-40280 by normalizing a parquet int/long that has optional metadata with it to look like the expected version that does not have the extra metadata. ## Why are the changes needed? This allows predicate push down in parquet to work when reading files that are complaint with the parquet specification, but different from what Spark writes. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I added unit tests that cover this use case. I also did some manual testing on some queries to verify that less data is actually read after this change. Closes #37747 from revans2/normalize_int_long_parquet_push. Authored-by: Robert (Bobby) Evans Signed-off-by: Thomas Graves (cherry picked from commit 24b3baf0177fc1446bf59bb34987296aefd4b318) Signed-off-by: Thomas Graves --- .../datasources/parquet/ParquetFilters.scala | 16 +++- .../test/resources/test-data/tagged_int.parquet| Bin 0 -> 305 bytes .../test/resources/test-data/tagged_long.parquet | Bin 0 -> 313 bytes .../datasources/parquet/ParquetFilterSuite.scala | 82 + 4 files changed, 96 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index d1b4506a0e4..e2cc3698a47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -30,7 +30,7 @@ import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.SparkFilterApi._ import org.apache.parquet.io.api.Binary import org.apache.parquet.schema.{GroupType, LogicalTypeAnnotation, MessageType, PrimitiveComparator, PrimitiveType, Type} -import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, TimeUnit} +import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, IntLogicalTypeAnnotation, TimeUnit} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition @@ -59,6 +59,18 @@ class ParquetFilters( // nested columns. If any part of the names contains `dots`, it is quoted to avoid confusion. // See `org.apache.spark.sql.connector.catalog.quote` for implementation details. private val nameToParquetField : Map[String, ParquetPrimitiveField] = { +def getNormalizedLogicalType(p: PrimitiveType): LogicalTypeAnnotation = { + // SPARK-40280: Signed 64 bits on an INT64 and signed 32 bits on an INT32 are optional, but + // the rest of the code here assumes they are not set, so normalize them to not being set. + (p.getPrimitiveTypeName, p.getLogicalTypeAnnotation) match { +case (INT32, intType: IntLogicalTypeAnnotation) + if intType.getBitWidth() == 32 && intType.isSigned() => null +case (INT64, intType: IntLogicalTypeAnnotation) + if intType.getBitWidth() == 64 && intType.isSigned() => null +case (_, otherType) => otherType + } +} + // Recursively traverse the parquet schema to get primitive fields that can be pushed-down. // `parentFieldNames` is used to keep track of the current nested level when traversing. def getPrimitiveFields( @@ -70,7 +82,7 @@ class ParquetFilters( //repeated columns (https://issues.apache.org/jira/browse/PARQUET-34) case p: PrimitiveType if p.getRepetition != Repetition.REPEATED => Some(ParquetPrimitiveField(fieldNames = parentFieldNames :+ p.getName, -fieldType = ParquetSchemaType(p.getLogicalTypeAnnotation, +fieldType = ParquetSchemaType(getNormalizedLogicalType(p), p.getPrimitiveTypeName, p.getTypeLength))) // Note that when g is a `Struct`, `g.getOriginalType` is `null`. // When g is a `Map`, `g.getOriginalType` is `MAP`. diff --git a/sql/core/src/test/resources/test-data/tagged_int.parquet b/sql
[spark] branch branch-3.3 updated: [SPARK-40280][SQL] Add support for parquet push down for annotated int and long
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.3 by this push: new 0cdb081670b [SPARK-40280][SQL] Add support for parquet push down for annotated int and long 0cdb081670b is described below commit 0cdb081670b55d9181d8ffb125911333e8ab339b Author: Robert (Bobby) Evans AuthorDate: Thu Sep 8 08:54:28 2022 -0500 [SPARK-40280][SQL] Add support for parquet push down for annotated int and long ### What changes were proposed in this pull request? This fixes SPARK-40280 by normalizing a parquet int/long that has optional metadata with it to look like the expected version that does not have the extra metadata. ## Why are the changes needed? This allows predicate push down in parquet to work when reading files that are complaint with the parquet specification, but different from what Spark writes. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I added unit tests that cover this use case. I also did some manual testing on some queries to verify that less data is actually read after this change. Closes #37747 from revans2/normalize_int_long_parquet_push. Authored-by: Robert (Bobby) Evans Signed-off-by: Thomas Graves (cherry picked from commit 24b3baf0177fc1446bf59bb34987296aefd4b318) Signed-off-by: Thomas Graves --- .../datasources/parquet/ParquetFilters.scala | 16 +++- .../test/resources/test-data/tagged_int.parquet| Bin 0 -> 305 bytes .../test/resources/test-data/tagged_long.parquet | Bin 0 -> 313 bytes .../datasources/parquet/ParquetFilterSuite.scala | 82 + 4 files changed, 96 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 9502ec0316c..e04019fa9a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -30,7 +30,7 @@ import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.SparkFilterApi._ import org.apache.parquet.io.api.Binary import org.apache.parquet.schema.{GroupType, LogicalTypeAnnotation, MessageType, PrimitiveComparator, PrimitiveType, Type} -import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, TimeUnit} +import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, IntLogicalTypeAnnotation, TimeUnit} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition @@ -59,6 +59,18 @@ class ParquetFilters( // nested columns. If any part of the names contains `dots`, it is quoted to avoid confusion. // See `org.apache.spark.sql.connector.catalog.quote` for implementation details. private val nameToParquetField : Map[String, ParquetPrimitiveField] = { +def getNormalizedLogicalType(p: PrimitiveType): LogicalTypeAnnotation = { + // SPARK-40280: Signed 64 bits on an INT64 and signed 32 bits on an INT32 are optional, but + // the rest of the code here assumes they are not set, so normalize them to not being set. + (p.getPrimitiveTypeName, p.getLogicalTypeAnnotation) match { +case (INT32, intType: IntLogicalTypeAnnotation) + if intType.getBitWidth() == 32 && intType.isSigned() => null +case (INT64, intType: IntLogicalTypeAnnotation) + if intType.getBitWidth() == 64 && intType.isSigned() => null +case (_, otherType) => otherType + } +} + // Recursively traverse the parquet schema to get primitive fields that can be pushed-down. // `parentFieldNames` is used to keep track of the current nested level when traversing. def getPrimitiveFields( @@ -70,7 +82,7 @@ class ParquetFilters( //repeated columns (https://issues.apache.org/jira/browse/PARQUET-34) case p: PrimitiveType if p.getRepetition != Repetition.REPEATED => Some(ParquetPrimitiveField(fieldNames = parentFieldNames :+ p.getName, -fieldType = ParquetSchemaType(p.getLogicalTypeAnnotation, +fieldType = ParquetSchemaType(getNormalizedLogicalType(p), p.getPrimitiveTypeName, p.getTypeLength))) // Note that when g is a `Struct`, `g.getOriginalType` is `null`. // When g is a `Map`, `g.getOriginalType` is `MAP`. diff --git a/sql/core/src/test/resources/test-data/tagged_int.parquet b/sql
[spark] branch master updated: [SPARK-40280][SQL] Add support for parquet push down for annotated int and long
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 24b3baf0177 [SPARK-40280][SQL] Add support for parquet push down for annotated int and long 24b3baf0177 is described below commit 24b3baf0177fc1446bf59bb34987296aefd4b318 Author: Robert (Bobby) Evans AuthorDate: Thu Sep 8 08:54:28 2022 -0500 [SPARK-40280][SQL] Add support for parquet push down for annotated int and long ### What changes were proposed in this pull request? This fixes SPARK-40280 by normalizing a parquet int/long that has optional metadata with it to look like the expected version that does not have the extra metadata. ## Why are the changes needed? This allows predicate push down in parquet to work when reading files that are complaint with the parquet specification, but different from what Spark writes. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? I added unit tests that cover this use case. I also did some manual testing on some queries to verify that less data is actually read after this change. Closes #37747 from revans2/normalize_int_long_parquet_push. Authored-by: Robert (Bobby) Evans Signed-off-by: Thomas Graves --- .../datasources/parquet/ParquetFilters.scala | 16 +++- .../test/resources/test-data/tagged_int.parquet| Bin 0 -> 305 bytes .../test/resources/test-data/tagged_long.parquet | Bin 0 -> 313 bytes .../datasources/parquet/ParquetFilterSuite.scala | 82 + 4 files changed, 96 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index f93419412e6..c34f2827659 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -30,7 +30,7 @@ import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.SparkFilterApi._ import org.apache.parquet.io.api.Binary import org.apache.parquet.schema.{GroupType, LogicalTypeAnnotation, MessageType, PrimitiveComparator, PrimitiveType, Type} -import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, TimeUnit} +import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, IntLogicalTypeAnnotation, TimeUnit} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition @@ -59,6 +59,18 @@ class ParquetFilters( // nested columns. If any part of the names contains `dots`, it is quoted to avoid confusion. // See `org.apache.spark.sql.connector.catalog.quote` for implementation details. private val nameToParquetField : Map[String, ParquetPrimitiveField] = { +def getNormalizedLogicalType(p: PrimitiveType): LogicalTypeAnnotation = { + // SPARK-40280: Signed 64 bits on an INT64 and signed 32 bits on an INT32 are optional, but + // the rest of the code here assumes they are not set, so normalize them to not being set. + (p.getPrimitiveTypeName, p.getLogicalTypeAnnotation) match { +case (INT32, intType: IntLogicalTypeAnnotation) + if intType.getBitWidth() == 32 && intType.isSigned() => null +case (INT64, intType: IntLogicalTypeAnnotation) + if intType.getBitWidth() == 64 && intType.isSigned() => null +case (_, otherType) => otherType + } +} + // Recursively traverse the parquet schema to get primitive fields that can be pushed-down. // `parentFieldNames` is used to keep track of the current nested level when traversing. def getPrimitiveFields( @@ -70,7 +82,7 @@ class ParquetFilters( //repeated columns (https://issues.apache.org/jira/browse/PARQUET-34) case p: PrimitiveType if p.getRepetition != Repetition.REPEATED => Some(ParquetPrimitiveField(fieldNames = parentFieldNames :+ p.getName, -fieldType = ParquetSchemaType(p.getLogicalTypeAnnotation, +fieldType = ParquetSchemaType(getNormalizedLogicalType(p), p.getPrimitiveTypeName, p.getTypeLength))) // Note that when g is a `Struct`, `g.getOriginalType` is `null`. // When g is a `Map`, `g.getOriginalType` is `MAP`. diff --git a/sql/core/src/test/resources/test-data/tagged_int.parquet b/sql/core/src/test/resources/test-data/tagged_int.parquet new file mode 100644 index 000..840f5dbdb93 Binary files /dev/null
[spark] branch master updated: [SPARK-38910][YARN] Clean spark staging before unregister
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2d730ab9df1 [SPARK-38910][YARN] Clean spark staging before unregister 2d730ab9df1 is described below commit 2d730ab9df1d667560540d51d2d7b8034f670c9a Author: Angerszh AuthorDate: Wed Aug 10 09:28:44 2022 -0500 [SPARK-38910][YARN] Clean spark staging before unregister ### What changes were proposed in this pull request? After discussing about https://github.com/apache/spark/pull/36207 and re-check the whole logic, we should revert https://github.com/apache/spark/pull/36207 and do some change 1. No matter whether it's client or cluster mode if it's the last attempt, anyway yarn won't rerun the job, we can clean staging dir first then we can avoid remaining staging dir if unregister failed. 2. If it's cluster or client mode, and it's not the last attempt and the final status is SUCCESS, if unregister failed, YARN will rerun the job again, we can't clean the staging dir before unregistering success because if we clean the staging dir before rerunning, yarn can't download the related files and fail. 3. If it's cluster unmanaged mode, if it failed, we can first delete the staging dir since it won't rerun. ### Why are the changes needed? Revert change and make it more accurate ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Closes #37162 from AngersZh/REVERT-SPARK-38910. Lead-authored-by: Angerszh Co-authored-by: AngersZh Signed-off-by: Thomas Graves --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 14 +++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index cc4a63c160f..8f8c08fbe74 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -260,7 +260,13 @@ private[spark] class ApplicationMaster( if (!unregistered) { // we only want to unregister if we don't want the RM to retry -if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { +if (isLastAttempt) { + cleanupStagingDir(new Path(System.getenv("SPARK_YARN_STAGING_DIR"))) + unregister(finalStatus, finalMsg) +} else if (finalStatus == FinalApplicationStatus.SUCCEEDED) { + // When it's not the last attempt, if unregister failed caused by timeout exception, + // YARN will rerun the application, AM should not clean staging dir before unregister + // success. unregister(finalStatus, finalMsg) cleanupStagingDir(new Path(System.getenv("SPARK_YARN_STAGING_DIR"))) } @@ -327,8 +333,9 @@ private[spark] class ApplicationMaster( ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, "Uncaught exception: " + StringUtils.stringifyException(e)) if (!unregistered) { - unregister(finalStatus, finalMsg) + // It's ok to clean staging dir first because unmanaged AM can't be retried. cleanupStagingDir(stagingDir) + unregister(finalStatus, finalMsg) } } finally { try { @@ -348,8 +355,9 @@ private[spark] class ApplicationMaster( finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) } if (!unregistered) { - unregister(finalStatus, finalMsg) + // It's ok to clean staging dir first because unmanaged AM can't be retried. cleanupStagingDir(stagingDir) + unregister(finalStatus, finalMsg) } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.3 updated: [SPARK-37618][CORE] Remove shuffle blocks using the shuffle service for released executors
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.3 by this push: new 2d47076 [SPARK-37618][CORE] Remove shuffle blocks using the shuffle service for released executors 2d47076 is described below commit 2d470763ecbcccde418956b03e503461352ab4c2 Author: Adam Binford AuthorDate: Fri Mar 25 13:00:17 2022 -0500 [SPARK-37618][CORE] Remove shuffle blocks using the shuffle service for released executors ### What changes were proposed in this pull request? Add support for removing shuffle files on released executors via the external shuffle service. The shuffle service already supports removing shuffle service cached RDD blocks, so I reused this mechanism to remove shuffle blocks as well, so as not to require updating the shuffle service itself. To support this change functioning in a secure Yarn environment, I updated permissions on some of the block manager folders and files. Specifically: - Block manager sub directories have the group write posix permission added to them. This gives the shuffle service permission to delete files from within these folders. - Shuffle files have the world readable posix permission added to them. This is because when the sub directories are marked group writable, they lose the setgid bit that gets set in a secure Yarn environment. Without this, the permissions on the files would be `rw-r-`, and since the group running Yarn (and therefore the shuffle service), is no longer the group owner of the file, it does not have access to read the file. The sub directories still do not have world execute permissio [...] Both of these changes are done after creating a file so that umasks don't affect the resulting permissions. ### Why are the changes needed? External shuffle services are very useful for long running jobs and dynamic allocation. However, currently if an executor is removed (either through dynamic deallocation or through some error), the shuffle files created by that executor will live until the application finishes. This results in local disks slowly filling up over time, eventually causing problems for long running applications. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. Not sure if there's a better way I could have tested for the files being deleted or any other tests I should add. Closes #35085 from Kimahriman/shuffle-service-remove-shuffle-blocks. Authored-by: Adam Binford Signed-off-by: Thomas Graves (cherry picked from commit 9a7596e1dde0f1dd596aa6d3b2efbcb5d1ef70ea) Signed-off-by: Thomas Graves --- .../network/shuffle/ExternalBlockStoreClient.java | 4 +- .../sort/io/LocalDiskShuffleMapOutputWriter.java | 3 +- .../scala/org/apache/spark/ContextCleaner.scala| 4 +- .../src/main/scala/org/apache/spark/SparkEnv.scala | 6 +- .../org/apache/spark/internal/config/package.scala | 10 ++ .../spark/shuffle/IndexShuffleBlockResolver.scala | 18 ++- .../spark/shuffle/ShuffleBlockResolver.scala | 8 ++ .../spark/storage/BlockManagerMasterEndpoint.scala | 89 +++ .../apache/spark/storage/DiskBlockManager.scala| 61 +- .../scala/org/apache/spark/storage/DiskStore.scala | 10 ++ .../shuffle/sort/UnsafeShuffleWriterSuite.java | 8 ++ .../apache/spark/ExternalShuffleServiceSuite.scala | 127 - .../sort/BypassMergeSortShuffleWriterSuite.scala | 11 ++ .../sort/IndexShuffleBlockResolverSuite.scala | 5 + .../io/LocalDiskShuffleMapOutputWriterSuite.scala | 5 + .../storage/BlockManagerReplicationSuite.scala | 3 +- .../apache/spark/storage/BlockManagerSuite.scala | 3 +- .../spark/storage/DiskBlockManagerSuite.scala | 26 - docs/configuration.md | 11 ++ .../streaming/ReceivedBlockHandlerSuite.scala | 3 +- 20 files changed, 372 insertions(+), 43 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index d2df776..b066d99 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -299,7 +299,7 @@ public class ExternalBlockStoreClient extends BlockStoreClient { BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response); numRemovedBlocksFuture.complete(((BlocksRemoved) msgObj).numRemovedBlocks); } catch (Throwable t) { - l
[spark] branch master updated: [SPARK-37618][CORE] Remove shuffle blocks using the shuffle service for released executors
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9a7596e [SPARK-37618][CORE] Remove shuffle blocks using the shuffle service for released executors 9a7596e is described below commit 9a7596e1dde0f1dd596aa6d3b2efbcb5d1ef70ea Author: Adam Binford AuthorDate: Fri Mar 25 13:00:17 2022 -0500 [SPARK-37618][CORE] Remove shuffle blocks using the shuffle service for released executors ### What changes were proposed in this pull request? Add support for removing shuffle files on released executors via the external shuffle service. The shuffle service already supports removing shuffle service cached RDD blocks, so I reused this mechanism to remove shuffle blocks as well, so as not to require updating the shuffle service itself. To support this change functioning in a secure Yarn environment, I updated permissions on some of the block manager folders and files. Specifically: - Block manager sub directories have the group write posix permission added to them. This gives the shuffle service permission to delete files from within these folders. - Shuffle files have the world readable posix permission added to them. This is because when the sub directories are marked group writable, they lose the setgid bit that gets set in a secure Yarn environment. Without this, the permissions on the files would be `rw-r-`, and since the group running Yarn (and therefore the shuffle service), is no longer the group owner of the file, it does not have access to read the file. The sub directories still do not have world execute permissio [...] Both of these changes are done after creating a file so that umasks don't affect the resulting permissions. ### Why are the changes needed? External shuffle services are very useful for long running jobs and dynamic allocation. However, currently if an executor is removed (either through dynamic deallocation or through some error), the shuffle files created by that executor will live until the application finishes. This results in local disks slowly filling up over time, eventually causing problems for long running applications. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. Not sure if there's a better way I could have tested for the files being deleted or any other tests I should add. Closes #35085 from Kimahriman/shuffle-service-remove-shuffle-blocks. Authored-by: Adam Binford Signed-off-by: Thomas Graves --- .../network/shuffle/ExternalBlockStoreClient.java | 4 +- .../sort/io/LocalDiskShuffleMapOutputWriter.java | 3 +- .../scala/org/apache/spark/ContextCleaner.scala| 4 +- .../src/main/scala/org/apache/spark/SparkEnv.scala | 6 +- .../org/apache/spark/internal/config/package.scala | 10 ++ .../spark/shuffle/IndexShuffleBlockResolver.scala | 18 ++- .../spark/shuffle/ShuffleBlockResolver.scala | 8 ++ .../spark/storage/BlockManagerMasterEndpoint.scala | 89 +++ .../apache/spark/storage/DiskBlockManager.scala| 61 +- .../scala/org/apache/spark/storage/DiskStore.scala | 10 ++ .../shuffle/sort/UnsafeShuffleWriterSuite.java | 8 ++ .../apache/spark/ExternalShuffleServiceSuite.scala | 127 - .../sort/BypassMergeSortShuffleWriterSuite.scala | 11 ++ .../sort/IndexShuffleBlockResolverSuite.scala | 5 + .../io/LocalDiskShuffleMapOutputWriterSuite.scala | 5 + .../storage/BlockManagerReplicationSuite.scala | 3 +- .../apache/spark/storage/BlockManagerSuite.scala | 3 +- .../spark/storage/DiskBlockManagerSuite.scala | 26 - docs/configuration.md | 11 ++ .../streaming/ReceivedBlockHandlerSuite.scala | 3 +- 20 files changed, 372 insertions(+), 43 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index d2df776..b066d99 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -299,7 +299,7 @@ public class ExternalBlockStoreClient extends BlockStoreClient { BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response); numRemovedBlocksFuture.complete(((BlocksRemoved) msgObj).numRemovedBlocks); } catch (Throwable t) { - logger.warn("Error trying to remove RDD blocks " + Arrays.toString(blockIds) + + logger.warn(&qu
[spark] branch branch-3.3 updated: [SPARK-38194][YARN][MESOS][K8S][3.3] Make memory overhead factor configurable
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.3 by this push: new 8a6d144 [SPARK-38194][YARN][MESOS][K8S][3.3] Make memory overhead factor configurable 8a6d144 is described below commit 8a6d1444d6a55783b92e9da00954b3766aef8090 Author: Adam Binford AuthorDate: Mon Mar 21 08:51:04 2022 -0500 [SPARK-38194][YARN][MESOS][K8S][3.3] Make memory overhead factor configurable ### What changes were proposed in this pull request? Add a new config to set the memory overhead factor for drivers and executors. Currently the memory overhead is hard coded to 10% (except in Kubernetes), and the only way to set it higher is to set it to a specific memory amount. ### Why are the changes needed? In dynamic environments where different people or use cases need different memory requirements, it would be helpful to set a higher memory overhead factor instead of having to set a higher specific memory overhead value. The kubernetes resource manager already makes this configurable. This makes it configurable across the board. ### Does this PR introduce _any_ user-facing change? No change to default behavior, just adds a new config users can change. ### How was this patch tested? New UT to check the memory calculation. Closes #35912 from Kimahriman/memory-overhead-factor-3.3. Authored-by: Adam Binford Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/SparkConf.scala| 4 +- .../org/apache/spark/internal/config/package.scala | 28 +++ docs/configuration.md | 30 ++- docs/running-on-kubernetes.md | 9 -- .../k8s/features/BasicDriverFeatureStep.scala | 19 +++-- .../k8s/features/BasicExecutorFeatureStep.scala| 7 +- .../k8s/features/BasicDriverFeatureStepSuite.scala | 96 +- .../features/BasicExecutorFeatureStepSuite.scala | 54 .../spark/deploy/rest/mesos/MesosRestServer.scala | 5 +- .../cluster/mesos/MesosSchedulerUtils.scala| 9 +- .../deploy/rest/mesos/MesosRestServerSuite.scala | 8 +- .../org/apache/spark/deploy/yarn/Client.scala | 14 +++- .../apache/spark/deploy/yarn/YarnAllocator.scala | 5 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 5 +- .../spark/deploy/yarn/YarnAllocatorSuite.scala | 29 +++ 15 files changed, 287 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 5f37a1a..cf12174 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -636,7 +636,9 @@ private[spark] object SparkConf extends Logging { DeprecatedConfig("spark.blacklist.killBlacklistedExecutors", "3.1.0", "Please use spark.excludeOnFailure.killExcludedExecutors"), DeprecatedConfig("spark.yarn.blacklist.executor.launch.blacklisting.enabled", "3.1.0", -"Please use spark.yarn.executor.launch.excludeOnFailure.enabled") +"Please use spark.yarn.executor.launch.excludeOnFailure.enabled"), + DeprecatedConfig("spark.kubernetes.memoryOverheadFactor", "3.3.0", +"Please use spark.driver.memoryOverheadFactor and spark.executor.memoryOverheadFactor") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index dbec61a..ffe4501 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -105,6 +105,22 @@ package object config { .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val DRIVER_MEMORY_OVERHEAD_FACTOR = +ConfigBuilder("spark.driver.memoryOverheadFactor") + .doc("Fraction of driver memory to be allocated as additional non-heap memory per driver " + +"process in cluster mode. This is memory that accounts for things like VM overheads, " + +"interned strings, other native overheads, etc. This tends to grow with the container " + +"size. This value defaults to 0.10 except for Kubernetes non-JVM jobs, which defaults to " + +"0.40. This is done as non-JVM tasks need more non-JVM heap space and such tasks " + +"commonly fail with \"Memory Overhead Exceeded\" errors. This preempts this error " + +"with a higher default. This value is ignored if spark.driver
[spark] branch master updated: [SPARK-38194] Followup: Fix k8s memory overhead passing to executor pods
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f36a5fb [SPARK-38194] Followup: Fix k8s memory overhead passing to executor pods f36a5fb is described below commit f36a5fb2b88620c1c490d087b0293c4e58d29979 Author: Adam Binford - Customer Site (Virginia) - CW 121796 AuthorDate: Thu Mar 17 18:32:29 2022 -0500 [SPARK-38194] Followup: Fix k8s memory overhead passing to executor pods ### What changes were proposed in this pull request? Follow up to https://github.com/apache/spark/pull/35504 to fix k8s memory overhead handling. ### Why are the changes needed? https://github.com/apache/spark/pull/35504 introduced a bug only caught by the K8S integration tests. ### Does this PR introduce _any_ user-facing change? Fix back to old behavior. ### How was this patch tested? See if IT passes Closes #35901 from Kimahriman/k8s-memory-overhead-executors. Authored-by: Adam Binford - Customer Site (Virginia) - CW 121796 Signed-off-by: Thomas Graves --- .../k8s/features/BasicDriverFeatureStep.scala | 30 +++- .../k8s/features/BasicDriverFeatureStepSuite.scala | 57 +- 2 files changed, 63 insertions(+), 24 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 9715149..413f5bc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -53,28 +53,32 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) // Memory settings private val driverMemoryMiB = conf.get(DRIVER_MEMORY) - private val memoryOverheadFactor = if (conf.contains(DRIVER_MEMORY_OVERHEAD_FACTOR)) { -conf.get(DRIVER_MEMORY_OVERHEAD_FACTOR) - } else { -conf.get(MEMORY_OVERHEAD_FACTOR) - } - // The memory overhead factor to use. If the user has not set it, then use a different - // value for non-JVM apps. This value is propagated to executors. - private val overheadFactor = + // The default memory overhead factor to use, derived from the deprecated + // `spark.kubernetes.memoryOverheadFactor` config or the default overhead values. + // If the user has not set it, then use a different default for non-JVM apps. This value is + // propagated to executors and used if the executor overhead factor is not set explicitly. + private val defaultOverheadFactor = if (conf.mainAppResource.isInstanceOf[NonJVMResource]) { - if (conf.contains(MEMORY_OVERHEAD_FACTOR) || conf.contains(DRIVER_MEMORY_OVERHEAD_FACTOR)) { -memoryOverheadFactor + if (conf.contains(MEMORY_OVERHEAD_FACTOR)) { +conf.get(MEMORY_OVERHEAD_FACTOR) } else { NON_JVM_MEMORY_OVERHEAD_FACTOR } } else { - memoryOverheadFactor + conf.get(MEMORY_OVERHEAD_FACTOR) } + // Prefer the driver memory overhead factor if set explicitly + private val memoryOverheadFactor = if (conf.contains(DRIVER_MEMORY_OVERHEAD_FACTOR)) { +conf.get(DRIVER_MEMORY_OVERHEAD_FACTOR) + } else { +defaultOverheadFactor + } + private val memoryOverheadMiB = conf .get(DRIVER_MEMORY_OVERHEAD) -.getOrElse(math.max((overheadFactor * driverMemoryMiB).toInt, +.getOrElse(math.max((memoryOverheadFactor * driverMemoryMiB).toInt, ResourceProfile.MEMORY_OVERHEAD_MIN_MIB)) private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB @@ -169,7 +173,7 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) KUBERNETES_DRIVER_POD_NAME.key -> driverPodName, "spark.app.id" -> conf.appId, KUBERNETES_DRIVER_SUBMIT_CHECK.key -> "true", - DRIVER_MEMORY_OVERHEAD_FACTOR.key -> overheadFactor.toString) + MEMORY_OVERHEAD_FACTOR.key -> defaultOverheadFactor.toString) // try upload local, resolvable files to a hadoop compatible file system Seq(JARS, FILES, ARCHIVES, SUBMIT_PYTHON_FILES).foreach { key => val uris = conf.get(key).filter(uri => KubernetesUtils.isLocalAndResolvable(uri)) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index d45f5f9..9a3b06a 100644 --- a/resource-managers/kubernetes/core/src/test/s
[spark] branch branch-3.3 updated: [SPARK-38194][YARN][MESOS][K8S] Make memory overhead factor configurable
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.3 by this push: new 8405ec3 [SPARK-38194][YARN][MESOS][K8S] Make memory overhead factor configurable 8405ec3 is described below commit 8405ec352dbed6a3199fc2af3c60fae7186d15b5 Author: Adam Binford AuthorDate: Wed Mar 16 10:54:18 2022 -0500 [SPARK-38194][YARN][MESOS][K8S] Make memory overhead factor configurable ### What changes were proposed in this pull request? Add a new config to set the memory overhead factor for drivers and executors. Currently the memory overhead is hard coded to 10% (except in Kubernetes), and the only way to set it higher is to set it to a specific memory amount. ### Why are the changes needed? In dynamic environments where different people or use cases need different memory requirements, it would be helpful to set a higher memory overhead factor instead of having to set a higher specific memory overhead value. The kubernetes resource manager already makes this configurable. This makes it configurable across the board. ### Does this PR introduce _any_ user-facing change? No change to default behavior, just adds a new config users can change. ### How was this patch tested? New UT to check the memory calculation. Closes #35504 from Kimahriman/yarn-configurable-memory-overhead-factor. Authored-by: Adam Binford Signed-off-by: Thomas Graves (cherry picked from commit 71e2110b799220adc107c9ac5ce737281f2b65cc) Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/SparkConf.scala| 4 +- .../org/apache/spark/internal/config/package.scala | 28 ++ docs/configuration.md | 30 ++- docs/running-on-kubernetes.md | 9 .../k8s/features/BasicDriverFeatureStep.scala | 13 +++-- .../k8s/features/BasicExecutorFeatureStep.scala| 7 ++- .../k8s/features/BasicDriverFeatureStepSuite.scala | 63 -- .../features/BasicExecutorFeatureStepSuite.scala | 54 +++ .../spark/deploy/rest/mesos/MesosRestServer.scala | 5 +- .../cluster/mesos/MesosSchedulerUtils.scala| 9 ++-- .../deploy/rest/mesos/MesosRestServerSuite.scala | 8 ++- .../org/apache/spark/deploy/yarn/Client.scala | 14 +++-- .../apache/spark/deploy/yarn/YarnAllocator.scala | 5 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 5 +- .../spark/deploy/yarn/YarnAllocatorSuite.scala | 29 ++ 15 files changed, 248 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 5f37a1a..cf12174 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -636,7 +636,9 @@ private[spark] object SparkConf extends Logging { DeprecatedConfig("spark.blacklist.killBlacklistedExecutors", "3.1.0", "Please use spark.excludeOnFailure.killExcludedExecutors"), DeprecatedConfig("spark.yarn.blacklist.executor.launch.blacklisting.enabled", "3.1.0", -"Please use spark.yarn.executor.launch.excludeOnFailure.enabled") +"Please use spark.yarn.executor.launch.excludeOnFailure.enabled"), + DeprecatedConfig("spark.kubernetes.memoryOverheadFactor", "3.3.0", +"Please use spark.driver.memoryOverheadFactor and spark.executor.memoryOverheadFactor") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index dbec61a..ffe4501 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -105,6 +105,22 @@ package object config { .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val DRIVER_MEMORY_OVERHEAD_FACTOR = +ConfigBuilder("spark.driver.memoryOverheadFactor") + .doc("Fraction of driver memory to be allocated as additional non-heap memory per driver " + +"process in cluster mode. This is memory that accounts for things like VM overheads, " + +"interned strings, other native overheads, etc. This tends to grow with the container " + +"size. This value defaults to 0.10 except for Kubernetes non-JVM jobs, which defaults to " + +"0.40. This is done as non-JVM tasks need more non-JVM heap space and such tasks " + +"commonly fail with \"Memor
[spark] branch master updated: [SPARK-38194][YARN][MESOS][K8S] Make memory overhead factor configurable
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 71e2110 [SPARK-38194][YARN][MESOS][K8S] Make memory overhead factor configurable 71e2110 is described below commit 71e2110b799220adc107c9ac5ce737281f2b65cc Author: Adam Binford AuthorDate: Wed Mar 16 10:54:18 2022 -0500 [SPARK-38194][YARN][MESOS][K8S] Make memory overhead factor configurable ### What changes were proposed in this pull request? Add a new config to set the memory overhead factor for drivers and executors. Currently the memory overhead is hard coded to 10% (except in Kubernetes), and the only way to set it higher is to set it to a specific memory amount. ### Why are the changes needed? In dynamic environments where different people or use cases need different memory requirements, it would be helpful to set a higher memory overhead factor instead of having to set a higher specific memory overhead value. The kubernetes resource manager already makes this configurable. This makes it configurable across the board. ### Does this PR introduce _any_ user-facing change? No change to default behavior, just adds a new config users can change. ### How was this patch tested? New UT to check the memory calculation. Closes #35504 from Kimahriman/yarn-configurable-memory-overhead-factor. Authored-by: Adam Binford Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/SparkConf.scala| 4 +- .../org/apache/spark/internal/config/package.scala | 28 ++ docs/configuration.md | 30 ++- docs/running-on-kubernetes.md | 9 .../k8s/features/BasicDriverFeatureStep.scala | 13 +++-- .../k8s/features/BasicExecutorFeatureStep.scala| 7 ++- .../k8s/features/BasicDriverFeatureStepSuite.scala | 63 -- .../features/BasicExecutorFeatureStepSuite.scala | 54 +++ .../spark/deploy/rest/mesos/MesosRestServer.scala | 5 +- .../cluster/mesos/MesosSchedulerUtils.scala| 9 ++-- .../deploy/rest/mesos/MesosRestServerSuite.scala | 8 ++- .../org/apache/spark/deploy/yarn/Client.scala | 14 +++-- .../apache/spark/deploy/yarn/YarnAllocator.scala | 5 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 5 +- .../spark/deploy/yarn/YarnAllocatorSuite.scala | 29 ++ 15 files changed, 248 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 5f37a1a..cf12174 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -636,7 +636,9 @@ private[spark] object SparkConf extends Logging { DeprecatedConfig("spark.blacklist.killBlacklistedExecutors", "3.1.0", "Please use spark.excludeOnFailure.killExcludedExecutors"), DeprecatedConfig("spark.yarn.blacklist.executor.launch.blacklisting.enabled", "3.1.0", -"Please use spark.yarn.executor.launch.excludeOnFailure.enabled") +"Please use spark.yarn.executor.launch.excludeOnFailure.enabled"), + DeprecatedConfig("spark.kubernetes.memoryOverheadFactor", "3.3.0", +"Please use spark.driver.memoryOverheadFactor and spark.executor.memoryOverheadFactor") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index dbec61a..ffe4501 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -105,6 +105,22 @@ package object config { .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val DRIVER_MEMORY_OVERHEAD_FACTOR = +ConfigBuilder("spark.driver.memoryOverheadFactor") + .doc("Fraction of driver memory to be allocated as additional non-heap memory per driver " + +"process in cluster mode. This is memory that accounts for things like VM overheads, " + +"interned strings, other native overheads, etc. This tends to grow with the container " + +"size. This value defaults to 0.10 except for Kubernetes non-JVM jobs, which defaults to " + +"0.40. This is done as non-JVM tasks need more non-JVM heap space and such tasks " + +"commonly fail with \"Memory Overhead Exceeded\" errors. This preempts this error " + +"with a higher default. This value is igno
[spark] branch master updated: [SPARK-36540][YARN] YARN-CLIENT mode should check Shutdown message when AMEndpoint disconencted
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 20051eb [SPARK-36540][YARN] YARN-CLIENT mode should check Shutdown message when AMEndpoint disconencted 20051eb is described below commit 20051eb69904de6afc27fe5adb18bcc760c78701 Author: Angerszh AuthorDate: Mon Oct 11 08:24:49 2021 -0500 [SPARK-36540][YARN] YARN-CLIENT mode should check Shutdown message when AMEndpoint disconencted ### What changes were proposed in this pull request? We meet a case AM lose connection ``` 21/08/18 02:14:15 ERROR TransportRequestHandler: Error sending result RpcResponse{requestId=5675952834716124039, body=NioManagedBuffer{buf=java.nio.HeapByteBuffer[pos=0 lim=47 cap=64]}} to xx.xx.xx.xx:41420; closing connection java.nio.channels.ClosedChannelException at io.netty.channel.AbstractChannel$AbstractUnsafe.newClosedChannelException(AbstractChannel.java:957) at io.netty.channel.AbstractChannel$AbstractUnsafe.write(AbstractChannel.java:865) at io.netty.channel.DefaultChannelPipeline$HeadContext.write(DefaultChannelPipeline.java:1367) at io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:717) at io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(AbstractChannelHandlerContext.java:764) at io.netty.channel.AbstractChannelHandlerContext$WriteTask.run(AbstractChannelHandlerContext.java:1104) at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164) at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:500) at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989) at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:748) ``` Check the code about client, when AMEndpoint disconnected, will finish Application with SUCCESS final status ``` override def onDisconnected(remoteAddress: RpcAddress): Unit = { // In cluster mode or unmanaged am case, do not rely on the disassociated event to exit // This avoids potentially reporting incorrect exit codes if the driver fails if (!(isClusterMode || sparkConf.get(YARN_UNMANAGED_AM))) { logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress") finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) } } ``` Normally say in client mode, when application success, driver will stop and AM loss connection, it's ok that exit with SUCCESS, but if there is a not work problem cause disconnected. Still finish with final status is not correct. Then YarnClientSchedulerBackend will receive application report with final status with success and stop SparkContext cause application failed but mark it as a normal stop. ``` private class MonitorThread extends Thread { private var allowInterrupt = true override def run() { try { val YarnAppReport(_, state, diags) = client.monitorApplication(appId.get, logApplicationReport = false) logError(s"YARN application has exited unexpectedly with state $state! " + "Check the YARN application logs for more details.") diags.foreach { err => logError(s"Diagnostics message: $err") } allowInterrupt = false sc.stop() } catch { case e: InterruptedException => logInfo("Interrupting monitor thread") } } def stopMonitor(): Unit = { if (allowInterrupt) { this.interrupt() } ``` IMO, we should send a `Shutdown` message to yarn client mode AM to make sure the shut down case ### Why are the changes needed? Fix bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Closes #33780 from AngersZh/SPARK-36540. Authored-by: Angerszh Signed-off-by: Thomas Graves --- docs/running-on-yarn.md | 13 + .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 15 +-- .../main/scala/org/apache/spark/deploy/yarn/config.scala | 15 +++ .../scheduler/cluster/YarnClientSchedulerBackend.scala|
[spark] branch master updated: [SPARK-36624][YARN] In yarn client mode, when ApplicationMaster failed with KILLED/FAILED, driver should exit with code not 0
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new dcada3d [SPARK-36624][YARN] In yarn client mode, when ApplicationMaster failed with KILLED/FAILED, driver should exit with code not 0 dcada3d is described below commit dcada3d48c51f4855c600dc254883bd9eb3a0a1c Author: Angerszh AuthorDate: Wed Sep 29 11:12:01 2021 -0500 [SPARK-36624][YARN] In yarn client mode, when ApplicationMaster failed with KILLED/FAILED, driver should exit with code not 0 ### What changes were proposed in this pull request? In current code for yarn client mode, even when use use `yarn application -kill` to kill the application, driver side still exit with code 0. This behavior make job scheduler can't know the job is not success. and user don't know too. In this case we should exit program with a non 0 code. ### Why are the changes needed? Make scheduler/user more clear about application's status ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Closes #33873 from AngersZh/SPDI-36624. Authored-by: Angerszh Signed-off-by: Thomas Graves --- docs/running-on-yarn.md | 10 ++ .../src/main/scala/org/apache/spark/deploy/yarn/config.scala | 11 +++ .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 10 +- 3 files changed, 30 insertions(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9930f3e..37ff479 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -442,6 +442,16 @@ To use a custom metrics.properties for the application master and executors, upd 1.6.0 + spark.yarn.am.clientModeExitOnError + false + + In yarn-client mode, when this is true, if driver got application report with final status of KILLED or FAILED, + driver will stop corresponding SparkContext and exit program with code 1. + Note, if this is true and called from another application, it will terminate the parent application as well. + + 3.3.0 + + spark.yarn.executor.failuresValidityInterval (none) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index 89a4af2..ab2063c 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -52,6 +52,17 @@ package object config extends Logging { .timeConf(TimeUnit.MILLISECONDS) .createOptional + private[spark] val AM_CLIENT_MODE_EXIT_ON_ERROR = +ConfigBuilder("spark.yarn.am.clientModeExitOnError") + .doc("In yarn-client mode, when this is true, if driver got " + +"application report with final status of KILLED or FAILED, " + +"driver will stop corresponding SparkContext and exit program with code 1. " + +"Note, if this is true and called from another application, it will terminate " + +"the parent application as well.") + .version("3.3.0") + .booleanConf + .createWithDefault(false) + private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS = ConfigBuilder("spark.yarn.executor.failuresValidityInterval") .doc("Interval after which Executor failures will be considered independent and not " + diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 8a55e61..28c8652 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -21,7 +21,7 @@ import java.io.InterruptedIOException import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.yarn.api.records.YarnApplicationState +import org.apache.hadoop.yarn.api.records.{FinalApplicationStatus, YarnApplicationState} import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnAppReport} @@ -122,6 +122,14 @@ private[spark] class YarnClientSchedulerBackend( } allowInterrupt = false sc.stop() +state match { + case FinalApplicationStatus.FAILED | FinalApplicationStatus.KILLED +if conf.get(AM_CLIENT_MODE_EXIT_ON_ERROR) => +logWarning(s"ApplicationMaster finis
[spark] branch branch-3.2 updated: [SPARK-595][DOCS] Add local-cluster mode option in Documentation
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new a5d0eaf [SPARK-595][DOCS] Add local-cluster mode option in Documentation a5d0eaf is described below commit a5d0eafa324279e4516bd4c6b544b0cc7dbbd4e3 Author: Yuto Akutsu AuthorDate: Fri Aug 6 09:26:13 2021 -0500 [SPARK-595][DOCS] Add local-cluster mode option in Documentation ### What changes were proposed in this pull request? Add local-cluster mode option to submitting-applications.md ### Why are the changes needed? Help users to find/use this option for unit tests. ### Does this PR introduce _any_ user-facing change? Yes, docs changed. ### How was this patch tested? `SKIP_API=1 bundle exec jekyll build` https://user-images.githubusercontent.com/87687356/127125380-6beb4601-7cf4-4876-b2c6-459454ce2a02.png";> Closes #33537 from yutoacts/SPARK-595. Lead-authored-by: Yuto Akutsu Co-authored-by: Yuto Akutsu Co-authored-by: Yuto Akutsu <87687356+yutoa...@users.noreply.github.com> Signed-off-by: Thomas Graves (cherry picked from commit 41b011e416286374e2e8e8dea36ba79f4c403040) Signed-off-by: Thomas Graves --- docs/submitting-applications.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 0319859..402dd06 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -162,9 +162,10 @@ The master URL passed to Spark can be in one of the following formats: Master URLMeaning local Run Spark locally with one worker thread (i.e. no parallelism at all). local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). - local[K,F] Run Spark locally with K worker threads and F maxFailures (see spark.task.maxFailures for an explanation of this variable) + local[K,F] Run Spark locally with K worker threads and F maxFailures (see spark.task.maxFailures for an explanation of this variable). local[*] Run Spark locally with as many worker threads as logical cores on your machine. local[*,F] Run Spark locally with as many worker threads as logical cores on your machine and F maxFailures. + local-cluster[N,C,M] Local-cluster mode is only for unit tests. It emulates a distributed cluster in a single JVM with N number of workers, C cores per worker and M MiB of memory per worker. spark://HOST:PORT Connect to the given Spark standalone cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-595][DOCS] Add local-cluster mode option in Documentation
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 41b011e [SPARK-595][DOCS] Add local-cluster mode option in Documentation 41b011e is described below commit 41b011e416286374e2e8e8dea36ba79f4c403040 Author: Yuto Akutsu AuthorDate: Fri Aug 6 09:26:13 2021 -0500 [SPARK-595][DOCS] Add local-cluster mode option in Documentation ### What changes were proposed in this pull request? Add local-cluster mode option to submitting-applications.md ### Why are the changes needed? Help users to find/use this option for unit tests. ### Does this PR introduce _any_ user-facing change? Yes, docs changed. ### How was this patch tested? `SKIP_API=1 bundle exec jekyll build` https://user-images.githubusercontent.com/87687356/127125380-6beb4601-7cf4-4876-b2c6-459454ce2a02.png";> Closes #33537 from yutoacts/SPARK-595. Lead-authored-by: Yuto Akutsu Co-authored-by: Yuto Akutsu Co-authored-by: Yuto Akutsu <87687356+yutoa...@users.noreply.github.com> Signed-off-by: Thomas Graves --- docs/submitting-applications.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 0319859..402dd06 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -162,9 +162,10 @@ The master URL passed to Spark can be in one of the following formats: Master URLMeaning local Run Spark locally with one worker thread (i.e. no parallelism at all). local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). - local[K,F] Run Spark locally with K worker threads and F maxFailures (see spark.task.maxFailures for an explanation of this variable) + local[K,F] Run Spark locally with K worker threads and F maxFailures (see spark.task.maxFailures for an explanation of this variable). local[*] Run Spark locally with as many worker threads as logical cores on your machine. local[*,F] Run Spark locally with as many worker threads as logical cores on your machine and F maxFailures. + local-cluster[N,C,M] Local-cluster mode is only for unit tests. It emulates a distributed cluster in a single JVM with N number of workers, C cores per worker and M MiB of memory per worker. spark://HOST:PORT Connect to the given Spark standalone cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.2 updated: [SPARK-35881][SQL] Add support for columnar execution of final query stage in AdaptiveSparkPlanExec
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new f9f5656 [SPARK-35881][SQL] Add support for columnar execution of final query stage in AdaptiveSparkPlanExec f9f5656 is described below commit f9f5656491c1edbfbc9f8c13840aaa935c49037f Author: Andy Grove AuthorDate: Fri Jul 30 13:21:50 2021 -0500 [SPARK-35881][SQL] Add support for columnar execution of final query stage in AdaptiveSparkPlanExec ### What changes were proposed in this pull request? Changes in this PR: - `AdaptiveSparkPlanExec` has new methods `finalPlanSupportsColumnar` and `doExecuteColumnar` to support adaptive queries where the final query stage produces columnar data. - `SessionState` now has a new set of injectable rules named `finalQueryStagePrepRules` that can be applied to the final query stage. - `AdaptiveSparkPlanExec` can now safely be wrapped by either `RowToColumnarExec` or `ColumnarToRowExec`. A Spark plugin can use the new rules to remove the root `ColumnarToRowExec` transition that is inserted by previous rules and at execution time can call `finalPlanSupportsColumnar` to see if the final query stage is columnar. If the plan is columnar then the plugin can safely call `doExecuteColumnar`. The adaptive plan can be wrapped in either `RowToColumnarExec` or `ColumnarToRowExec` to force a particular output format. There are fast paths in both of these operators to avoid any re [...] ### Why are the changes needed? Without this change it is necessary to use reflection to get the final physical plan to determine whether it is columnar and to execute it is a columnar plan. `AdaptiveSparkPlanExec` only provides public methods for row-based execution. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I have manually tested this patch with the RAPIDS Accelerator for Apache Spark. Closes #33140 from andygrove/support-columnar-adaptive. Authored-by: Andy Grove Signed-off-by: Thomas Graves (cherry picked from commit 0f538402fb76e4d6182cc881219d53b5fdf73af1) Signed-off-by: Thomas Graves --- .../apache/spark/sql/SparkSessionExtensions.scala | 22 +++- .../org/apache/spark/sql/execution/Columnar.scala | 134 - .../execution/adaptive/AdaptiveSparkPlanExec.scala | 46 +-- .../sql/internal/BaseSessionStateBuilder.scala | 7 +- .../apache/spark/sql/internal/SessionState.scala | 3 +- 5 files changed, 141 insertions(+), 71 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index b14dce6..18ebae5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -47,6 +47,7 @@ import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} * (External) Catalog listeners. * Columnar Rules. * Adaptive Query Stage Preparation Rules. + * Adaptive Query Post Stage Preparation Rules. * * * The extensions can be used by calling `withExtensions` on the [[SparkSession.Builder]], for @@ -110,9 +111,12 @@ class SparkSessionExtensions { type TableFunctionDescription = (FunctionIdentifier, ExpressionInfo, TableFunctionBuilder) type ColumnarRuleBuilder = SparkSession => ColumnarRule type QueryStagePrepRuleBuilder = SparkSession => Rule[SparkPlan] + type PostStageCreationRuleBuilder = SparkSession => Rule[SparkPlan] private[this] val columnarRuleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder] private[this] val queryStagePrepRuleBuilders = mutable.Buffer.empty[QueryStagePrepRuleBuilder] + private[this] val postStageCreationRuleBuilders = +mutable.Buffer.empty[PostStageCreationRuleBuilder] /** * Build the override rules for columnar execution. @@ -129,6 +133,14 @@ class SparkSessionExtensions { } /** + * Build the override rules for the final query stage preparation phase of adaptive query + * execution. + */ + private[sql] def buildPostStageCreationRules(session: SparkSession): Seq[Rule[SparkPlan]] = { +postStageCreationRuleBuilders.map(_.apply(session)).toSeq + } + + /** * Inject a rule that can override the columnar execution of an executor. */ def injectColumnar(builder: ColumnarRuleBuilder): Unit = { @@ -136,13 +148,21 @@ class SparkSessionExtensions { } /** - * Inject a rule that can override the the query stage preparation phase of adaptive query + * Inject a rule that can override the query stage preparation phase of adaptive query * execution. */ def injectQueryStagePrepRul
[spark] branch master updated: [SPARK-35881][SQL] Add support for columnar execution of final query stage in AdaptiveSparkPlanExec
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0f53840 [SPARK-35881][SQL] Add support for columnar execution of final query stage in AdaptiveSparkPlanExec 0f53840 is described below commit 0f538402fb76e4d6182cc881219d53b5fdf73af1 Author: Andy Grove AuthorDate: Fri Jul 30 13:21:50 2021 -0500 [SPARK-35881][SQL] Add support for columnar execution of final query stage in AdaptiveSparkPlanExec ### What changes were proposed in this pull request? Changes in this PR: - `AdaptiveSparkPlanExec` has new methods `finalPlanSupportsColumnar` and `doExecuteColumnar` to support adaptive queries where the final query stage produces columnar data. - `SessionState` now has a new set of injectable rules named `finalQueryStagePrepRules` that can be applied to the final query stage. - `AdaptiveSparkPlanExec` can now safely be wrapped by either `RowToColumnarExec` or `ColumnarToRowExec`. A Spark plugin can use the new rules to remove the root `ColumnarToRowExec` transition that is inserted by previous rules and at execution time can call `finalPlanSupportsColumnar` to see if the final query stage is columnar. If the plan is columnar then the plugin can safely call `doExecuteColumnar`. The adaptive plan can be wrapped in either `RowToColumnarExec` or `ColumnarToRowExec` to force a particular output format. There are fast paths in both of these operators to avoid any re [...] ### Why are the changes needed? Without this change it is necessary to use reflection to get the final physical plan to determine whether it is columnar and to execute it is a columnar plan. `AdaptiveSparkPlanExec` only provides public methods for row-based execution. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I have manually tested this patch with the RAPIDS Accelerator for Apache Spark. Closes #33140 from andygrove/support-columnar-adaptive. Authored-by: Andy Grove Signed-off-by: Thomas Graves --- .../apache/spark/sql/SparkSessionExtensions.scala | 22 +++- .../org/apache/spark/sql/execution/Columnar.scala | 134 - .../execution/adaptive/AdaptiveSparkPlanExec.scala | 46 +-- .../sql/internal/BaseSessionStateBuilder.scala | 7 +- .../apache/spark/sql/internal/SessionState.scala | 3 +- 5 files changed, 141 insertions(+), 71 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index b14dce6..18ebae5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -47,6 +47,7 @@ import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} * (External) Catalog listeners. * Columnar Rules. * Adaptive Query Stage Preparation Rules. + * Adaptive Query Post Stage Preparation Rules. * * * The extensions can be used by calling `withExtensions` on the [[SparkSession.Builder]], for @@ -110,9 +111,12 @@ class SparkSessionExtensions { type TableFunctionDescription = (FunctionIdentifier, ExpressionInfo, TableFunctionBuilder) type ColumnarRuleBuilder = SparkSession => ColumnarRule type QueryStagePrepRuleBuilder = SparkSession => Rule[SparkPlan] + type PostStageCreationRuleBuilder = SparkSession => Rule[SparkPlan] private[this] val columnarRuleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder] private[this] val queryStagePrepRuleBuilders = mutable.Buffer.empty[QueryStagePrepRuleBuilder] + private[this] val postStageCreationRuleBuilders = +mutable.Buffer.empty[PostStageCreationRuleBuilder] /** * Build the override rules for columnar execution. @@ -129,6 +133,14 @@ class SparkSessionExtensions { } /** + * Build the override rules for the final query stage preparation phase of adaptive query + * execution. + */ + private[sql] def buildPostStageCreationRules(session: SparkSession): Seq[Rule[SparkPlan]] = { +postStageCreationRuleBuilders.map(_.apply(session)).toSeq + } + + /** * Inject a rule that can override the columnar execution of an executor. */ def injectColumnar(builder: ColumnarRuleBuilder): Unit = { @@ -136,13 +148,21 @@ class SparkSessionExtensions { } /** - * Inject a rule that can override the the query stage preparation phase of adaptive query + * Inject a rule that can override the query stage preparation phase of adaptive query * execution. */ def injectQueryStagePrepRule(builder: QueryStagePrepRuleBuilder): Unit = { queryStagePrepRuleBuilders += builder } + /** + * Inject a
[spark] branch master updated: [SPARK-35672][CORE][YARN] Pass user classpath entries to executors using config instead of command line
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 866df69 [SPARK-35672][CORE][YARN] Pass user classpath entries to executors using config instead of command line 866df69 is described below commit 866df69c6290b2f8e2726f1325969d23c938c0f2 Author: Erik Krogen AuthorDate: Fri Jun 25 08:53:57 2021 -0500 [SPARK-35672][CORE][YARN] Pass user classpath entries to executors using config instead of command line ### What changes were proposed in this pull request? Refactor the logic for constructing the user classpath from `yarn.ApplicationMaster` into `yarn.Client` so that it can be leveraged on the executor side as well, instead of having the driver construct it and pass it to the executor via command-line arguments. A new method, `getUserClassPath`, is added to `CoarseGrainedExecutorBackend` which defaults to `Nil` (consistent with the existing behavior where non-YARN resource managers do not configure the user classpath). `YarnCoarseGrained [...] ### Why are the changes needed? User-provided JARs are made available to executors using a custom classloader, so they do not appear on the standard Java classpath. Instead, they are passed as a list to the executor which then creates a classloader out of the URLs. Currently in the case of YARN, this list of JARs is crafted by the Driver (in `ExecutorRunnable`), which then passes the information to the executors (`CoarseGrainedExecutorBackend`) by specifying each JAR on the executor command line as `--user-class-pat [...] > /bin/bash: Argument list too long A [Google search](https://www.google.com/search?q=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22&oq=spark%20%22%2Fbin%2Fbash%3A%20argument%20list%20too%20long%22) indicates that this is not a theoretical problem and afflicts real users, including ours. Passing this list using the configurations instead resolves this issue. ### Does this PR introduce _any_ user-facing change? No, except for fixing the bug, allowing for larger JAR lists to be passed successfully. Configuration of JARs is identical to before. ### How was this patch tested? New unit tests were added in `YarnClusterSuite`. Also, we have been running a similar fix internally for 4 months with great success. Closes #32810 from xkrogen/xkrogen-SPARK-35672-classpath-scalable. Authored-by: Erik Krogen Signed-off-by: Thomas Graves --- .../executor/CoarseGrainedExecutorBackend.scala| 17 +++--- .../scala/org/apache/spark/executor/Executor.scala | 2 + .../CoarseGrainedExecutorBackendSuite.scala| 17 +++--- .../spark/deploy/yarn/ApplicationMaster.scala | 9 ++-- .../org/apache/spark/deploy/yarn/Client.scala | 32 +-- .../spark/deploy/yarn/ExecutorRunnable.scala | 12 - .../YarnCoarseGrainedExecutorBackend.scala | 8 +-- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 35 .../spark/deploy/yarn/YarnClusterSuite.scala | 63 ++ 9 files changed, 142 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 8568fdd..f8ff27d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -52,7 +52,6 @@ private[spark] class CoarseGrainedExecutorBackend( bindAddress: String, hostname: String, cores: Int, -userClassPath: Seq[URL], env: SparkEnv, resourcesFileOpt: Option[String], resourceProfile: ResourceProfile) @@ -124,7 +123,7 @@ private[spark] class CoarseGrainedExecutorBackend( */ private def createClassLoader(): MutableURLClassLoader = { val currentLoader = Utils.getContextOrSparkClassLoader -val urls = userClassPath.toArray +val urls = getUserClassPath.toArray if (env.conf.get(EXECUTOR_USER_CLASS_PATH_FIRST)) { new ChildFirstURLClassLoader(urls, currentLoader) } else { @@ -149,6 +148,8 @@ private[spark] class CoarseGrainedExecutorBackend( } } + def getUserClassPath: Seq[URL] = Nil + def extractLogUrls: Map[String, String] = { val prefix = "SPARK_LOG_URL_" sys.env.filterKeys(_.startsWith(prefix)) @@ -165,7 +166,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") try { -executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false, +executor = new Executor(executorId, hostname, env, getUserClassPath, isLocal
[spark] branch branch-3.1 updated: [SPARK-35391] Fix memory leak in ExecutorAllocationListener
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.1 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.1 by this push: new a415df3 [SPARK-35391] Fix memory leak in ExecutorAllocationListener a415df3 is described below commit a415df341a3d5f4e895c0ca186dd4ffeccc777da Author: Vasily Kolpakov AuthorDate: Mon Jun 21 08:23:20 2021 -0500 [SPARK-35391] Fix memory leak in ExecutorAllocationListener ### What changes were proposed in this pull request? This PR fixes a memory leak in ExecutorAllocationListener. ### Why are the changes needed? Dynamic allocation stops working under high load (~100 tasks/s, ~5 stages/s) in long-lived (~10 days) spark applications. This PR addresses the problem. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual tests. The patch fixed dynamic allocation in production cluster. Closes #32526 from VasilyKolpakov/SPARK-35391_fix_ExecutorAllocationListener. Authored-by: Vasily Kolpakov Signed-off-by: Thomas Graves (cherry picked from commit 844f10c7426a76fb29ee91223c8af43825e147c5) Signed-off-by: Thomas Graves --- .../apache/spark/ExecutorAllocationManager.scala | 40 ++ 1 file changed, 25 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index bdb768e..f2078f4 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -736,6 +736,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToTaskIndices -= stageAttempt stageAttemptToSpeculativeTaskIndices -= stageAttempt stageAttemptToExecutorPlacementHints -= stageAttempt +removeStageFromResourceProfileIfUnused(stageAttempt) // Update the executor placement hints updateExecutorPlacementHints() @@ -780,20 +781,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask(stageAttempt) -= 1 if (stageAttemptToNumRunningTask(stageAttempt) == 0) { stageAttemptToNumRunningTask -= stageAttempt -if (!stageAttemptToNumTasks.contains(stageAttempt)) { - val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => -v.contains(stageAttempt) - }.keys - if (rpForStage.size == 1) { -// be careful about the removal from here due to late tasks, make sure stage is -// really complete and no tasks left -resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt - } else { -logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + - s" but have $rpForStage") - } -} - +removeStageFromResourceProfileIfUnused(stageAttempt) } } if (taskEnd.taskInfo.speculative) { @@ -858,6 +846,28 @@ private[spark] class ExecutorAllocationManager( allocationManager.synchronized { // Clear unschedulableTaskSets since atleast one task becomes schedulable now unschedulableTaskSets.remove(stageAttempt) +removeStageFromResourceProfileIfUnused(stageAttempt) + } +} + +def removeStageFromResourceProfileIfUnused(stageAttempt: StageAttempt): Unit = { + if (!stageAttemptToNumRunningTask.contains(stageAttempt) && + !stageAttemptToNumTasks.contains(stageAttempt) && + !stageAttemptToNumSpeculativeTasks.contains(stageAttempt) && + !stageAttemptToTaskIndices.contains(stageAttempt) && + !stageAttemptToSpeculativeTaskIndices.contains(stageAttempt) + ) { +val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => + v.contains(stageAttempt) +}.keys +if (rpForStage.size == 1) { + // be careful about the removal from here due to late tasks, make sure stage is + // really complete and no tasks left + resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt +} else { + logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + + s" but have $rpForStage") +} } } @@ -931,7 +941,7 @@ private[spark] class ExecutorAllocationManager( val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq // attempts is a Set, change to Seq so we keep all values attempts.map { attempt => -stageAttemptToNumRunningTask.getOrElseUpdate(att
[spark] branch master updated: [SPARK-35391] Fix memory leak in ExecutorAllocationListener
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 844f10c [SPARK-35391] Fix memory leak in ExecutorAllocationListener 844f10c is described below commit 844f10c7426a76fb29ee91223c8af43825e147c5 Author: Vasily Kolpakov AuthorDate: Mon Jun 21 08:23:20 2021 -0500 [SPARK-35391] Fix memory leak in ExecutorAllocationListener ### What changes were proposed in this pull request? This PR fixes a memory leak in ExecutorAllocationListener. ### Why are the changes needed? Dynamic allocation stops working under high load (~100 tasks/s, ~5 stages/s) in long-lived (~10 days) spark applications. This PR addresses the problem. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual tests. The patch fixed dynamic allocation in production cluster. Closes #32526 from VasilyKolpakov/SPARK-35391_fix_ExecutorAllocationListener. Authored-by: Vasily Kolpakov Signed-off-by: Thomas Graves --- .../apache/spark/ExecutorAllocationManager.scala | 40 ++ 1 file changed, 25 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 779559b..c4b6193 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -737,6 +737,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToTaskIndices -= stageAttempt stageAttemptToSpeculativeTaskIndices -= stageAttempt stageAttemptToExecutorPlacementHints -= stageAttempt +removeStageFromResourceProfileIfUnused(stageAttempt) // Update the executor placement hints updateExecutorPlacementHints() @@ -781,20 +782,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask(stageAttempt) -= 1 if (stageAttemptToNumRunningTask(stageAttempt) == 0) { stageAttemptToNumRunningTask -= stageAttempt -if (!stageAttemptToNumTasks.contains(stageAttempt)) { - val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => -v.contains(stageAttempt) - }.keys - if (rpForStage.size == 1) { -// be careful about the removal from here due to late tasks, make sure stage is -// really complete and no tasks left -resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt - } else { -logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + - s" but have $rpForStage") - } -} - +removeStageFromResourceProfileIfUnused(stageAttempt) } } if (taskEnd.taskInfo.speculative) { @@ -859,6 +847,28 @@ private[spark] class ExecutorAllocationManager( allocationManager.synchronized { // Clear unschedulableTaskSets since atleast one task becomes schedulable now unschedulableTaskSets.remove(stageAttempt) +removeStageFromResourceProfileIfUnused(stageAttempt) + } +} + +def removeStageFromResourceProfileIfUnused(stageAttempt: StageAttempt): Unit = { + if (!stageAttemptToNumRunningTask.contains(stageAttempt) && + !stageAttemptToNumTasks.contains(stageAttempt) && + !stageAttemptToNumSpeculativeTasks.contains(stageAttempt) && + !stageAttemptToTaskIndices.contains(stageAttempt) && + !stageAttemptToSpeculativeTaskIndices.contains(stageAttempt) + ) { +val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => + v.contains(stageAttempt) +}.keys +if (rpForStage.size == 1) { + // be careful about the removal from here due to late tasks, make sure stage is + // really complete and no tasks left + resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt +} else { + logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + + s" but have $rpForStage") +} } } @@ -920,7 +930,7 @@ private[spark] class ExecutorAllocationManager( val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq // attempts is a Set, change to Seq so we keep all values attempts.map { attempt => -stageAttemptToNumRunningTask.getOrElseUpdate(attempt, 0) +stageAttemptToNumRunningTask.getOrElse(attempt, 0) }.sum } -
[spark] branch master updated: [SPARK-35074][CORE] hardcoded configs move to config package
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6c3b7f9 [SPARK-35074][CORE] hardcoded configs move to config package 6c3b7f9 is described below commit 6c3b7f92cfaf4d11c8c9c984082ea40bd1f86abd Author: dgd-contributor AuthorDate: Mon Jun 7 09:55:03 2021 -0500 [SPARK-35074][CORE] hardcoded configs move to config package ### What changes were proposed in this pull request? Currently spark.jars.xxx property keys (e.g. spark.jars.ivySettings and spark.jars.packages) are hardcoded in multiple places within Spark code across multiple modules. We should define them in config/package.scala and reference them in all other places. ### Why are the changes needed? improvement ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? no Closes #32746 from dgd-contributor/SPARK-35074_configs_should_be_moved_to_config_package.scala. Authored-by: dgd-contributor Signed-off-by: Thomas Graves --- .../org/apache/spark/deploy/SparkSubmit.scala | 15 +++--- .../apache/spark/deploy/SparkSubmitArguments.scala | 14 +++--- .../org/apache/spark/internal/config/package.scala | 56 ++ .../org/apache/spark/util/DependencyUtils.scala| 11 +++-- 4 files changed, 77 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 568bcf9..a65be54 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -588,7 +588,8 @@ private[spark] class SparkSubmit extends Logging { OptionAssigner(args.deployMode, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = SUBMIT_DEPLOY_MODE.key), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = "spark.app.name"), - OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, confKey = "spark.jars.ivy"), + OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, +confKey = JAR_IVY_REPO_PATH.key), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, confKey = DRIVER_MEMORY.key), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, @@ -605,13 +606,13 @@ private[spark] class SparkSubmit extends Logging { // Propagate attributes for dependency resolution at the driver side OptionAssigner(args.packages, STANDALONE | MESOS | KUBERNETES, -CLUSTER, confKey = "spark.jars.packages"), +CLUSTER, confKey = JAR_PACKAGES.key), OptionAssigner(args.repositories, STANDALONE | MESOS | KUBERNETES, -CLUSTER, confKey = "spark.jars.repositories"), +CLUSTER, confKey = JAR_REPOSITORIES.key), OptionAssigner(args.ivyRepoPath, STANDALONE | MESOS | KUBERNETES, -CLUSTER, confKey = "spark.jars.ivy"), +CLUSTER, confKey = JAR_IVY_REPO_PATH.key), OptionAssigner(args.packagesExclusions, STANDALONE | MESOS | KUBERNETES, -CLUSTER, confKey = "spark.jars.excludes"), +CLUSTER, confKey = JAR_PACKAGES_EXCLUSIONS.key), // Yarn only OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.queue"), @@ -646,7 +647,7 @@ private[spark] class SparkSubmit extends Logging { confKey = DRIVER_CORES.key), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, confKey = DRIVER_SUPERVISE.key), - OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, confKey = "spark.jars.ivy"), + OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, confKey = JAR_IVY_REPO_PATH.key), // An internal option used only for spark-shell to add user jars to repl's classloader, // previously it uses "spark.jars" or "spark.yarn.dist.jars" which now may be pointed to @@ -1299,7 +1300,7 @@ private[spark] object SparkSubmitUtils extends Logging { val file = Option(uri.getScheme).getOrElse("file") match { case "file" => new File(uri.getPath) case scheme => throw new IllegalArgumentException(s"Scheme $scheme not supported in " + -"spark.jars.ivySettings") +JAR_IVY_SETTING_PATH.key) } require(file.exists(), s"Ivy settings file $file does not exist") require(file.isFile(), s"Ivy settings file $file is not a normal file") diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 692e7ea..47fbab5 100644 --- a/core/src/main
[spark] branch branch-3.0 updated: [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 1606791 [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use 1606791 is described below commit 1606791bf54d785eca1ae42fbd98e68a1d58e884 Author: Andy Grove AuthorDate: Wed May 19 07:45:26 2021 -0500 [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use ### What changes were proposed in this pull request? AQE has an optimization where it attempts to reuse compatible exchanges but it does not take into account whether the exchanges are columnar or not, resulting in incorrect reuse under some circumstances. This PR simply changes the key used to lookup cached stages. It now uses the canonicalized form of the new query stage (potentially created by a plugin) rather than using the canonicalized form of the original exchange. ### Why are the changes needed? When using the [RAPIDS Accelerator for Apache Spark](https://github.com/NVIDIA/spark-rapids) we sometimes see a new query stage correctly create a row-based exchange and then Spark replaces it with a cached columnar exchange, which is not compatible, and this causes queries to fail. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The patch has been tested with the query that highlighted this issue. I looked at writing unit tests for this but it would involve implementing a mock columnar exchange in the tests so would be quite a bit of work. If anyone has ideas on other ways to test this I am happy to hear them. Closes #32195 from andygrove/SPARK-35093. Authored-by: Andy Grove Signed-off-by: Thomas Graves (cherry picked from commit 52e3cf9ff50b4209e29cb06df09b1ef3a18bc83b) Signed-off-by: Thomas Graves --- .../apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala| 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 187827c..e7a8034 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -349,7 +349,8 @@ case class AdaptiveSparkPlanExec( // Check the `stageCache` again for reuse. If a match is found, ditch the new stage // and reuse the existing stage found in the `stageCache`, otherwise update the // `stageCache` with the new stage. - val queryStage = context.stageCache.getOrElseUpdate(e.canonicalized, newStage) + val queryStage = context.stageCache.getOrElseUpdate( +newStage.plan.canonicalized, newStage) if (queryStage.ne(newStage)) { newStage = reuseQueryStage(queryStage, e) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.1 updated: [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.1 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.1 by this push: new 73abceb [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use 73abceb is described below commit 73abceb05d64eafeb39866c69a84d0b7f3c1f097 Author: Andy Grove AuthorDate: Wed May 19 07:45:26 2021 -0500 [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use ### What changes were proposed in this pull request? AQE has an optimization where it attempts to reuse compatible exchanges but it does not take into account whether the exchanges are columnar or not, resulting in incorrect reuse under some circumstances. This PR simply changes the key used to lookup cached stages. It now uses the canonicalized form of the new query stage (potentially created by a plugin) rather than using the canonicalized form of the original exchange. ### Why are the changes needed? When using the [RAPIDS Accelerator for Apache Spark](https://github.com/NVIDIA/spark-rapids) we sometimes see a new query stage correctly create a row-based exchange and then Spark replaces it with a cached columnar exchange, which is not compatible, and this causes queries to fail. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The patch has been tested with the query that highlighted this issue. I looked at writing unit tests for this but it would involve implementing a mock columnar exchange in the tests so would be quite a bit of work. If anyone has ideas on other ways to test this I am happy to hear them. Closes #32195 from andygrove/SPARK-35093. Authored-by: Andy Grove Signed-off-by: Thomas Graves (cherry picked from commit 52e3cf9ff50b4209e29cb06df09b1ef3a18bc83b) Signed-off-by: Thomas Graves --- .../apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala| 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 89d3b53..596c8b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -418,7 +418,8 @@ case class AdaptiveSparkPlanExec( // Check the `stageCache` again for reuse. If a match is found, ditch the new stage // and reuse the existing stage found in the `stageCache`, otherwise update the // `stageCache` with the new stage. - val queryStage = context.stageCache.getOrElseUpdate(e.canonicalized, newStage) + val queryStage = context.stageCache.getOrElseUpdate( +newStage.plan.canonicalized, newStage) if (queryStage.ne(newStage)) { newStage = reuseQueryStage(queryStage, e) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 52e3cf9 [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use 52e3cf9 is described below commit 52e3cf9ff50b4209e29cb06df09b1ef3a18bc83b Author: Andy Grove AuthorDate: Wed May 19 07:45:26 2021 -0500 [SPARK-35093][SQL] AQE now uses newQueryStage plan as key for looking up cached exchanges for re-use ### What changes were proposed in this pull request? AQE has an optimization where it attempts to reuse compatible exchanges but it does not take into account whether the exchanges are columnar or not, resulting in incorrect reuse under some circumstances. This PR simply changes the key used to lookup cached stages. It now uses the canonicalized form of the new query stage (potentially created by a plugin) rather than using the canonicalized form of the original exchange. ### Why are the changes needed? When using the [RAPIDS Accelerator for Apache Spark](https://github.com/NVIDIA/spark-rapids) we sometimes see a new query stage correctly create a row-based exchange and then Spark replaces it with a cached columnar exchange, which is not compatible, and this causes queries to fail. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The patch has been tested with the query that highlighted this issue. I looked at writing unit tests for this but it would involve implementing a mock columnar exchange in the tests so would be quite a bit of work. If anyone has ideas on other ways to test this I am happy to hear them. Closes #32195 from andygrove/SPARK-35093. Authored-by: Andy Grove Signed-off-by: Thomas Graves --- .../apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala| 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 256aacb..766788a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -435,7 +435,8 @@ case class AdaptiveSparkPlanExec( // Check the `stageCache` again for reuse. If a match is found, ditch the new stage // and reuse the existing stage found in the `stageCache`, otherwise update the // `stageCache` with the new stage. - val queryStage = context.stageCache.getOrElseUpdate(e.canonicalized, newStage) + val queryStage = context.stageCache.getOrElseUpdate( +newStage.plan.canonicalized, newStage) if (queryStage.ne(newStage)) { newStage = reuseQueryStage(queryStage, e) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-34472][YARN] Ship ivySettings file to driver in cluster mode
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 83f753e [SPARK-34472][YARN] Ship ivySettings file to driver in cluster mode 83f753e is described below commit 83f753e4e1412a896243f4016600552c0110c1b0 Author: Shardul Mahadik AuthorDate: Tue Apr 20 13:35:57 2021 -0500 [SPARK-34472][YARN] Ship ivySettings file to driver in cluster mode ### What changes were proposed in this pull request? In YARN, ship the `spark.jars.ivySettings` file to the driver when using `cluster` deploy mode so that `addJar` is able to find it in order to resolve ivy paths. ### Why are the changes needed? SPARK-33084 introduced support for Ivy paths in `sc.addJar` or Spark SQL `ADD JAR`. If we use a custom ivySettings file using `spark.jars.ivySettings`, it is loaded at https://github.com/apache/spark/blob/b26e7b510bbaee63c4095ab47e75ff2a70e377d7/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L1280. However, this file is only accessible on the client machine. In YARN cluster mode, this file is not available on the driver and so `addJar` fails to find it. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit tests to verify that the `ivySettings` file is localized by the YARN client and that a YARN cluster mode application is able to find to load the `ivySettings` file. Closes #31591 from shardulm94/SPARK-34472. Authored-by: Shardul Mahadik Signed-off-by: Thomas Graves --- .../org/apache/spark/deploy/SparkSubmit.scala | 7 +- docs/configuration.md | 7 +- .../org/apache/spark/deploy/yarn/Client.scala | 57 +--- .../spark/deploy/yarn/YarnClusterSuite.scala | 103 + 4 files changed, 161 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 0e31fcf..36873c7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -1286,7 +1286,12 @@ private[spark] object SparkSubmitUtils extends Logging { settingsFile: String, remoteRepos: Option[String], ivyPath: Option[String]): IvySettings = { -val file = new File(settingsFile) +val uri = new URI(settingsFile) +val file = Option(uri.getScheme).getOrElse("file") match { + case "file" => new File(uri.getPath) + case scheme => throw new IllegalArgumentException(s"Scheme $scheme not supported in " + +"spark.jars.ivySettings") +} require(file.exists(), s"Ivy settings file $file does not exist") require(file.isFile(), s"Ivy settings file $file is not a normal file") val ivySettings: IvySettings = new IvySettings diff --git a/docs/configuration.md b/docs/configuration.md index d9bbddc..c6b462a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -771,7 +771,12 @@ Apart from these, the following properties are also available, and may be useful option --repositories or spark.jars.repositories will also be included. Useful for allowing Spark to resolve artifacts from behind a firewall e.g. via an in-house artifact server like Artifactory. Details on the settings file format can be -found at http://ant.apache.org/ivy/history/latest-milestone/settings.html";>Settings Files +found at http://ant.apache.org/ivy/history/latest-milestone/settings.html";>Settings Files. +Only paths with file:// scheme are supported. Paths without a scheme are assumed to have +a file:// scheme. + +When running in YARN cluster mode, this file will also be localized to the remote driver for dependency +resolution within SparkContext#addJar 2.2.0 diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 618faef..427202f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -21,6 +21,7 @@ import java.io.{FileSystem => _, _} import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer import java.nio.charset.StandardCharsets +import java.nio.file.Files import java.util.{Locale, Properties, UUID} import java.util.zip.{ZipEntry, ZipOutputStream} @@ -30,7 +31,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.util.control.NonFata
[spark] branch master updated: [SPARK-34877][CORE][YARN] Add the code change for adding the Spark AM log link in spark UI
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1e64b4f [SPARK-34877][CORE][YARN] Add the code change for adding the Spark AM log link in spark UI 1e64b4f is described below commit 1e64b4fa27882f81989bda2aefdda66343625436 Author: SaurabhChawla AuthorDate: Tue Apr 20 08:56:07 2021 -0500 [SPARK-34877][CORE][YARN] Add the code change for adding the Spark AM log link in spark UI ### What changes were proposed in this pull request? On Running Spark job with yarn and deployment mode as client, Spark Driver and Spark Application master launch in two separate containers. In various scenarios there is need to see Spark Application master logs to see the resource allocation, Decommissioning status and other information shared between yarn RM and Spark Application master. In Cluster mode Spark driver and Spark AM is on same container, So Log link of the driver already there to see the logs in Spark UI This PR is for adding the spark AM log link for spark job running in the client mode for yarn. Instead of searching the container id and then find the logs. We can directly check in the Spark UI This change is only for showing the AM log links in the Client mode when resource manager is yarn. ### Why are the changes needed? Till now the only way to check this by finding the container id of the AM and check the logs either using Yarn utility or Yarn RM Application History server. This PR is for adding the spark AM log link for spark job running in the client mode for yarn. Instead of searching the container id and then find the logs. We can directly check in the Spark UI ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added the unit test also checked the Spark UI **In Yarn Client mode** Before Change ![image](https://user-images.githubusercontent.com/34540906/112644861-e1733200-8e6b-11eb-939b-c76ca9902a4e.png) After the Change - The AM info is there ![image](https://user-images.githubusercontent.com/34540906/115264198-b7075280-a153-11eb-98f3-2aed66ffad2a.png) AM Log ![image](https://user-images.githubusercontent.com/34540906/112645680-c0f7a780-8e6c-11eb-8b82-4ccc0aee927b.png) **In Yarn Cluster Mode** - The AM log link will not be there ![image](https://user-images.githubusercontent.com/34540906/112649512-86900980-8e70-11eb-9b37-69d5c4b53ffa.png) Closes #31974 from SaurabhChawla100/SPARK-34877. Authored-by: SaurabhChawla Signed-off-by: Thomas Graves --- .../spark/ui/static/executorspage-template.html| 17 .../org/apache/spark/ui/static/executorspage.js| 50 +- .../resources/org/apache/spark/ui/static/utils.js | 22 ++ .../scheduler/MiscellaneousProcessDetails.scala| 30 + .../org/apache/spark/scheduler/SparkListener.scala | 4 ++ .../cluster/CoarseGrainedClusterMessage.scala | 7 ++- .../cluster/CoarseGrainedSchedulerBackend.scala| 5 +++ .../apache/spark/status/AppStatusListener.scala| 22 ++ .../org/apache/spark/status/AppStatusStore.scala | 10 + .../scala/org/apache/spark/status/LiveEntity.scala | 24 +++ .../status/api/v1/OneApplicationResource.scala | 4 ++ .../scala/org/apache/spark/status/api/v1/api.scala | 9 .../scala/org/apache/spark/status/storeTypes.scala | 13 ++ .../miscellaneous_process_expectation.json | 11 + .../spark-events/application_1555004656427_0144| 1 + .../spark/deploy/history/HistoryServerSuite.scala | 3 +- .../spark/status/AppStatusListenerSuite.scala | 38 .../spark/deploy/yarn/ApplicationMaster.scala | 15 +++ .../scheduler/cluster/YarnSchedulerBackend.scala | 6 +++ 19 files changed, 288 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 7a064dc..be6d7bc 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -134,4 +134,21 @@ limitations under the License. + + +Miscellaneous Process + + + +Process ID +Address +Status +Cores +Logs + + + + + + diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index c8dc619..9133ef8 100644 --- a/core/src/main/resources/org/apache/spark/ui/static
[spark] branch master updated: [SPARK-34828][YARN] Make shuffle service name configurable on client side and allow for classpath-based config override on server side
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9f065ff [SPARK-34828][YARN] Make shuffle service name configurable on client side and allow for classpath-based config override on server side 9f065ff is described below commit 9f065ff375bb2146f54dc1583b52beaf24064284 Author: Erik Krogen AuthorDate: Tue Mar 30 10:09:00 2021 -0500 [SPARK-34828][YARN] Make shuffle service name configurable on client side and allow for classpath-based config override on server side ### What changes were proposed in this pull request? Add a new config, `spark.shuffle.service.name`, which allows for Spark applications to look for a YARN shuffle service which is defined at a name other than the default `spark_shuffle`. Add a new config, `spark.yarn.shuffle.service.metrics.namespace`, which allows for configuring the namespace used when emitting metrics from the shuffle service into the NodeManager's `metrics2` system. Add a new mechanism by which to override shuffle service configurations independently of the configurations in the NodeManager. When a resource `spark-shuffle-site.xml` is present on the classpath of the shuffle service, the configs present within it will be used to override the configs coming from `yarn-site.xml` (via the NodeManager). ### Why are the changes needed? There are two use cases which can benefit from these changes. One use case is to run multiple instances of the shuffle service side-by-side in the same NodeManager. This can be helpful, for example, when running a YARN cluster with a mixed workload of applications running multiple Spark versions, since a given version of the shuffle service is not always compatible with other versions of Spark (e.g. see SPARK-27780). With this PR, it is possible to run two shuffle services like `spark_shuffle` and `spark_shuffle_3.2.0`, one of which is "legacy" [...] Besides this, the separation of shuffle service configs into `spark-shuffle-site.xml` can be useful for administrators who want to change and/or deploy Spark shuffle service configurations independently of the configurations for the NodeManager (e.g., perhaps they are owned by two different teams). ### Does this PR introduce _any_ user-facing change? Yes. There are two new configurations related to the external shuffle service, and a new mechanism which can optionally be used to configure the shuffle service. `docs/running-on-yarn.md` has been updated to provide user instructions; please see this guide for more details. ### How was this patch tested? In addition to the new unit tests added, I have deployed this to a live YARN cluster and successfully deployed two Spark shuffle services simultaneously, one running a modified version of Spark 2.3.0 (which supports some of the newer shuffle protocols) and one running Spark 3.1.1. Spark applications of both versions are able to communicate with their respective shuffle services without issue. Closes #31936 from xkrogen/xkrogen-SPARK-34828-shufflecompat-config-from-classpath. Authored-by: Erik Krogen Signed-off-by: Thomas Graves --- .../spark/network/yarn/YarnShuffleService.java | 66 +++--- .../network/yarn/YarnShuffleServiceMetrics.java| 6 +- .../org/apache/spark/internal/config/package.scala | 10 +++ docs/running-on-yarn.md| 71 +++ .../spark/deploy/yarn/ExecutorRunnable.scala | 4 +- .../yarn/YarnShuffleAlternateNameConfigSuite.scala | 79 ++ .../network/yarn/YarnShuffleServiceSuite.scala | 13 .../spark/network/yarn/YarnTestAccessor.scala | 3 + 8 files changed, 240 insertions(+), 12 deletions(-) diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 548a5cc..cb6d5d0 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -19,6 +19,7 @@ package org.apache.spark.network.yarn; import java.io.File; import java.io.IOException; +import java.net.URL; import java.nio.charset.StandardCharsets; import java.nio.ByteBuffer; import java.util.List; @@ -75,6 +76,20 @@ import org.apache.spark.network.yarn.util.HadoopConfigProvider; * is because an application running on the same Yarn cluster may choose to not use the external * shuffle service, in which case its setting of `spark.authenticate` should be independent of * the service's. + * + * The shuffle service will produce metrics via the YARN
[spark] branch master updated: [SPARK-33739][SQL] Jobs committed through the S3A Magic committer don't track bytes
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ff5115c [SPARK-33739][SQL] Jobs committed through the S3A Magic committer don't track bytes ff5115c is described below commit ff5115c3ac15cf718fbfe98e07c56f3dde79a602 Author: Steve Loughran AuthorDate: Thu Feb 18 08:43:18 2021 -0600 [SPARK-33739][SQL] Jobs committed through the S3A Magic committer don't track bytes BasicWriteStatsTracker to probe for a custom Xattr if the size of the generated file is 0 bytes; if found and parseable use that as the declared length of the output. The matching Hadoop patch in HADOOP-17414: * Returns all S3 object headers as XAttr attributes prefixed "header." * Sets the custom header x-hadoop-s3a-magic-data-length to the length of the data in the marker file. As a result, spark job tracking will correctly report the amount of data uploaded and yet to materialize. ### Why are the changes needed? Now that S3 is consistent, it's a lot easier to use the S3A "magic" committer which redirects a file written to `dest/__magic/job_0011/task_1245/__base/year=2020/output.avro` to its final destination `dest/year=2020/output.avro` , adding a zero byte marker file at the end and a json file `dest/__magic/job_0011/task_1245/__base/year=2020/output.avro.pending` containing all the information for the job committer to complete the upload. But: the write tracker statictics don't show progress as they measure the length of the created file, find the marker file and report 0 bytes. By probing for a specific HTTP header in the marker file and parsing that if retrieved, the real progress can be reported. There's a matching change in Hadoop [https://github.com/apache/hadoop/pull/2530](https://github.com/apache/hadoop/pull/2530) which adds getXAttr API support to the S3A connector and returns the headers; the magic committer adds the relevant attributes. If the FS being probed doesn't support the XAttr API, the header is missing or the value not a positive long then the size of 0 is returned. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New tests in BasicWriteTaskStatsTrackerSuite which use a filter FS to implement getXAttr on top of LocalFS; this is used to explore the set of options: * no XAttr API implementation (existing tests; what callers would see with most filesystems) * no attribute found (HDFS, ABFS without the attribute) * invalid data of different forms All of these return Some(0) as file length. The Hadoop PR verifies XAttr implementation in S3A and that the commit protocol attaches the header to the files. External downstream testing has done the full hadoop+spark end to end operation, with manual review of logs to verify that the data was successfully collected from the attribute. Closes #30714 from steveloughran/cdpd/SPARK-33739-magic-commit-tracking-master. Authored-by: Steve Loughran Signed-off-by: Thomas Graves --- docs/cloud-integration.md | 43 +-- .../datasources/BasicWriteStatsTracker.scala | 63 +- .../BasicWriteTaskStatsTrackerSuite.scala | 128 - 3 files changed, 221 insertions(+), 13 deletions(-) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 1eaa8ab..d801df7 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -49,7 +49,6 @@ They cannot be used as a direct replacement for a cluster filesystem such as HDF Key differences are: -* Changes to stored objects may not be immediately visible, both in directory listings and actual data access. * The means by which directories are emulated may make working with them slow. * Rename operations may be very slow and, on failure, leave the store in an unknown state. * Seeking within a file may require new HTTP calls, hurting performance. @@ -58,7 +57,6 @@ How does this affect Spark? 1. Reading and writing data can be significantly slower than working with a normal filesystem. 1. Some directory structures may be very inefficient to scan during query split calculation. -1. The output of work may not be immediately visible to a follow-on query. 1. The rename-based algorithm by which Spark normally commits work when saving an RDD, DataFrame or Dataset is potentially both slow and unreliable. @@ -66,8 +64,28 @@ For these reasons, it is not always safe to use an object store as a direct dest an intermediate store in a chain of queries. Consult the documentation of the objec
[spark] branch master updated: [SPARK-33741][CORE] Add min threshold time speculation config
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bd5039f [SPARK-33741][CORE] Add min threshold time speculation config bd5039f is described below commit bd5039fc3542dc4ac96bea28639f1896d7919388 Author: schintap AuthorDate: Wed Jan 13 08:57:56 2021 -0600 [SPARK-33741][CORE] Add min threshold time speculation config ### What changes were proposed in this pull request? Add min threshold time speculation config ### Why are the changes needed? When we turn on speculation with default configs we have the last 10% of the tasks subject to speculation. There are a lot of stages where the stage runs for few seconds to minutes. Also in general we don't want to speculate tasks that run within a minimum threshold. By setting a minimum threshold for speculation config gives us better control for speculative tasks ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test Closes #30710 from redsanket/SPARK-33741. Lead-authored-by: schintap Co-authored-by: Sanket Chintapalli Signed-off-by: Thomas Graves --- .../org/apache/spark/internal/config/package.scala | 8 + .../scala/org/apache/spark/scheduler/Pool.scala| 2 +- .../org/apache/spark/scheduler/Schedulable.scala | 2 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../apache/spark/scheduler/TaskSetManager.scala| 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 37 -- docs/configuration.md | 9 ++ 7 files changed, 55 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 84c6647..f962bc6 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1889,6 +1889,14 @@ package object config { .doubleConf .createWithDefault(0.75) + private[spark] val SPECULATION_MIN_THRESHOLD = +ConfigBuilder("spark.speculation.min.threshold") + .doc("Minimum amount of time a task runs before being considered for speculation. " + +"This can be used to avoid launching speculative copies of tasks that are very short.") + .version("3.2.0") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(100) + private[spark] val SPECULATION_TASK_DURATION_THRESHOLD = ConfigBuilder("spark.speculation.task.duration.threshold") .doc("Task duration after which scheduler would try to speculative run the task. If " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 7333b31..de4c9d3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -94,7 +94,7 @@ private[spark] class Pool( schedulableQueue.asScala.foreach(_.executorDecommission(executorId)) } - override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = { + override def checkSpeculatableTasks(minTimeToSpeculation: Long): Boolean = { var shouldRevive = false for (schedulable <- schedulableQueue.asScala) { shouldRevive |= schedulable.checkSpeculatableTasks(minTimeToSpeculation) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index 0626f8f..e549ce6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -45,6 +45,6 @@ private[spark] trait Schedulable { def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String, reason: ExecutorLossReason): Unit def executorDecommission(executorId: String): Unit - def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean + def checkSpeculatableTasks(minTimeToSpeculation: Long): Boolean def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index b939e40..71b8bc2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -100,7 +100,7 @@ private[spark] class TaskSchedulerImpl( // Duplicate copies of a task will only be launched if the original copy has been running for // at least this amount of time. This is t
[spark] branch branch-3.0 updated: Revert "[SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted"
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 3fb9f6f Revert "[SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted" 3fb9f6f is described below commit 3fb9f6f670328d31bf24fcb6b805715f5828ce06 Author: Thomas Graves AuthorDate: Wed Dec 2 14:38:19 2020 -0600 Revert "[SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted" ### What changes were proposed in this pull request? Revert SPARK-33504 on branch-3.0 compilation error. Original PR https://github.com/apache/spark/pull/30446 This reverts commit e59179b7326112f526e4c000e21146df283d861c. ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Closes #30576 from tgravescs/revert33504. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../spark/scheduler/EventLoggingListener.scala | 24 +--- .../scheduler/EventLoggingListenerSuite.scala | 64 +- 2 files changed, 3 insertions(+), 85 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 5673c02..24e2a5e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -18,9 +18,7 @@ package org.apache.spark.scheduler import java.net.URI -import java.util.Properties -import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -105,7 +103,7 @@ private[spark] class EventLoggingListener( // Events that do not trigger a flush override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { -logEvent(event.copy(properties = redactProperties(event.properties))) +logEvent(event) if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), @@ -158,9 +156,7 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } - override def onJobStart(event: SparkListenerJobStart): Unit = { -logEvent(event.copy(properties = redactProperties(event.properties)), flushLogger = true) - } + override def onJobStart(event: SparkListenerJobStart): Unit = logEvent(event, flushLogger = true) override def onJobEnd(event: SparkListenerJobEnd): Unit = logEvent(event, flushLogger = true) @@ -250,22 +246,6 @@ private[spark] class EventLoggingListener( logWriter.stop() } - private def redactProperties(properties: Properties): Properties = { -if (properties == null) { - return properties -} -val redactedProperties = new Properties -// properties may contain some custom local properties such as stage/job description -// only properties in sparkConf need to be redacted. -val (globalProperties, localProperties) = properties.asScala.toSeq.partition { - case (key, _) => sparkConf.contains(key) -} -(Utils.redact(sparkConf, globalProperties) ++ localProperties).foreach { - case (key, value) => redactedProperties.setProperty(key, value) -} -redactedProperties - } - private[spark] def redactEvent( event: SparkListenerEnvironmentUpdate): SparkListenerEnvironmentUpdate = { // environmentDetails maps a string descriptor to a set of properties diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index e0e6406..046564d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, InputStream} -import java.util.{Arrays, Properties} +import java.util.Arrays import scala.collection.immutable.Map import scala.collection.mutable @@ -96,68 +96,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(redactedProps(key) == "*(redacted)") } - test("Spark-33504 sensitive attributes redaction in properties") { -val (secretKey, secretPassword) = ("spark.executorEnv.HADOOP_CREDSTORE_PASSWORD", - "secret_password") -val (customKey, customValue) = ("parse_token", "secret_password") - -
[spark] branch branch-3.0 updated: [SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new e59179b [SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted e59179b is described below commit e59179b7326112f526e4c000e21146df283d861c Author: neko AuthorDate: Wed Dec 2 09:24:19 2020 -0600 [SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted ### What changes were proposed in this pull request? To make sure the sensitive attributes to be redacted in the history server log. ### Why are the changes needed? We found the secure attributes like password in SparkListenerJobStart and SparkListenerStageSubmitted events would not been redated, resulting in sensitive attributes can be viewd directly. The screenshot can be viewed in the attachment of JIRA spark-33504 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? muntual test works well, I have also added unit testcase. Closes #30446 from akiyamaneko/eventlog_unredact. Authored-by: neko Signed-off-by: Thomas Graves (cherry picked from commit 28dad1ba770e5b7f7cf542da1ae3f05975a969c6) Signed-off-by: Thomas Graves --- .../spark/scheduler/EventLoggingListener.scala | 24 +++- .../scheduler/EventLoggingListenerSuite.scala | 64 +- 2 files changed, 85 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 24e2a5e..5673c02 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -18,7 +18,9 @@ package org.apache.spark.scheduler import java.net.URI +import java.util.Properties +import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -103,7 +105,7 @@ private[spark] class EventLoggingListener( // Events that do not trigger a flush override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { -logEvent(event) +logEvent(event.copy(properties = redactProperties(event.properties))) if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), @@ -156,7 +158,9 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } - override def onJobStart(event: SparkListenerJobStart): Unit = logEvent(event, flushLogger = true) + override def onJobStart(event: SparkListenerJobStart): Unit = { +logEvent(event.copy(properties = redactProperties(event.properties)), flushLogger = true) + } override def onJobEnd(event: SparkListenerJobEnd): Unit = logEvent(event, flushLogger = true) @@ -246,6 +250,22 @@ private[spark] class EventLoggingListener( logWriter.stop() } + private def redactProperties(properties: Properties): Properties = { +if (properties == null) { + return properties +} +val redactedProperties = new Properties +// properties may contain some custom local properties such as stage/job description +// only properties in sparkConf need to be redacted. +val (globalProperties, localProperties) = properties.asScala.toSeq.partition { + case (key, _) => sparkConf.contains(key) +} +(Utils.redact(sparkConf, globalProperties) ++ localProperties).foreach { + case (key, value) => redactedProperties.setProperty(key, value) +} +redactedProperties + } + private[spark] def redactEvent( event: SparkListenerEnvironmentUpdate): SparkListenerEnvironmentUpdate = { // environmentDetails maps a string descriptor to a set of properties diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 046564d..e0e6406 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, InputStream} -import java.util.Arrays +import java.util.{Arrays, Properties} import scala.collection.immutable.Map import scala.collection.mutable @@ -96,6 +96,68 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(redactedProps(key) == "*(redacted)") } + test("Spark-33504
[spark] branch master updated: [SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 28dad1b [SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted 28dad1b is described below commit 28dad1ba770e5b7f7cf542da1ae3f05975a969c6 Author: neko AuthorDate: Wed Dec 2 09:24:19 2020 -0600 [SPARK-33504][CORE] The application log in the Spark history server contains sensitive attributes should be redacted ### What changes were proposed in this pull request? To make sure the sensitive attributes to be redacted in the history server log. ### Why are the changes needed? We found the secure attributes like password in SparkListenerJobStart and SparkListenerStageSubmitted events would not been redated, resulting in sensitive attributes can be viewd directly. The screenshot can be viewed in the attachment of JIRA spark-33504 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? muntual test works well, I have also added unit testcase. Closes #30446 from akiyamaneko/eventlog_unredact. Authored-by: neko Signed-off-by: Thomas Graves --- .../spark/scheduler/EventLoggingListener.scala | 24 +++- .../scheduler/EventLoggingListenerSuite.scala | 64 +- 2 files changed, 85 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 1fda03f..d4e22d7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -18,7 +18,9 @@ package org.apache.spark.scheduler import java.net.URI +import java.util.Properties +import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -103,7 +105,7 @@ private[spark] class EventLoggingListener( // Events that do not trigger a flush override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { -logEvent(event) +logEvent(event.copy(properties = redactProperties(event.properties))) if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), @@ -156,7 +158,9 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } - override def onJobStart(event: SparkListenerJobStart): Unit = logEvent(event, flushLogger = true) + override def onJobStart(event: SparkListenerJobStart): Unit = { +logEvent(event.copy(properties = redactProperties(event.properties)), flushLogger = true) + } override def onJobEnd(event: SparkListenerJobEnd): Unit = logEvent(event, flushLogger = true) @@ -276,6 +280,22 @@ private[spark] class EventLoggingListener( logWriter.stop() } + private def redactProperties(properties: Properties): Properties = { +if (properties == null) { + return properties +} +val redactedProperties = new Properties +// properties may contain some custom local properties such as stage/job description +// only properties in sparkConf need to be redacted. +val (globalProperties, localProperties) = properties.asScala.toSeq.partition { + case (key, _) => sparkConf.contains(key) +} +(Utils.redact(sparkConf, globalProperties) ++ localProperties).foreach { + case (key, value) => redactedProperties.setProperty(key, value) +} +redactedProperties + } + private[spark] def redactEvent( event: SparkListenerEnvironmentUpdate): SparkListenerEnvironmentUpdate = { // environmentDetails maps a string descriptor to a set of properties diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index c4a8bcb..7acb845 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, InputStream} -import java.util.Arrays +import java.util.{Arrays, Properties} import scala.collection.immutable.Map import scala.collection.mutable @@ -98,6 +98,68 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(redactedProps(key) == "*(redacted)") } + test("Spark-33504 sensitive attributes redaction in properties") { +val (secretKey, secretPassword) = ("spark.execut
[spark] branch master updated: [SPARK-32916][SHUFFLE][TEST-MAVEN][TEST-HADOOP2.7] Remove the newly added YarnShuffleServiceSuite.java
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 423ba5a [SPARK-32916][SHUFFLE][TEST-MAVEN][TEST-HADOOP2.7] Remove the newly added YarnShuffleServiceSuite.java 423ba5a is described below commit 423ba5a16038c1cb28d0973e18518645e69d5ff1 Author: Chandni Singh AuthorDate: Fri Nov 13 16:16:23 2020 -0600 [SPARK-32916][SHUFFLE][TEST-MAVEN][TEST-HADOOP2.7] Remove the newly added YarnShuffleServiceSuite.java ### What changes were proposed in this pull request? This is a follow-up fix for the failing tests in `YarnShuffleServiceSuite.java`. This java class was introduced in https://github.com/apache/spark/pull/30062. The tests in the class fail when run with hadoop-2.7 profile: ``` [ERROR] testCreateDefaultMergedShuffleFileManagerInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite) Time elapsed: 0.627 s <<< ERROR! java.lang.NoClassDefFoundError: org/apache/commons/logging/LogFactory at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateDefaultMergedShuffleFileManagerInstance(YarnShuffleServiceSuite.java:37) Caused by: java.lang.ClassNotFoundException: org.apache.commons.logging.LogFactory at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateDefaultMergedShuffleFileManagerInstance(YarnShuffleServiceSuite.java:37) [ERROR] testCreateRemoteBlockPushResolverInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite) Time elapsed: 0 s <<< ERROR! java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.network.yarn.YarnShuffleService at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateRemoteBlockPushResolverInstance(YarnShuffleServiceSuite.java:47) [ERROR] testInvalidClassNameOfMergeManagerWillUseNoOpInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite) Time elapsed: 0.001 s <<< ERROR! java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.network.yarn.YarnShuffleService at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testInvalidClassNameOfMergeManagerWillUseNoOpInstance(YarnShuffleServiceSuite.java:57) ``` A test suit for `YarnShuffleService` did exist here: `resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala` I missed this when I created `common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java`. Moving all the new tests to the earlier test suite fixes the failures with hadoop-2.7 even though why this happened is not clear. ### Why are the changes needed? The newly added tests are failing when run with hadoop profile 2.7 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Ran the unit tests with the default profile as well as hadoop 2.7 profile. `build/mvn test -Dtest=none -DwildcardSuites=org.apache.spark.network.yarn.YarnShuffleServiceSuite -Phadoop-2.7 -Pyarn` ``` Run starting. Expected test count is: 11 YarnShuffleServiceSuite: - executor state kept across NM restart - removed applications should not be in registered executor file - shuffle service should be robust to corrupt registered executor file - get correct recovery path - moving recovery file from NM local dir to recovery path - service throws error if cannot start - recovery db should not be created if NM recovery is not enabled - SPARK-31646: metrics should be registered into Node Manager's metrics system - create default merged shuffle file manager instance - create remote block push resolver instance - invalid class name of merge manager will use noop instance Run completed in 2 seconds, 572 milliseconds. Total number of tests run: 11 Suites: completed 2, aborted 0 Tests: succeeded 11, failed 0, canceled 0, ignored 0, pending 0 All tests passed. ``` Closes #30349 from otterc/SPARK-32916-followup. Authored-by: Chandni Singh Signed-off-by: Thomas Graves --- .../network/yarn/YarnShuffleServiceSuite.java | 61 -- .../network/yarn/YarnShuffleServiceSuite.scala | 27 +- 2 files changed, 26 insertions(+), 62 deletions(-) diff --git a/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java b/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java deleted file mode 100644 index 09bc4d8..000 --- a/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * cont
[spark] branch master updated: [SPARK-33288][SPARK-32661][K8S] Stage level scheduling support for Kubernetes
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new acfd846 [SPARK-33288][SPARK-32661][K8S] Stage level scheduling support for Kubernetes acfd846 is described below commit acfd8467534fbf58c12e9f2d993b7d135fb8d32b Author: Thomas Graves AuthorDate: Fri Nov 13 16:04:13 2020 -0600 [SPARK-33288][SPARK-32661][K8S] Stage level scheduling support for Kubernetes ### What changes were proposed in this pull request? This adds support for Stage level scheduling to kubernetes. Kubernetes can support dynamic allocation via the shuffle tracking option which means we can support stage level scheduling by getting new executors. The main changes here are having the k8s cluster manager pass the resource profile id into the executors and then the ExecutorsPodsAllocator has to request executors based on the individual resource profiles. I tried to keep code changes here to a minimum. I specifically choose to leave the ExecutorPodsSnapshot the way it was and construct the resource profile to pod states on the fly, with a fast path when not using other resource profiles, to keep the impact to a minimum. This res [...] This also adds support for [SPARK-32661]Spark executors on K8S should request extra memory for off-heap allocations because the stage level scheduling api has support for this and it made sense to make consistent with YARN. This was started with PR https://github.com/apache/spark/pull/29477 but never updated so I just did it here. To do this I moved a few functions around that were now used by both YARN and kubernetes so you will see some changes in Utils. ### Why are the changes needed? Add the feature to Kubernetes based on customer feedback. ### Does this PR introduce _any_ user-facing change? Yes the feature now works with K8s, but not underlying API changes. ### How was this patch tested? Tested manually on kubernetes cluster and with unit tests. Closes #30204 from tgravescs/stagek8sOrigSnapshotsRebase. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../apache/spark/resource/ResourceProfile.scala| 133 +- .../spark/resource/ResourceProfileManager.scala| 21 +- .../main/scala/org/apache/spark/util/Utils.scala | 21 ++ .../resource/ResourceProfileManagerSuite.scala | 27 +- .../spark/resource/ResourceProfileSuite.scala | 8 +- .../scala/org/apache/spark/util/UtilsSuite.scala | 27 ++ docs/configuration.md | 2 +- docs/running-on-kubernetes.md | 4 + docs/running-on-yarn.md| 1 + .../org/apache/spark/deploy/k8s/Constants.scala| 3 +- .../apache/spark/deploy/k8s/KubernetesConf.scala | 12 +- .../k8s/features/BasicDriverFeatureStep.scala | 4 +- .../k8s/features/BasicExecutorFeatureStep.scala| 89 --- .../cluster/k8s/ExecutorPodsAllocator.scala| 285 - .../k8s/KubernetesClusterSchedulerBackend.scala| 5 +- .../cluster/k8s/KubernetesExecutorBuilder.scala| 8 +- .../spark/deploy/k8s/KubernetesConfSuite.scala | 15 +- .../k8s/features/BasicDriverFeatureStepSuite.scala | 5 +- .../features/BasicExecutorFeatureStepSuite.scala | 106 ++-- .../cluster/k8s/ExecutorLifecycleTestUtils.scala | 40 +-- .../cluster/k8s/ExecutorPodsAllocatorSuite.scala | 109 +++- .../KubernetesClusterSchedulerBackendSuite.scala | 5 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 4 +- .../src/main/dockerfiles/spark/entrypoint.sh | 1 + .../org/apache/spark/deploy/yarn/Client.scala | 9 +- .../spark/deploy/yarn/ResourceRequestHelper.scala | 4 +- .../apache/spark/deploy/yarn/YarnAllocator.scala | 136 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 24 +- .../spark/deploy/yarn/YarnAllocatorSuite.scala | 20 +- .../deploy/yarn/YarnSparkHadoopUtilSuite.scala | 27 -- 30 files changed, 772 insertions(+), 383 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 8a37670..ac7e8e8 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -29,6 +29,7 @@ import org.apache.spark.annotation.{Evolving, Since} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.util.Utils /** * Resource profile to associate with an RDD. A ResourceProfile allows the user to
[spark] branch master updated: [SPARK-31711][CORE] Register the executor source with the metrics system when running in local mode
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b7fff03 [SPARK-31711][CORE] Register the executor source with the metrics system when running in local mode b7fff03 is described below commit b7fff0397319efd2987d4cceff4f738f1c06409d Author: Luca Canali AuthorDate: Wed Nov 4 16:48:55 2020 -0600 [SPARK-31711][CORE] Register the executor source with the metrics system when running in local mode ### What changes were proposed in this pull request? This PR proposes to register the executor source with the Spark metrics system when running in local mode. ### Why are the changes needed? The Apache Spark metrics system provides many useful insights on the Spark workload. In particular, the [executor source metrics](https://github.com/apache/spark/blob/master/docs/monitoring.md#component-instance--executor) provide detailed info, including the number of active tasks, I/O metrics, and several task metrics details. The executor source metrics, contrary to other sources (for example ExecutorMetrics source), is not available when running in local mode. Having executor metrics in local mode can be useful when testing and troubleshooting Spark workloads in a development environment. The metrics can be fed to a dashboard to see the evolution of resource usage and can be used to troubleshoot performance, as [in this example](https://github.com/cerndb/spark-dashboard). Currently users will have to deploy on a cluster to be able to collect executor source metrics, while the possibility of having them in local mode is handy for testing. ### Does this PR introduce _any_ user-facing change? - This PR exposes executor source metrics data when running in local mode. ### How was this patch tested? - Manually tested by running in local mode and inspecting the metrics listed in http://localhost:4040/metrics/json/ - Also added a test in `SourceConfigSuite` Closes #28528 from LucaCanali/metricsWithLocalMode. Authored-by: Luca Canali Signed-off-by: Thomas Graves --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 - core/src/main/scala/org/apache/spark/executor/Executor.scala | 8 .../org/apache/spark/metrics/source/SourceConfigSuite.scala | 12 docs/monitoring.md | 8 ++-- 4 files changed, 30 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b357682..d680154 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -42,7 +42,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} -import org.apache.spark.executor.{ExecutorMetrics, ExecutorMetricsSource} +import org.apache.spark.executor.{Executor, ExecutorMetrics, ExecutorMetricsSource} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -625,6 +625,9 @@ class SparkContext(config: SparkConf) extends Logging { // Post init _taskScheduler.postStartHook() +if (isLocal) { + _env.metricsSystem.registerSource(Executor.executorSourceLocalModeOnly) +} _env.metricsSystem.registerSource(_dagScheduler.metricsSource) _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) _env.metricsSystem.registerSource(new JVMCPUSource()) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 6653650..1a0ad56 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -135,6 +135,11 @@ private[spark] class Executor( env.metricsSystem.registerSource(new JVMCPUSource()) executorMetricsSource.foreach(_.register(env.metricsSystem)) env.metricsSystem.registerSource(env.blockManager.shuffleMetricsSource) + } else { +// This enable the registration of the executor source in local mode. +// The actual registration happens in SparkContext, +// it cannot be done here as the appId is not available yet +Executor.executorSourceLocalModeOnly = executorSource } // Whether to load classes in user jars before those in Spark jars @@ -987,4 +992,7 @@ private[spark] object Executor { // task is fu
[spark] branch branch-3.0 updated: [SPARK-32332][SQL][3.0] Support columnar exchanges
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 7c91b15 [SPARK-32332][SQL][3.0] Support columnar exchanges 7c91b15 is described below commit 7c91b15c22fe875a44e08781caf3422bfca81b19 Author: Andy Grove AuthorDate: Fri Jul 31 11:14:33 2020 -0500 [SPARK-32332][SQL][3.0] Support columnar exchanges ### What changes were proposed in this pull request? Backports SPARK-32332 to 3.0 branch. ### Why are the changes needed? Plugins cannot replace exchanges with columnar versions when AQE is enabled without this patch. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests included. Closes #29310 from andygrove/backport-SPARK-32332. Authored-by: Andy Grove Signed-off-by: Thomas Graves --- .../execution/adaptive/AdaptiveSparkPlanExec.scala | 30 -- .../adaptive/CustomShuffleReaderExec.scala | 37 --- .../adaptive/OptimizeLocalShuffleReader.scala | 5 +- .../execution/adaptive/OptimizeSkewedJoin.scala| 17 +-- .../sql/execution/adaptive/QueryStageExec.scala| 24 +++-- .../sql/execution/adaptive/simpleCosting.scala | 6 +- .../execution/exchange/BroadcastExchangeExec.scala | 42 +++- .../execution/exchange/ShuffleExchangeExec.scala | 55 +- .../execution/streaming/IncrementalExecution.scala | 4 +- .../spark/sql/SparkSessionExtensionSuite.scala | 120 + 10 files changed, 272 insertions(+), 68 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 5714c33..8b59b12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -100,7 +100,12 @@ case class AdaptiveSparkPlanExec( // The following two rules need to make use of 'CustomShuffleReaderExec.partitionSpecs' // added by `CoalesceShufflePartitions`. So they must be executed after it. OptimizeSkewedJoin(conf), -OptimizeLocalShuffleReader(conf), +OptimizeLocalShuffleReader(conf) + ) + + // A list of physical optimizer rules to be applied right after a new stage is created. The input + // plan to these rules has exchange as its root node. + @transient private val postStageCreationRules = Seq( ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules), CollapseCodegenStages(conf) ) @@ -227,7 +232,8 @@ case class AdaptiveSparkPlanExec( } // Run the final plan when there's no more unfinished stages. - currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) + currentPhysicalPlan = applyPhysicalRules( +result.newPlan, queryStageOptimizerRules ++ postStageCreationRules) isFinalPlan = true executionId.foreach(onUpdatePlan(_, Seq(currentPhysicalPlan))) currentPhysicalPlan @@ -375,10 +381,22 @@ case class AdaptiveSparkPlanExec( private def newQueryStage(e: Exchange): QueryStageExec = { val optimizedPlan = applyPhysicalRules(e.child, queryStageOptimizerRules) val queryStage = e match { - case s: ShuffleExchangeExec => -ShuffleQueryStageExec(currentStageId, s.copy(child = optimizedPlan)) - case b: BroadcastExchangeExec => -BroadcastQueryStageExec(currentStageId, b.copy(child = optimizedPlan)) + case s: ShuffleExchangeLike => +val newShuffle = applyPhysicalRules( + s.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) +if (!newShuffle.isInstanceOf[ShuffleExchangeLike]) { + throw new IllegalStateException( +"Custom columnar rules cannot transform shuffle node to something else.") +} +ShuffleQueryStageExec(currentStageId, newShuffle) + case b: BroadcastExchangeLike => +val newBroadcast = applyPhysicalRules( + b.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) +if (!newBroadcast.isInstanceOf[BroadcastExchangeLike]) { + throw new IllegalStateException( +"Custom columnar rules cannot transform broadcast node to something else.") +} +BroadcastQueryStageExec(currentStageId, newBroadcast) } currentStageId += 1 setLogicalLinkForNewQueryStage(queryStage, e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala index b
[spark] branch master updated: [SPARK-32332][SQL] Support columnar exchanges
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a025a89 [SPARK-32332][SQL] Support columnar exchanges a025a89 is described below commit a025a89f4ef3a05d7e70c02f03a9826bb97eceac Author: Wenchen Fan AuthorDate: Wed Jul 29 14:21:47 2020 -0500 [SPARK-32332][SQL] Support columnar exchanges ### What changes were proposed in this pull request? This PR adds abstract classes for shuffle and broadcast, so that users can provide their columnar implementations. This PR updates several places to use the abstract exchange classes, and also update `AdaptiveSparkPlanExec` so that the columnar rules can see exchange nodes. This is an alternative of https://github.com/apache/spark/pull/29134 . Close https://github.com/apache/spark/pull/29134 ### Why are the changes needed? To allow columnar exchanges. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? new tests Closes #29262 from cloud-fan/columnar. Authored-by: Wenchen Fan Signed-off-by: Thomas Graves --- .../execution/adaptive/AdaptiveSparkPlanExec.scala | 30 -- .../adaptive/CustomShuffleReaderExec.scala | 21 ++-- .../adaptive/OptimizeLocalShuffleReader.scala | 5 +- .../execution/adaptive/OptimizeSkewedJoin.scala| 4 +- .../sql/execution/adaptive/QueryStageExec.scala| 37 --- .../sql/execution/adaptive/simpleCosting.scala | 6 +- .../execution/exchange/BroadcastExchangeExec.scala | 46 ++-- .../execution/exchange/ShuffleExchangeExec.scala | 57 +- .../execution/streaming/IncrementalExecution.scala | 4 +- .../spark/sql/SparkSessionExtensionSuite.scala | 120 + 10 files changed, 260 insertions(+), 70 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 34db0a3..b160b8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -100,7 +100,12 @@ case class AdaptiveSparkPlanExec( // The following two rules need to make use of 'CustomShuffleReaderExec.partitionSpecs' // added by `CoalesceShufflePartitions`. So they must be executed after it. OptimizeSkewedJoin(conf), -OptimizeLocalShuffleReader(conf), +OptimizeLocalShuffleReader(conf) + ) + + // A list of physical optimizer rules to be applied right after a new stage is created. The input + // plan to these rules has exchange as its root node. + @transient private val postStageCreationRules = Seq( ApplyColumnarRulesAndInsertTransitions(conf, context.session.sessionState.columnarRules), CollapseCodegenStages(conf) ) @@ -227,7 +232,8 @@ case class AdaptiveSparkPlanExec( } // Run the final plan when there's no more unfinished stages. - currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) + currentPhysicalPlan = applyPhysicalRules( +result.newPlan, queryStageOptimizerRules ++ postStageCreationRules) isFinalPlan = true executionId.foreach(onUpdatePlan(_, Seq(currentPhysicalPlan))) currentPhysicalPlan @@ -376,10 +382,22 @@ case class AdaptiveSparkPlanExec( private def newQueryStage(e: Exchange): QueryStageExec = { val optimizedPlan = applyPhysicalRules(e.child, queryStageOptimizerRules) val queryStage = e match { - case s: ShuffleExchangeExec => -ShuffleQueryStageExec(currentStageId, s.copy(child = optimizedPlan)) - case b: BroadcastExchangeExec => -BroadcastQueryStageExec(currentStageId, b.copy(child = optimizedPlan)) + case s: ShuffleExchangeLike => +val newShuffle = applyPhysicalRules( + s.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) +if (!newShuffle.isInstanceOf[ShuffleExchangeLike]) { + throw new IllegalStateException( +"Custom columnar rules cannot transform shuffle node to something else.") +} +ShuffleQueryStageExec(currentStageId, newShuffle) + case b: BroadcastExchangeLike => +val newBroadcast = applyPhysicalRules( + b.withNewChildren(Seq(optimizedPlan)), postStageCreationRules) +if (!newBroadcast.isInstanceOf[BroadcastExchangeLike]) { + throw new IllegalStateException( +"Custom columnar rules cannot transform broadcast node to something else.") +} +BroadcastQueryStageExec(currentStageId
[spark] branch master updated: [SPARK-30322][DOCS] Add stage level scheduling docs
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e926d41 [SPARK-30322][DOCS] Add stage level scheduling docs e926d41 is described below commit e926d419d305c9400f6f2426ca3e8d04a9180005 Author: Thomas Graves AuthorDate: Wed Jul 29 13:46:28 2020 -0500 [SPARK-30322][DOCS] Add stage level scheduling docs ### What changes were proposed in this pull request? Document the stage level scheduling feature. ### Why are the changes needed? Document the stage level scheduling feature. ### Does this PR introduce _any_ user-facing change? Documentation. ### How was this patch tested? n/a docs only Closes #29292 from tgravescs/SPARK-30322. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- docs/configuration.md | 7 +++ docs/running-on-yarn.md | 4 2 files changed, 11 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index abf7610..62799db 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3028,3 +3028,10 @@ There are configurations available to request resources for the driver: sp Spark will use the configurations specified to first request containers with the corresponding resources from the cluster manager. Once it gets the container, Spark launches an Executor in that container which will discover what resources the container has and the addresses associated with each resource. The Executor will register with the Driver and report back the resources available to that Executor. The Spark scheduler can then schedule tasks to each Executor and assign specific reso [...] See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with Mesos or local mode. And please also note that local-cluster mode with multiple workers is not supported(see Standalon [...] + +# Stage Level Scheduling Overview + +The stage level scheduling feature allows users to specify task and executor resource requirements at the stage level. This allows for different stages to run with executors that have different resources. A prime example of this is one ETL stage runs with executors with just CPUs, the next stage is an ML stage that needs GPUs. Stage level scheduling allows for user to request different executors that have GPUs when the ML stage runs rather then having to acquire executors with GPUs at th [...] +This is only available for the RDD API in Scala, Java, and Python and requires dynamic allocation to be enabled. It is only available on YARN at this time. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page for more implementation details. + +See the `RDD.withResources` and `ResourceProfileBuilder` API's for using this feature. The current implementation acquires new executors for each `ResourceProfile` created and currently has to be an exact match. Spark does not try to fit tasks into an executor that require a different ResourceProfile than the executor was created with. Executors that are not in use will idle timeout with the dynamic allocation logic. The default configuration for this feature is to only allow one Resour [...] diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 36d8f0b..6f7aaf2b 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -641,6 +641,10 @@ If the user has a user defined YARN resource, lets call it `acceleratorX` then t YARN does not tell Spark the addresses of the resources allocated to each container. For that reason, the user must specify a discovery script that gets run by the executor on startup to discover what resources are available to that executor. You can find an example scripts in `examples/src/main/scripts/getGpusResources.sh`. The script must have execute permissions set and the user should setup permissions to not allow malicious users to modify it. The script should write to STDOUT a JSO [...] +# Stage Level Scheduling Overview + +Stage level scheduling is supported on YARN when dynamic allocation is enabled. One thing to note that is YARN specific is that each ResourceProfile requires a different container priority on YARN. The mapping is simply the ResourceProfile id becomes the priority, on YARN lower numbers are higher priority. This means that profiles created earlier will have a higher priority in YARN. Normally this won't matter as Spark finishes one stage before starting another one, the only cas
[spark] branch branch-3.0 updated: [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new e5b5b7e [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread e5b5b7e is described below commit e5b5b7e507ab974bbca3abb0bbf56bf67696d53e Author: Kousuke Saruta AuthorDate: Wed Jul 29 08:44:56 2020 -0500 [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread ### What changes were proposed in this pull request? This PR changes the order between initialization for ExecutorPlugin and starting heartbeat thread in Executor. ### Why are the changes needed? In the current master, heartbeat thread in a executor starts after plugin initialization so if the initialization takes long time, heartbeat is not sent to driver and the executor will be removed from cluster. ### Does this PR introduce _any_ user-facing change? Yes. Plugins for executors will be allowed to take long time for initialization. ### How was this patch tested? New testcase. Closes #29002 from sarutak/fix-heartbeat-issue. Authored-by: Kousuke Saruta Signed-off-by: Thomas Graves (cherry picked from commit 9be088357eff4328248b29a3a49a816756745345) Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/TestUtils.scala| 15 - .../scala/org/apache/spark/executor/Executor.scala | 12 ++-- .../org/apache/spark/executor/ExecutorSuite.scala | 72 +- 3 files changed, 89 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index d459627..1e00769 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -179,11 +179,20 @@ private[spark] object TestUtils { destDir: File, toStringValue: String = "", baseClass: String = null, - classpathUrls: Seq[URL] = Seq.empty): File = { + classpathUrls: Seq[URL] = Seq.empty, + implementsClasses: Seq[String] = Seq.empty, + extraCodeBody: String = ""): File = { val extendsText = Option(baseClass).map { c => s" extends ${c}" }.getOrElse("") +val implementsText = + "implements " + (implementsClasses :+ "java.io.Serializable").mkString(", ") val sourceFile = new JavaSourceFromString(className, - "public class " + className + extendsText + " implements java.io.Serializable {" + - " @Override public String toString() { return \"" + toStringValue + "\"; }}") + s""" + |public class $className $extendsText $implementsText { + | @Override public String toString() { return "$toStringValue"; } + | + | $extraCodeBody + |} +""".stripMargin) createCompiledClass(className, destDir, sourceFile, classpathUrls) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 8aeb16f..e9f1d9c 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -153,11 +153,6 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) - // Plugins need to load using a class loader that includes the executor's user classpath - private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { -PluginContainer(env, resources.asJava) - } - // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. private val maxDirectResultSize = Math.min( @@ -218,6 +213,13 @@ private[spark] class Executor( heartbeater.start() + // Plugins need to load using a class loader that includes the executor's user classpath. + // Plugins also needs to be initialized after the heartbeater started + // to avoid blocking to send heartbeat (see SPARK-32175). + private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { +PluginContainer(env, resources.asJava) + } + metricsPoller.start() private[executor] def numRunningTasks: Int = runningTasks.size() diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 31049d1..b198448 100644
[spark] branch master updated: [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9be0883 [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread 9be0883 is described below commit 9be088357eff4328248b29a3a49a816756745345 Author: Kousuke Saruta AuthorDate: Wed Jul 29 08:44:56 2020 -0500 [SPARK-32175][CORE] Fix the order between initialization for ExecutorPlugin and starting heartbeat thread ### What changes were proposed in this pull request? This PR changes the order between initialization for ExecutorPlugin and starting heartbeat thread in Executor. ### Why are the changes needed? In the current master, heartbeat thread in a executor starts after plugin initialization so if the initialization takes long time, heartbeat is not sent to driver and the executor will be removed from cluster. ### Does this PR introduce _any_ user-facing change? Yes. Plugins for executors will be allowed to take long time for initialization. ### How was this patch tested? New testcase. Closes #29002 from sarutak/fix-heartbeat-issue. Authored-by: Kousuke Saruta Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/TestUtils.scala| 15 - .../scala/org/apache/spark/executor/Executor.scala | 12 ++-- .../org/apache/spark/executor/ExecutorSuite.scala | 72 +- 3 files changed, 89 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 259cc43..6947d1c 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -179,11 +179,20 @@ private[spark] object TestUtils { destDir: File, toStringValue: String = "", baseClass: String = null, - classpathUrls: Seq[URL] = Seq.empty): File = { + classpathUrls: Seq[URL] = Seq.empty, + implementsClasses: Seq[String] = Seq.empty, + extraCodeBody: String = ""): File = { val extendsText = Option(baseClass).map { c => s" extends ${c}" }.getOrElse("") +val implementsText = + "implements " + (implementsClasses :+ "java.io.Serializable").mkString(", ") val sourceFile = new JavaSourceFromString(className, - "public class " + className + extendsText + " implements java.io.Serializable {" + - " @Override public String toString() { return \"" + toStringValue + "\"; }}") + s""" + |public class $className $extendsText $implementsText { + | @Override public String toString() { return "$toStringValue"; } + | + | $extraCodeBody + |} +""".stripMargin) createCompiledClass(className, destDir, sourceFile, classpathUrls) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bc0f0c0..d220029 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -154,11 +154,6 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) - // Plugins need to load using a class loader that includes the executor's user classpath - private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { -PluginContainer(env, resources.asJava) - } - // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. private val maxDirectResultSize = Math.min( @@ -225,6 +220,13 @@ private[spark] class Executor( heartbeater.start() + // Plugins need to load using a class loader that includes the executor's user classpath. + // Plugins also needs to be initialized after the heartbeater started + // to avoid blocking to send heartbeat (see SPARK-32175). + private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { +PluginContainer(env, resources.asJava) + } + metricsPoller.start() private[executor] def numRunningTasks: Int = runningTasks.size() diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 31049d1..b198448 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/E
[spark] branch master updated: [SPARK-30794][CORE] Stage Level scheduling: Add ability to set off heap memory
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 998086c [SPARK-30794][CORE] Stage Level scheduling: Add ability to set off heap memory 998086c is described below commit 998086c9a179692b2687bc9a104dbbb35f5a44e2 Author: Warren Zhu AuthorDate: Mon Jul 27 08:16:13 2020 -0500 [SPARK-30794][CORE] Stage Level scheduling: Add ability to set off heap memory ### What changes were proposed in this pull request? Support set off heap memory in `ExecutorResourceRequests` ### Why are the changes needed? Support stage level scheduling ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added UT in `ResourceProfileSuite` and `DAGSchedulerSuite` Closes #28972 from warrenzhu25/30794. Authored-by: Warren Zhu Signed-off-by: Thomas Graves --- .../spark/resource/ExecutorResourceRequests.scala | 14 ++ .../apache/spark/resource/ResourceProfile.scala| 8 +++- .../spark/resource/ResourceProfileSuite.scala | 50 +++--- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 5 ++- python/pyspark/resource/requests.py| 9 python/pyspark/resource/tests/test_resources.py| 5 ++- .../apache/spark/deploy/yarn/YarnAllocator.scala | 3 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 23 -- 8 files changed, 102 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index 9da6ffb..654afa0 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -55,6 +55,20 @@ class ExecutorResourceRequests() extends Serializable { } /** + * Specify off heap memory. The value specified will be converted to MiB. + * This value only take effect when MEMORY_OFFHEAP_ENABLED is true. + * + * @param amount Amount of memory. In the same format as JVM memory strings (e.g. 512m, 2g). + * Default unit is MiB if not specified. + */ + def offHeapMemory(amount: String): this.type = { +val amountMiB = JavaUtils.byteStringAsMb(amount) +val req = new ExecutorResourceRequest(OFFHEAP_MEM, amountMiB) +_executorResources.put(OFFHEAP_MEM, req) +this + } + + /** * Specify overhead memory. The value specified will be converted to MiB. * * @param amount Amount of memory. In the same format as JVM memory strings (e.g. 512m, 2g). diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index f56ea69..8a37670 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -243,13 +243,15 @@ object ResourceProfile extends Logging { // task resources val CPUS = "cpus" // Executor resources + // Make sure add new executor resource in below allSupportedExecutorResources val CORES = "cores" val MEMORY = "memory" + val OFFHEAP_MEM = "offHeap" val OVERHEAD_MEM = "memoryOverhead" val PYSPARK_MEM = "pyspark.memory" // all supported spark executor resources (minus the custom resources like GPUs/FPGAs) - val allSupportedExecutorResources = Seq(CORES, MEMORY, OVERHEAD_MEM, PYSPARK_MEM) + val allSupportedExecutorResources = Seq(CORES, MEMORY, OVERHEAD_MEM, PYSPARK_MEM, OFFHEAP_MEM) val UNKNOWN_RESOURCE_PROFILE_ID = -1 val DEFAULT_RESOURCE_PROFILE_ID = 0 @@ -295,6 +297,10 @@ object ResourceProfile extends Logging { ereqs.memory(conf.get(EXECUTOR_MEMORY).toString) conf.get(EXECUTOR_MEMORY_OVERHEAD).map(mem => ereqs.memoryOverhead(mem.toString)) conf.get(PYSPARK_EXECUTOR_MEMORY).map(mem => ereqs.pysparkMemory(mem.toString)) +if (conf.get(MEMORY_OFFHEAP_ENABLED)) { + // Explicitly add suffix b as default unit of offHeapMemory is Mib + ereqs.offHeapMemory(conf.get(MEMORY_OFFHEAP_SIZE).toString + "b") +} val execReq = ResourceUtils.parseAllResourceRequests(conf, SPARK_EXECUTOR_PREFIX) execReq.foreach { req => val name = req.id.resourceName diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index 29d3ef1..d0479ca 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -18,7 +18,7 @@ package org.apac
[spark] branch master updated: [SPARK-31418][SCHEDULER] Request more executors in case of dynamic allocation is enabled and a task becomes unschedulable due to spark's blacklisting feature
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e7fb67c [SPARK-31418][SCHEDULER] Request more executors in case of dynamic allocation is enabled and a task becomes unschedulable due to spark's blacklisting feature e7fb67c is described below commit e7fb67cd880511452b94c2077429868e72998c05 Author: Venkata krishnan Sowrirajan AuthorDate: Thu Jul 23 12:33:22 2020 -0500 [SPARK-31418][SCHEDULER] Request more executors in case of dynamic allocation is enabled and a task becomes unschedulable due to spark's blacklisting feature ### What changes were proposed in this pull request? In this change, when dynamic allocation is enabled instead of aborting immediately when there is an unschedulable taskset due to blacklisting, pass an event saying `SparkListenerUnschedulableTaskSetAdded` which will be handled by `ExecutorAllocationManager` and request more executors needed to schedule the unschedulable blacklisted tasks. Once the event is sent, we start the abortTimer similar to [SPARK-22148][SPARK-15815] to abort in the case when no new executors launched either due [...] ### Why are the changes needed? This is an improvement. In the case when dynamic allocation is enabled, this would request more executors to schedule the unschedulable tasks instead of aborting the stage without even retrying upto spark.task.maxFailures times (in some cases not retrying at all). This is a potential issue with respect to Spark's Fault tolerance. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added unit tests both in ExecutorAllocationManagerSuite and TaskSchedulerImplSuite Closes #28287 from venkata91/SPARK-31418. Authored-by: Venkata krishnan Sowrirajan Signed-off-by: Thomas Graves --- .../org/apache/spark/SparkFirehoseListener.java| 10 ++ .../apache/spark/ExecutorAllocationManager.scala | 55 ++- .../org/apache/spark/scheduler/DAGScheduler.scala | 38 + .../apache/spark/scheduler/DAGSchedulerEvent.scala | 8 + .../org/apache/spark/scheduler/SparkListener.scala | 30 .../apache/spark/scheduler/SparkListenerBus.scala | 4 + .../apache/spark/scheduler/TaskSchedulerImpl.scala | 55 +-- .../spark/ExecutorAllocationManagerSuite.scala | 175 - .../spark/scheduler/TaskSchedulerImplSuite.scala | 36 + 9 files changed, 392 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 579e7ff..c0e72b5 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -162,6 +162,16 @@ public class SparkFirehoseListener implements SparkListenerInterface { onEvent(speculativeTask); } + public void onUnschedulableTaskSetAdded( + SparkListenerUnschedulableTaskSetAdded unschedulableTaskSetAdded) { +onEvent(unschedulableTaskSetAdded); + } + + public void onUnschedulableTaskSetRemoved( + SparkListenerUnschedulableTaskSetRemoved unschedulableTaskSetRemoved) { +onEvent(unschedulableTaskSetRemoved); + } + @Override public void onResourceProfileAdded(SparkListenerResourceProfileAdded event) { onEvent(event); diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 620a6fe..85409d5 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -281,6 +281,7 @@ private[spark] class ExecutorAllocationManager( private def maxNumExecutorsNeededPerResourceProfile(rpId: Int): Int = { val pending = listener.totalPendingTasksPerResourceProfile(rpId) val pendingSpeculative = listener.pendingSpeculativeTasksPerResourceProfile(rpId) +val unschedulableTaskSets = listener.pendingUnschedulableTaskSetsPerResourceProfile(rpId) val running = listener.totalRunningTasksPerResourceProfile(rpId) val numRunningOrPendingTasks = pending + running val rp = resourceProfileManager.resourceProfileFromId(rpId) @@ -289,13 +290,27 @@ private[spark] class ExecutorAllocationManager( s" tasksperexecutor: $tasksPerExecutor") val maxNeeded = math.ceil(numRunningOrPendingTasks * executorAllocationRatio / tasksPerExecutor).toInt -if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) { + +val maxNeededWithSpeculationLocalityOffset = + if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) {
[spark] branch 3.0.0 created (now 4da93b0)
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a change to branch 3.0.0 in repository https://gitbox.apache.org/repos/asf/spark.git. at 4da93b0 [SPARK-32363][PYTHON][BUILD] Fix flakiness in pip package testing in Jenkins No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch 3.0.0 created (now 4da93b0)
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a change to branch 3.0.0 in repository https://gitbox.apache.org/repos/asf/spark.git. at 4da93b0 [SPARK-32363][PYTHON][BUILD] Fix flakiness in pip package testing in Jenkins No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch 3.0.0 created (now 4da93b0)
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a change to branch 3.0.0 in repository https://gitbox.apache.org/repos/asf/spark.git. at 4da93b0 [SPARK-32363][PYTHON][BUILD] Fix flakiness in pip package testing in Jenkins No new revisions were added by this update. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new cf22d94 [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature cf22d94 is described below commit cf22d947fb8f37aa4d394b6633d6f08dbbf6dc1c Author: Erik Krogen AuthorDate: Wed Jul 15 11:40:55 2020 -0500 [SPARK-32036] Replace references to blacklist/whitelist language with more appropriate terminology, excluding the blacklisting feature ### What changes were proposed in this pull request? This PR will remove references to these "blacklist" and "whitelist" terms besides the blacklisting feature as a whole, which can be handled in a separate JIRA/PR. This touches quite a few files, but the changes are straightforward (variable/method/etc. name changes) and most quite self-contained. ### Why are the changes needed? As per discussion on the Spark dev list, it will be beneficial to remove references to problematic language that can alienate potential community members. One such reference is "blacklist" and "whitelist". While it seems to me that there is some valid debate as to whether these terms have racist origins, the cultural connotations are inescapable in today's world. ### Does this PR introduce _any_ user-facing change? In the test file `HiveQueryFileTest`, a developer has the ability to specify the system property `spark.hive.whitelist` to specify a list of Hive query files that should be tested. This system property has been renamed to `spark.hive.includelist`. The old property has been kept for compatibility, but will log a warning if used. I am open to feedback from others on whether keeping a deprecated property here is unnecessary given that this is just for developers running tests. ### How was this patch tested? Existing tests should be suitable since no behavior changes are expected as a result of this PR. Closes #28874 from xkrogen/xkrogen-SPARK-32036-rename-blacklists. Authored-by: Erik Krogen Signed-off-by: Thomas Graves --- R/pkg/tests/fulltests/test_context.R | 2 +- R/pkg/tests/fulltests/test_sparkSQL.R | 8 ++-- R/pkg/tests/run-all.R | 4 +- .../java/org/apache/spark/network/crypto/README.md | 2 +- .../spark/deploy/history/FsHistoryProvider.scala | 29 +++-- .../spark/deploy/rest/RestSubmissionClient.scala | 4 +- .../spark/scheduler/OutputCommitCoordinator.scala | 2 +- .../scala/org/apache/spark/util/JsonProtocol.scala | 4 +- .../test/scala/org/apache/spark/ThreadAudit.scala | 4 +- .../org/apache/spark/deploy/SparkSubmitSuite.scala | 22 +- .../deploy/history/FsHistoryProviderSuite.scala| 8 ++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 14 +++--- dev/sparktestsupport/modules.py| 10 ++--- docs/streaming-programming-guide.md| 50 +++--- .../streaming/JavaRecoverableNetworkWordCount.java | 20 - .../streaming/recoverable_network_wordcount.py | 16 +++ .../streaming/RecoverableNetworkWordCount.scala| 16 +++ .../scala/org/apache/spark/util/DockerUtils.scala | 6 +-- project/SparkBuild.scala | 4 +- python/pylintrc| 2 +- python/pyspark/cloudpickle.py | 6 +-- python/pyspark/sql/functions.py| 4 +- python/pyspark/sql/pandas/typehints.py | 4 +- python/run-tests.py| 2 +- .../cluster/mesos/MesosClusterScheduler.scala | 4 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 2 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../spark/sql/catalyst/json/JSONOptions.scala | 10 ++--- .../spark/sql/catalyst/optimizer/Optimizer.scala | 34 +++ .../spark/sql/catalyst/optimizer/expressions.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 2 +- .../spark/sql/catalyst/rules/RuleExecutor.scala| 6 +-- .../catalyst/optimizer/FilterPushdownSuite.scala | 2 +- .../PullupCorrelatedPredicatesSuite.scala | 2 +- .../datasources/json/JsonOutputWriter.scala| 2 +- .../inputs/{blacklist.sql => ignored.sql} | 2 +- .../org/apache/spark/sql/SQLQueryTestSuite.scala | 6 +-- .../org/apache/spark/sql/TPCDSQuerySuite.scala | 4 +- .../sql/execution/datasources/json/JsonSuite.scala | 2 +- .../thriftserver/ThriftServerQueryTestSuite.scala | 4 +- .../hive/execution/HiveCompatibilitySuite.scala| 16 +++ .../execution/HiveWindowFunctionQuerySuite.scala | 8 ++-- .
[spark] branch branch-3.0 updated: [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 7d1b6b1 [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab 7d1b6b1 is described below commit 7d1b6b148bd6dc84b214e471707e20569258a3d7 Author: TJX2014 AuthorDate: Tue Jun 30 08:56:59 2020 -0500 [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab ### What changes were proposed in this pull request? `formatDate` in utils.js `org/apache/spark/ui/static/utils.js` is partly refactored. ### Why are the changes needed? In branch-2.4,task launch time is returned as html string from driver, while in branch-3.x,this is returned in JSON Object as`Date`type from `org.apache.spark.status.api.v1.TaskData` Due to: LaunchTime from jersey server in spark driver is correct, which will be converted to date string like `2020-06-28T02:57:42.605GMT` in json object, then the formatDate in utils.js treat it as date.split(".")[0].replace("T", " "). So `2020-06-28T02:57:42.605GMT` will be converted to `2020-06-28 02:57:42`, but correct is `2020-06-28 10:57:42` in GMT+8 timezone. ![选区_071](https://user-images.githubusercontent.com/7149304/85937186-b6d36780-b933-11ea-8382-80a3891f1c2a.png) ![选区_070](https://user-images.githubusercontent.com/7149304/85937190-bcc94880-b933-11ea-8860-2083c97ea269.png) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual test. Closes #28918 from TJX2014/master-SPARK-32068-ui-task-lauch-time-tz. Authored-by: TJX2014 Signed-off-by: Thomas Graves (cherry picked from commit 165c948e3297fd5ebcee771fcbf7a0c20fccfaae) Signed-off-by: Thomas Graves --- core/src/main/resources/org/apache/spark/ui/static/utils.js | 11 +-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js index 6fc34a9..2e46111 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/utils.js +++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js @@ -56,13 +56,17 @@ function formatTimeMillis(timeMillis) { return "-"; } else { var dt = new Date(timeMillis); +return formatDateString(dt); + } +} + +function formatDateString(dt) { return dt.getFullYear() + "-" + padZeroes(dt.getMonth() + 1) + "-" + padZeroes(dt.getDate()) + " " + padZeroes(dt.getHours()) + ":" + padZeroes(dt.getMinutes()) + ":" + padZeroes(dt.getSeconds()); - } } function getTimeZone() { @@ -161,7 +165,10 @@ function setDataTableDefaults() { function formatDate(date) { if (date <= 0) return "-"; - else return date.split(".")[0].replace("T", " "); + else { + var dt = new Date(date.replace("GMT", "Z")) + return formatDateString(dt); + } } function createRESTEndPointForExecutorsPage(appId) { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 165c948 [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab 165c948 is described below commit 165c948e3297fd5ebcee771fcbf7a0c20fccfaae Author: TJX2014 AuthorDate: Tue Jun 30 08:56:59 2020 -0500 [SPARK-32068][WEBUI] Correct task lauchtime show issue due to timezone in stage tab ### What changes were proposed in this pull request? `formatDate` in utils.js `org/apache/spark/ui/static/utils.js` is partly refactored. ### Why are the changes needed? In branch-2.4,task launch time is returned as html string from driver, while in branch-3.x,this is returned in JSON Object as`Date`type from `org.apache.spark.status.api.v1.TaskData` Due to: LaunchTime from jersey server in spark driver is correct, which will be converted to date string like `2020-06-28T02:57:42.605GMT` in json object, then the formatDate in utils.js treat it as date.split(".")[0].replace("T", " "). So `2020-06-28T02:57:42.605GMT` will be converted to `2020-06-28 02:57:42`, but correct is `2020-06-28 10:57:42` in GMT+8 timezone. ![选区_071](https://user-images.githubusercontent.com/7149304/85937186-b6d36780-b933-11ea-8382-80a3891f1c2a.png) ![选区_070](https://user-images.githubusercontent.com/7149304/85937190-bcc94880-b933-11ea-8860-2083c97ea269.png) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manual test. Closes #28918 from TJX2014/master-SPARK-32068-ui-task-lauch-time-tz. Authored-by: TJX2014 Signed-off-by: Thomas Graves --- core/src/main/resources/org/apache/spark/ui/static/utils.js | 11 +-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js index 6fc34a9..2e46111 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/utils.js +++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js @@ -56,13 +56,17 @@ function formatTimeMillis(timeMillis) { return "-"; } else { var dt = new Date(timeMillis); +return formatDateString(dt); + } +} + +function formatDateString(dt) { return dt.getFullYear() + "-" + padZeroes(dt.getMonth() + 1) + "-" + padZeroes(dt.getDate()) + " " + padZeroes(dt.getHours()) + ":" + padZeroes(dt.getMinutes()) + ":" + padZeroes(dt.getSeconds()); - } } function getTimeZone() { @@ -161,7 +165,10 @@ function setDataTableDefaults() { function formatDate(date) { if (date <= 0) return "-"; - else return date.split(".")[0].replace("T", " "); + else { + var dt = new Date(date.replace("GMT", "Z")) + return formatDateString(dt); + } } function createRESTEndPointForExecutorsPage(appId) { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-31029] Avoid using global execution context in driver main thread for YarnSchedulerBackend
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3c34e45 [SPARK-31029] Avoid using global execution context in driver main thread for YarnSchedulerBackend 3c34e45 is described below commit 3c34e45df4b3e0610dde5334716025a85cbbc05b Author: Shanyu Zhao AuthorDate: Fri Jun 19 09:59:14 2020 -0500 [SPARK-31029] Avoid using global execution context in driver main thread for YarnSchedulerBackend #31029 # What changes were proposed in this pull request? In YarnSchedulerBackend, we should avoid using the global execution context for its Future. Otherwise if user's Spark application also uses global execution context for its Future, the user is facing indeterministic behavior in terms of the thread's context class loader. ### Why are the changes needed? When running tpc-ds test (https://github.com/databricks/spark-sql-perf), occasionally we see error related to class not found: 2020-02-04 20:00:26,673 ERROR yarn.ApplicationMaster: User class threw exception: scala.ScalaReflectionException: class com.databricks.spark.sql.perf.ExperimentRun in JavaMirror with sun.misc.Launcher$AppClassLoader28ba21f3 of type class sun.misc.Launcher$AppClassLoader with classpath [...] and parent being sun.misc.Launcher$ExtClassLoader3ff5d147 of type class sun.misc.Launcher$ExtClassLoader with classpath [...] and parent being primordial classloader with boot classpath [...] not found. This is the root cause for the problem: Spark driver starts ApplicationMaster in the main thread, which starts a user thread and set MutableURLClassLoader to that thread's ContextClassLoader. userClassThread = startUserApplication() The main thread then setup YarnSchedulerBackend RPC endpoints, which handles these calls using scala Future with the default global ExecutionContext: doRequestTotalExecutors doKillExecutors So for the main thread and user thread, whoever starts the future first get a chance to set ContextClassLoader to the default thread pool: - If main thread starts a future to handle doKillExecutors() before user thread does then the default thread pool thread's ContextClassLoader would be the default (AppClassLoader). - If user thread starts a future first then the thread pool thread will have MutableURLClassLoader. Note that only MutableURLClassLoader can load user provided class for the Spark app, you will see errors related to class not found if the ContextClassLoader is AppClassLoader. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing unit tests and manual tests Closes #27843 from shanyu/shanyu-31029. Authored-by: Shanyu Zhao Signed-off-by: Thomas Graves --- .../apache/spark/scheduler/cluster/YarnSchedulerBackend.scala | 11 +-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index e428bab..0475b0a 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -21,8 +21,7 @@ import java.util.EnumSet import java.util.concurrent.atomic.{AtomicBoolean} import javax.servlet.DispatcherType -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.Future +import scala.concurrent.{ExecutionContext, Future} import scala.util.{Failure, Success} import scala.util.control.NonFatal @@ -67,6 +66,14 @@ private[spark] abstract class YarnSchedulerBackend( private implicit val askTimeout = RpcUtils.askRpcTimeout(sc.conf) + /** + * Declare implicit single thread execution context for futures doRequestTotalExecutors and + * doKillExecutors below, avoiding using the global execution context that may cause conflict + * with user code's execution of futures. + */ + private implicit val schedulerEndpointEC = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonSingleThreadExecutor("yarn-scheduler-endpoint")) + /** Application ID. */ protected var appId: Option[ApplicationId] = None - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-30845] Do not upload local pyspark archives for spark-submit on Yarn
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 37b7d32 [SPARK-30845] Do not upload local pyspark archives for spark-submit on Yarn 37b7d32 is described below commit 37b7d32dbd3546c303d31305ed40c6435390bb4d Author: Shanyu Zhao AuthorDate: Mon Jun 8 15:55:49 2020 -0500 [SPARK-30845] Do not upload local pyspark archives for spark-submit on Yarn ### What changes were proposed in this pull request? Use spark-submit to submit a pyspark app on Yarn, and set this in spark-env.sh: export PYSPARK_ARCHIVES_PATH=local:/opt/spark/python/lib/pyspark.zip,local:/opt/spark/python/lib/py4j-0.10.7-src.zip You can see that these local archives are still uploaded to Yarn distributed cache: yarn.Client: Uploading resource file:/opt/spark/python/lib/pyspark.zip -> hdfs://myhdfs/user/test1/.sparkStaging/application_1581024490249_0001/pyspark.zip This PR fix this issue by checking the files specified in PYSPARK_ARCHIVES_PATH, if they are local archives, don't distribute to Yarn dist cache. ### Why are the changes needed? For pyspark appp to support local pyspark archives set in PYSPARK_ARCHIVES_PATH. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests and manual tests. Closes #27598 from shanyu/shanyu-30845. Authored-by: Shanyu Zhao Signed-off-by: Thomas Graves --- .../yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index fc429d6..7b12119 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -635,7 +635,12 @@ private[spark] class Client( distribute(args.primaryPyFile, appMasterOnly = true) } -pySparkArchives.foreach { f => distribute(f) } +pySparkArchives.foreach { f => + val uri = Utils.resolveURI(f) + if (uri.getScheme != Utils.LOCAL_SCHEME) { +distribute(f) + } +} // The python files list needs to be treated especially. All files that are not an // archive need to be placed in a subdirectory that will be added to PYTHONPATH. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-29303][WEB UI] Add UI support for stage level scheduling
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b64688e [SPARK-29303][WEB UI] Add UI support for stage level scheduling b64688e is described below commit b64688ebbaac7afd3734c0d84d1e77b1fd2d2e9d Author: Thomas Graves AuthorDate: Thu May 21 13:11:35 2020 -0500 [SPARK-29303][WEB UI] Add UI support for stage level scheduling ### What changes were proposed in this pull request? This adds UI updates to support stage level scheduling and ResourceProfiles. 3 main things have been added. ResourceProfile id added to the Stage page, the Executors page now has an optional selectable column to show the ResourceProfile Id of each executor, and the Environment page now has a section with the ResourceProfile ids. Along with this the rest api for environment page was updated to include the Resource profile information. I debating on splitting the resource profile information into its own page but I wasn't sure it called for a completely separate page. Open to peoples thoughts on this. Screen shots: ![Screen Shot 2020-04-01 at 3 07 46 PM](https://user-images.githubusercontent.com/4563792/78185169-469a7000-7430-11ea-8b0c-d9ede2d41df8.png) ![Screen Shot 2020-04-01 at 3 08 14 PM](https://user-images.githubusercontent.com/4563792/78185175-48fcca00-7430-11ea-8d1d-6b9333700f32.png) ![Screen Shot 2020-04-01 at 3 09 03 PM](https://user-images.githubusercontent.com/4563792/78185176-4a2df700-7430-11ea-92d9-73c382bb0f32.png) ![Screen Shot 2020-04-01 at 11 05 48 AM](https://user-images.githubusercontent.com/4563792/78185186-4dc17e00-7430-11ea-8962-f749dd47ea60.png) ### Why are the changes needed? For user to be able to know what resource profile was used with which stage and executors. The resource profile information is also available so user debugging can see exactly what resources were requested with that profile. ### Does this PR introduce any user-facing change? Yes, UI updates. ### How was this patch tested? Unit tests and tested on yarn both active applications and with the history server. Closes #28094 from tgravescs/SPARK-29303-pr. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../org/apache/spark/SparkFirehoseListener.java| 5 + .../spark/ui/static/executorspage-template.html| 1 + .../org/apache/spark/ui/static/executorspage.js| 7 +- .../main/scala/org/apache/spark/SparkContext.scala | 2 +- .../deploy/history/HistoryAppStatusStore.scala | 3 +- .../spark/resource/ResourceProfileManager.scala| 7 +- .../spark/scheduler/EventLoggingListener.scala | 4 + .../org/apache/spark/scheduler/SparkListener.scala | 12 +++ .../apache/spark/scheduler/SparkListenerBus.scala | 2 + .../apache/spark/status/AppStatusListener.scala| 33 +- .../org/apache/spark/status/AppStatusStore.scala | 8 +- .../scala/org/apache/spark/status/LiveEntity.scala | 25 - .../status/api/v1/OneApplicationResource.scala | 4 +- .../scala/org/apache/spark/status/api/v1/api.scala | 18 +++- .../scala/org/apache/spark/status/storeTypes.scala | 7 ++ .../org/apache/spark/ui/env/EnvironmentPage.scala | 48 + .../scala/org/apache/spark/ui/jobs/JobPage.scala | 4 +- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 4 + .../scala/org/apache/spark/util/JsonProtocol.scala | 101 +-- .../app_environment_expectation.json | 3 +- .../application_list_json_expectation.json | 15 +++ .../blacklisting_for_stage_expectation.json| 3 +- .../blacklisting_node_for_stage_expectation.json | 3 +- .../complete_stage_list_json_expectation.json | 9 +- .../completed_app_list_json_expectation.json | 15 +++ .../executor_list_json_expectation.json| 3 +- ...ist_with_executor_metrics_json_expectation.json | 12 ++- .../executor_memory_usage_expectation.json | 15 ++- .../executor_node_blacklisting_expectation.json| 15 ++- ...de_blacklisting_unblacklisting_expectation.json | 15 ++- .../executor_resource_information_expectation.json | 9 +- .../failed_stage_list_json_expectation.json| 3 +- .../limit_app_list_json_expectation.json | 30 +++--- .../minDate_app_list_json_expectation.json | 18 +++- .../minEndDate_app_list_json_expectation.json | 15 +++ .../multiple_resource_profiles_expectation.json| 112 + .../one_stage_attempt_json_expectation.json| 3 +- .../one_stage_json_expectation.json| 3 +- .../stage_list_json_expectation.json | 12 ++- ...age_list_with_accumulable_json_expectation
[spark] branch master updated: [SPARK-31235][FOLLOWUP][TESTS][TEST-HADOOP3.2] Fix test "specify a more specific type for the ap…
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c1801fd [SPARK-31235][FOLLOWUP][TESTS][TEST-HADOOP3.2] Fix test "specify a more specific type for the ap… c1801fd is described below commit c1801fd6da4a2dd5f37dc366b92bede669e8fda0 Author: wang-zhun AuthorDate: Fri May 8 15:41:23 2020 -0500 [SPARK-31235][FOLLOWUP][TESTS][TEST-HADOOP3.2] Fix test "specify a more specific type for the ap… ### What changes were proposed in this pull request? Update the input parameters for instantiating `RMAppManager` and `ClientRMService` ### Why are the changes needed? For hadoop3.2, if `RMAppManager` is not created correctly, the following exception will occur: ``` java.lang.RuntimeException: java.lang.NullPointerException at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:135) at org.apache.hadoop.yarn.security.YarnAuthorizationProvider.getInstance(YarnAuthorizationProvider.java:55) at org.apache.hadoop.yarn.server.resourcemanager.RMAppManager.(RMAppManager.java:117) ``` ### How was this patch tested? UTs Closes #28456 from wang-zhun/Fix-SPARK-31235. Authored-by: wang-zhun Signed-off-by: Thomas Graves --- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 86 +++--- 1 file changed, 43 insertions(+), 43 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index b335e7f..7611ccd 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.{ClientRMService, RMAppMana import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler import org.apache.hadoop.yarn.server.security.ApplicationACLsManager import org.apache.hadoop.yarn.util.Records import org.mockito.ArgumentMatchers.{any, anyBoolean, anyShort, eq => meq} @@ -222,11 +223,50 @@ class ClientSuite extends SparkFunSuite with Matchers { 3 -> ("SPARK-SQL", "SPARK-SQL"), 4 -> ("012345678901234567890123", "01234567890123456789")) +// Mock yarn submit application +val yarnClient = mock(classOf[YarnClient]) +val rmApps = new ConcurrentHashMap[ApplicationId, RMApp]() +val rmContext = mock(classOf[RMContext]) +when(rmContext.getRMApps).thenReturn(rmApps) +val dispatcher = mock(classOf[Dispatcher]) +when(rmContext.getDispatcher).thenReturn(dispatcher) +when[EventHandler[_]](dispatcher.getEventHandler).thenReturn( + new EventHandler[Event[_]] { +override def handle(event: Event[_]): Unit = {} + } +) +val writer = mock(classOf[RMApplicationHistoryWriter]) +when(rmContext.getRMApplicationHistoryWriter).thenReturn(writer) +val publisher = mock(classOf[SystemMetricsPublisher]) +when(rmContext.getSystemMetricsPublisher).thenReturn(publisher) +val yarnScheduler = mock(classOf[YarnScheduler]) +val rmAppManager = new RMAppManager(rmContext, + yarnScheduler, + null, + mock(classOf[ApplicationACLsManager]), + new Configuration()) +val clientRMService = new ClientRMService(rmContext, + yarnScheduler, + rmAppManager, + null, + null, + null) +clientRMService.init(new Configuration()) +when(yarnClient.submitApplication(any())).thenAnswer((invocationOnMock: InvocationOnMock) => { + val subContext = invocationOnMock.getArguments()(0) +.asInstanceOf[ApplicationSubmissionContext] + val request = Records.newRecord(classOf[SubmitApplicationRequest]) + request.setApplicationSubmissionContext(subContext) + clientRMService.submitApplication(request) + null +}) + +// Spark submit application +val appContext = spy(Records.newRecord(classOf[ApplicationSubmissionContext])) +when(appContext.getUnmanagedAM).thenReturn(true) for ((id, (sourceType, targetType)) <- appTypes) { val sparkConf = new SparkConf().set("spark.yarn.applicationType", sourceType) val args = new ClientArguments(Array()) - - val appContext = spy(Records.newRecord(classOf[ApplicationSubmissionContext])) val appId = ApplicationId.newInstance(123456, id) appContext.setApplication
[spark] branch branch-3.0 updated: [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 8c077b0 [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver 8c077b0 is described below commit 8c077b0a3e27e86ce3dbbf491521cd15b8f9fe79 Author: Akshat Bordia AuthorDate: Tue May 5 08:58:37 2020 -0500 [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver ### What changes were proposed in this pull request? Fixing an issue where Spark Master UI Fails to load if the application is waiting for workers to launch driver. **Root Cause:** This is happening due to the fact that the submitted application is waiting for a worker to be free to run the driver. Due to this resource is set to null in the formatResourcesAddresses method and this is running into null pointer exception. ![image](https://user-images.githubusercontent.com/31816865/80801557-77ee9300-8bca-11ea-92b7-b8df58b68de3.png) **Fix:** Added a null check before forming a resource address and display "None" if the driver isn't launched yet. ### Why are the changes needed? Spark Master UI should load as expected when applications are waiting for workers to run driver. ### Does this PR introduce _any_ user-facing change? The worker column in Spark Master UI will show "None" if the driver hasn't been launched yet. ![image](https://user-images.githubusercontent.com/31816865/80801671-be43f200-8bca-11ea-86c3-381925f82cc7.png) ### How was this patch tested? Tested on a local setup. Launched 2 applications and ensured that Spark Master UI loads fine. ![image](https://user-images.githubusercontent.com/31816865/80801883-5b9f2600-8bcb-11ea-8a1a-cc597aabc4c2.png) Closes #28429 from akshatb1/MasterUIBug. Authored-by: Akshat Bordia Signed-off-by: Thomas Graves (cherry picked from commit c71198ab6c8c9ded6a52eb97859b39dc2119b5fd) Signed-off-by: Thomas Graves --- core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index bf68ba8..252e704 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -37,7 +37,7 @@ private[deploy] class DriverInfo( @transient var worker: Option[WorkerInfo] = None // resources(e.f. gpu/fpga) allocated to this driver // map from resource name to ResourceInformation - private var _resources: Map[String, ResourceInformation] = _ + private var _resources: Map[String, ResourceInformation] = Map.empty init() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c71198a [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver c71198a is described below commit c71198ab6c8c9ded6a52eb97859b39dc2119b5fd Author: Akshat Bordia AuthorDate: Tue May 5 08:58:37 2020 -0500 [SPARK-31621][CORE] Fixing Spark Master UI Issue when application is waiting for workers to launch driver ### What changes were proposed in this pull request? Fixing an issue where Spark Master UI Fails to load if the application is waiting for workers to launch driver. **Root Cause:** This is happening due to the fact that the submitted application is waiting for a worker to be free to run the driver. Due to this resource is set to null in the formatResourcesAddresses method and this is running into null pointer exception. ![image](https://user-images.githubusercontent.com/31816865/80801557-77ee9300-8bca-11ea-92b7-b8df58b68de3.png) **Fix:** Added a null check before forming a resource address and display "None" if the driver isn't launched yet. ### Why are the changes needed? Spark Master UI should load as expected when applications are waiting for workers to run driver. ### Does this PR introduce _any_ user-facing change? The worker column in Spark Master UI will show "None" if the driver hasn't been launched yet. ![image](https://user-images.githubusercontent.com/31816865/80801671-be43f200-8bca-11ea-86c3-381925f82cc7.png) ### How was this patch tested? Tested on a local setup. Launched 2 applications and ensured that Spark Master UI loads fine. ![image](https://user-images.githubusercontent.com/31816865/80801883-5b9f2600-8bcb-11ea-8a1a-cc597aabc4c2.png) Closes #28429 from akshatb1/MasterUIBug. Authored-by: Akshat Bordia Signed-off-by: Thomas Graves --- core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index bf68ba8..252e704 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -37,7 +37,7 @@ private[deploy] class DriverInfo( @transient var worker: Option[WorkerInfo] = None // resources(e.f. gpu/fpga) allocated to this driver // map from resource name to ResourceInformation - private var _resources: Map[String, ResourceInformation] = _ + private var _resources: Map[String, ResourceInformation] = Map.empty init() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-31235][YARN] Separates different categories of applications
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f3891e3 [SPARK-31235][YARN] Separates different categories of applications f3891e3 is described below commit f3891e377f320ad212d198521dcdf5414830c063 Author: wang-zhun AuthorDate: Tue May 5 08:40:57 2020 -0500 [SPARK-31235][YARN] Separates different categories of applications ### What changes were proposed in this pull request? This PR adds `spark.yarn.applicationType` to identify the application type ### Why are the changes needed? The current application defaults to the SPARK type. In fact, different types of applications have different characteristics and are suitable for different scenarios.For example: SPAKR-SQL, SPARK-STREAMING. I recommend distinguishing them by the parameter `spark.yarn.applicationType` so that we can more easily manage and maintain different types of applications. ### How was this patch tested? 1.add UT 2.Tested by verifying Yarn-UI `ApplicationType` in the following cases: - client and cluster mode Need additional explanation: limit cannot exceed 20 characters, can be empty or space The reasons are as follows: ``` // org.apache.hadoop.yarn.server.resourcemanager.submitApplication. if (submissionContext.getApplicationType() == null) { submissionContext .setApplicationType(YarnConfiguration.DEFAULT_APPLICATION_TYPE); } else { // APPLICATION_TYPE_LENGTH = 20 if (submissionContext.getApplicationType().length() > YarnConfiguration.APPLICATION_TYPE_LENGTH) { submissionContext.setApplicationType(submissionContext .getApplicationType().substring(0, YarnConfiguration.APPLICATION_TYPE_LENGTH)); } } ``` Closes #28009 from wang-zhun/SPARK-31235. Authored-by: wang-zhun Signed-off-by: Thomas Graves --- docs/running-on-yarn.md| 9 +++ .../org/apache/spark/deploy/yarn/Client.scala | 2 +- .../org/apache/spark/deploy/yarn/config.scala | 8 +++ .../org/apache/spark/deploy/yarn/ClientSuite.scala | 82 +- 4 files changed, 97 insertions(+), 4 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 166fb87..b58cd24 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -157,6 +157,15 @@ To use a custom metrics.properties for the application master and executors, upd 3.0.0 + spark.yarn.applicationType + SPARK + +Defines more specific application types, e.g. SPARK, SPARK-SQL, SPARK-STREAMING, +SPARK-MLLIB and SPARK-GRAPH. Please be careful not to exceed 20 characters. + + 3.1.0 + + spark.yarn.driver.resource.{resource-type}.amount (none) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 6da6a8d..fc429d6 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -256,7 +256,7 @@ private[spark] class Client( appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark")) appContext.setQueue(sparkConf.get(QUEUE_NAME)) appContext.setAMContainerSpec(containerContext) -appContext.setApplicationType("SPARK") +appContext.setApplicationType(sparkConf.get(APPLICATION_TYPE)) sparkConf.get(APPLICATION_TAGS).foreach { tags => appContext.setApplicationTags(new java.util.HashSet[String](tags.asJava)) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index 3797491..b4257a4 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -111,6 +111,14 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val APPLICATION_TYPE = ConfigBuilder("spark.yarn.applicationType") +.doc("Type of this application," + + "it allows user to specify a more specific type for the application, such as SPARK," + + "SPARK-SQL, SPARK-STREAMING, SPARK-MLLIB and SPARK-GRAPH") +.version("3.1.0") +.stringConf +.createWithDefault("SPARK") + /* File distribution. */ private[spark] val SPARK_ARCHIVE = ConfigBuilder("spark.yarn.archive") diff --git a/resource-managers/yarn/src/test/scala/org/apache/
[spark] branch revert-28307-locality-fix created (now 1243ec0)
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a change to branch revert-28307-locality-fix in repository https://gitbox.apache.org/repos/asf/spark.git. at 1243ec0 Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)" This branch includes the following new commits: new 1243ec0 Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)" The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] 01/01: Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)"
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch revert-28307-locality-fix in repository https://gitbox.apache.org/repos/asf/spark.git commit 1243ec0cd06d5fd9b0ccd8c850bd1c3df2603534 Author: Thomas Graves AuthorDate: Thu Apr 23 14:38:48 2020 -0500 Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)" This reverts commit f093480af99063ad89273ffb3bf97d61269611e4. --- core/src/main/scala/org/apache/spark/internal/config/package.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5006da0..981d5f9 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -550,7 +550,7 @@ package object config { "anytime a task is scheduled. See Delay Scheduling section of TaskSchedulerImpl's class " + "documentation for more details.") .internal() -.version("3.1.0") +.version("3.0.0") .booleanConf .createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d1fc3a5..d327099 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -356,7 +356,7 @@ private[spark] class TaskSchedulerImpl( * value at index 'i' corresponds to shuffledOffers[i] * @param tasks tasks scheduled per offer, value at index 'i' corresponds to shuffledOffers[i] * @param addressesWithDescs tasks scheduler per host:port, used for barrier tasks - * @return tuple of (no delay schedule rejects?, option of min locality of launched task) + * @return tuple of (had delay schedule rejects?, option of min locality of launched task) */ private def resourceOfferSingleTaskSet( taskSet: TaskSetManager, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (8dc2c02 -> f093480)
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 8dc2c02 [SPARK-31522][SQL] Hive metastore client initialization related configurations should be static add f093480 fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307) No new revisions were added by this update. Summary of changes: core/src/main/scala/org/apache/spark/internal/config/package.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch revert-28307-locality-fix created (now 1243ec0)
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a change to branch revert-28307-locality-fix in repository https://gitbox.apache.org/repos/asf/spark.git. at 1243ec0 Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)" This branch includes the following new commits: new 1243ec0 Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)" The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] 01/01: Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)"
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch revert-28307-locality-fix in repository https://gitbox.apache.org/repos/asf/spark.git commit 1243ec0cd06d5fd9b0ccd8c850bd1c3df2603534 Author: Thomas Graves AuthorDate: Thu Apr 23 14:38:48 2020 -0500 Revert "fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307)" This reverts commit f093480af99063ad89273ffb3bf97d61269611e4. --- core/src/main/scala/org/apache/spark/internal/config/package.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5006da0..981d5f9 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -550,7 +550,7 @@ package object config { "anytime a task is scheduled. See Delay Scheduling section of TaskSchedulerImpl's class " + "documentation for more details.") .internal() -.version("3.1.0") +.version("3.0.0") .booleanConf .createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d1fc3a5..d327099 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -356,7 +356,7 @@ private[spark] class TaskSchedulerImpl( * value at index 'i' corresponds to shuffledOffers[i] * @param tasks tasks scheduled per offer, value at index 'i' corresponds to shuffledOffers[i] * @param addressesWithDescs tasks scheduler per host:port, used for barrier tasks - * @return tuple of (no delay schedule rejects?, option of min locality of launched task) + * @return tuple of (had delay schedule rejects?, option of min locality of launched task) */ private def resourceOfferSingleTaskSet( taskSet: TaskSetManager, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (8dc2c02 -> f093480)
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 8dc2c02 [SPARK-31522][SQL] Hive metastore client initialization related configurations should be static add f093480 fix version for config spark.locality.wait.legacyResetOnTaskLaunch (#28307) No new revisions were added by this update. Summary of changes: core/src/main/scala/org/apache/spark/internal/config/package.scala | 2 +- core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-18886][CORE][FOLLOWUP] allow follow up locality resets even if no task was launched
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8b77b31 [SPARK-18886][CORE][FOLLOWUP] allow follow up locality resets even if no task was launched 8b77b31 is described below commit 8b77b3183576da9c85d3939f5759e011c93d4fc5 Author: Nicholas Marcott <481161+bmarc...@users.noreply.github.com> AuthorDate: Wed Apr 22 08:25:24 2020 -0500 [SPARK-18886][CORE][FOLLOWUP] allow follow up locality resets even if no task was launched ### What changes were proposed in this pull request? Remove the requirement to launch a task in order to reset locality wait timer. ### Why are the changes needed? Recently https://github.com/apache/spark/pull/27207 was merged, but contained a bug which leads to undesirable behavior. The crux of the issue is that single resource offers couldn't reset the timer, if there had been a previous reject followed by an allResourceOffer with no available resources. This lead to a problem where once locality level reached ANY, single resource offers are all accepted, leading allResourceOffers to be left with no resources to utilize (hence no task being launched on an all resource offer -> no timer reset). The task manager would be stuck in ANY locality level. Noting down here the downsides of using below reset conditions, in case we want to follow up. As this is quite complex, I could easily be missing something, so please comment/respond if you have more bad behavior scenarios or find something wrong here: The format is: > **Reset condition** > - the unwanted side effect > - the cause/use case Below references to locality increase/decrease mean: ``` PROCESS_LOCAL, NODE_LOCAL ... .. ANY -- locality decrease ---> <- locality increase - ``` **Task launch:** - locality decrease: - Blacklisting, FAIR/FIFO scheduling, or task resource requirements can minimize tasks launched - locality increase: - single task launch decreases locality despite many tasks remaining **No delay schedule reject since last allFreeResource offer** - locality decrease: - locality wait less than allFreeResource offer frequency, which occurs at least 1 per second - locality increase: - single resource (or none) not rejected despite many tasks remaining (other lower priority tasks utilizing resources) **Current impl - No delay schedule reject since last (allFreeResource offer + task launch)** - locality decrease: - all from above - locality increase: - single resource accepted and task launched despite many tasks remaining The current impl is an improvement on the legacy (task launch) in that unintended locality decrease case is similar and the unintended locality increase case only occurs when the cluster is fully utilized. For the locality increase cases, perhaps a config which specifies a certain % of tasks in a taskset to finish before resetting locality levels would be helpful. **If** that was considered a good approach then perhaps removing the task launch as a requirement would eliminate most of downsides listed above. Lemme know if you have more ideas for eliminating locality increase downside of **No delay schedule reject since last allFreeResource offer** ### Does this PR introduce any user-facing change? No ### How was this patch tested? TaskSchedulerImplSuite Also manually tested similar to how I tested in https://github.com/apache/spark/pull/27207 using [this simple app](https://github.com/bmarcott/spark-test-apps/blob/master/src/main/scala/TestLocalityWait.scala). With the new changes, given locality wait of 10s the behavior is generally: 10 seconds of locality being respected, followed by a single full utilization of resources using ANY locality level, followed by 10 seconds of locality being respected, and so on If the legacy flag is enabled (spark.locality.wait.legacyResetOnTaskLaunch=true), the behavior is only scheduling PROCESS_LOCAL tasks (only utilizing a single executor) cloud-fan tgravescs Closes #28188 from bmarcott/nmarcott-locality-fix. Authored-by: Nicholas Marcott <481161+bmarc...@users.noreply.github.com> Signed-off-by: Thomas Graves --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 16 ++--- .../spark/scheduler/TaskSchedulerImplSuite.scala | 69 +- 2 files changed, 75 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2e3
[spark] branch master updated: [SPARK-29153][CORE] Add ability to merge resource profiles within a stage with Stage Level Scheduling
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 55dea9b [SPARK-29153][CORE] Add ability to merge resource profiles within a stage with Stage Level Scheduling 55dea9b is described below commit 55dea9be62019d64d5d76619e1551956c8bb64d0 Author: Thomas Graves AuthorDate: Thu Apr 2 08:30:18 2020 -0500 [SPARK-29153][CORE] Add ability to merge resource profiles within a stage with Stage Level Scheduling ### What changes were proposed in this pull request? For the stage level scheduling feature, add the ability to optionally merged resource profiles if they were specified on multiple RDD within a stage. There is a config to enable this feature, its off by default (spark.scheduler.resourceProfile.mergeConflicts). When the config is set to true, Spark will merge the profiles selecting the max value of each resource (cores, memory, gpu, etc). further documentation will be added with SPARK-30322. This also added in the ability to check if an equivalent resource profile already exists. This is so that if a user is running stages and combining the same profiles over and over again we don't get an explosion in the number of profiles. ### Why are the changes needed? To allow users to specify resource on multiple RDD and not worry as much about if they go into the same stage and fail. ### Does this PR introduce any user-facing change? Yes, when the config is turned on it now merges the profiles instead of errorring out. ### How was this patch tested? Unit tests Closes #28053 from tgravescs/SPARK-29153. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../org/apache/spark/internal/config/package.scala | 10 ++ .../apache/spark/resource/ResourceProfile.scala| 5 + .../spark/resource/ResourceProfileManager.scala| 55 ++-- .../org/apache/spark/scheduler/DAGScheduler.scala | 48 ++- .../resource/ResourceProfileManagerSuite.scala | 30 + .../spark/resource/ResourceProfileSuite.scala | 20 +++ .../apache/spark/scheduler/DAGSchedulerSuite.scala | 140 - docs/configuration.md | 11 ++ 8 files changed, 304 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 8f8b6ad..76791ab 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1816,4 +1816,14 @@ package object config { .bytesConf(ByteUnit.BYTE) .createOptional + private[spark] val RESOURCE_PROFILE_MERGE_CONFLICTS = +ConfigBuilder("spark.scheduler.resource.profileMergeConflicts") + .doc("If set to true, Spark will merge ResourceProfiles when different profiles " + +"are specified in RDDs that get combined into a single stage. When they are merged, " + +"Spark chooses the maximum of each resource and creates a new ResourceProfile. The " + +"default of false results in Spark throwing an exception if multiple different " + +"ResourceProfiles are found in RDDs going into the same stage.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 96c456e..97186fb 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -222,6 +222,11 @@ class ResourceProfile( } } + // check that the task resources and executor resources are equal, but id's could be different + private[spark] def resourcesEqual(rp: ResourceProfile): Boolean = { +rp.taskResources == taskResources && rp.executorResources == executorResources + } + override def hashCode(): Int = Seq(taskResources, executorResources).hashCode() override def toString(): String = { diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index fabc0bd..c3e2444 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -17,7 +17,9 @@ package org.apache.spark.resource -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.locks.ReentrantReadWriteLock + +impor
[spark] branch master updated: [SPARK-31179] Fast fail the connection while last connection failed in fast fail time window
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ec28925 [SPARK-31179] Fast fail the connection while last connection failed in fast fail time window ec28925 is described below commit ec289252368127f8261eb6a2270362ba0b65db36 Author: turbofei AuthorDate: Thu Apr 2 08:18:14 2020 -0500 [SPARK-31179] Fast fail the connection while last connection failed in fast fail time window ## What changes were proposed in this pull request? For TransportFactory, the requests sent to the same address share a clientPool. Specially, when the io.numConnectionPerPeer is 1, these requests would share a same client. When this address is unreachable, the createClient operation would be still timeout. And these requests would block each other during createClient, because there is a lock for this shared client. It would cost connectionNum \* connectionTimeOut \* maxRetry to retry, and then fail the task. It fact, it is expected that this task could fail in connectionTimeOut * maxRetry. In this PR, I set a fastFail time window for the clientPool, if the last connection failed in this time window, the new connection would fast fail. ## Why are the changes needed? It can save time for some cases. ## Does this PR introduce any user-facing change? No. ## How was this patch tested? Existing UT. Closes #27943 from turboFei/SPARK-31179-fast-fail-connection. Authored-by: turbofei Signed-off-by: Thomas Graves --- .../network/client/TransportClientFactory.java | 42 +++--- .../{ => client}/TransportClientFactorySuite.java | 28 +-- .../network/shuffle/ExternalBlockStoreClient.java | 4 +-- .../network/netty/NettyBlockTransferService.scala | 4 +-- .../netty/NettyBlockTransferServiceSuite.scala | 2 +- 5 files changed, 68 insertions(+), 12 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index c9ef9f9..24c436a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -27,6 +27,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; import com.codahale.metrics.MetricSet; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -61,6 +62,7 @@ public class TransportClientFactory implements Closeable { private static class ClientPool { TransportClient[] clients; Object[] locks; +volatile long lastConnectionFailed; ClientPool(int size) { clients = new TransportClient[size]; @@ -68,6 +70,7 @@ public class TransportClientFactory implements Closeable { for (int i = 0; i < size; i++) { locks[i] = new Object(); } + lastConnectionFailed = 0; } } @@ -86,6 +89,7 @@ public class TransportClientFactory implements Closeable { private EventLoopGroup workerGroup; private final PooledByteBufAllocator pooledAllocator; private final NettyMemoryMetrics metrics; + private final int fastFailTimeWindow; public TransportClientFactory( TransportContext context, @@ -112,6 +116,7 @@ public class TransportClientFactory implements Closeable { } this.metrics = new NettyMemoryMetrics( this.pooledAllocator, conf.getModuleName() + "-client", conf); +fastFailTimeWindow = (int)(conf.ioRetryWaitTimeMs() * 0.95); } public MetricSet getAllMetrics() { @@ -121,18 +126,27 @@ public class TransportClientFactory implements Closeable { /** * Create a {@link TransportClient} connecting to the given remote host / port. * - * We maintains an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) + * We maintain an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) * and randomly picks one to use. If no client was previously created in the randomly selected * spot, this function creates a new client and places it there. * + * If the fastFail parameter is true, fail immediately when the last attempt to the same address + * failed within the fast fail time window (95 percent of the io wait retry timeout). The + * assumption is the caller will handle retrying. + * * Prior to the creation of a new TransportClient, we will execute all * {@link TransportClientBootstrap}s that are registered w
[spark] branch branch-3.0 updated: [SPARK-31219][YARN] Enable closeIdleConnections in YarnShuffleService
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 7329c25 [SPARK-31219][YARN] Enable closeIdleConnections in YarnShuffleService 7329c25 is described below commit 7329c256c6d02cbc700d367320ef20d215bca8aa Author: manuzhang AuthorDate: Mon Mar 30 12:44:46 2020 -0500 [SPARK-31219][YARN] Enable closeIdleConnections in YarnShuffleService ### What changes were proposed in this pull request? Close idle connections at shuffle server side when an `IdleStateEvent` is triggered after `spark.shuffle.io.connectionTimeout` or `spark.network.timeout` time. It's based on following investigations. 1. We found connections on our clusters building up continuously (> 10k for some nodes). Is that normal ? We don't think so. 2. We looked into the connections on one node and found there were a lot of half-open connections. (connections only existed on one node) 3. We also checked those connections were very old (> 21 hours). (FYI, https://superuser.com/questions/565991/how-to-determine-the-socket-connection-up-time-on-linux) 4. Looking at the code, TransportContext registers an IdleStateHandler which should fire an IdleStateEvent when timeout. We did a heap dump of the YarnShuffleService and checked the attributes of IdleStateHandler. It turned out firstAllIdleEvent of many IdleStateHandlers were already false so IdleStateEvent were already fired. 5. Finally, we realized the IdleStateEvent would not be handled since closeIdleConnections are hardcoded to false for YarnShuffleService. ### Why are the changes needed? Idle connections to YarnShuffleService could never be closed, and will be accumulating and taking up memory and file descriptors. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #27998 from manuzhang/spark-31219. Authored-by: manuzhang Signed-off-by: Thomas Graves (cherry picked from commit 0d997e5156a751c99cd6f8be1528ed088a585d1f) Signed-off-by: Thomas Graves --- .../src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 815a56d..c41efba 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -188,7 +188,7 @@ public class YarnShuffleService extends AuxiliaryService { int port = conf.getInt( SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - transportContext = new TransportContext(transportConf, blockHandler); + transportContext = new TransportContext(transportConf, blockHandler, true); shuffleServer = transportContext.createServer(port, bootstraps); // the port should normally be fixed, but for tests its useful to find an open port port = shuffleServer.getPort(); - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-31219][YARN] Enable closeIdleConnections in YarnShuffleService
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0d997e5 [SPARK-31219][YARN] Enable closeIdleConnections in YarnShuffleService 0d997e5 is described below commit 0d997e5156a751c99cd6f8be1528ed088a585d1f Author: manuzhang AuthorDate: Mon Mar 30 12:44:46 2020 -0500 [SPARK-31219][YARN] Enable closeIdleConnections in YarnShuffleService ### What changes were proposed in this pull request? Close idle connections at shuffle server side when an `IdleStateEvent` is triggered after `spark.shuffle.io.connectionTimeout` or `spark.network.timeout` time. It's based on following investigations. 1. We found connections on our clusters building up continuously (> 10k for some nodes). Is that normal ? We don't think so. 2. We looked into the connections on one node and found there were a lot of half-open connections. (connections only existed on one node) 3. We also checked those connections were very old (> 21 hours). (FYI, https://superuser.com/questions/565991/how-to-determine-the-socket-connection-up-time-on-linux) 4. Looking at the code, TransportContext registers an IdleStateHandler which should fire an IdleStateEvent when timeout. We did a heap dump of the YarnShuffleService and checked the attributes of IdleStateHandler. It turned out firstAllIdleEvent of many IdleStateHandlers were already false so IdleStateEvent were already fired. 5. Finally, we realized the IdleStateEvent would not be handled since closeIdleConnections are hardcoded to false for YarnShuffleService. ### Why are the changes needed? Idle connections to YarnShuffleService could never be closed, and will be accumulating and taking up memory and file descriptors. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #27998 from manuzhang/spark-31219. Authored-by: manuzhang Signed-off-by: Thomas Graves --- .../src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 815a56d..c41efba 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -188,7 +188,7 @@ public class YarnShuffleService extends AuxiliaryService { int port = conf.getInt( SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - transportContext = new TransportContext(transportConf, blockHandler); + transportContext = new TransportContext(transportConf, blockHandler, true); shuffleServer = transportContext.createServer(port, bootstraps); // the port should normally be fixed, but for tests its useful to find an open port port = shuffleServer.getPort(); - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-29154][CORE] Update Spark scheduler for stage level scheduling
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 474b1bb [SPARK-29154][CORE] Update Spark scheduler for stage level scheduling 474b1bb is described below commit 474b1bb5c2bce2f83c4dd8e19b9b7c5b3aebd6c4 Author: Thomas Graves AuthorDate: Thu Mar 26 09:46:36 2020 -0500 [SPARK-29154][CORE] Update Spark scheduler for stage level scheduling ### What changes were proposed in this pull request? This is the core scheduler changes to support Stage level scheduling. The main changes here include modification to the DAGScheduler to look at the ResourceProfiles associated with an RDD and have those applied inside the scheduler. Currently if multiple RDD's in a stage have conflicting ResourceProfiles we throw an error. logic to allow this will happen in SPARK-29153. I added the interfaces to RDD to add and get the REsourceProfile so that I could add unit tests for the scheduler. These are marked as private for now until we finish the feature and will be exposed in SPARK-29150. If you think this is confusing I can remove those and remove the tests and add them back later. I modified the task scheduler to make sure to only schedule on executor that exactly match the resource profile. It will then check those executors to make sure the current resources meet the task needs before assigning it. In here I changed the way we do the custom resource assignment. Other changes here include having the cpus per task passed around so that we can properly account for them. Previously we just used the one global config, but now it can change based on the ResourceProfile. I removed the exceptions that require the cores to be the limiting resource. With this change all the places I found that used executor cores /task cpus as slots has been updated to use the ResourceProfile logic and look to see what resource is limiting. ### Why are the changes needed? Stage level sheduling feature ### Does this PR introduce any user-facing change? No ### How was this patch tested? unit tests and lots of manual testing Closes #27773 from tgravescs/SPARK-29154. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/SparkContext.scala | 27 +-- .../org/apache/spark/internal/config/Tests.scala | 9 + core/src/main/scala/org/apache/spark/rdd/RDD.scala | 27 +++ .../apache/spark/resource/ResourceProfile.scala| 42 +++-- .../spark/resource/ResourceProfileManager.scala| 11 +- .../org/apache/spark/resource/ResourceUtils.scala | 13 +- .../spark/resource/TaskResourceRequests.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 70 +--- .../apache/spark/scheduler/SchedulerBackend.scala | 8 +- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 178 ++- .../scala/org/apache/spark/scheduler/TaskSet.scala | 3 +- .../apache/spark/scheduler/TaskSetManager.scala| 32 ++-- .../org/apache/spark/scheduler/WorkerOffer.scala | 5 +- .../cluster/CoarseGrainedSchedulerBackend.scala| 24 ++- .../scheduler/local/LocalSchedulerBackend.scala| 9 +- .../deploy/StandaloneDynamicAllocationSuite.scala | 1 - .../CoarseGrainedExecutorBackendSuite.scala| 4 +- .../CoarseGrainedSchedulerBackendSuite.scala | 13 +- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 115 +++- .../scheduler/ExternalClusterManagerSuite.scala| 3 +- .../org/apache/spark/scheduler/FakeTask.scala | 31 +++- .../org/apache/spark/scheduler/PoolSuite.scala | 4 +- .../scheduler/SchedulerIntegrationSuite.scala | 5 +- .../spark/scheduler/TaskSchedulerImplSuite.scala | 192 - .../spark/scheduler/TaskSetManagerSuite.scala | 91 ++ .../mesos/MesosFineGrainedSchedulerBackend.scala | 3 +- 26 files changed, 704 insertions(+), 218 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cdb98db..588e7dc 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1597,13 +1597,17 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Get the max number of tasks that can be concurrent launched currently. + * Get the max number of tasks that can be concurrent launched based on the ResourceProfile + * being used. * Note that please don't cache the value returned by this method, because the number can change * due to add/remove executors. * + * @param rp ResourceProfile which to use to calculate max concur
[spark] branch master updated: [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3ff2135 [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character 3ff2135 is described below commit 3ff213568694e265466d8480b61fd38f4fd8fdff Author: Javier AuthorDate: Tue Mar 3 09:55:15 2020 -0600 [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character ### What changes were proposed in this pull request? A SQL statement that contains a comment with an unmatched quote character can lead to a parse error: - Added a insideComment flag in the splitter method to avoid checking single and double quotes within a comment: ``` spark-sql> SELECT 1 -- someone's comment here > ; Error in query: extraneous input ';' expecting (line 2, pos 0) == SQL == SELECT 1 -- someone's comment here ; ^^^ ``` ### Why are the changes needed? This misbehaviour was not present on previous spark versions. ### Does this PR introduce any user-facing change? - No ### How was this patch tested? - New tests were added. Closes #27321 from javierivanov/SPARK-30049B. Lead-authored-by: Javier Co-authored-by: Javier Fuentes Signed-off-by: Thomas Graves --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 24 ++ .../spark/sql/hive/thriftserver/CliSuite.scala | 22 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index b665d4a..19f7ea8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -509,24 +509,40 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { private def splitSemiColon(line: String): JList[String] = { var insideSingleQuote = false var insideDoubleQuote = false +var insideComment = false var escape = false var beginIndex = 0 +var endIndex = line.length val ret = new JArrayList[String] + for (index <- 0 until line.length) { - if (line.charAt(index) == '\'') { + if (line.charAt(index) == '\'' && !insideComment) { // take a look to see if it is escaped if (!escape) { // flip the boolean variable insideSingleQuote = !insideSingleQuote } - } else if (line.charAt(index) == '\"') { + } else if (line.charAt(index) == '\"' && !insideComment) { // take a look to see if it is escaped if (!escape) { // flip the boolean variable insideDoubleQuote = !insideDoubleQuote } + } else if (line.charAt(index) == '-') { +val hasNext = index + 1 < line.length +if (insideDoubleQuote || insideSingleQuote || insideComment) { + // Ignores '-' in any case of quotes or comment. + // Avoids to start a comment(--) within a quoted segment or already in a comment. + // Sample query: select "quoted value --" + //^^ avoids starting a comment if it's inside quotes. +} else if (hasNext && line.charAt(index + 1) == '-') { + // ignore quotes and ; + insideComment = true + // ignore eol + endIndex = index +} } else if (line.charAt(index) == ';') { -if (insideSingleQuote || insideDoubleQuote) { +if (insideSingleQuote || insideDoubleQuote || insideComment) { // do not split } else { // split, do not include ; itself @@ -543,7 +559,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { escape = true } } -ret.add(line.substring(beginIndex)) +ret.add(line.substring(beginIndex, endIndex)) ret } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 6609701..43aafc3 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -400,4 +400,26 @@ class CliSuite extends S
[spark] branch branch-3.0 updated: [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 4be2a79 [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character 4be2a79 is described below commit 4be2a79c7a9a1b1e3b0c3704e94da19c2b87ba47 Author: Javier AuthorDate: Tue Mar 3 09:55:15 2020 -0600 [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character ### What changes were proposed in this pull request? A SQL statement that contains a comment with an unmatched quote character can lead to a parse error: - Added a insideComment flag in the splitter method to avoid checking single and double quotes within a comment: ``` spark-sql> SELECT 1 -- someone's comment here > ; Error in query: extraneous input ';' expecting (line 2, pos 0) == SQL == SELECT 1 -- someone's comment here ; ^^^ ``` ### Why are the changes needed? This misbehaviour was not present on previous spark versions. ### Does this PR introduce any user-facing change? - No ### How was this patch tested? - New tests were added. Closes #27321 from javierivanov/SPARK-30049B. Lead-authored-by: Javier Co-authored-by: Javier Fuentes Signed-off-by: Thomas Graves (cherry picked from commit 3ff213568694e265466d8480b61fd38f4fd8fdff) Signed-off-by: Thomas Graves --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 24 ++ .../spark/sql/hive/thriftserver/CliSuite.scala | 22 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index b665d4a..19f7ea8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -509,24 +509,40 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { private def splitSemiColon(line: String): JList[String] = { var insideSingleQuote = false var insideDoubleQuote = false +var insideComment = false var escape = false var beginIndex = 0 +var endIndex = line.length val ret = new JArrayList[String] + for (index <- 0 until line.length) { - if (line.charAt(index) == '\'') { + if (line.charAt(index) == '\'' && !insideComment) { // take a look to see if it is escaped if (!escape) { // flip the boolean variable insideSingleQuote = !insideSingleQuote } - } else if (line.charAt(index) == '\"') { + } else if (line.charAt(index) == '\"' && !insideComment) { // take a look to see if it is escaped if (!escape) { // flip the boolean variable insideDoubleQuote = !insideDoubleQuote } + } else if (line.charAt(index) == '-') { +val hasNext = index + 1 < line.length +if (insideDoubleQuote || insideSingleQuote || insideComment) { + // Ignores '-' in any case of quotes or comment. + // Avoids to start a comment(--) within a quoted segment or already in a comment. + // Sample query: select "quoted value --" + //^^ avoids starting a comment if it's inside quotes. +} else if (hasNext && line.charAt(index + 1) == '-') { + // ignore quotes and ; + insideComment = true + // ignore eol + endIndex = index +} } else if (line.charAt(index) == ';') { -if (insideSingleQuote || insideDoubleQuote) { +if (insideSingleQuote || insideDoubleQuote || insideComment) { // do not split } else { // split, do not include ; itself @@ -543,7 +559,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { escape = true } } -ret.add(line.substring(beginIndex)) +ret.add(line.substring(beginIndex, endIndex)) ret } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 6609701..43aafc3 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thrif
[spark] branch branch-3.0 updated: [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 0aace99 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks 0aace99 is described below commit 0aace99d1162348269848665725c7db2541807cc Author: xuesenliang AuthorDate: Tue Mar 3 09:29:43 2020 -0600 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks ### What changes were proposed in this pull request? When a job finished, its running (re-submitted) map stages should be marked as finished if not used by other jobs. The running tasks of these stages are cancelled. And the ListenerBus should be notified too, otherwise, these map stage items will stay on the "Active Stages" page of web UI and never gone. For example: Suppose job 0 has two stages: map stage 0 and result stage 1. Map stage 0 has two partitions, and its result stage 1 has two partitions too. **Steps to reproduce the bug:** 1. map stage 0:start task 0(```TID 0```) and task 1 (```TID 1```), then both finished successfully. 2. result stage 1: start task 0(```TID 2```) and task 1 (```TID 3```) 3. result stage 1: task 0(```TID 2```) finished successfully 4. result stage 1: speculative task 1.1(```TID 4```) launched, but then failed due to FetchFailedException. 5. driver re-submits map stage 0 and result stage 1. 6. map stage 0 (retry 1): task0(```TID 5```) launched 7. result stage 1: task 1(```TID 3```) finished successfully, so job 0 finished. 8. map stage 0 is removed from ```runningStages``` and ```stageIdToStage```, because it doesn't belong to any job. ``` private def DAGScheduler#cleanupStateForJobAndIndependentStages(job: ActiveJob): HashSet[Stage] = { ... stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { case (stageId, stage) => ... def removeStage(stageId: Int): Unit = { for (stage <- stageIdToStage.get(stageId)) { if (runningStages.contains(stage)) { logDebug("Removing running stage %d".format(stageId)) runningStages -= stage } ... } stageIdToStage -= stageId } jobSet -= job.jobId if (jobSet.isEmpty) { // no other job needs this stage removeStage(stageId) } } ... } ``` 9. map stage 0 (retry 1): task0(TID 5) finished successfully, but its stage 0 is not in ```stageIdToStage```, so the stage not ```markStageAsFinished``` ``` private[scheduler] def DAGScheduler#handleTaskCompletion(event: CompletionEvent): Unit = { val task = event.task val stageId = task.stageId ... if (!stageIdToStage.contains(task.stageId)) { postTaskEnd(event) // Skip all the actions if the stage has been cancelled. return } ... ``` Relevant spark driver logs as follows: ``` 20/01/02 11:21:45 INFO DAGScheduler: Got job 0 (main at NativeMethodAccessorImpl.java:0) with 2 output partitions 20/01/02 11:21:45 INFO DAGScheduler: Final stage: ResultStage 1 (main at NativeMethodAccessorImpl.java:0) 20/01/02 11:21:45 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 0) 20/01/02 11:21:45 INFO DAGScheduler: Missing parents: List(ShuffleMapStage 0) 20/01/02 11:21:45 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents 20/01/02 11:21:45 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0, 1)) 20/01/02 11:21:45 INFO YarnClusterScheduler: Adding task set 0.0 with 2 tasks 20/01/02 11:21:45 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, 9.179.143.4, executor 1, partition 0, PROCESS_LOCAL, 7704 bytes) 20/01/02 11:21:45 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, 9.76.13.26, executor 2, partition 1, PROCESS_LOCAL, 7705 bytes) 20/01/02 11:22:18 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 32491 ms on 9.179.143.4 (executor 1) (1/2) 20/01/02 11:22:26 INFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 40544 ms on 9.76.13.26 (executor 2) (2/2) 20/01/02 11:22:26 INFO DAGScheduler: ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0) finished in 4
[spark] branch master updated: [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 7a4cf33 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks 7a4cf33 is described below commit 7a4cf339d7082b576624940253e8283de9e83e19 Author: xuesenliang AuthorDate: Tue Mar 3 09:29:43 2020 -0600 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks ### What changes were proposed in this pull request? When a job finished, its running (re-submitted) map stages should be marked as finished if not used by other jobs. The running tasks of these stages are cancelled. And the ListenerBus should be notified too, otherwise, these map stage items will stay on the "Active Stages" page of web UI and never gone. For example: Suppose job 0 has two stages: map stage 0 and result stage 1. Map stage 0 has two partitions, and its result stage 1 has two partitions too. **Steps to reproduce the bug:** 1. map stage 0:start task 0(```TID 0```) and task 1 (```TID 1```), then both finished successfully. 2. result stage 1: start task 0(```TID 2```) and task 1 (```TID 3```) 3. result stage 1: task 0(```TID 2```) finished successfully 4. result stage 1: speculative task 1.1(```TID 4```) launched, but then failed due to FetchFailedException. 5. driver re-submits map stage 0 and result stage 1. 6. map stage 0 (retry 1): task0(```TID 5```) launched 7. result stage 1: task 1(```TID 3```) finished successfully, so job 0 finished. 8. map stage 0 is removed from ```runningStages``` and ```stageIdToStage```, because it doesn't belong to any job. ``` private def DAGScheduler#cleanupStateForJobAndIndependentStages(job: ActiveJob): HashSet[Stage] = { ... stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { case (stageId, stage) => ... def removeStage(stageId: Int): Unit = { for (stage <- stageIdToStage.get(stageId)) { if (runningStages.contains(stage)) { logDebug("Removing running stage %d".format(stageId)) runningStages -= stage } ... } stageIdToStage -= stageId } jobSet -= job.jobId if (jobSet.isEmpty) { // no other job needs this stage removeStage(stageId) } } ... } ``` 9. map stage 0 (retry 1): task0(TID 5) finished successfully, but its stage 0 is not in ```stageIdToStage```, so the stage not ```markStageAsFinished``` ``` private[scheduler] def DAGScheduler#handleTaskCompletion(event: CompletionEvent): Unit = { val task = event.task val stageId = task.stageId ... if (!stageIdToStage.contains(task.stageId)) { postTaskEnd(event) // Skip all the actions if the stage has been cancelled. return } ... ``` Relevant spark driver logs as follows: ``` 20/01/02 11:21:45 INFO DAGScheduler: Got job 0 (main at NativeMethodAccessorImpl.java:0) with 2 output partitions 20/01/02 11:21:45 INFO DAGScheduler: Final stage: ResultStage 1 (main at NativeMethodAccessorImpl.java:0) 20/01/02 11:21:45 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 0) 20/01/02 11:21:45 INFO DAGScheduler: Missing parents: List(ShuffleMapStage 0) 20/01/02 11:21:45 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents 20/01/02 11:21:45 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0, 1)) 20/01/02 11:21:45 INFO YarnClusterScheduler: Adding task set 0.0 with 2 tasks 20/01/02 11:21:45 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, 9.179.143.4, executor 1, partition 0, PROCESS_LOCAL, 7704 bytes) 20/01/02 11:21:45 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, 9.76.13.26, executor 2, partition 1, PROCESS_LOCAL, 7705 bytes) 20/01/02 11:22:18 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 32491 ms on 9.179.143.4 (executor 1) (1/2) 20/01/02 11:22:26 INFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 40544 ms on 9.76.13.26 (executor 2) (2/2) 20/01/02 11:22:26 INFO DAGScheduler: ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0) finished in 4
[spark] branch master updated: [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2d22498 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks 2d22498 is described below commit 2d22498d6a1f290f9ca54404a6e83ed4b61431d2 Author: xuesenliang AuthorDate: Tue Mar 3 09:27:07 2020 -0600 [SPARK-30388][CORE] Mark running map stages of finished job as finished, and cancel running tasks ### What changes were proposed in this pull request? When a job finished, its running (re-submitted) map stages should be marked as finished if not used by other jobs. The running tasks of these stages are cancelled. And the ListenerBus should be notified too, otherwise, these map stage items will stay on the "Active Stages" page of web UI and never gone. For example: Suppose job 0 has two stages: map stage 0 and result stage 1. Map stage 0 has two partitions, and its result stage 1 has two partitions too. **Steps to reproduce the bug:** 1. map stage 0:start task 0(```TID 0```) and task 1 (```TID 1```), then both finished successfully. 2. result stage 1: start task 0(```TID 2```) and task 1 (```TID 3```) 3. result stage 1: task 0(```TID 2```) finished successfully 4. result stage 1: speculative task 1.1(```TID 4```) launched, but then failed due to FetchFailedException. 5. driver re-submits map stage 0 and result stage 1. 6. map stage 0 (retry 1): task0(```TID 5```) launched 7. result stage 1: task 1(```TID 3```) finished successfully, so job 0 finished. 8. map stage 0 is removed from ```runningStages``` and ```stageIdToStage```, because it doesn't belong to any job. ``` private def DAGScheduler#cleanupStateForJobAndIndependentStages(job: ActiveJob): HashSet[Stage] = { ... stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { case (stageId, stage) => ... def removeStage(stageId: Int): Unit = { for (stage <- stageIdToStage.get(stageId)) { if (runningStages.contains(stage)) { logDebug("Removing running stage %d".format(stageId)) runningStages -= stage } ... } stageIdToStage -= stageId } jobSet -= job.jobId if (jobSet.isEmpty) { // no other job needs this stage removeStage(stageId) } } ... } ``` 9. map stage 0 (retry 1): task0(TID 5) finished successfully, but its stage 0 is not in ```stageIdToStage```, so the stage not ```markStageAsFinished``` ``` private[scheduler] def DAGScheduler#handleTaskCompletion(event: CompletionEvent): Unit = { val task = event.task val stageId = task.stageId ... if (!stageIdToStage.contains(task.stageId)) { postTaskEnd(event) // Skip all the actions if the stage has been cancelled. return } ... ``` Relevant spark driver logs as follows: ``` 20/01/02 11:21:45 INFO DAGScheduler: Got job 0 (main at NativeMethodAccessorImpl.java:0) with 2 output partitions 20/01/02 11:21:45 INFO DAGScheduler: Final stage: ResultStage 1 (main at NativeMethodAccessorImpl.java:0) 20/01/02 11:21:45 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 0) 20/01/02 11:21:45 INFO DAGScheduler: Missing parents: List(ShuffleMapStage 0) 20/01/02 11:21:45 INFO DAGScheduler: Submitting ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents 20/01/02 11:21:45 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0, 1)) 20/01/02 11:21:45 INFO YarnClusterScheduler: Adding task set 0.0 with 2 tasks 20/01/02 11:21:45 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, 9.179.143.4, executor 1, partition 0, PROCESS_LOCAL, 7704 bytes) 20/01/02 11:21:45 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1, 9.76.13.26, executor 2, partition 1, PROCESS_LOCAL, 7705 bytes) 20/01/02 11:22:18 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 32491 ms on 9.179.143.4 (executor 1) (1/2) 20/01/02 11:22:26 INFO TaskSetManager: Finished task 1.0 in stage 0.0 (TID 1) in 40544 ms on 9.76.13.26 (executor 2) (2/2) 20/01/02 11:22:26 INFO DAGScheduler: ShuffleMapStage 0 (main at NativeMethodAccessorImpl.java:0) finished in 4
[spark] branch master updated: [SPARK-29149][YARN] Update YARN cluster manager For Stage Level Scheduling
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0e2ca11 [SPARK-29149][YARN] Update YARN cluster manager For Stage Level Scheduling 0e2ca11 is described below commit 0e2ca11d80c3921387d7b077cb64c3a0c06b08d7 Author: Thomas Graves AuthorDate: Fri Feb 28 15:23:33 2020 -0600 [SPARK-29149][YARN] Update YARN cluster manager For Stage Level Scheduling ### What changes were proposed in this pull request? Yarn side changes for Stage level scheduling. The previous PR for dynamic allocation changes was https://github.com/apache/spark/pull/27313 Modified the data structures to store things on a per ResourceProfile basis. I tried to keep the code changes to a minimum, the main loop that requests just goes through each Resourceprofile and the logic inside for each one stayed very close to the same. On submission we now have to give each ResourceProfile a separate yarn Priority because yarn doesn't support asking for containers with different resources at the same Priority. We just use the profile id as the priority level. Using a different Priority actually makes things easier when the containers come back to match them again which ResourceProfile they were requested for. The expectation is that yarn will only give you a container with resource amounts you requested or more. It should never give you a container if it doesn't satisfy your resource requests. If you want to see the full feature changes you can look at https://github.com/apache/spark/pull/27053/files for reference ### Why are the changes needed? For stage level scheduling YARN support. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Tested manually on YARN cluster and then unit tests. Closes #27583 from tgravescs/SPARK-29149. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../cluster/CoarseGrainedClusterMessage.scala | 6 +- .../org/apache/spark/HeartbeatReceiverSuite.scala | 30 +- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +- .../deploy/yarn/ApplicationMasterSource.scala | 4 +- ...calityPreferredContainerPlacementStrategy.scala | 25 +- .../spark/deploy/yarn/ResourceRequestHelper.scala | 11 + .../apache/spark/deploy/yarn/YarnAllocator.scala | 556 ++--- .../scheduler/cluster/YarnSchedulerBackend.scala | 8 +- .../yarn/ContainerPlacementStrategySuite.scala | 46 +- .../yarn/LocalityPlacementStrategySuite.scala | 6 +- .../spark/deploy/yarn/YarnAllocatorSuite.scala | 258 +++--- .../cluster/YarnSchedulerBackendSuite.scala| 20 +- 12 files changed, 663 insertions(+), 316 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 8db0122..465c0d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -117,9 +117,9 @@ private[spark] object CoarseGrainedClusterMessages { // Request executors by specifying the new total number of executors desired // This includes executors already pending or running case class RequestExecutors( - requestedTotal: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int], + resourceProfileToTotalExecs: Map[ResourceProfile, Int], + numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], + hostToLocalTaskCount: Map[Int, Map[String, Int]], nodeBlacklist: Set[String]) extends CoarseGrainedClusterMessage diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index a929695..3126913 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -30,7 +30,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.DYN_ALLOCATION_TESTING -import org.apache.spark.resource.ResourceProfile +import org.apache.spark.resource.{ResourceProfile, ResourceProfileManager} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -61,6 +61,7 @@ class HeartbeatReceiverSuite PrivateMethod[collection.Map[String, Long]](Symbol("executorLastSeen")) private va
[spark] branch master updated: [SPARK-30942] Fix the warning for requiring cores to be limiting resources
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c46c067 [SPARK-30942] Fix the warning for requiring cores to be limiting resources c46c067 is described below commit c46c067f39213df9b3ee5a51e7d7803b867a0d54 Author: Thomas Graves AuthorDate: Tue Feb 25 10:55:56 2020 -0600 [SPARK-30942] Fix the warning for requiring cores to be limiting resources ### What changes were proposed in this pull request? fix the warning for limiting resources when we don't know the number of executor cores. The issue is that there are places in the Spark code that use cores/task cpus to calculate slots and until the entire Stage level scheduling feature is in, we have to rely on the cores being the limiting resource. Change the check to only warn when custom resources are specified. ### Why are the changes needed? fix the check and warn when we should ### Does this PR introduce any user-facing change? A warning is printed ### How was this patch tested? manually tested spark-shell with standalone mode, yarn, local mode. Closes #27686 from tgravescs/SPARK-30942. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- core/src/main/scala/org/apache/spark/SparkContext.scala| 2 +- .../src/main/scala/org/apache/spark/resource/ResourceProfile.scala | 7 +++ 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a47136e..f377f13 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2798,7 +2798,7 @@ object SparkContext extends Logging { defaultProf.maxTasksPerExecutor(sc.conf) < cpuSlots) { throw new IllegalArgumentException("The number of slots on an executor has to be " + "limited by the number of cores, otherwise you waste resources and " + - "dynamic allocation doesn't work properly. Your configuration has " + + "some scheduling doesn't work properly. Your configuration has " + s"core/task cpu slots = ${cpuSlots} and " + s"${limitingResource} = " + s"${defaultProf.maxTasksPerExecutor(sc.conf)}. Please adjust your configuration " + diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 2608ab9..5b2476c 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -168,7 +168,7 @@ class ResourceProfile( // limiting resource because the scheduler code uses that for slots throw new IllegalArgumentException("The number of slots on an executor has to be " + "limited by the number of cores, otherwise you waste resources and " + - "dynamic allocation doesn't work properly. Your configuration has " + + "some scheduling doesn't work properly. Your configuration has " + s"core/task cpu slots = ${taskLimit} and " + s"${execReq.resourceName} = ${numTasks}. " + "Please adjust your configuration so that all resources require same number " + @@ -183,12 +183,11 @@ class ResourceProfile( "no corresponding task resource request was specified.") } } -if(!shouldCheckExecCores && Utils.isDynamicAllocationEnabled(sparkConf)) { +if(!shouldCheckExecCores && execResourceToCheck.nonEmpty) { // if we can't rely on the executor cores config throw a warning for user logWarning("Please ensure that the number of slots available on your " + "executors is limited by the number of cores to task cpus and not another " + -"custom resource. If cores is not the limiting resource then dynamic " + -"allocation will not work properly!") +"custom resource.") } if (taskResourcesToCheck.nonEmpty) { throw new SparkException("No executor resource configs were not specified for the " + - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 878094f [SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling 878094f is described below commit 878094f9720d3c1866cbc01fb24c9794fe34edd9 Author: Thomas Graves AuthorDate: Fri Jan 31 22:20:28 2020 -0600 [SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling ### What changes were proposed in this pull request? This change is to allow custom resource scheduler (GPUs,FPGAs,etc) resource discovery to be more flexible. Users are asking for it to work with hadoop 2.x versions that do not support resource scheduling in YARN and/or also they may not run in an isolated environment. This change creates a plugin api that users can write their own resource discovery class that allows a lot more flexibility. The user can chain plugins for different resource types. The user specified plugins execute in the order specified and will fall back to use the discovery script plugin if they don't return information for a particular resource. I had to open up a few of the classes to be public and change them to not be case classes and make them developer api in order for the the plugin to get enough information it needs. I also relaxed the yarn side so that if yarn isn't configured for resource scheduling we just warn and go on. This helps users that have yarn 3.1 but haven't configured the resource scheduling side on their cluster yet, or aren't running in isolated environment. The user would configured this like: --conf spark.resources.discovery.plugin="org.apache.spark.resource.ResourceDiscoveryFPGAPlugin, org.apache.spark.resource.ResourceDiscoveryGPUPlugin" Note the executor side had to be wrapped with a classloader to make sure we include the user classpath for jars they specified on submission. Note this is more flexible because the discovery script has limitations such as spawning it in a separate process. This means if you are trying to allocate resources in that process they might be released when the script returns. Other things are the class makes it more flexible to be able to integrate with existing systems and solutions for assigning resources. ### Why are the changes needed? to more easily use spark resource scheduling with older versions of hadoop or in non-isolated enivronments. ### Does this PR introduce any user-facing change? Yes a plugin api ### How was this patch tested? Unit tests added and manual testing done on yarn and standalone modes. Closes #27410 from tgravescs/hadoop27spark3. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../api/resource/ResourceDiscoveryPlugin.java | 63 +++ .../main/scala/org/apache/spark/SparkContext.scala | 8 +- .../spark/deploy/StandaloneResourceUtils.scala | 4 +- .../executor/CoarseGrainedExecutorBackend.scala| 36 +++- .../org/apache/spark/internal/config/package.scala | 12 ++ .../resource/ResourceDiscoveryScriptPlugin.scala | 62 +++ .../apache/spark/resource/ResourceProfile.scala| 4 +- .../org/apache/spark/resource/ResourceUtils.scala | 136 +-- .../scala/org/apache/spark/SparkConfSuite.scala| 2 +- .../CoarseGrainedExecutorBackendSuite.scala| 3 +- .../resource/ResourceDiscoveryPluginSuite.scala| 194 + .../apache/spark/resource/ResourceUtilsSuite.scala | 65 --- .../apache/spark/resource/TestResourceIDs.scala| 16 +- docs/configuration.md | 12 ++ .../apache/spark/deploy/k8s/KubernetesUtils.scala | 8 +- .../k8s/features/BasicDriverFeatureStepSuite.scala | 2 +- .../features/BasicExecutorFeatureStepSuite.scala | 4 +- .../spark/deploy/yarn/ResourceRequestHelper.scala | 31 +++- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 6 +- 19 files changed, 560 insertions(+), 108 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/resource/ResourceDiscoveryPlugin.java b/core/src/main/java/org/apache/spark/api/resource/ResourceDiscoveryPlugin.java new file mode 100644 index 000..ffd2f83 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/resource/ResourceDiscoveryPlugin.java @@ -0,0 +1,63 @@ +/* + * 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 +
[spark] branch master updated: [SPARK-30511][SPARK-28403][CORE] Don't treat failed/killed speculative tasks as pending in ExecutorAllocationManager
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 21bc047 [SPARK-30511][SPARK-28403][CORE] Don't treat failed/killed speculative tasks as pending in ExecutorAllocationManager 21bc047 is described below commit 21bc0474bbb16c7648aed40f25a2945d98d2a167 Author: zebi...@fb.com AuthorDate: Fri Jan 31 08:49:34 2020 -0600 [SPARK-30511][SPARK-28403][CORE] Don't treat failed/killed speculative tasks as pending in ExecutorAllocationManager ### What changes were proposed in this pull request? Currently, when speculative tasks fail/get killed, they are still considered as pending and count towards the calculation of number of needed executors. To be more accurate: `stageAttemptToNumSpeculativeTasks(stageAttempt)` is incremented on onSpeculativeTaskSubmitted, but never decremented. `stageAttemptToNumSpeculativeTasks -= stageAttempt` is performed on stage completion. **This means Spark is marking ended speculative tasks as pending, which leads to Spark to hold more executors [...] This PR fixes this issue by updating `stageAttemptToSpeculativeTaskIndices` and `stageAttemptToNumSpeculativeTasks` on speculative tasks completion. This PR also addresses some other minor issues: scheduler behavior after receiving an intentionally killed task event; try to address [SPARK-28403](https://issues.apache.org/jira/browse/SPARK-28403). ### Why are the changes needed? This has caused resource wastage in our production with speculation enabled. With aggressive speculation, we found data skewed jobs can hold hundreds of idle executors with less than 10 tasks running. An easy repro of the issue (`--conf spark.speculation=true --conf spark.executor.cores=4 --conf spark.dynamicAllocation.maxExecutors=1000` in cluster mode): ``` val n = 4000 val someRDD = sc.parallelize(1 to n, n) someRDD.mapPartitionsWithIndex( (index: Int, it: Iterator[Int]) => { if (index < 300 && index >= 150) { Thread.sleep(index * 1000) // Fake running tasks } else if (index == 300) { Thread.sleep(1000 * 1000) // Fake long running tasks } it.toList.map(x => index + ", " + x).iterator }).collect ``` You will see when running the last task, we would be hold 38 executors (see below), which is exactly (152 + 3) / 4 = 38. ![image](https://user-images.githubusercontent.com/9404831/72469112-9a7fac00-3793-11ea-8f50-74d0ab7325a4.png) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added a comprehensive unit test. Test with the above repro shows that we are holding 2 executors at the end ![image](https://user-images.githubusercontent.com/9404831/72469177-bbe09800-3793-11ea-850f-4a2c67142899.png) Closes #27223 from linzebing/speculation_fix. Authored-by: zebi...@fb.com Signed-off-by: Thomas Graves --- .../apache/spark/ExecutorAllocationManager.scala | 61 ++ .../spark/ExecutorAllocationManagerSuite.scala | 135 + 2 files changed, 172 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index bff854a..677386c 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -263,9 +263,16 @@ private[spark] class ExecutorAllocationManager( */ private def maxNumExecutorsNeeded(): Int = { val numRunningOrPendingTasks = listener.totalPendingTasks + listener.totalRunningTasks -math.ceil(numRunningOrPendingTasks * executorAllocationRatio / - tasksPerExecutorForFullParallelism) - .toInt +val maxNeeded = math.ceil(numRunningOrPendingTasks * executorAllocationRatio / + tasksPerExecutorForFullParallelism).toInt +if (tasksPerExecutorForFullParallelism > 1 && maxNeeded == 1 && + listener.pendingSpeculativeTasks > 0) { + // If we have pending speculative tasks and only need a single executor, allocate one more + // to satisfy the locality requirements of speculation + maxNeeded + 1 +} else { + maxNeeded +} } private def totalRunningTasks(): Int = synchronized { @@ -377,14 +384,8 @@ private[spark] class ExecutorAllocationManager( // If our target has not changed, do not send a message // to the cluster manager and reset our exponential growth if (delta == 0) { - // Check if there is any speculative jobs pending - if (listener.pendingTasks == 0 && listener.pendingSpeculativeTasks > 0) { -nu
[spark] branch master updated: [SPARK-30638][CORE] Add resources allocated to PluginContext
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3d2b8d8 [SPARK-30638][CORE] Add resources allocated to PluginContext 3d2b8d8 is described below commit 3d2b8d8b13eff0faa02316542a343e7a64873b8a Author: Thomas Graves AuthorDate: Fri Jan 31 08:25:32 2020 -0600 [SPARK-30638][CORE] Add resources allocated to PluginContext ### What changes were proposed in this pull request? Add the allocated resources to parameters to the PluginContext so that any plugins in driver or executor could use this information to initialize devices or use this information in a useful manner. ### Why are the changes needed? To allow users to initialize/track devices once at the executor level before each task runs to use them. ### Does this PR introduce any user-facing change? Yes to the people using the Executor/Driver plugin interface. ### How was this patch tested? Unit tests and manually by writing a plugin that initialized GPU's using this interface. Closes #27367 from tgravescs/pluginWithResources. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../org/apache/spark/api/plugin/PluginContext.java | 5 + .../main/scala/org/apache/spark/SparkContext.scala | 2 +- .../executor/CoarseGrainedExecutorBackend.scala| 10 +- .../scala/org/apache/spark/executor/Executor.scala | 7 +- .../spark/internal/plugin/PluginContainer.scala| 36 +-- .../spark/internal/plugin/PluginContextImpl.scala | 6 +- .../scheduler/local/LocalSchedulerBackend.scala| 5 +- .../org/apache/spark/executor/ExecutorSuite.scala | 12 ++- .../internal/plugin/PluginContainerSuite.scala | 109 +++-- .../spark/executor/MesosExecutorBackend.scala | 4 +- 10 files changed, 167 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java b/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java index b9413cf..36d8275 100644 --- a/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java +++ b/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java @@ -18,11 +18,13 @@ package org.apache.spark.api.plugin; import java.io.IOException; +import java.util.Map; import com.codahale.metrics.MetricRegistry; import org.apache.spark.SparkConf; import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.resource.ResourceInformation; /** * :: DeveloperApi :: @@ -54,6 +56,9 @@ public interface PluginContext { /** The host name which is being used by the Spark process for communication. */ String hostname(); + /** The custom resources (GPUs, FPGAs, etc) allocated to driver or executor. */ + Map resources(); + /** * Send a message to the plugin's driver-side component. * diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3262631..6e0c7ac 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -542,7 +542,7 @@ class SparkContext(config: SparkConf) extends Logging { HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) // Initialize any plugins before the task scheduler is initialized. -_plugins = PluginContainer(this) +_plugins = PluginContainer(this, _resources.asJava) // Create and start the scheduler val (sched, ts) = SparkContext.createTaskScheduler(this, master, deployMode) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 511c63a..ce211ce 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -69,6 +69,8 @@ private[spark] class CoarseGrainedExecutorBackend( // to be changed so that we don't share the serializer instance across threads private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() + private var _resources = Map.empty[String, ResourceInformation] + /** * Map each taskId to the information about the resource allocated to it, Please refer to * [[ResourceInformation]] for specifics. @@ -78,9 +80,8 @@ private[spark] class CoarseGrainedExecutorBackend( override def onStart(): Unit = { logInfo("Connecting to driver: " + driverUrl) -var resources = Map.empty[String, ResourceInformation] try { - resources = parseOrFindResources(resourcesFileOpt) + _resources = parseOrFindR
[spark] branch branch-2.4 updated: [SPARK-30512] Added a dedicated boss event loop group
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.4 by this push: new 12f4492 [SPARK-30512] Added a dedicated boss event loop group 12f4492 is described below commit 12f449220dde0b8476f58806f7dee06fcb54da87 Author: Chandni Singh AuthorDate: Wed Jan 29 15:02:48 2020 -0600 [SPARK-30512] Added a dedicated boss event loop group ### What changes were proposed in this pull request? Adding a dedicated boss event loop group to the Netty pipeline in the External Shuffle Service to avoid the delay in channel registration. ``` EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1, conf.getModuleName() + "-boss"); EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) .channel(NettyUtils.getServerChannelClass(ioMode)) .option(ChannelOption.ALLOCATOR, allocator) ``` ### Why are the changes needed? We have been seeing a large number of SASL authentication (RPC requests) timing out with the external shuffle service. ``` java.lang.RuntimeException: java.util.concurrent.TimeoutException: Timeout waiting for task. at org.spark-project.guava.base.Throwables.propagate(Throwables.java:160) at org.apache.spark.network.client.TransportClient.sendRpcSync(TransportClient.java:278) at org.apache.spark.network.sasl.SaslClientBootstrap.doBootstrap(SaslClientBootstrap.java:80) at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:228) at org.apache.spark.network.client.TransportClientFactory.createUnmanagedClient(TransportClientFactory.java:181) at org.apache.spark.network.shuffle.ExternalShuffleClient.registerWithShuffleServer(ExternalShuffleClient.java:141) at org.apache.spark.storage.BlockManager$$anonfun$registerWithExternalShuffleServer$1.apply$mcVI$sp(BlockManager.scala:218) ``` The investigation that we have done is described here: https://github.com/netty/netty/issues/9890 After adding `LoggingHandler` to the netty pipeline, we saw that the registration of the channel was getting delay which is because the worker threads are busy with the existing channels. ### Does this PR introduce any user-facing change? No ### How was this patch tested? We have tested the patch on our clusters and with a stress testing tool. After this change, we didn't see any SASL requests timing out. Existing unit tests pass. Closes #27240 from otterc/SPARK-30512. Authored-by: Chandni Singh Signed-off-by: Thomas Graves (cherry picked from commit 6b47ace27d04012bcff47951ea1eea2aa6fb7d60) Signed-off-by: Thomas Graves --- .../main/java/org/apache/spark/network/server/TransportServer.java | 7 --- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index 9c85ab2..da3fe30 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -91,9 +91,10 @@ public class TransportServer implements Closeable { private void init(String hostToBind, int portToBind) { IOMode ioMode = IOMode.valueOf(conf.ioMode()); -EventLoopGroup bossGroup = - NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); -EventLoopGroup workerGroup = bossGroup; +EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1, + conf.getModuleName() + "-boss"); +EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(), + conf.getModuleName() + "-server"); PooledByteBufAllocator allocator = NettyUtils.createPooledByteBufAllocator( conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-30512] Added a dedicated boss event loop group
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6b47ace [SPARK-30512] Added a dedicated boss event loop group 6b47ace is described below commit 6b47ace27d04012bcff47951ea1eea2aa6fb7d60 Author: Chandni Singh AuthorDate: Wed Jan 29 15:02:48 2020 -0600 [SPARK-30512] Added a dedicated boss event loop group ### What changes were proposed in this pull request? Adding a dedicated boss event loop group to the Netty pipeline in the External Shuffle Service to avoid the delay in channel registration. ``` EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1, conf.getModuleName() + "-boss"); EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) .channel(NettyUtils.getServerChannelClass(ioMode)) .option(ChannelOption.ALLOCATOR, allocator) ``` ### Why are the changes needed? We have been seeing a large number of SASL authentication (RPC requests) timing out with the external shuffle service. ``` java.lang.RuntimeException: java.util.concurrent.TimeoutException: Timeout waiting for task. at org.spark-project.guava.base.Throwables.propagate(Throwables.java:160) at org.apache.spark.network.client.TransportClient.sendRpcSync(TransportClient.java:278) at org.apache.spark.network.sasl.SaslClientBootstrap.doBootstrap(SaslClientBootstrap.java:80) at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:228) at org.apache.spark.network.client.TransportClientFactory.createUnmanagedClient(TransportClientFactory.java:181) at org.apache.spark.network.shuffle.ExternalShuffleClient.registerWithShuffleServer(ExternalShuffleClient.java:141) at org.apache.spark.storage.BlockManager$$anonfun$registerWithExternalShuffleServer$1.apply$mcVI$sp(BlockManager.scala:218) ``` The investigation that we have done is described here: https://github.com/netty/netty/issues/9890 After adding `LoggingHandler` to the netty pipeline, we saw that the registration of the channel was getting delay which is because the worker threads are busy with the existing channels. ### Does this PR introduce any user-facing change? No ### How was this patch tested? We have tested the patch on our clusters and with a stress testing tool. After this change, we didn't see any SASL requests timing out. Existing unit tests pass. Closes #27240 from otterc/SPARK-30512. Authored-by: Chandni Singh Signed-off-by: Thomas Graves --- .../main/java/org/apache/spark/network/server/TransportServer.java | 7 --- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index 8396e69..f0ff9f5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -100,9 +100,10 @@ public class TransportServer implements Closeable { private void init(String hostToBind, int portToBind) { IOMode ioMode = IOMode.valueOf(conf.ioMode()); -EventLoopGroup bossGroup = - NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); -EventLoopGroup workerGroup = bossGroup; +EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1, + conf.getModuleName() + "-boss"); +EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(), + conf.getModuleName() + "-server"); bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-29306][CORE] Stage Level Sched: Executors need to track what ResourceProfile they are created with
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6dbfa2b [SPARK-29306][CORE] Stage Level Sched: Executors need to track what ResourceProfile they are created with 6dbfa2b is described below commit 6dbfa2bb9c5215aab97ec3f86b3325a11a7ff4d1 Author: Thomas Graves AuthorDate: Fri Jan 17 08:15:25 2020 -0600 [SPARK-29306][CORE] Stage Level Sched: Executors need to track what ResourceProfile they are created with ### What changes were proposed in this pull request? This is the second PR for the Stage Level Scheduling. This is adding in the necessary executor side changes: 1) executors to know what ResourceProfile they should be using 2) handle parsing the resource profile settings - these are not in the global configs 3) then reporting back to the driver what resource profile it was started with. This PR adds all the piping for YARN to pass the information all the way to executors, but it just uses the default ResourceProfile (which is the global applicatino level configs). At a high level these changes include: 1) adding a new --resourceProfileId option to the CoarseGrainedExecutorBackend 2) Add the ResourceProfile settings to new internal confs that gets passed into the Executor 3) Executor changes that use the resource profile id passed in to read the corresponding ResourceProfile confs and then parse those requests and discover resources as necessary 4) Executor registers to Driver with the Resource profile id so that the ExecutorMonitor can track how many executor with each profile are running 5) YARN side changes to show that passing the resource profile id and confs actually works. Just uses the DefaultResourceProfile for now. I also removed a check from the CoarseGrainedExecutorBackend that used to check to make sure there were task requirements before parsing any custom resource executor requests. With the resource profiles this becomes much more expensive because we would then have to pass the task requests to each executor and the check was just a short cut and not really needed. It was much cleaner just to remove it. Note there were some changes to the ResourceProfile, ExecutorResourceRequests, and TaskResourceRequests in this PR as well because I discovered some issues with things not being immutable. That api now look like: val rpBuilder = new ResourceProfileBuilder() val ereq = new ExecutorResourceRequests() val treq = new TaskResourceRequests() ereq.cores(2).memory("6g").memoryOverhead("2g").pysparkMemory("2g").resource("gpu", 2, "/home/tgraves/getGpus") treq.cpus(2).resource("gpu", 2) val resourceProfile = rpBuilder.require(ereq).require(treq).build This makes is so that ResourceProfile is immutable and Spark can use it directly without worrying about the user changing it. ### Why are the changes needed? These changes are needed for the executor to report which ResourceProfile they are using so that ultimately the dynamic allocation manager can use that information to know how many with a profile are running and how many more it needs to request. Its also needed to get the resource profile confs to the executor so that it can run the appropriate discovery script if needed. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit tests and manually on YARN. Closes #26682 from tgravescs/SPARK-29306. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../executor/CoarseGrainedExecutorBackend.scala| 56 .../spark/resource/ExecutorResourceRequest.scala | 39 +++-- .../spark/resource/ExecutorResourceRequests.scala | 37 +++-- .../apache/spark/resource/ResourceProfile.scala| 160 + .../spark/resource/ResourceProfileBuilder.scala| 84 +++ .../org/apache/spark/resource/ResourceUtils.scala | 113 --- .../spark/resource/TaskResourceRequest.scala | 20 ++- .../spark/resource/TaskResourceRequests.scala | 25 ++-- .../cluster/CoarseGrainedClusterMessage.scala | 10 +- .../cluster/CoarseGrainedSchedulerBackend.scala| 21 ++- .../spark/scheduler/cluster/ExecutorData.scala | 7 +- .../spark/scheduler/cluster/ExecutorInfo.scala | 32 +++-- .../spark/scheduler/dynalloc/ExecutorMonitor.scala | 60 ++-- .../spark/resource/JavaResourceProfileSuite.java | 2 +- .../spark/ExecutorAllocationManagerSuite.scala | 6 +- .../org/apache/spark/HeartbeatReceiverSuite.scala | 5 +- .../scala/org/apache/spark/LocalSparkContext.scala | 3
[spark] branch master updated: [SPARK-30448][CORE] accelerator aware scheduling enforce cores as limiting resource
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d6532c7 [SPARK-30448][CORE] accelerator aware scheduling enforce cores as limiting resource d6532c7 is described below commit d6532c7079f22f32e90e1c69c25bdfab51c7c53e Author: Thomas Graves AuthorDate: Fri Jan 10 08:32:28 2020 -0600 [SPARK-30448][CORE] accelerator aware scheduling enforce cores as limiting resource ### What changes were proposed in this pull request? This PR is to make sure cores is the limiting resource when using accelerator aware scheduling and fix a few issues with SparkContext.checkResourcesPerTask For the first version of accelerator aware scheduling(SPARK-27495), the SPIP had a condition that we can support dynamic allocation because we were going to have a strict requirement that we don't waste any resources. This means that the number of slots each executor has could be calculated from the number of cores and task cpus just as is done today. Somewhere along the line of development we relaxed that and only warn when we are wasting resources. This breaks the dynamic allocation logic if the limiting resource is no longer the cores because its using the cores and task cpus to calculate the number of executors it needs. This means we will request less executors then we really need to run everything. We have to enforce that cores is always the limiting resource so we should throw if its not. The only issue with us enforcing this is on cluster managers (standalone and mesos coarse grained) where we don't know the executor cores up front by default. Meaning the spark.executor.cores config defaults to 1 but when the executor is started by default it gets all the cores of the Worker. So we have to add logic specifically to handle that and we can't enforce this requirements, we can just warn when dynamic allocation is enabled for those. ### Why are the changes needed? Bug in dynamic allocation if cores is not limiting resource and warnings not correct. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Unit test added and manually tested the confiditions on local mode, local cluster mode, standalone mode, and yarn. Closes #27138 from tgravescs/SPARK-30446. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/SparkContext.scala | 39 +- .../scala/org/apache/spark/SparkContextSuite.scala | 22 ++-- .../CoarseGrainedSchedulerBackendSuite.scala | 2 +- 3 files changed, 51 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 94a0ce7..3262631 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2779,9 +2779,13 @@ object SparkContext extends Logging { } else { executorCores.get } + // some cluster managers don't set the EXECUTOR_CORES config by default (standalone + // and mesos coarse grained), so we can't rely on that config for those. + val shouldCheckExecCores = executorCores.isDefined || sc.conf.contains(EXECUTOR_CORES) || +(master.equalsIgnoreCase("yarn") || master.startsWith("k8s")) // Number of cores per executor must meet at least one task requirement. - if (execCores < taskCores) { + if (shouldCheckExecCores && execCores < taskCores) { throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " + s"the task config: ${CPUS_PER_TASK.key} = $taskCores when run on $master.") } @@ -2789,11 +2793,14 @@ object SparkContext extends Logging { // Calculate the max slots each executor can provide based on resources available on each // executor and resources required by each task. val taskResourceRequirements = parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX) - val executorResourcesAndAmounts = -parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX) + val executorResourcesAndAmounts = parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX) .map(request => (request.id.resourceName, request.amount)).toMap - var numSlots = execCores / taskCores - var limitingResourceName = "CPU" + + var (numSlots, limitingResourceName) = if (shouldCheckExecCores) { +(execCores / taskCores, "CPU") + } else { +(-1, "") + } taskResourceRequirements.foreach { taskReq =>
[spark] branch master updated: [SPARK-30300][SQL][WEB-UI] Fix updating the UI max value string when driver updates the same metric id as the tasks
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 0d2ef3a [SPARK-30300][SQL][WEB-UI] Fix updating the UI max value string when driver updates the same metric id as the tasks 0d2ef3a is described below commit 0d2ef3ae2b2d0b66f763d6bb2e490a667c83f9f2 Author: Niranjan Artal AuthorDate: Fri Dec 20 07:29:28 2019 -0600 [SPARK-30300][SQL][WEB-UI] Fix updating the UI max value string when driver updates the same metric id as the tasks ### What changes were proposed in this pull request? In this PR, For a given metrics id we are checking if the driver side accumulator's value is greater than max of all stages value. If it's true, then we are removing that entry from the Hashmap. By doing this, for this metrics, "driver" would be displayed on the UI(As the driver would have the maximum value) ### Why are the changes needed? This PR fixes https://issues.apache.org/jira/browse/SPARK-30300. Currently driver's metric value is not compared while caluculating the max. ### Does this PR introduce any user-facing change? For the metrics where driver's value is greater than max of all stages, this is the change. Previous : (min, median, max (stageId 0( attemptId 1): taskId 2)) Now: (min, median, max (driver)) ### How was this patch tested? Ran unit tests. Closes #26941 from nartal1/SPARK-30300. Authored-by: Niranjan Artal Signed-off-by: Thomas Graves --- .../org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala| 6 ++ 1 file changed, 6 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index 64d2f33..d5bb36e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -237,6 +237,12 @@ class SQLAppStatusListener( if (metricTypes.contains(id)) { val prev = allMetrics.getOrElse(id, null) val updated = if (prev != null) { + // If the driver updates same metrics as tasks and has higher value then remove + // that entry from maxMetricsFromAllStage. This would make stringValue function default + // to "driver" that would be displayed on UI. + if (maxMetricsFromAllStages.contains(id) && value > maxMetricsFromAllStages(id)(0)) { +maxMetricsFromAllStages.remove(id) + } val _copy = Arrays.copyOf(prev, prev.length + 1) _copy(prev.length) = value _copy - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-30209][SQL][WEB-UI] Display stageId, attemptId and taskId for max metrics in Spark UI
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new dddfeca [SPARK-30209][SQL][WEB-UI] Display stageId, attemptId and taskId for max metrics in Spark UI dddfeca is described below commit dddfeca175bdce5294debe00d4a993daef92ca60 Author: Niranjan Artal AuthorDate: Mon Dec 16 15:27:34 2019 -0600 [SPARK-30209][SQL][WEB-UI] Display stageId, attemptId and taskId for max metrics in Spark UI ### What changes were proposed in this pull request? SPARK-30209 discusses about adding additional metrics such as stageId, attempId and taskId for max metrics. We have the data required to display in LiveStageMetrics. Need to capture and pass these metrics to display on the UI. To minimize memory used for variables, we are saving maximum of each metric id per stage. So per stage additional memory usage is (#metrics * 4 * sizeof(Long)). Then max is calculated for each metric id among all stages which is passed in the stringValue method. Memory used is minimal. Ran the benchmark for runtime. Stage.Proc time has increased to around 1.5-2.5x but the Aggregate time has decreased. ### Why are the changes needed? These additional metrics stageId, attemptId and taskId could help in debugging the jobs quicker. For a given operator, it will be easy to identify the task which is taking maximum time to complete from the SQL tab itself. ### Does this PR introduce any user-facing change? Yes. stageId, attemptId and taskId is shown only for executor side metrics. For driver metrics, "(driver)" is displayed on UI. ![image (3)](https://user-images.githubusercontent.com/50492963/70763041-929d9980-1d07-11ea-940f-88ac6bdce9b5.png) "Driver" ![image (4)](https://user-images.githubusercontent.com/50492963/70763043-94675d00-1d07-11ea-95ab-3478728cb435.png) ### How was this patch tested? Manually tested, ran benchmark script for runtime. Closes #26843 from nartal1/SPARK-30209. Authored-by: Niranjan Artal Signed-off-by: Thomas Graves --- .../spark/sql/execution/metric/SQLMetrics.scala| 52 -- .../sql/execution/ui/SQLAppStatusListener.scala| 63 +- .../sql/execution/metric/SQLMetricsSuite.scala | 39 +- .../sql/execution/metric/SQLMetricsTestUtils.scala | 26 ++--- .../execution/ui/SQLAppStatusListenerSuite.scala | 5 +- 5 files changed, 137 insertions(+), 48 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index b7f0ab2..45b1c86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -111,7 +111,8 @@ object SQLMetrics { // data size total (min, med, max): // 100GB (100MB, 1GB, 10GB) val acc = new SQLMetric(SIZE_METRIC, -1) -acc.register(sc, name = Some(s"$name total (min, med, max)"), countFailedValues = false) +acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } @@ -120,14 +121,16 @@ object SQLMetrics { // duration(min, med, max): // 5s (800ms, 1s, 2s) val acc = new SQLMetric(TIMING_METRIC, -1) -acc.register(sc, name = Some(s"$name total (min, med, max)"), countFailedValues = false) +acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } def createNanoTimingMetric(sc: SparkContext, name: String): SQLMetric = { // Same with createTimingMetric, just normalize the unit of time to millisecond. val acc = new SQLMetric(NS_TIMING_METRIC, -1) -acc.register(sc, name = Some(s"$name total (min, med, max)"), countFailedValues = false) +acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } @@ -142,31 +145,46 @@ object SQLMetrics { // probe avg (min, med, max): // (1.2, 2.2, 6.3) val acc = new SQLMetric(AVERAGE_METRIC) -acc.register(sc, name = Some(s"$name (min, med, max)"), countFailedValues = false) +acc.register(sc, name = Some(s"$name (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } + private def toNumberFormat(value: Long): String = { +val numberFormat = NumberFormat.getNumberInstance(Locale.US) +numberFormat.format(value.toDouble / baseForAvgMetric) + } + + def metricNeedsMax(
[spark] branch master updated: [SPARK-29976][CORE] Trigger speculation for stages with too few tasks
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ad238a2 [SPARK-29976][CORE] Trigger speculation for stages with too few tasks ad238a2 is described below commit ad238a2238a9d0da89be4424574436cbfaee579d Author: Yuchen Huo AuthorDate: Tue Dec 10 14:43:26 2019 -0600 [SPARK-29976][CORE] Trigger speculation for stages with too few tasks ### What changes were proposed in this pull request? This PR add an optional spark conf for speculation to allow speculative runs for stages where there are only a few tasks. ``` spark.speculation.task.duration.threshold ``` If provided, tasks would be speculatively run if the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold. ### Why are the changes needed? This change helps avoid scenarios where there is single executor that could hang forever due to disk issue and we unfortunately assigned the single task in a TaskSet to that executor and cause the whole job to hang forever. ### Does this PR introduce any user-facing change? yes. If the new config `spark.speculation.task.duration.threshold` is provided and the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold, then speculative tasks would be submitted for the running tasks in the TaskSet. ### How was this patch tested? Unit tests are added to TaskSetManagerSuite. Closes #26614 from yuchenhuo/SPARK-29976. Authored-by: Yuchen Huo Signed-off-by: Thomas Graves --- .../org/apache/spark/internal/config/package.scala | 12 +++ .../apache/spark/scheduler/TaskSetManager.scala| 60 + .../spark/scheduler/TaskSetManagerSuite.scala | 98 ++ docs/configuration.md | 13 +++ 4 files changed, 167 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 25dc4c6..9d7b31a 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1499,6 +1499,18 @@ package object config { .doubleConf .createWithDefault(0.75) + private[spark] val SPECULATION_TASK_DURATION_THRESHOLD = +ConfigBuilder("spark.speculation.task.duration.threshold") + .doc("Task duration after which scheduler would try to speculative run the task. If " + +"provided, tasks would be speculatively run if current stage contains less tasks " + +"than or equal to the number of slots on a single executor and the task is taking " + +"longer time than the threshold. This config helps speculate stage with very few " + +"tasks. Regular speculation configs may also apply if the executor slots are " + +"large enough. E.g. tasks might be re-launched if there are enough successful runs " + +"even though the threshold hasn't been reached.") + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir") .doc("Staging directory used while submitting applications.") .stringConf diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 5c0bc49..e026e90 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -81,6 +81,13 @@ private[spark] class TaskSetManager( val speculationQuantile = conf.get(SPECULATION_QUANTILE) val speculationMultiplier = conf.get(SPECULATION_MULTIPLIER) val minFinishedForSpeculation = math.max((speculationQuantile * numTasks).floor.toInt, 1) + // User provided threshold for speculation regardless of whether the quantile has been reached + val speculationTaskDurationThresOpt = conf.get(SPECULATION_TASK_DURATION_THRESHOLD) + // SPARK-29976: Only when the total number of tasks in the stage is less than or equal to the + // number of slots on a single executor, would the task manager speculative run the tasks if + // their duration is longer than the given threshold. In this way, we wouldn't speculate too + // aggressively but still handle basic cases. + val speculationTasksLessEqToSlots = numTasks <= (conf.get(EXECUTOR_CORES) / sched.CPUS_PER_TASK) // For each task, tracks whether a copy of the task has succeeded. A task wi
[spark] branch master updated: [SPARK-29415][CORE] Stage Level Sched: Add base ResourceProfile and Request classes
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 2d5de25 [SPARK-29415][CORE] Stage Level Sched: Add base ResourceProfile and Request classes 2d5de25 is described below commit 2d5de25a999e0e5580cf4024937b61e6c9265672 Author: Thomas Graves AuthorDate: Mon Nov 25 09:36:39 2019 -0600 [SPARK-29415][CORE] Stage Level Sched: Add base ResourceProfile and Request classes ### What changes were proposed in this pull request? This PR is adding the base classes needed for Stage level scheduling. Its adding a ResourceProfile and the executor and task resource request classes. These are made private for now until we get all the parts implemented, at which point this will become public interfaces. I am adding them first as all the other subtasks for this feature require these classes. If people have better ideas on breaking this feature up please let me know. See https://issues.apache.org/jira/browse/SPARK-29415 for more detailed design. ### Why are the changes needed? New API for stage level scheduling. Its easier to add these first because the other jira for this features will all use them. ### Does this PR introduce any user-facing change? Yes adds API to create a ResourceProfile with executor/task resources, see the spip jira https://issues.apache.org/jira/browse/SPARK-27495 Example of the api: val rp = new ResourceProfile() rp.require(new ExecutorResourceRequest("cores", 2)) rp.require(new ExecutorResourceRequest("gpu", 1, Some("/opt/gpuScripts/getGpus"))) rp.require(new TaskResourceRequest("gpu", 1)) ### How was this patch tested? Tested using Unit tests added with this PR. Closes #26284 from tgravescs/SPARK-29415. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../spark/resource/ExecutorResourceRequest.scala | 77 ++ .../spark/resource/ExecutorResourceRequests.scala | 122 +++ .../apache/spark/resource/ResourceProfile.scala| 147 ++ .../org/apache/spark/resource/ResourceUtils.scala | 7 +- .../spark/resource/TaskResourceRequest.scala | 43 ++ .../spark/resource/TaskResourceRequests.scala | 75 ++ .../spark/resource/JavaResourceProfileSuite.java | 66 .../spark/resource/ResourceProfileSuite.scala | 166 + 8 files changed, 701 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala new file mode 100644 index 000..88ceaad --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala @@ -0,0 +1,77 @@ +/* + * 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. + */ + +package org.apache.spark.resource + +import scala.collection.mutable + +import org.apache.spark.resource.ResourceUtils.RESOURCE_DOT + +/** + * An Executor resource request. This is used in conjunction with the ResourceProfile to + * programmatically specify the resources needed for an RDD that will be applied at the + * stage level. + * + * This is used to specify what the resource requirements are for an Executor and how + * Spark can find out specific details about those resources. Not all the parameters are + * required for every resource type. The resources names supported + * correspond to the regular Spark configs with the prefix removed. For instance overhead + * memory in this api is memoryOverhead, which is spark.executor.memoryOverhead with + * spark.executor removed. Resources like GPUs are resource.gpu + * (spark configs spark.executor.resource.gpu.*). The amount, discoveryScript, and vendor + * parameters for resources are all the same parameters a user would specify through the + * configs: spark.executor.resource.{resourceName}.{amount, discoveryScript, vendor}. + * + * For instance
[spark] branch master updated: [SPARK-29151][CORE] Support fractional resources for task resource scheduling
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3cb18d9 [SPARK-29151][CORE] Support fractional resources for task resource scheduling 3cb18d9 is described below commit 3cb18d90c441bbaa64c693e276793b670213e599 Author: Alessandro Bellina AuthorDate: Tue Nov 5 08:57:43 2019 -0600 [SPARK-29151][CORE] Support fractional resources for task resource scheduling ### What changes were proposed in this pull request? This PR adds the ability for tasks to request fractional resources, in order to be able to execute more than 1 task per resource. For example, if you have 1 GPU in the executor, and the task configuration is 0.5 GPU/task, the executor can schedule two tasks to run on that 1 GPU. ### Why are the changes needed? Currently there is no good way to share a resource such that multiple tasks can run on a single unit. This allows multiple tasks to share an executor resource. ### Does this PR introduce any user-facing change? Yes: There is a configuration change where `spark.task.resource.[resource type].amount` can now be fractional. ### How was this patch tested? Unit tests and manually on standalone mode, and yarn. Closes #26078 from abellina/SPARK-29151. Authored-by: Alessandro Bellina Signed-off-by: Thomas Graves --- .../main/scala/org/apache/spark/SparkContext.scala | 21 ++-- .../apache/spark/deploy/master/WorkerInfo.scala| 1 + .../apache/spark/resource/ResourceAllocator.scala | 39 +++ .../org/apache/spark/resource/ResourceUtils.scala | 58 -- .../spark/scheduler/ExecutorResourceInfo.scala | 7 ++- .../cluster/CoarseGrainedSchedulerBackend.scala| 15 +- .../org/apache/spark/HeartbeatReceiverSuite.scala | 1 + .../scala/org/apache/spark/SparkConfSuite.scala| 51 +++ .../scala/org/apache/spark/SparkContextSuite.scala | 3 +- .../deploy/StandaloneDynamicAllocationSuite.scala | 1 + .../CoarseGrainedSchedulerBackendSuite.scala | 1 + .../scheduler/ExecutorResourceInfoSuite.scala | 34 +++-- docs/configuration.md | 12 +++-- 13 files changed, 214 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cad88ad..3cea2ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2799,7 +2799,10 @@ object SparkContext extends Logging { s" = ${taskReq.amount}") } // Compare and update the max slots each executor can provide. -val resourceNumSlots = execAmount / taskReq.amount +// If the configured amount per task was < 1.0, a task is subdividing +// executor resources. If the amount per task was > 1.0, the task wants +// multiple executor resources. +val resourceNumSlots = Math.floor(execAmount * taskReq.numParts / taskReq.amount).toInt if (resourceNumSlots < numSlots) { numSlots = resourceNumSlots limitingResourceName = taskReq.resourceName @@ -2809,11 +2812,19 @@ object SparkContext extends Logging { // large enough if any task resources were specified. taskResourceRequirements.foreach { taskReq => val execAmount = executorResourcesAndAmounts(taskReq.resourceName) -if (taskReq.amount * numSlots < execAmount) { +if ((numSlots * taskReq.amount / taskReq.numParts) < execAmount) { + val taskReqStr = if (taskReq.numParts > 1) { +s"${taskReq.amount}/${taskReq.numParts}" + } else { +s"${taskReq.amount}" + } + val resourceNumSlots = Math.floor(execAmount * taskReq.numParts/taskReq.amount).toInt val message = s"The configuration of resource: ${taskReq.resourceName} " + -s"(exec = ${execAmount}, task = ${taskReq.amount}) will result in wasted " + -s"resources due to resource ${limitingResourceName} limiting the number of " + -s"runnable tasks per executor to: ${numSlots}. Please adjust your configuration." +s"(exec = ${execAmount}, task = ${taskReqStr}, " + +s"runnable tasks = ${resourceNumSlots}) will " + +s"result in wasted resources due to resource ${limitingResourceName} limiting the " + +s"number of runnable tasks per executor to: ${numSlots}. Please adjust " + +s"your configuration." if (Utils.isTesting) { throw new SparkException(message)
[spark] branch master updated: [SPARK-27492][DOC][YARN][K8S][CORE] Resource scheduling high level user docs
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new b425f8e [SPARK-27492][DOC][YARN][K8S][CORE] Resource scheduling high level user docs b425f8e is described below commit b425f8ee6599f53f47d7d4a8f0c27f2ba7d2eab9 Author: Thomas Graves AuthorDate: Wed Sep 11 08:22:36 2019 -0500 [SPARK-27492][DOC][YARN][K8S][CORE] Resource scheduling high level user docs ### What changes were proposed in this pull request? Document the resource scheduling feature - https://issues.apache.org/jira/browse/SPARK-24615 Add general docs, yarn, kubernetes, and standalone cluster specific ones. ### Why are the changes needed? Help users understand the feature ### Does this PR introduce any user-facing change? docs ### How was this patch tested? N/A Closes #25698 from tgravescs/SPARK-27492-gpu-sched-docs. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- docs/configuration.md | 14 +- docs/running-on-kubernetes.md | 11 +++ docs/running-on-yarn.md | 14 ++ docs/spark-standalone.md | 12 4 files changed, 50 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 9933283..5cf42d5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -230,7 +230,7 @@ of the most common options to set are: write to STDOUT a JSON string in the format of the ResourceInformation class. This has a name and an array of addresses. For a client-submitted driver in Standalone, discovery script must assign different resource addresses to this driver comparing to workers' and -other dirvers' when spark.resources.coordinate.enable is off. +other drivers' when spark.resources.coordinate.enable is off. @@ -2617,3 +2617,15 @@ Also, you can modify or add configurations at runtime: --conf spark.hadoop.abc.def=xyz \ myApp.jar {% endhighlight %} + +# Custom Resource Scheduling and Configuration Overview + +GPUs and other accelerators have been widely used for accelerating special workloads, e.g., +deep learning and signal processing. Spark now supports requesting and scheduling generic resources, such as GPUs, with a few caveats. The current implementation requires that the resource have addresses that can be allocated by the scheduler. It requires your cluster manager to support and be properly configured with the resources. + +There are configurations available to request resources for the driver: spark.driver.resource.{resourceName}.amount, request resources for the executor(s): spark.executor.resource.{resourceName}.amount and specify the requirements for each task: spark.task.resource.{resourceName}.amount. The spark.driver.resource.{resourceName}.discoveryScript config is required on YARN, Kubernetes and a client side Driver on Spark Standalone. spa [...] + +Spark will use the configurations specified to first request containers with the corresponding resources from the cluster manager. Once it gets the container, Spark launches an Executor in that container which will discover what resources the container has and the addresses associated with each resource. The Executor will register with the Driver and report back the resources available to that Executor. The Spark scheduler can then schedule tasks to each Executor and assign specific reso [...] + +See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with Mesos or local mode. If using local-cluster mode see the Spark Standalone documentation but be aware only a single wor [...] + diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 2d4e5cd..4ef738e 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1266,3 +1266,14 @@ The following affect the driver and executor containers. All other containers in + +### Resource Allocation and Configuration Overview + +Please make sure to have read the Custom Resource Scheduling and Configuration Overview section on the [configuration page](configuration.html). This section only talks about the Kubernetes specific aspects of resource scheduling. + +The user is responsible to properly configuring the Kubernetes cluster to have the resources available and ideally isolate each resource per container so that a resource is not shared between multiple containers. If the resource is not isolated the user is
[spark] branch master updated: [SPARK-28577][YARN] Resource capability requested for each executor add offHeapMemorySize
This is an automated email from the ASF dual-hosted git repository. tgraves pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a07f795 [SPARK-28577][YARN] Resource capability requested for each executor add offHeapMemorySize a07f795 is described below commit a07f795aead3bd81e7cccad30a7f6148c09ed8ad Author: yangjie01 AuthorDate: Wed Sep 4 09:00:12 2019 -0500 [SPARK-28577][YARN] Resource capability requested for each executor add offHeapMemorySize ## What changes were proposed in this pull request? If MEMORY_OFFHEAP_ENABLED is true, add MEMORY_OFFHEAP_SIZE to resource requested for executor to ensure instance has enough memory to use. In this pr add a helper method `executorOffHeapMemorySizeAsMb` in `YarnSparkHadoopUtil`. ## How was this patch tested? Add 3 new test suite to test `YarnSparkHadoopUtil#executorOffHeapMemorySizeAsMb` Closes #25309 from LuciferYang/spark-28577. Authored-by: yangjie01 Signed-off-by: Thomas Graves --- docs/configuration.md | 17 ++--- .../org/apache/spark/deploy/yarn/Client.scala | 14 +++ .../apache/spark/deploy/yarn/YarnAllocator.scala | 4 +++- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 16 +++-- .../spark/deploy/yarn/YarnAllocatorSuite.scala | 21 .../deploy/yarn/YarnSparkHadoopUtilSuite.scala | 28 ++ 6 files changed, 82 insertions(+), 18 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index aad496d..9933283 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -271,17 +271,17 @@ of the most common options to set are: spark.executor.memoryOverhead executorMemory * 0.10, with minimum of 384 -Amount of non-heap memory to be allocated per executor process in cluster mode, in MiB unless +Amount of additional memory to be allocated per executor process in cluster mode, in MiB unless otherwise specified. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%). This option is currently supported on YARN and Kubernetes. -Note: Non-heap memory includes off-heap memory -(when spark.memory.offHeap.enabled=true) and memory used by other executor processes -(e.g. python process that goes with a PySpark executor) and memory used by other non-executor -processes running in the same container. The maximum memory size of container to running executor -is determined by the sum of spark.executor.memoryOverhead and -spark.executor.memory. +Note: Additional memory includes PySpark executor memory +(when spark.executor.pyspark.memory is not configured) and memory used by other +non-executor processes running in the same container. The maximum memory size of container to +running executor is determined by the sum of spark.executor.memoryOverhead, +spark.executor.memory, spark.memory.offHeap.size and +spark.executor.pyspark.memory. @@ -1378,9 +1378,6 @@ Apart from these, the following properties are also available, and may be useful If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive. -Note: If off-heap memory is enabled, may need to raise the non-heap memory size -(e.g. increase spark.driver.memoryOverhead or -spark.executor.memoryOverhead). diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 5fc6894..9be3e7b 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -97,6 +97,8 @@ private[spark] class Client( // Executor related configurations private val executorMemory = sparkConf.get(EXECUTOR_MEMORY) + // Executor offHeap memory in MiB. + protected val executorOffHeapMemory = YarnSparkHadoopUtil.executorOffHeapMemorySizeAsMb(sparkConf) private val executorMemoryOverhead = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse( math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt @@ -346,12 +348,14 @@ private[spark] class Client( val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() logInfo("Verifying our application has not requested more than the maximum " + s"memory capability of the cluster ($maxMem MB per container)") -val executorMem = executorMemory + executorMemoryOverhead + pysparkWorkerMemory