spark git commit: [SPARK-11667] Update dynamic allocation docs to reflect supported cluster managers
Repository: spark Updated Branches: refs/heads/branch-1.6 a98cac26f -> 782885786 [SPARK-11667] Update dynamic allocation docs to reflect supported cluster managers Author: Andrew Or <and...@databricks.com> Closes #9637 from andrewor14/update-da-docs. (cherry picked from commit 12a0784ac0f314a606f1237e7144eb1355421307) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/78288578 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/78288578 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/78288578 Branch: refs/heads/branch-1.6 Commit: 782885786032da72e9a76e93e1dbeb9643e572dd Parents: a98cac2 Author: Andrew Or <and...@databricks.com> Authored: Thu Nov 12 15:48:42 2015 -0800 Committer: Andrew Or <and...@databricks.com> Committed: Thu Nov 12 15:48:59 2015 -0800 -- docs/job-scheduling.md | 55 ++--- 1 file changed, 27 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/78288578/docs/job-scheduling.md -- diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 8d9c2ba..a3c34cb 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -56,36 +56,32 @@ provide another approach to share RDDs. ## Dynamic Resource Allocation -Spark 1.2 introduces the ability to dynamically scale the set of cluster resources allocated to -your application up and down based on the workload. This means that your application may give -resources back to the cluster if they are no longer used and request them again later when there -is demand. This feature is particularly useful if multiple applications share resources in your -Spark cluster. If a subset of the resources allocated to an application becomes idle, it can be -returned to the cluster's pool of resources and acquired by other applications. In Spark, dynamic -resource allocation is performed on the granularity of the executor and can be enabled through -`spark.dynamicAllocation.enabled`. - -This feature is currently disabled by default and available only on [YARN](running-on-yarn.html). -A future release will extend this to [standalone mode](spark-standalone.html) and -[Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes). Note that although Spark on -Mesos already has a similar notion of dynamic resource sharing in fine-grained mode, enabling -dynamic allocation allows your Mesos application to take advantage of coarse-grained low-latency -scheduling while sharing cluster resources efficiently. +Spark provides a mechanism to dynamically adjust the resources your application occupies based +on the workload. This means that your application may give resources back to the cluster if they +are no longer used and request them again later when there is demand. This feature is particularly +useful if multiple applications share resources in your Spark cluster. + +This feature is disabled by default and available on all coarse-grained cluster managers, i.e. +[standalone mode](spark-standalone.html), [YARN mode](running-on-yarn.html), and +[Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes). ### Configuration and Setup -All configurations used by this feature live under the `spark.dynamicAllocation.*` namespace. -To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true`. -Other relevant configurations are described on the -[configurations page](configuration.html#dynamic-allocation) and in the subsequent sections in -detail. +There are two requirements for using this feature. First, your application must set +`spark.dynamicAllocation.enabled` to `true`. Second, you must set up an *external shuffle service* +on each worker node in the same cluster and set `spark.shuffle.service.enabled` to true in your +application. The purpose of the external shuffle service is to allow executors to be removed +without deleting shuffle files written by them (more detail described +[below](job-scheduling.html#graceful-decommission-of-executors)). The way to set up this service +varies across cluster managers: + +In standalone mode, simply start your workers with `spark.shuffle.service.enabled` set to `true`. -Additionally, your application must use an external shuffle service. The purpose of the service is -to preserve the shuffle files written by executors so the executors can be safely removed (more -detail described [below](job-scheduling.html#graceful-decommission-of-executors)). To enable -this service, set `spark.shuffle.service.enabled` to `true`. In YARN, this external shuffle service -is implemented in `org.apache.spark.yarn.network.Ya
spark git commit: [SPARK-2533] Add locality levels on stage summary view
Repository: spark Updated Branches: refs/heads/master 380dfcc0d -> 74c30049a [SPARK-2533] Add locality levels on stage summary view Author: Jean-Baptiste OnofréCloses #9487 from jbonofre/SPARK-2533-2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/74c30049 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/74c30049 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/74c30049 Branch: refs/heads/master Commit: 74c30049a8bf9841eeca48f827572c2044912e21 Parents: 380dfcc Author: Jean-Baptiste Onofré Authored: Thu Nov 12 15:46:21 2015 -0800 Committer: Andrew Or Committed: Thu Nov 12 15:46:21 2015 -0800 -- .../org/apache/spark/ui/jobs/StagePage.scala| 21 +++- 1 file changed, 20 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/74c30049/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 51425e5..1b34ba9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,7 +28,7 @@ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.{InternalAccumulator, SparkConf} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} import org.apache.spark.ui._ import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.util.{Utils, Distribution} @@ -70,6 +70,21 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { private val displayPeakExecutionMemory = parent.conf.getBoolean("spark.sql.unsafe.enabled", true) + private def getLocalitySummaryString(stageData: StageUIData): String = { +val localities = stageData.taskData.values.map(_.taskInfo.taskLocality) +val localityCounts = localities.groupBy(identity).mapValues(_.size) +val localityNamesAndCounts = localityCounts.toSeq.map { case (locality, count) => + val localityName = locality match { +case TaskLocality.PROCESS_LOCAL => "Process local" +case TaskLocality.NODE_LOCAL => "Node local" +case TaskLocality.RACK_LOCAL => "Rack local" +case TaskLocality.ANY => "Any" + } + s"$localityName: $count" +} +localityNamesAndCounts.sorted.mkString("; ") + } + def render(request: HttpServletRequest): Seq[Node] = { progressListener.synchronized { val parameterId = request.getParameter("id") @@ -129,6 +144,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Total Time Across All Tasks: {UIUtils.formatDuration(stageData.executorRunTime)} + + Locality Level Summary: + {getLocalitySummaryString(stageData)} + {if (stageData.hasInput) { Input Size / Records: - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-2533] Add locality levels on stage summary view
Repository: spark Updated Branches: refs/heads/branch-1.6 340ca9e76 -> 069591799 [SPARK-2533] Add locality levels on stage summary view Author: Jean-Baptiste OnofréCloses #9487 from jbonofre/SPARK-2533-2. (cherry picked from commit 74c30049a8bf9841eeca48f827572c2044912e21) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/06959179 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/06959179 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/06959179 Branch: refs/heads/branch-1.6 Commit: 06959179985e929bf118b530da115f0a81ccb9b3 Parents: 340ca9e Author: Jean-Baptiste Onofré Authored: Thu Nov 12 15:46:21 2015 -0800 Committer: Andrew Or Committed: Thu Nov 12 15:46:29 2015 -0800 -- .../org/apache/spark/ui/jobs/StagePage.scala| 21 +++- 1 file changed, 20 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/06959179/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 51425e5..1b34ba9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,7 +28,7 @@ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.{InternalAccumulator, SparkConf} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} import org.apache.spark.ui._ import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.util.{Utils, Distribution} @@ -70,6 +70,21 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { private val displayPeakExecutionMemory = parent.conf.getBoolean("spark.sql.unsafe.enabled", true) + private def getLocalitySummaryString(stageData: StageUIData): String = { +val localities = stageData.taskData.values.map(_.taskInfo.taskLocality) +val localityCounts = localities.groupBy(identity).mapValues(_.size) +val localityNamesAndCounts = localityCounts.toSeq.map { case (locality, count) => + val localityName = locality match { +case TaskLocality.PROCESS_LOCAL => "Process local" +case TaskLocality.NODE_LOCAL => "Node local" +case TaskLocality.RACK_LOCAL => "Rack local" +case TaskLocality.ANY => "Any" + } + s"$localityName: $count" +} +localityNamesAndCounts.sorted.mkString("; ") + } + def render(request: HttpServletRequest): Seq[Node] = { progressListener.synchronized { val parameterId = request.getParameter("id") @@ -129,6 +144,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Total Time Across All Tasks: {UIUtils.formatDuration(stageData.executorRunTime)} + + Locality Level Summary: + {getLocalitySummaryString(stageData)} + {if (stageData.hasInput) { Input Size / Records: - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11658] simplify documentation for PySpark combineByKey
Repository: spark Updated Branches: refs/heads/branch-1.6 782885786 -> 0dd6c2987 [SPARK-11658] simplify documentation for PySpark combineByKey Author: Chris SnowCloses #9640 from snowch/patch-3. (cherry picked from commit 68ef61bb656bd9c08239726913ca8ab271d52786) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0dd6c298 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0dd6c298 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0dd6c298 Branch: refs/heads/branch-1.6 Commit: 0dd6c2987fd80531bae501394e93d6510f022f20 Parents: 7828857 Author: Chris Snow Authored: Thu Nov 12 15:50:47 2015 -0800 Committer: Andrew Or Committed: Thu Nov 12 15:50:53 2015 -0800 -- python/pyspark/rdd.py | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0dd6c298/python/pyspark/rdd.py -- diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 56e8922..4b4d596 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1760,7 +1760,6 @@ class RDD(object): In addition, users can control the partitioning of the output RDD. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) ->>> def f(x): return x >>> def add(a, b): return a + str(b) >>> sorted(x.combineByKey(str, add, add).collect()) [('a', '11'), ('b', '1')] - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11658] simplify documentation for PySpark combineByKey
Repository: spark Updated Branches: refs/heads/master 12a0784ac -> 68ef61bb6 [SPARK-11658] simplify documentation for PySpark combineByKey Author: Chris SnowCloses #9640 from snowch/patch-3. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/68ef61bb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/68ef61bb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/68ef61bb Branch: refs/heads/master Commit: 68ef61bb656bd9c08239726913ca8ab271d52786 Parents: 12a0784 Author: Chris Snow Authored: Thu Nov 12 15:50:47 2015 -0800 Committer: Andrew Or Committed: Thu Nov 12 15:50:47 2015 -0800 -- python/pyspark/rdd.py | 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/68ef61bb/python/pyspark/rdd.py -- diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 56e8922..4b4d596 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1760,7 +1760,6 @@ class RDD(object): In addition, users can control the partitioning of the output RDD. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) ->>> def f(x): return x >>> def add(a, b): return a + str(b) >>> sorted(x.combineByKey(str, add, add).collect()) [('a', '11'), ('b', '1')] - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10192][CORE] simple test w/ failure involving a shared dependency
Repository: spark Updated Branches: refs/heads/master c0e48dfa6 -> 33112f9c4 [SPARK-10192][CORE] simple test w/ failure involving a shared dependency just trying to increase test coverage in the scheduler, this already works. It includes a regression test for SPARK-9809 copied some test utils from https://github.com/apache/spark/pull/5636, we can wait till that is merged first Author: Imran RashidCloses #8402 from squito/test_retry_in_shared_shuffle_dep. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/33112f9c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/33112f9c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/33112f9c Branch: refs/heads/master Commit: 33112f9c48680c33d663978f76806ebf0ea39789 Parents: c0e48df Author: Imran Rashid Authored: Tue Nov 10 16:50:22 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:50:22 2015 -0800 -- .../spark/scheduler/DAGSchedulerSuite.scala | 51 +++- 1 file changed, 49 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/33112f9c/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 3816b8c..068b49b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -594,11 +594,17 @@ class DAGSchedulerSuite * @param stageId - The current stageId * @param attemptIdx - The current attempt count */ - private def completeNextResultStageWithSuccess(stageId: Int, attemptIdx: Int): Unit = { + private def completeNextResultStageWithSuccess( + stageId: Int, + attemptIdx: Int, + partitionToResult: Int => Int = _ => 42): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) assert(scheduler.stageIdToStage(stageId).isInstanceOf[ResultStage]) -complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq) +val taskResults = stageAttempt.tasks.zipWithIndex.map { case (task, idx) => + (Success, partitionToResult(idx)) +} +complete(stageAttempt, taskResults.toSeq) } /** @@ -1055,6 +1061,47 @@ class DAGSchedulerSuite } /** + * Run two jobs, with a shared dependency. We simulate a fetch failure in the second job, which + * requires regenerating some outputs of the shared dependency. One key aspect of this test is + * that the second job actually uses a different stage for the shared dependency (a "skipped" + * stage). + */ + test("shuffle fetch failure in a reused shuffle dependency") { +// Run the first job successfully, which creates one shuffle dependency + +val shuffleMapRdd = new MyRDD(sc, 2, Nil) +val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) +val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) +submit(reduceRdd, Array(0, 1)) + +completeShuffleMapStageSuccessfully(0, 0, 2) +completeNextResultStageWithSuccess(1, 0) +assert(results === Map(0 -> 42, 1 -> 42)) +assertDataStructuresEmpty() + +// submit another job w/ the shared dependency, and have a fetch failure +val reduce2 = new MyRDD(sc, 2, List(shuffleDep)) +submit(reduce2, Array(0, 1)) +// Note that the stage numbering here is only b/c the shared dependency produces a new, skipped +// stage. If instead it reused the existing stage, then this would be stage 2 +completeNextStageWithFetchFailure(3, 0, shuffleDep) +scheduler.resubmitFailedStages() + +// the scheduler now creates a new task set to regenerate the missing map output, but this time +// using a different stage, the "skipped" one + +// SPARK-9809 -- this stage is submitted without a task for each partition (because some of +// the shuffle map output is still available from stage 0); make sure we've still got internal +// accumulators setup +assert(scheduler.stageIdToStage(2).internalAccumulators.nonEmpty) +completeShuffleMapStageSuccessfully(2, 0, 2) +completeNextResultStageWithSuccess(3, 1, idx => idx + 1234) +assert(results === Map(0 -> 1234, 1 -> 1235)) + +assertDataStructuresEmpty() + } + + /** * This test runs a three stage job, with a fetch failure in stage 1. but during the retry, we * have completions from both the first & second attempt of stage 1. So all the map output is * available before we finish any task set for stage 1. We want to make sure that we don't
spark git commit: [SPARK-10192][CORE] simple test w/ failure involving a shared dependency
Repository: spark Updated Branches: refs/heads/branch-1.6 d8bfc025c -> 10272d5c9 [SPARK-10192][CORE] simple test w/ failure involving a shared dependency just trying to increase test coverage in the scheduler, this already works. It includes a regression test for SPARK-9809 copied some test utils from https://github.com/apache/spark/pull/5636, we can wait till that is merged first Author: Imran RashidCloses #8402 from squito/test_retry_in_shared_shuffle_dep. (cherry picked from commit 33112f9c48680c33d663978f76806ebf0ea39789) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/10272d5c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/10272d5c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/10272d5c Branch: refs/heads/branch-1.6 Commit: 10272d5c98694e5a0cfef2587a81be7ce609cbb7 Parents: d8bfc02 Author: Imran Rashid Authored: Tue Nov 10 16:50:22 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:50:34 2015 -0800 -- .../spark/scheduler/DAGSchedulerSuite.scala | 51 +++- 1 file changed, 49 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/10272d5c/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 3816b8c..068b49b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -594,11 +594,17 @@ class DAGSchedulerSuite * @param stageId - The current stageId * @param attemptIdx - The current attempt count */ - private def completeNextResultStageWithSuccess(stageId: Int, attemptIdx: Int): Unit = { + private def completeNextResultStageWithSuccess( + stageId: Int, + attemptIdx: Int, + partitionToResult: Int => Int = _ => 42): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) assert(scheduler.stageIdToStage(stageId).isInstanceOf[ResultStage]) -complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq) +val taskResults = stageAttempt.tasks.zipWithIndex.map { case (task, idx) => + (Success, partitionToResult(idx)) +} +complete(stageAttempt, taskResults.toSeq) } /** @@ -1055,6 +1061,47 @@ class DAGSchedulerSuite } /** + * Run two jobs, with a shared dependency. We simulate a fetch failure in the second job, which + * requires regenerating some outputs of the shared dependency. One key aspect of this test is + * that the second job actually uses a different stage for the shared dependency (a "skipped" + * stage). + */ + test("shuffle fetch failure in a reused shuffle dependency") { +// Run the first job successfully, which creates one shuffle dependency + +val shuffleMapRdd = new MyRDD(sc, 2, Nil) +val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) +val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) +submit(reduceRdd, Array(0, 1)) + +completeShuffleMapStageSuccessfully(0, 0, 2) +completeNextResultStageWithSuccess(1, 0) +assert(results === Map(0 -> 42, 1 -> 42)) +assertDataStructuresEmpty() + +// submit another job w/ the shared dependency, and have a fetch failure +val reduce2 = new MyRDD(sc, 2, List(shuffleDep)) +submit(reduce2, Array(0, 1)) +// Note that the stage numbering here is only b/c the shared dependency produces a new, skipped +// stage. If instead it reused the existing stage, then this would be stage 2 +completeNextStageWithFetchFailure(3, 0, shuffleDep) +scheduler.resubmitFailedStages() + +// the scheduler now creates a new task set to regenerate the missing map output, but this time +// using a different stage, the "skipped" one + +// SPARK-9809 -- this stage is submitted without a task for each partition (because some of +// the shuffle map output is still available from stage 0); make sure we've still got internal +// accumulators setup +assert(scheduler.stageIdToStage(2).internalAccumulators.nonEmpty) +completeShuffleMapStageSuccessfully(2, 0, 2) +completeNextResultStageWithSuccess(3, 1, idx => idx + 1234) +assert(results === Map(0 -> 1234, 1 -> 1235)) + +assertDataStructuresEmpty() + } + + /** * This test runs a three stage job, with a fetch failure in stage 1. but during the retry, we * have completions from both the first & second attempt of stage 1.
spark git commit: [SPARK-11361][STREAMING] Show scopes of RDD operations inside DStream.foreachRDD and DStream.transform in DAG viz
Repository: spark Updated Branches: refs/heads/master 900917541 -> 6600786dd [SPARK-11361][STREAMING] Show scopes of RDD operations inside DStream.foreachRDD and DStream.transform in DAG viz Currently, when a DStream sets the scope for RDD generated by it, that scope is not allowed to be overridden by the RDD operations. So in case of `DStream.foreachRDD`, all the RDDs generated inside the foreachRDD get the same scope - `foreachRDD `, as set by the `ForeachDStream`. So it is hard to debug generated RDDs in the RDD DAG viz in the Spark UI. This patch allows the RDD operations inside `DStream.transform` and `DStream.foreachRDD` to append their own scopes to the earlier DStream scope. I have also slightly tweaked how callsites are set such that the short callsite reflects the RDD operation name and line number. This tweak is necessary as callsites are not managed through scopes (which support nesting and overriding) and I didnt want to add another local property to control nesting and overriding of callsites. ## Before: ![image](https://cloud.githubusercontent.com/assets/663212/10808548/fa71c0c4-7da9-11e5-9af0-5737793a146f.png) ## After: ![image](https://cloud.githubusercontent.com/assets/663212/10808659/37bc45b6-7dab-11e5-8041-c20be6a9bc26.png) The code that was used to generate this is: ``` val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.foreachRDD { rdd => val temp = rdd.map { _ -> 1 }.reduceByKey( _ + _) val temp2 = temp.map { _ -> 1}.reduceByKey(_ + _) val count = temp2.count println(count) } ``` Note - The inner scopes of the RDD operations map/reduceByKey inside foreachRDD is visible - The short callsites of stages refers to the line number of the RDD ops rather than the same line number of foreachRDD in all three cases. Author: Tathagata DasCloses #9315 from tdas/SPARK-11361. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6600786d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6600786d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6600786d Branch: refs/heads/master Commit: 6600786dddc89cb16779ee56b9173f63a3af3f27 Parents: 9009175 Author: Tathagata Das Authored: Tue Nov 10 16:54:06 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:54:06 2015 -0800 -- .../scala/org/apache/spark/SparkContext.scala | 9 +-- .../spark/streaming/TestOutputStream.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 63 +--- .../streaming/dstream/ForEachDStream.scala | 14 +++- .../streaming/dstream/TransformedDStream.scala | 13 .../spark/streaming/DStreamScopeSuite.scala | 75 .../apache/spark/streaming/TestSuiteBase.scala | 4 +- 7 files changed, 147 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6600786d/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7421821..67270c3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1787,10 +1787,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * has overridden the call site using `setCallSite()`, this will return the user's version. */ private[spark] def getCallSite(): CallSite = { -Option(getLocalProperty(CallSite.SHORT_FORM)).map { case shortCallSite => - val longCallSite = Option(getLocalProperty(CallSite.LONG_FORM)).getOrElse("") - CallSite(shortCallSite, longCallSite) -}.getOrElse(Utils.getCallSite()) +val callSite = Utils.getCallSite() +CallSite( + Option(getLocalProperty(CallSite.SHORT_FORM)).getOrElse(callSite.shortForm), + Option(getLocalProperty(CallSite.LONG_FORM)).getOrElse(callSite.longForm) +) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/6600786d/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala -- diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala index 1a9..79077e4 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala +++
spark git commit: [SPARK-11361][STREAMING] Show scopes of RDD operations inside DStream.foreachRDD and DStream.transform in DAG viz
Repository: spark Updated Branches: refs/heads/branch-1.6 fafeca36e -> b34819c7b [SPARK-11361][STREAMING] Show scopes of RDD operations inside DStream.foreachRDD and DStream.transform in DAG viz Currently, when a DStream sets the scope for RDD generated by it, that scope is not allowed to be overridden by the RDD operations. So in case of `DStream.foreachRDD`, all the RDDs generated inside the foreachRDD get the same scope - `foreachRDD `, as set by the `ForeachDStream`. So it is hard to debug generated RDDs in the RDD DAG viz in the Spark UI. This patch allows the RDD operations inside `DStream.transform` and `DStream.foreachRDD` to append their own scopes to the earlier DStream scope. I have also slightly tweaked how callsites are set such that the short callsite reflects the RDD operation name and line number. This tweak is necessary as callsites are not managed through scopes (which support nesting and overriding) and I didnt want to add another local property to control nesting and overriding of callsites. ## Before: ![image](https://cloud.githubusercontent.com/assets/663212/10808548/fa71c0c4-7da9-11e5-9af0-5737793a146f.png) ## After: ![image](https://cloud.githubusercontent.com/assets/663212/10808659/37bc45b6-7dab-11e5-8041-c20be6a9bc26.png) The code that was used to generate this is: ``` val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.foreachRDD { rdd => val temp = rdd.map { _ -> 1 }.reduceByKey( _ + _) val temp2 = temp.map { _ -> 1}.reduceByKey(_ + _) val count = temp2.count println(count) } ``` Note - The inner scopes of the RDD operations map/reduceByKey inside foreachRDD is visible - The short callsites of stages refers to the line number of the RDD ops rather than the same line number of foreachRDD in all three cases. Author: Tathagata DasCloses #9315 from tdas/SPARK-11361. (cherry picked from commit 6600786dddc89cb16779ee56b9173f63a3af3f27) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b34819c7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b34819c7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b34819c7 Branch: refs/heads/branch-1.6 Commit: b34819c7b720d6407d428deca4b00a02ab7c3503 Parents: fafeca3 Author: Tathagata Das Authored: Tue Nov 10 16:54:06 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:54:23 2015 -0800 -- .../scala/org/apache/spark/SparkContext.scala | 9 +-- .../spark/streaming/TestOutputStream.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 63 +--- .../streaming/dstream/ForEachDStream.scala | 14 +++- .../streaming/dstream/TransformedDStream.scala | 13 .../spark/streaming/DStreamScopeSuite.scala | 75 .../apache/spark/streaming/TestSuiteBase.scala | 4 +- 7 files changed, 147 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b34819c7/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7421821..67270c3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1787,10 +1787,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * has overridden the call site using `setCallSite()`, this will return the user's version. */ private[spark] def getCallSite(): CallSite = { -Option(getLocalProperty(CallSite.SHORT_FORM)).map { case shortCallSite => - val longCallSite = Option(getLocalProperty(CallSite.LONG_FORM)).getOrElse("") - CallSite(shortCallSite, longCallSite) -}.getOrElse(Utils.getCallSite()) +val callSite = Utils.getCallSite() +CallSite( + Option(getLocalProperty(CallSite.SHORT_FORM)).getOrElse(callSite.shortForm), + Option(getLocalProperty(CallSite.LONG_FORM)).getOrElse(callSite.longForm) +) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/b34819c7/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala -- diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala index 1a9..79077e4 100644
spark git commit: [SPARK-11615] Drop @VisibleForTesting annotation
Repository: spark Updated Branches: refs/heads/branch-1.6 93ac30741 -> fafeca36e [SPARK-11615] Drop @VisibleForTesting annotation See http://search-hadoop.com/m/q3RTtjpe8r1iRbTj2 for discussion. Summary: addition of VisibleForTesting annotation resulted in spark-shell malfunctioning. Author: tedyuCloses #9585 from tedyu/master. (cherry picked from commit 900917541651abe7125f0d205085d2ab6a00d92c) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fafeca36 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fafeca36 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fafeca36 Branch: refs/heads/branch-1.6 Commit: fafeca36eaa04e6e54bcab863959cce710698e30 Parents: 93ac307 Author: tedyu Authored: Tue Nov 10 16:52:26 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:53:09 2015 -0800 -- core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala | 8 .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 2 -- .../org/apache/spark/util/AsynchronousListenerBus.scala | 5 ++--- .../org/apache/spark/util/collection/ExternalSorter.scala| 3 +-- scalastyle-config.xml| 7 +++ .../org/apache/spark/sql/execution/QueryExecution.scala | 3 --- .../apache/spark/network/shuffle/ShuffleTestAccessor.scala | 1 - 7 files changed, 14 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fafeca36/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala -- diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index c72b588..464027f 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -21,8 +21,6 @@ import javax.annotation.concurrent.GuardedBy import scala.util.control.NonFatal -import com.google.common.annotations.VisibleForTesting - import org.apache.spark.{Logging, SparkException} import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, ThreadSafeRpcEndpoint} @@ -193,8 +191,10 @@ private[netty] class Inbox( def isEmpty: Boolean = inbox.synchronized { messages.isEmpty } - /** Called when we are dropping a message. Test cases override this to test message dropping. */ - @VisibleForTesting + /** + * Called when we are dropping a message. Test cases override this to test message dropping. + * Exposed for testing. + */ protected def onDrop(message: InboxMessage): Unit = { logWarning(s"Drop $message because $endpointRef is stopped") } http://git-wip-us.apache.org/repos/asf/spark/blob/fafeca36/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 77d034f..ca37829 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -21,8 +21,6 @@ import java.util.concurrent.TimeoutException import scala.collection.mutable.{HashMap, HashSet, ListBuffer} -import com.google.common.annotations.VisibleForTesting - import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics http://git-wip-us.apache.org/repos/asf/spark/blob/fafeca36/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala index b8481ea..b3b54af 100644 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala @@ -20,7 +20,6 @@ package org.apache.spark.util import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean -import com.google.common.annotations.VisibleForTesting import org.apache.spark.SparkContext /** @@ -119,8 +118,8 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri * For testing only. Wait until there are no more events in the queue, or until the specified * time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue * emptied. + * Exposed for testing. */ - @VisibleForTesting
spark git commit: [MINOR] Fix typo in AggregationQuerySuite.scala
Repository: spark Updated Branches: refs/heads/master 6600786dd -> 12c7635dc [MINOR] Fix typo in AggregationQuerySuite.scala Author: Forest FangCloses #9357 from saurfang/patch-1. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/12c7635d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/12c7635d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/12c7635d Branch: refs/heads/master Commit: 12c7635dc025239d3b69b9adef2f4eebb28edf48 Parents: 6600786 Author: Forest Fang Authored: Tue Nov 10 16:55:58 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:56:06 2015 -0800 -- .../apache/spark/sql/hive/execution/AggregationQuerySuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/12c7635d/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 8253921..22d2aef 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -795,14 +795,14 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te val df = sqlContext.createDataFrame(rdd, schema) val allColumns = df.schema.fields.map(f => col(f.name)) - val expectedAnaswer = + val expectedAnswer = data .find(r => r.getInt(0) == 50) .getOrElse(fail("A row with id 50 should be the expected answer.")) checkAnswer( df.groupBy().agg(udaf(allColumns: _*)), // udaf returns a Row as the output value. -Row(expectedAnaswer) +Row(expectedAnswer) ) } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11572] Exit AsynchronousListenerBus thread when stop() is called
Repository: spark Updated Branches: refs/heads/master 33112f9c4 -> 3e0a6cf1e [SPARK-11572] Exit AsynchronousListenerBus thread when stop() is called As vonnagy reported in the following thread: http://search-hadoop.com/m/q3RTtk982kvIow22 Attempts to join the thread in AsynchronousListenerBus resulted in lock up because AsynchronousListenerBus thread was still getting messages `SparkListenerExecutorMetricsUpdate` from the DAGScheduler Author: tedyuCloses #9546 from ted-yu/master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3e0a6cf1 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3e0a6cf1 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3e0a6cf1 Branch: refs/heads/master Commit: 3e0a6cf1e02a19b37c68d3026415d53bb57a576b Parents: 33112f9 Author: tedyu Authored: Tue Nov 10 16:51:25 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:51:25 2015 -0800 -- .../org/apache/spark/util/AsynchronousListenerBus.scala | 9 +++-- 1 file changed, 3 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3e0a6cf1/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala index 61b5a4c..b8481ea 100644 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala @@ -67,15 +67,12 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri processingEvent = true } try { - val event = eventQueue.poll - if (event == null) { + if (stopped.get()) { // Get out of the while loop and shutdown the daemon thread -if (!stopped.get) { - throw new IllegalStateException("Polling `null` from eventQueue means" + -" the listener bus has been stopped. So `stopped` must be true") -} return } + val event = eventQueue.poll + assert(event != null, "event queue was empty but the listener bus was not stopped") postToAll(event) } finally { self.synchronized { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11572] Exit AsynchronousListenerBus thread when stop() is called
Repository: spark Updated Branches: refs/heads/branch-1.6 10272d5c9 -> 93ac30741 [SPARK-11572] Exit AsynchronousListenerBus thread when stop() is called As vonnagy reported in the following thread: http://search-hadoop.com/m/q3RTtk982kvIow22 Attempts to join the thread in AsynchronousListenerBus resulted in lock up because AsynchronousListenerBus thread was still getting messages `SparkListenerExecutorMetricsUpdate` from the DAGScheduler Author: tedyuCloses #9546 from ted-yu/master. (cherry picked from commit 3e0a6cf1e02a19b37c68d3026415d53bb57a576b) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/93ac3074 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/93ac3074 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/93ac3074 Branch: refs/heads/branch-1.6 Commit: 93ac30741cd0ed99512f01525fbda8a08c87967a Parents: 10272d5 Author: tedyu Authored: Tue Nov 10 16:51:25 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:51:33 2015 -0800 -- .../org/apache/spark/util/AsynchronousListenerBus.scala | 9 +++-- 1 file changed, 3 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/93ac3074/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala index 61b5a4c..b8481ea 100644 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala @@ -67,15 +67,12 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri processingEvent = true } try { - val event = eventQueue.poll - if (event == null) { + if (stopped.get()) { // Get out of the while loop and shutdown the daemon thread -if (!stopped.get) { - throw new IllegalStateException("Polling `null` from eventQueue means" + -" the listener bus has been stopped. So `stopped` must be true") -} return } + val event = eventQueue.poll + assert(event != null, "event queue was empty but the listener bus was not stopped") postToAll(event) } finally { self.synchronized { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11615] Drop @VisibleForTesting annotation
Repository: spark Updated Branches: refs/heads/master 3e0a6cf1e -> 900917541 [SPARK-11615] Drop @VisibleForTesting annotation See http://search-hadoop.com/m/q3RTtjpe8r1iRbTj2 for discussion. Summary: addition of VisibleForTesting annotation resulted in spark-shell malfunctioning. Author: tedyuCloses #9585 from tedyu/master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/90091754 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/90091754 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/90091754 Branch: refs/heads/master Commit: 900917541651abe7125f0d205085d2ab6a00d92c Parents: 3e0a6cf Author: tedyu Authored: Tue Nov 10 16:52:26 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:52:59 2015 -0800 -- core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala | 8 .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 2 -- .../org/apache/spark/util/AsynchronousListenerBus.scala | 5 ++--- .../org/apache/spark/util/collection/ExternalSorter.scala| 3 +-- scalastyle-config.xml| 7 +++ .../org/apache/spark/sql/execution/QueryExecution.scala | 3 --- .../apache/spark/network/shuffle/ShuffleTestAccessor.scala | 1 - 7 files changed, 14 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/90091754/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala -- diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index c72b588..464027f 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -21,8 +21,6 @@ import javax.annotation.concurrent.GuardedBy import scala.util.control.NonFatal -import com.google.common.annotations.VisibleForTesting - import org.apache.spark.{Logging, SparkException} import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, ThreadSafeRpcEndpoint} @@ -193,8 +191,10 @@ private[netty] class Inbox( def isEmpty: Boolean = inbox.synchronized { messages.isEmpty } - /** Called when we are dropping a message. Test cases override this to test message dropping. */ - @VisibleForTesting + /** + * Called when we are dropping a message. Test cases override this to test message dropping. + * Exposed for testing. + */ protected def onDrop(message: InboxMessage): Unit = { logWarning(s"Drop $message because $endpointRef is stopped") } http://git-wip-us.apache.org/repos/asf/spark/blob/90091754/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 77d034f..ca37829 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -21,8 +21,6 @@ import java.util.concurrent.TimeoutException import scala.collection.mutable.{HashMap, HashSet, ListBuffer} -import com.google.common.annotations.VisibleForTesting - import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics http://git-wip-us.apache.org/repos/asf/spark/blob/90091754/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala index b8481ea..b3b54af 100644 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala @@ -20,7 +20,6 @@ package org.apache.spark.util import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean -import com.google.common.annotations.VisibleForTesting import org.apache.spark.SparkContext /** @@ -119,8 +118,8 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri * For testing only. Wait until there are no more events in the queue, or until the specified * time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue * emptied. + * Exposed for testing. */ - @VisibleForTesting @throws(classOf[TimeoutException]) def waitUntilEmpty(timeoutMillis: Long): Unit = { val finishTime = System.currentTimeMillis +
spark git commit: [MINOR] License header formatting fix
Repository: spark Updated Branches: refs/heads/master 12c7635dc -> 745e45d5f [MINOR] License header formatting fix The header wasn't indented properly. Author: Marc Prud'hommeauxCloses #9312 from mprudhom/patch-1. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/745e45d5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/745e45d5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/745e45d5 Branch: refs/heads/master Commit: 745e45d5ff7fe251c0d5197b7e08b1f80807b005 Parents: 12c7635 Author: Marc Prud'hommeaux Authored: Tue Nov 10 16:57:08 2015 -0800 Committer: Andrew Or Committed: Tue Nov 10 16:57:12 2015 -0800 -- .../datasources/jdbc/DefaultSource.scala| 30 ++-- 1 file changed, 15 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/745e45d5/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala index 6773afc..f522303 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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.sql.execution.datasources.jdbc - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10471][CORE][MESOS] prevent getting offers for unmet constraints
Repository: spark Updated Branches: refs/heads/master 88a3fdcc7 -> 5039a49b6 [SPARK-10471][CORE][MESOS] prevent getting offers for unmet constraints this change rejects offers for slaves with unmet constraints for 120s to mitigate offer starvation. this prevents mesos to send us these offers again and again. in return, we get more offers for slaves which might meet our constraints. and it enables mesos to send the rejected offers to other frameworks. Author: Felix BechsteinCloses #8639 from felixb/decline_offers_constraint_mismatch. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5039a49b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5039a49b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5039a49b Branch: refs/heads/master Commit: 5039a49b636325f321daa089971107003fae9d4b Parents: 88a3fdc Author: Felix Bechstein Authored: Mon Nov 9 13:36:14 2015 -0800 Committer: Andrew Or Committed: Mon Nov 9 13:36:14 2015 -0800 -- .../mesos/CoarseMesosSchedulerBackend.scala | 92 +++- .../cluster/mesos/MesosSchedulerBackend.scala | 48 +++--- .../cluster/mesos/MesosSchedulerUtils.scala | 4 + 3 files changed, 91 insertions(+), 53 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5039a49b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d10a77f..2de9b6a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -101,6 +101,10 @@ private[spark] class CoarseMesosSchedulerBackend( private val slaveOfferConstraints = parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + // reject offers with mismatched constraints in seconds + private val rejectOfferDurationForUnmetConstraints = +getRejectOfferDurationForUnmetConstraints(sc) + // A client for talking to the external shuffle service, if it is a private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { if (shuffleServiceEnabled) { @@ -249,48 +253,56 @@ private[spark] class CoarseMesosSchedulerBackend( val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt val id = offer.getId.getValue -if (taskIdToSlaveId.size < executorLimit && -totalCoresAcquired < maxCores && -meetsConstraints && -mem >= calculateTotalMemory(sc) && -cpus >= 1 && -failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && -!slaveIdsWithExecutors.contains(slaveId)) { - // Launch an executor on the slave - val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired) - totalCoresAcquired += cpusToUse - val taskId = newMesosTaskId() - taskIdToSlaveId.put(taskId, slaveId) - slaveIdsWithExecutors += slaveId - coresByTaskId(taskId) = cpusToUse - // Gather cpu resources from the available resources and use them in the task. - val (remainingResources, cpuResourcesToUse) = -partitionResources(offer.getResourcesList, "cpus", cpusToUse) - val (_, memResourcesToUse) = -partitionResources(remainingResources.asJava, "mem", calculateTotalMemory(sc)) - val taskBuilder = MesosTaskInfo.newBuilder() -.setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) -.setSlaveId(offer.getSlaveId) -.setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave, taskId)) -.setName("Task " + taskId) -.addAllResources(cpuResourcesToUse.asJava) -.addAllResources(memResourcesToUse.asJava) - - sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => -MesosSchedulerBackendUtil - .setupContainerBuilderDockerInfo(image, sc.conf, taskBuilder.getContainerBuilder()) +if (meetsConstraints) { + if (taskIdToSlaveId.size < executorLimit && + totalCoresAcquired < maxCores && + mem >= calculateTotalMemory(sc) && + cpus >= 1 && + failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && + !slaveIdsWithExecutors.contains(slaveId)) { +//
spark git commit: [SPARK-10471][CORE][MESOS] prevent getting offers for unmet constraints
Repository: spark Updated Branches: refs/heads/branch-1.6 2459b3432 -> 74f50275e [SPARK-10471][CORE][MESOS] prevent getting offers for unmet constraints this change rejects offers for slaves with unmet constraints for 120s to mitigate offer starvation. this prevents mesos to send us these offers again and again. in return, we get more offers for slaves which might meet our constraints. and it enables mesos to send the rejected offers to other frameworks. Author: Felix BechsteinCloses #8639 from felixb/decline_offers_constraint_mismatch. (cherry picked from commit 5039a49b636325f321daa089971107003fae9d4b) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/74f50275 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/74f50275 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/74f50275 Branch: refs/heads/branch-1.6 Commit: 74f50275e429e649212928a9f36552941b862edc Parents: 2459b34 Author: Felix Bechstein Authored: Mon Nov 9 13:36:14 2015 -0800 Committer: Andrew Or Committed: Mon Nov 9 13:36:28 2015 -0800 -- .../mesos/CoarseMesosSchedulerBackend.scala | 92 +++- .../cluster/mesos/MesosSchedulerBackend.scala | 48 +++--- .../cluster/mesos/MesosSchedulerUtils.scala | 4 + 3 files changed, 91 insertions(+), 53 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/74f50275/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d10a77f..2de9b6a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -101,6 +101,10 @@ private[spark] class CoarseMesosSchedulerBackend( private val slaveOfferConstraints = parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + // reject offers with mismatched constraints in seconds + private val rejectOfferDurationForUnmetConstraints = +getRejectOfferDurationForUnmetConstraints(sc) + // A client for talking to the external shuffle service, if it is a private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { if (shuffleServiceEnabled) { @@ -249,48 +253,56 @@ private[spark] class CoarseMesosSchedulerBackend( val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt val id = offer.getId.getValue -if (taskIdToSlaveId.size < executorLimit && -totalCoresAcquired < maxCores && -meetsConstraints && -mem >= calculateTotalMemory(sc) && -cpus >= 1 && -failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && -!slaveIdsWithExecutors.contains(slaveId)) { - // Launch an executor on the slave - val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired) - totalCoresAcquired += cpusToUse - val taskId = newMesosTaskId() - taskIdToSlaveId.put(taskId, slaveId) - slaveIdsWithExecutors += slaveId - coresByTaskId(taskId) = cpusToUse - // Gather cpu resources from the available resources and use them in the task. - val (remainingResources, cpuResourcesToUse) = -partitionResources(offer.getResourcesList, "cpus", cpusToUse) - val (_, memResourcesToUse) = -partitionResources(remainingResources.asJava, "mem", calculateTotalMemory(sc)) - val taskBuilder = MesosTaskInfo.newBuilder() -.setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) -.setSlaveId(offer.getSlaveId) -.setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave, taskId)) -.setName("Task " + taskId) -.addAllResources(cpuResourcesToUse.asJava) -.addAllResources(memResourcesToUse.asJava) - - sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => -MesosSchedulerBackendUtil - .setupContainerBuilderDockerInfo(image, sc.conf, taskBuilder.getContainerBuilder()) +if (meetsConstraints) { + if (taskIdToSlaveId.size < executorLimit && + totalCoresAcquired < maxCores && + mem >= calculateTotalMemory(sc) && + cpus >= 1 && +
spark git commit: [SPARK-11112] Fix Scala 2.11 compilation error in RDDInfo.scala
Repository: spark Updated Branches: refs/heads/branch-1.6 2946c85f5 -> fb469e76a [SPARK-2] Fix Scala 2.11 compilation error in RDDInfo.scala As shown in https://amplab.cs.berkeley.edu/jenkins/view/Spark-QA-Compile/job/Spark-Master-Scala211-Compile/1946/console , compilation fails with: ``` [error] /home/jenkins/workspace/Spark-Master-Scala211-Compile/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala:25: in class RDDInfo, multiple overloaded alternatives of constructor RDDInfo define default arguments. [error] class RDDInfo( [error] ``` This PR tries to fix the compilation error Author: tedyuCloses #9538 from tedyu/master. (cherry picked from commit 404a28f4edd09cf17361dcbd770e4cafde51bf6d) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fb469e76 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fb469e76 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fb469e76 Branch: refs/heads/branch-1.6 Commit: fb469e76ae28d50f77ebf38cfc2d96237a07c79e Parents: 2946c85 Author: tedyu Authored: Mon Nov 9 10:07:58 2015 -0800 Committer: Andrew Or Committed: Mon Nov 9 10:08:04 2015 -0800 -- .../main/scala/org/apache/spark/storage/RDDInfo.scala | 12 +--- 1 file changed, 1 insertion(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fb469e76/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 3fa209b..87c1b98 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -28,20 +28,10 @@ class RDDInfo( val numPartitions: Int, var storageLevel: StorageLevel, val parentIds: Seq[Int], -val callSite: CallSite, +val callSite: CallSite = CallSite.empty, val scope: Option[RDDOperationScope] = None) extends Ordered[RDDInfo] { - def this( - id: Int, - name: String, - numPartitions: Int, - storageLevel: StorageLevel, - parentIds: Seq[Int], - scope: Option[RDDOperationScope] = None) { -this(id, name, numPartitions, storageLevel, parentIds, CallSite.empty, scope) - } - var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11112] Fix Scala 2.11 compilation error in RDDInfo.scala
Repository: spark Updated Branches: refs/heads/master 08a7a836c -> 404a28f4e [SPARK-2] Fix Scala 2.11 compilation error in RDDInfo.scala As shown in https://amplab.cs.berkeley.edu/jenkins/view/Spark-QA-Compile/job/Spark-Master-Scala211-Compile/1946/console , compilation fails with: ``` [error] /home/jenkins/workspace/Spark-Master-Scala211-Compile/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala:25: in class RDDInfo, multiple overloaded alternatives of constructor RDDInfo define default arguments. [error] class RDDInfo( [error] ``` This PR tries to fix the compilation error Author: tedyuCloses #9538 from tedyu/master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/404a28f4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/404a28f4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/404a28f4 Branch: refs/heads/master Commit: 404a28f4edd09cf17361dcbd770e4cafde51bf6d Parents: 08a7a83 Author: tedyu Authored: Mon Nov 9 10:07:58 2015 -0800 Committer: Andrew Or Committed: Mon Nov 9 10:07:58 2015 -0800 -- .../main/scala/org/apache/spark/storage/RDDInfo.scala | 12 +--- 1 file changed, 1 insertion(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/404a28f4/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 3fa209b..87c1b98 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -28,20 +28,10 @@ class RDDInfo( val numPartitions: Int, var storageLevel: StorageLevel, val parentIds: Seq[Int], -val callSite: CallSite, +val callSite: CallSite = CallSite.empty, val scope: Option[RDDOperationScope] = None) extends Ordered[RDDInfo] { - def this( - id: Int, - name: String, - numPartitions: Int, - storageLevel: StorageLevel, - parentIds: Seq[Int], - scope: Option[RDDOperationScope] = None) { -this(id, name, numPartitions, storageLevel, parentIds, CallSite.empty, scope) - } - var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11362] [SQL] Use Spark BitSet in BroadcastNestedLoopJoin
Repository: spark Updated Branches: refs/heads/branch-1.6 fc2942d12 -> 2946c85f5 [SPARK-11362] [SQL] Use Spark BitSet in BroadcastNestedLoopJoin JIRA: https://issues.apache.org/jira/browse/SPARK-11362 We use scala.collection.mutable.BitSet in BroadcastNestedLoopJoin now. We should use Spark's BitSet. Author: Liang-Chi HsiehCloses #9316 from viirya/use-spark-bitset. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2946c85f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2946c85f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2946c85f Branch: refs/heads/branch-1.6 Commit: 2946c85f5f48516637a6ce52ba9e31caf3c8ee3a Parents: fc2942d Author: Liang-Chi Hsieh Authored: Sat Nov 7 19:44:45 2015 -0800 Committer: Andrew Or Committed: Mon Nov 9 10:02:46 2015 -0800 -- .../execution/joins/BroadcastNestedLoopJoin.scala | 18 -- 1 file changed, 8 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2946c85f/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index 05d20f5..aab177b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.util.collection.CompactBuffer +import org.apache.spark.util.collection.{BitSet, CompactBuffer} case class BroadcastNestedLoopJoin( @@ -95,9 +95,7 @@ case class BroadcastNestedLoopJoin( /** All rows that either match both-way, or rows from streamed joined with nulls. */ val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => val matchedRows = new CompactBuffer[InternalRow] - // TODO: Use Spark's BitSet. - val includedBroadcastTuples = -new scala.collection.mutable.BitSet(broadcastedRelation.value.size) + val includedBroadcastTuples = new BitSet(broadcastedRelation.value.size) val joinedRow = new JoinedRow val leftNulls = new GenericMutableRow(left.output.size) @@ -115,11 +113,11 @@ case class BroadcastNestedLoopJoin( case BuildRight if boundCondition(joinedRow(streamedRow, broadcastedRow)) => matchedRows += resultProj(joinedRow(streamedRow, broadcastedRow)).copy() streamRowMatched = true - includedBroadcastTuples += i + includedBroadcastTuples.set(i) case BuildLeft if boundCondition(joinedRow(broadcastedRow, streamedRow)) => matchedRows += resultProj(joinedRow(broadcastedRow, streamedRow)).copy() streamRowMatched = true - includedBroadcastTuples += i + includedBroadcastTuples.set(i) case _ => } i += 1 @@ -138,8 +136,8 @@ case class BroadcastNestedLoopJoin( val includedBroadcastTuples = matchesOrStreamedRowsWithNulls.map(_._2) val allIncludedBroadcastTuples = includedBroadcastTuples.fold( - new scala.collection.mutable.BitSet(broadcastedRelation.value.size) -)(_ ++ _) + new BitSet(broadcastedRelation.value.size) +)(_ | _) val leftNulls = new GenericMutableRow(left.output.size) val rightNulls = new GenericMutableRow(right.output.size) @@ -155,7 +153,7 @@ case class BroadcastNestedLoopJoin( val joinedRow = new JoinedRow joinedRow.withLeft(leftNulls) while (i < rel.length) { -if (!allIncludedBroadcastTuples.contains(i)) { +if (!allIncludedBroadcastTuples.get(i)) { buf += resultProj(joinedRow.withRight(rel(i))).copy() } i += 1 @@ -164,7 +162,7 @@ case class BroadcastNestedLoopJoin( val joinedRow = new JoinedRow joinedRow.withRight(rightNulls) while (i < rel.length) { -if (!allIncludedBroadcastTuples.contains(i)) { +if (!allIncludedBroadcastTuples.get(i)) { buf += resultProj(joinedRow.withLeft(rel(i))).copy() } i += 1 - To unsubscribe, e-mail:
spark git commit: [SPARK-11112] DAG visualization: display RDD callsite
Repository: spark Updated Branches: refs/heads/master 30b706b7b -> 7f741905b [SPARK-2] DAG visualization: display RDD callsite https://cloud.githubusercontent.com/assets/2133137/10870343/2a8cd070-807d-11e5-857a-4ebcace77b5b.png;> mateiz sarutak Author: Andrew Or <and...@databricks.com> Closes #9398 from andrewor14/rdd-callsite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7f741905 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7f741905 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7f741905 Branch: refs/heads/master Commit: 7f741905b06ed6d3dfbff6db41a3355dab71aa3c Parents: 30b706b Author: Andrew Or <and...@databricks.com> Authored: Sat Nov 7 05:35:53 2015 +0100 Committer: Andrew Or <and...@databricks.com> Committed: Sat Nov 7 05:35:53 2015 +0100 -- .../apache/spark/ui/static/spark-dag-viz.css| 4 +++ .../org/apache/spark/storage/RDDInfo.scala | 16 +++-- .../spark/ui/scope/RDDOperationGraph.scala | 10 +++--- .../org/apache/spark/util/JsonProtocol.scala| 17 - .../scala/org/apache/spark/util/Utils.scala | 1 + .../org/apache/spark/ui/UISeleniumSuite.scala | 14 .../apache/spark/util/JsonProtocolSuite.scala | 37 7 files changed, 79 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7f741905/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css -- diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css index 3b4ae2e..9cc5c79 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.css @@ -122,3 +122,7 @@ stroke: #52C366; stroke-width: 2px; } + +.tooltip-inner { + white-space: pre-wrap; +} http://git-wip-us.apache.org/repos/asf/spark/blob/7f741905/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 9606262..3fa209b 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.{RDDOperationScope, RDD} -import org.apache.spark.util.Utils +import org.apache.spark.util.{CallSite, Utils} @DeveloperApi class RDDInfo( @@ -28,9 +28,20 @@ class RDDInfo( val numPartitions: Int, var storageLevel: StorageLevel, val parentIds: Seq[Int], +val callSite: CallSite, val scope: Option[RDDOperationScope] = None) extends Ordered[RDDInfo] { + def this( + id: Int, + name: String, + numPartitions: Int, + storageLevel: StorageLevel, + parentIds: Seq[Int], + scope: Option[RDDOperationScope] = None) { +this(id, name, numPartitions, storageLevel, parentIds, CallSite.empty, scope) + } + var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L @@ -56,6 +67,7 @@ private[spark] object RDDInfo { def fromRdd(rdd: RDD[_]): RDDInfo = { val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd)) val parentIds = rdd.dependencies.map(_.rdd.id) -new RDDInfo(rdd.id, rddName, rdd.partitions.length, rdd.getStorageLevel, parentIds, rdd.scope) +new RDDInfo(rdd.id, rddName, rdd.partitions.length, + rdd.getStorageLevel, parentIds, rdd.creationSite, rdd.scope) } } http://git-wip-us.apache.org/repos/asf/spark/blob/7f741905/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index 81f168a..2427456 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{StringBuilder, ListBuffer} import org.apache.spark.Logging import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.CallSite /** * A representation of a generic cluster graph used for storing information on RDD operations. @@ -38,7 +39,7 @@ private[ui] case class RDDOperationGraph( rootCluster: RDDOperationCluster) /** A node in an RDDOperati
spark git commit: [SPARK-11114][PYSPARK] add getOrCreate for SparkContext/SQLContext in Python
Repository: spark Updated Branches: refs/heads/master a1413b366 -> 232d7f8d4 [SPARK-4][PYSPARK] add getOrCreate for SparkContext/SQLContext in Python Also added SQLContext.newSession() Author: Davies LiuCloses #9122 from davies/py_create. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/232d7f8d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/232d7f8d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/232d7f8d Branch: refs/heads/master Commit: 232d7f8d42950431f1d9be2a6bb3591fb6ea20d6 Parents: a1413b3 Author: Davies Liu Authored: Mon Oct 19 16:18:20 2015 -0700 Committer: Andrew Or Committed: Mon Oct 19 16:18:20 2015 -0700 -- python/pyspark/context.py | 16 ++-- python/pyspark/sql/context.py | 27 +++ python/pyspark/sql/tests.py | 14 ++ python/pyspark/tests.py | 4 4 files changed, 59 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/232d7f8d/python/pyspark/context.py -- diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 4969d85..afd74d9 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -21,7 +21,7 @@ import os import shutil import signal import sys -from threading import Lock +from threading import RLock from tempfile import NamedTemporaryFile from pyspark import accumulators @@ -65,7 +65,7 @@ class SparkContext(object): _jvm = None _next_accum_id = 0 _active_spark_context = None -_lock = Lock() +_lock = RLock() _python_includes = None # zip and egg files that need to be added to PYTHONPATH PACKAGE_EXTENSIONS = ('.zip', '.egg', '.jar') @@ -280,6 +280,18 @@ class SparkContext(object): """ self.stop() +@classmethod +def getOrCreate(cls, conf=None): +""" +Get or instantiate a SparkContext and register it as a singleton object. + +:param conf: SparkConf (optional) +""" +with SparkContext._lock: +if SparkContext._active_spark_context is None: +SparkContext(conf=conf or SparkConf()) +return SparkContext._active_spark_context + def setLogLevel(self, logLevel): """ Control our logLevel. This overrides any user-defined log settings. http://git-wip-us.apache.org/repos/asf/spark/blob/232d7f8d/python/pyspark/sql/context.py -- diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 89c8c6e..7945365 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -75,6 +75,8 @@ class SQLContext(object): SQLContext in the JVM, instead we make all calls to this object. """ +_instantiatedContext = None + @ignore_unicode_prefix def __init__(self, sparkContext, sqlContext=None): """Creates a new SQLContext. @@ -99,6 +101,8 @@ class SQLContext(object): self._scala_SQLContext = sqlContext _monkey_patch_RDD(self) install_exception_handler() +if SQLContext._instantiatedContext is None: +SQLContext._instantiatedContext = self @property def _ssql_ctx(self): @@ -111,6 +115,29 @@ class SQLContext(object): self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext +@classmethod +@since(1.6) +def getOrCreate(cls, sc): +""" +Get the existing SQLContext or create a new one with given SparkContext. + +:param sc: SparkContext +""" +if cls._instantiatedContext is None: +jsqlContext = sc._jvm.SQLContext.getOrCreate(sc._jsc.sc()) +cls(sc, jsqlContext) +return cls._instantiatedContext + +@since(1.6) +def newSession(self): +""" +Returns a new SQLContext as new session, that has separate SQLConf, +registered temporary tables and UDFs, but shared SparkContext and +table cache. +""" +jsqlContext = self._ssql_ctx.newSession() +return self.__class__(self._sc, jsqlContext) + @since(1.3) def setConf(self, key, value): """Sets the given Spark SQL configuration property. http://git-wip-us.apache.org/repos/asf/spark/blob/232d7f8d/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 645133b..f465e1f 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -174,6 +174,20 @@ class
spark git commit: [SPARK-9708][MESOS] Spark should create local temporary directories in Mesos sandbox when launched with Mesos
Repository: spark Updated Branches: refs/heads/master 232d7f8d4 -> fc26f32cf [SPARK-9708][MESOS] Spark should create local temporary directories in Mesos sandbox when launched with Mesos This is my own original work and I license this to the project under the project's open source license Author: Chris BannisterAuthor: Chris Bannister Closes #8358 from Zariel/mesos-local-dir. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fc26f32c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fc26f32c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fc26f32c Branch: refs/heads/master Commit: fc26f32cf1bede8b9a1343dca0c0182107c9985e Parents: 232d7f8 Author: Chris Bannister Authored: Mon Oct 19 16:24:40 2015 -0700 Committer: Andrew Or Committed: Mon Oct 19 16:24:40 2015 -0700 -- .../main/scala/org/apache/spark/util/Utils.scala | 17 ++--- 1 file changed, 14 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fc26f32c/core/src/main/scala/org/apache/spark/util/Utils.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 22c05a2..5595040 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -649,6 +649,7 @@ private[spark] object Utils extends Logging { * logic of locating the local directories according to deployment mode. */ def getConfiguredLocalDirs(conf: SparkConf): Array[String] = { +val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) if (isRunningInYarnContainer(conf)) { // If we are in yarn mode, systems can have different disk layouts so we must set it // to what Yarn on this system said was available. Note this assumes that Yarn has @@ -657,13 +658,23 @@ private[spark] object Utils extends Logging { getYarnLocalDirs(conf).split(",") } else if (conf.getenv("SPARK_EXECUTOR_DIRS") != null) { conf.getenv("SPARK_EXECUTOR_DIRS").split(File.pathSeparator) +} else if (conf.getenv("SPARK_LOCAL_DIRS") != null) { + conf.getenv("SPARK_LOCAL_DIRS").split(",") +} else if (conf.getenv("MESOS_DIRECTORY") != null && !shuffleServiceEnabled) { + // Mesos already creates a directory per Mesos task. Spark should use that directory + // instead so all temporary files are automatically cleaned up when the Mesos task ends. + // Note that we don't want this if the shuffle service is enabled because we want to + // continue to serve shuffle files after the executors that wrote them have already exited. + Array(conf.getenv("MESOS_DIRECTORY")) } else { + if (conf.getenv("MESOS_DIRECTORY") != null && shuffleServiceEnabled) { +logInfo("MESOS_DIRECTORY available but not using provided Mesos sandbox because " + + "spark.shuffle.service.enabled is enabled.") + } // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user // configuration to point to a secure directory. So create a subdirectory with restricted // permissions under each listed directory. - Option(conf.getenv("SPARK_LOCAL_DIRS")) -.getOrElse(conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) -.split(",") + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")).split(",") } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-11131][CORE] Fix race in worker registration protocol.
Repository: spark Updated Branches: refs/heads/master 67582132b -> 7ab0ce650 [SPARK-11131][CORE] Fix race in worker registration protocol. Because the registration RPC was not really an RPC, but a bunch of disconnected messages, it was possible for other messages to be sent before the reply to the registration arrived, and that would confuse the Worker. Especially in local-cluster mode, the worker was succeptible to receiving an executor request before it received a message from the master saying registration succeeded. On top of the above, the change also fixes a ClassCastException when the registration fails, which also affects the executor registration protocol. Because the `ask` is issued with a specific return type, if the error message (of a different type) was returned instead, the code would just die with an exception. This is fixed by having a common base trait for these reply messages. Author: Marcelo VanzinCloses #9138 from vanzin/SPARK-11131. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7ab0ce65 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7ab0ce65 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7ab0ce65 Branch: refs/heads/master Commit: 7ab0ce6501c37f0fc3a49e3332573ae4e4def3e8 Parents: 6758213 Author: Marcelo Vanzin Authored: Mon Oct 19 16:14:50 2015 -0700 Committer: Andrew Or Committed: Mon Oct 19 16:14:50 2015 -0700 -- .../org/apache/spark/deploy/DeployMessage.scala | 7 +- .../org/apache/spark/deploy/master/Master.scala | 50 +++--- .../org/apache/spark/deploy/worker/Worker.scala | 73 +--- .../executor/CoarseGrainedExecutorBackend.scala | 4 +- .../cluster/CoarseGrainedClusterMessage.scala | 4 ++ .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- 6 files changed, 86 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7ab0ce65/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index d8084a5..3feb7ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -69,9 +69,14 @@ private[deploy] object DeployMessages { // Master to Worker + sealed trait RegisterWorkerResponse + case class RegisteredWorker(master: RpcEndpointRef, masterWebUiUrl: String) extends DeployMessage +with RegisterWorkerResponse + + case class RegisterWorkerFailed(message: String) extends DeployMessage with RegisterWorkerResponse - case class RegisterWorkerFailed(message: String) extends DeployMessage + case object MasterInStandby extends DeployMessage with RegisterWorkerResponse case class ReconnectWorker(masterUrl: String) extends DeployMessage http://git-wip-us.apache.org/repos/asf/spark/blob/7ab0ce65/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index d518e92..6715d6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -233,31 +233,6 @@ private[deploy] class Master( System.exit(0) } -case RegisterWorker( -id, workerHost, workerPort, workerRef, cores, memory, workerUiPort, publicAddress) => { - logInfo("Registering worker %s:%d with %d cores, %s RAM".format( -workerHost, workerPort, cores, Utils.megabytesToString(memory))) - if (state == RecoveryState.STANDBY) { -// ignore, don't send response - } else if (idToWorker.contains(id)) { -workerRef.send(RegisterWorkerFailed("Duplicate worker ID")) - } else { -val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, - workerRef, workerUiPort, publicAddress) -if (registerWorker(worker)) { - persistenceEngine.addWorker(worker) - workerRef.send(RegisteredWorker(self, masterWebUiUrl)) - schedule() -} else { - val workerAddress = worker.endpoint.address - logWarning("Worker registration failed. Attempted to re-register worker at same " + -"address: " + workerAddress) - workerRef.send(RegisterWorkerFailed("Attempted to re-register worker at same address: " -+ workerAddress)) -} - } -} - case RegisterApplication(description,
spark git commit: [SPARK-11120] Allow sane default number of executor failures when dynamically allocating in YARN
Repository: spark Updated Branches: refs/heads/master fc26f32cf -> 16906ef23 [SPARK-11120] Allow sane default number of executor failures when dynamically allocating in YARN I also added some information to container-failure error msgs about what host they failed on, which would have helped me identify the problem that lead me to this JIRA and PR sooner. Author: Ryan WilliamsCloses #9147 from ryan-williams/dyn-exec-failures. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/16906ef2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/16906ef2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/16906ef2 Branch: refs/heads/master Commit: 16906ef23a7aa2854c8cdcaa3bb3808ab39e0eec Parents: fc26f32 Author: Ryan Williams Authored: Mon Oct 19 16:34:15 2015 -0700 Committer: Andrew Or Committed: Mon Oct 19 16:34:15 2015 -0700 -- .../main/scala/org/apache/spark/SparkConf.scala | 4 +++- .../spark/deploy/yarn/ApplicationMaster.scala| 19 +++ .../apache/spark/deploy/yarn/YarnAllocator.scala | 19 +++ 3 files changed, 29 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/16906ef2/core/src/main/scala/org/apache/spark/SparkConf.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 1a0ac3d..58d3b84 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -595,7 +595,9 @@ private[spark] object SparkConf extends Logging { "spark.rpc.lookupTimeout" -> Seq( AlternateConfig("spark.akka.lookupTimeout", "1.4")), "spark.streaming.fileStream.minRememberDuration" -> Seq( - AlternateConfig("spark.streaming.minRememberDuration", "1.5")) + AlternateConfig("spark.streaming.minRememberDuration", "1.5")), +"spark.yarn.max.executor.failures" -> Seq( + AlternateConfig("spark.yarn.max.worker.failures", "1.5")) ) /** http://git-wip-us.apache.org/repos/asf/spark/blob/16906ef2/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala -- diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index d1d248b..4b4d999 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -62,10 +62,21 @@ private[spark] class ApplicationMaster( .asInstanceOf[YarnConfiguration] private val isClusterMode = args.userClass != null - // Default to numExecutors * 2, with minimum of 3 - private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", -sparkConf.getInt("spark.yarn.max.worker.failures", - math.max(sparkConf.getInt("spark.executor.instances", 0) * 2, 3))) + // Default to twice the number of executors (twice the maximum number of executors if dynamic + // allocation is enabled), with a minimum of 3. + + private val maxNumExecutorFailures = { +val defaultKey = + if (Utils.isDynamicAllocationEnabled(sparkConf)) { +"spark.dynamicAllocation.maxExecutors" + } else { +"spark.executor.instances" + } +val effectiveNumExecutors = sparkConf.getInt(defaultKey, 0) +val defaultMaxNumExecutorFailures = math.max(3, 2 * effectiveNumExecutors) + +sparkConf.getInt("spark.yarn.max.executor.failures", defaultMaxNumExecutorFailures) + } @volatile private var exitCode = 0 @volatile private var unregistered = false http://git-wip-us.apache.org/repos/asf/spark/blob/16906ef2/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala -- diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 9e1ef1b..1deaa37 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -430,17 +430,20 @@ private[yarn] class YarnAllocator( for (completedContainer <- completedContainers) { val containerId = completedContainer.getContainerId val alreadyReleased = releasedContainers.remove(containerId) + val hostOpt = allocatedContainerToHostMap.get(containerId) + val onHostStr = hostOpt.map(host => s" on host: $host").getOrElse("") val
spark git commit: [SPARK-11051][CORE] Do not allow local checkpointing after the RDD is materialized and checkpointed
Repository: spark Updated Branches: refs/heads/master 7ab0ce650 -> a1413b366 [SPARK-11051][CORE] Do not allow local checkpointing after the RDD is materialized and checkpointed JIRA: https://issues.apache.org/jira/browse/SPARK-11051 When a `RDD` is materialized and checkpointed, its partitions and dependencies are cleared. If we allow local checkpointing on it and assign `LocalRDDCheckpointData` to its `checkpointData`. Next time when the RDD is materialized again, the error will be thrown. Author: Liang-Chi HsiehCloses #9072 from viirya/no-localcheckpoint-after-checkpoint. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a1413b36 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a1413b36 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a1413b36 Branch: refs/heads/master Commit: a1413b3662250dd5e980e8b1f7c3dc4585ab4766 Parents: 7ab0ce6 Author: Liang-Chi Hsieh Authored: Mon Oct 19 16:16:31 2015 -0700 Committer: Andrew Or Committed: Mon Oct 19 16:16:31 2015 -0700 -- .../main/scala/org/apache/spark/rdd/RDD.scala | 35 .../org/apache/spark/CheckpointSuite.scala | 4 +++ 2 files changed, 32 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a1413b36/core/src/main/scala/org/apache/spark/rdd/RDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a56e542..a97bb17 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -294,7 +294,11 @@ abstract class RDD[T: ClassTag]( */ private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = { -if (isCheckpointed) firstParent[T].iterator(split, context) else compute(split, context) +if (isCheckpointedAndMaterialized) { + firstParent[T].iterator(split, context) +} else { + compute(split, context) +} } /** @@ -1520,21 +1524,38 @@ abstract class RDD[T: ClassTag]( persist(LocalRDDCheckpointData.transformStorageLevel(storageLevel), allowOverride = true) } -checkpointData match { - case Some(reliable: ReliableRDDCheckpointData[_]) => logWarning( -"RDD was already marked for reliable checkpointing: overriding with local checkpoint.") - case _ => +// If this RDD is already checkpointed and materialized, its lineage is already truncated. +// We must not override our `checkpointData` in this case because it is needed to recover +// the checkpointed data. If it is overridden, next time materializing on this RDD will +// cause error. +if (isCheckpointedAndMaterialized) { + logWarning("Not marking RDD for local checkpoint because it was already " + +"checkpointed and materialized") +} else { + // Lineage is not truncated yet, so just override any existing checkpoint data with ours + checkpointData match { +case Some(_: ReliableRDDCheckpointData[_]) => logWarning( + "RDD was already marked for reliable checkpointing: overriding with local checkpoint.") +case _ => + } + checkpointData = Some(new LocalRDDCheckpointData(this)) } -checkpointData = Some(new LocalRDDCheckpointData(this)) this } /** - * Return whether this RDD is marked for checkpointing, either reliably or locally. + * Return whether this RDD is checkpointed and materialized, either reliably or locally. */ def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) /** + * Return whether this RDD is checkpointed and materialized, either reliably or locally. + * This is introduced as an alias for `isCheckpointed` to clarify the semantics of the + * return value. Exposed for testing. + */ + private[spark] def isCheckpointedAndMaterialized: Boolean = isCheckpointed + + /** * Return whether this RDD is marked for local checkpointing. * Exposed for testing. */ http://git-wip-us.apache.org/repos/asf/spark/blob/a1413b36/core/src/test/scala/org/apache/spark/CheckpointSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 4d70bfe..119e5fc 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -241,9 +241,13 @@ class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging val rdd = new BlockRDD[Int](sc,
spark git commit: [SPARK-11051][CORE] Do not allow local checkpointing after the RDD is materialized and checkpointed
Repository: spark Updated Branches: refs/heads/branch-1.5 5186ec8ac -> 2195fec12 [SPARK-11051][CORE] Do not allow local checkpointing after the RDD is materialized and checkpointed JIRA: https://issues.apache.org/jira/browse/SPARK-11051 When a `RDD` is materialized and checkpointed, its partitions and dependencies are cleared. If we allow local checkpointing on it and assign `LocalRDDCheckpointData` to its `checkpointData`. Next time when the RDD is materialized again, the error will be thrown. Author: Liang-Chi HsiehCloses #9072 from viirya/no-localcheckpoint-after-checkpoint. (cherry picked from commit a1413b3662250dd5e980e8b1f7c3dc4585ab4766) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2195fec1 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2195fec1 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2195fec1 Branch: refs/heads/branch-1.5 Commit: 2195fec125b2027c734ba4c6940b7fd8dfc5d10b Parents: 5186ec8 Author: Liang-Chi Hsieh Authored: Mon Oct 19 16:16:31 2015 -0700 Committer: Andrew Or Committed: Mon Oct 19 16:16:39 2015 -0700 -- .../main/scala/org/apache/spark/rdd/RDD.scala | 35 .../org/apache/spark/CheckpointSuite.scala | 4 +++ 2 files changed, 32 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2195fec1/core/src/main/scala/org/apache/spark/rdd/RDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 081c721..20b115f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -294,7 +294,11 @@ abstract class RDD[T: ClassTag]( */ private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = { -if (isCheckpointed) firstParent[T].iterator(split, context) else compute(split, context) +if (isCheckpointedAndMaterialized) { + firstParent[T].iterator(split, context) +} else { + compute(split, context) +} } /** @@ -1526,21 +1530,38 @@ abstract class RDD[T: ClassTag]( persist(LocalRDDCheckpointData.transformStorageLevel(storageLevel), allowOverride = true) } -checkpointData match { - case Some(reliable: ReliableRDDCheckpointData[_]) => logWarning( -"RDD was already marked for reliable checkpointing: overriding with local checkpoint.") - case _ => +// If this RDD is already checkpointed and materialized, its lineage is already truncated. +// We must not override our `checkpointData` in this case because it is needed to recover +// the checkpointed data. If it is overridden, next time materializing on this RDD will +// cause error. +if (isCheckpointedAndMaterialized) { + logWarning("Not marking RDD for local checkpoint because it was already " + +"checkpointed and materialized") +} else { + // Lineage is not truncated yet, so just override any existing checkpoint data with ours + checkpointData match { +case Some(_: ReliableRDDCheckpointData[_]) => logWarning( + "RDD was already marked for reliable checkpointing: overriding with local checkpoint.") +case _ => + } + checkpointData = Some(new LocalRDDCheckpointData(this)) } -checkpointData = Some(new LocalRDDCheckpointData(this)) this } /** - * Return whether this RDD is marked for checkpointing, either reliably or locally. + * Return whether this RDD is checkpointed and materialized, either reliably or locally. */ def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) /** + * Return whether this RDD is checkpointed and materialized, either reliably or locally. + * This is introduced as an alias for `isCheckpointed` to clarify the semantics of the + * return value. Exposed for testing. + */ + private[spark] def isCheckpointedAndMaterialized: Boolean = isCheckpointed + + /** * Return whether this RDD is marked for local checkpointing. * Exposed for testing. */ http://git-wip-us.apache.org/repos/asf/spark/blob/2195fec1/core/src/test/scala/org/apache/spark/CheckpointSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d343bb9..608f7ca 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -241,9 +241,13 @@
spark git commit: [SPARK-10515] When killing executor, the pending replacement executors should not be lost
Repository: spark Updated Branches: refs/heads/branch-1.5 166fdf4e3 -> 13920d5fe [SPARK-10515] When killing executor, the pending replacement executors should not be lost If the heartbeat receiver kills executors (and new ones are not registered to replace them), the idle timeout for the old executors will be lost (and then change a total number of executors requested by Driver), So new ones will be not to asked to replace them. For example, executorsPendingToRemove=Set(1), and executor 2 is idle timeout before a new executor is asked to replace executor 1. Then driver kill executor 2, and sending RequestExecutors to AM. But executorsPendingToRemove=Set(1,2), So AM doesn't allocate a executor to replace 1. see: https://github.com/apache/spark/pull/8668 Author: KaiXinXiaoLeiAuthor: huleilei Closes #8945 from KaiXinXiaoLei/pendingexecutor. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/13920d5f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/13920d5f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/13920d5f Branch: refs/heads/branch-1.5 Commit: 13920d5fecbb8f1aaeb8a5013b1062b91cbabc7b Parents: 166fdf4 Author: KaiXinXiaoLei Authored: Thu Oct 15 14:48:01 2015 -0700 Committer: Andrew Or Committed: Thu Oct 15 14:48:46 2015 -0700 -- .../cluster/CoarseGrainedSchedulerBackend.scala | 2 ++ .../StandaloneDynamicAllocationSuite.scala | 35 2 files changed, 37 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/13920d5f/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 5730a87..6ae8fed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -432,6 +432,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp if (!replace) { doRequestTotalExecutors( numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) +} else { + numPendingExecutors += knownExecutors.size } doKillExecutors(executorsToKill) http://git-wip-us.apache.org/repos/asf/spark/blob/13920d5f/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 2e2fa22..d145e78 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -369,6 +369,41 @@ class StandaloneDynamicAllocationSuite assert(apps.head.getExecutorLimit === 1) } + test("the pending replacement executors should not be lost (SPARK-10515)") { +sc = new SparkContext(appConf) +val appId = sc.applicationId +eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.executors.size === 2) + assert(apps.head.getExecutorLimit === Int.MaxValue) +} +// sync executors between the Master and the driver, needed because +// the driver refuses to kill executors it does not know about +syncExecutors(sc) +val executors = getExecutorIds(sc) +assert(executors.size === 2) +// kill executor 1, and replace it +assert(sc.killAndReplaceExecutor(executors.head)) +eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.head.executors.size === 2) +} + +var apps = getApplications() +// kill executor 1 +assert(sc.killExecutor(executors.head)) +apps = getApplications() +assert(apps.head.executors.size === 2) +assert(apps.head.getExecutorLimit === 2) +// kill executor 2 +assert(sc.killExecutor(executors(1))) +apps = getApplications() +assert(apps.head.executors.size === 1) +assert(apps.head.getExecutorLimit === 1) + } + // === // | Utility methods for testing | // === - To
spark git commit: [SPARK-11078] Ensure spilling tests actually spill
Repository: spark Updated Branches: refs/heads/master 2d000124b -> 3b364ff0a [SPARK-11078] Ensure spilling tests actually spill #9084 uncovered that many tests that test spilling don't actually spill. This is a follow-up patch to fix that to ensure our unit tests actually catch potential bugs in spilling. The size of this patch is inflated by the refactoring of `ExternalSorterSuite`, which had a lot of duplicate code and logic. Author: Andrew Or <and...@databricks.com> Closes #9124 from andrewor14/spilling-tests. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3b364ff0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3b364ff0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3b364ff0 Branch: refs/heads/master Commit: 3b364ff0a4f38c2b8023429a55623de32be5f329 Parents: 2d00012 Author: Andrew Or <and...@databricks.com> Authored: Thu Oct 15 14:50:01 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Thu Oct 15 14:50:01 2015 -0700 -- .../main/scala/org/apache/spark/TestUtils.scala | 51 ++ .../spark/shuffle/ShuffleMemoryManager.scala| 6 +- .../util/collection/ExternalAppendOnlyMap.scala | 6 + .../spark/util/collection/Spillable.scala | 37 +- .../org/apache/spark/DistributedSuite.scala | 39 +- .../collection/ExternalAppendOnlyMapSuite.scala | 103 ++- .../util/collection/ExternalSorterSuite.scala | 871 --- .../execution/TestShuffleMemoryManager.scala| 2 + 8 files changed, 534 insertions(+), 581 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3b364ff0/core/src/main/scala/org/apache/spark/TestUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 888763a..acfe751 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -24,10 +24,14 @@ import java.util.Arrays import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import com.google.common.io.{ByteStreams, Files} import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler._ import org.apache.spark.util.Utils /** @@ -154,4 +158,51 @@ private[spark] object TestUtils { " @Override public String toString() { return \"" + toStringValue + "\"; }}") createCompiledClass(className, destDir, sourceFile, classpathUrls) } + + /** + * Run some code involving jobs submitted to the given context and assert that the jobs spilled. + */ + def assertSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = { +val spillListener = new SpillListener +sc.addSparkListener(spillListener) +body +assert(spillListener.numSpilledStages > 0, s"expected $identifier to spill, but did not") + } + + /** + * Run some code involving jobs submitted to the given context and assert that the jobs + * did not spill. + */ + def assertNotSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = { +val spillListener = new SpillListener +sc.addSparkListener(spillListener) +body +assert(spillListener.numSpilledStages == 0, s"expected $identifier to not spill, but did") + } + +} + + +/** + * A [[SparkListener]] that detects whether spills have occurred in Spark jobs. + */ +private class SpillListener extends SparkListener { + private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]] + private val spilledStageIds = new mutable.HashSet[Int] + + def numSpilledStages: Int = spilledStageIds.size + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { +stageIdToTaskMetrics.getOrElseUpdate( + taskEnd.stageId, new ArrayBuffer[TaskMetrics]) += taskEnd.taskMetrics + } + + override def onStageCompleted(stageComplete: SparkListenerStageCompleted): Unit = { +val stageId = stageComplete.stageInfo.stageId +val metrics = stageIdToTaskMetrics.remove(stageId).toSeq.flatten +val spilled = metrics.map(_.memoryBytesSpilled).sum > 0 +if (spilled) { + spilledStageIds += stageId +} + } } http://git-wip-us.apache.org/repos/asf/spark/blob/3b364ff0/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/Shu
spark git commit: [SPARK-10515] When killing executor, the pending replacement executors should not be lost
Repository: spark Updated Branches: refs/heads/master 723aa75a9 -> 2d000124b [SPARK-10515] When killing executor, the pending replacement executors should not be lost If the heartbeat receiver kills executors (and new ones are not registered to replace them), the idle timeout for the old executors will be lost (and then change a total number of executors requested by Driver), So new ones will be not to asked to replace them. For example, executorsPendingToRemove=Set(1), and executor 2 is idle timeout before a new executor is asked to replace executor 1. Then driver kill executor 2, and sending RequestExecutors to AM. But executorsPendingToRemove=Set(1,2), So AM doesn't allocate a executor to replace 1. see: https://github.com/apache/spark/pull/8668 Author: KaiXinXiaoLeiAuthor: huleilei Closes #8945 from KaiXinXiaoLei/pendingexecutor. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2d000124 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2d000124 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2d000124 Branch: refs/heads/master Commit: 2d000124b72d0ff9e3ecefa03923405642516c4c Parents: 723aa75 Author: KaiXinXiaoLei Authored: Thu Oct 15 14:48:01 2015 -0700 Committer: Andrew Or Committed: Thu Oct 15 14:48:01 2015 -0700 -- .../cluster/CoarseGrainedSchedulerBackend.scala | 2 ++ .../StandaloneDynamicAllocationSuite.scala | 35 2 files changed, 37 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2d000124/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 18771f7..55a564b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -438,6 +438,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp if (!replace) { doRequestTotalExecutors( numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) +} else { + numPendingExecutors += knownExecutors.size } doKillExecutors(executorsToKill) http://git-wip-us.apache.org/repos/asf/spark/blob/2d000124/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 2e2fa22..d145e78 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -369,6 +369,41 @@ class StandaloneDynamicAllocationSuite assert(apps.head.getExecutorLimit === 1) } + test("the pending replacement executors should not be lost (SPARK-10515)") { +sc = new SparkContext(appConf) +val appId = sc.applicationId +eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.executors.size === 2) + assert(apps.head.getExecutorLimit === Int.MaxValue) +} +// sync executors between the Master and the driver, needed because +// the driver refuses to kill executors it does not know about +syncExecutors(sc) +val executors = getExecutorIds(sc) +assert(executors.size === 2) +// kill executor 1, and replace it +assert(sc.killAndReplaceExecutor(executors.head)) +eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.head.executors.size === 2) +} + +var apps = getApplications() +// kill executor 1 +assert(sc.killExecutor(executors.head)) +apps = getApplications() +assert(apps.head.executors.size === 2) +assert(apps.head.getExecutorLimit === 2) +// kill executor 2 +assert(sc.killExecutor(executors(1))) +apps = getApplications() +assert(apps.head.executors.size === 1) +assert(apps.head.getExecutorLimit === 1) + } + // === // | Utility methods for testing | // === - To unsubscribe,
spark git commit: [SPARK-11071] [LAUNCHER] Fix flakiness in LauncherServerSuite::timeout.
Repository: spark Updated Branches: refs/heads/master b591de7c0 -> a5719804c [SPARK-11071] [LAUNCHER] Fix flakiness in LauncherServerSuite::timeout. The test could fail depending on scheduling of the various threads involved; the change removes some sources of races, while making the test a little more resilient by trying a few times before giving up. Author: Marcelo VanzinCloses #9079 from vanzin/SPARK-11071. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a5719804 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a5719804 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a5719804 Branch: refs/heads/master Commit: a5719804c5ed99ce36bd0dd230ab8b3b7a3b92e3 Parents: b591de7 Author: Marcelo Vanzin Authored: Thu Oct 15 14:46:40 2015 -0700 Committer: Andrew Or Committed: Thu Oct 15 14:46:40 2015 -0700 -- .../apache/spark/launcher/LauncherServer.java | 9 - .../spark/launcher/LauncherServerSuite.java | 35 +++- 2 files changed, 34 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a5719804/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java -- diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index c5fd408..d099ee9 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -242,7 +242,14 @@ class LauncherServer implements Closeable { synchronized (clients) { clients.add(clientConnection); } - timeoutTimer.schedule(timeout, getConnectionTimeout()); + long timeoutMs = getConnectionTimeout(); + // 0 is used for testing to avoid issues with clock resolution / thread scheduling, + // and force an immediate timeout. + if (timeoutMs > 0) { +timeoutTimer.schedule(timeout, getConnectionTimeout()); + } else { +timeout.run(); + } } } } catch (IOException ioe) { http://git-wip-us.apache.org/repos/asf/spark/blob/a5719804/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java -- diff --git a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java index 27cd106..dc8fbb5 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java @@ -121,12 +121,12 @@ public class LauncherServerSuite extends BaseSuite { @Test public void testTimeout() throws Exception { -final long TEST_TIMEOUT = 10L; - ChildProcAppHandle handle = null; TestClient client = null; try { - SparkLauncher.setConfig(SparkLauncher.CHILD_CONNECTION_TIMEOUT, String.valueOf(TEST_TIMEOUT)); + // LauncherServer will immediately close the server-side socket when the timeout is set + // to 0. + SparkLauncher.setConfig(SparkLauncher.CHILD_CONNECTION_TIMEOUT, "0"); handle = LauncherServer.newAppHandle(); @@ -134,12 +134,29 @@ public class LauncherServerSuite extends BaseSuite { LauncherServer.getServerInstance().getPort()); client = new TestClient(s); - Thread.sleep(TEST_TIMEOUT * 10); - try { -client.send(new Hello(handle.getSecret(), "1.4.0")); -fail("Expected exception caused by connection timeout."); - } catch (IllegalStateException e) { -// Expected. + // Try a few times since the client-side socket may not reflect the server-side close + // immediately. + boolean helloSent = false; + int maxTries = 10; + for (int i = 0; i < maxTries; i++) { +try { + if (!helloSent) { +client.send(new Hello(handle.getSecret(), "1.4.0")); +helloSent = true; + } else { +client.send(new SetAppId("appId")); + } + fail("Expected exception caused by connection timeout."); +} catch (IllegalStateException | IOException e) { + // Expected. + break; +} catch (AssertionError e) { + if (i < maxTries - 1) { +Thread.sleep(100); + } else { +throw new AssertionError("Test failed after " + maxTries + " attempts.", e); + } +} } } finally {
spark git commit: fix typo bellow -> below
Repository: spark Updated Branches: refs/heads/master a5719804c -> 723aa75a9 fix typo bellow -> below Author: Britta WeberCloses #9136 from brwe/typo-bellow. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/723aa75a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/723aa75a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/723aa75a Branch: refs/heads/master Commit: 723aa75a9d566c698aa49597f4f655396fef77bd Parents: a571980 Author: Britta Weber Authored: Thu Oct 15 14:47:11 2015 -0700 Committer: Andrew Or Committed: Thu Oct 15 14:47:11 2015 -0700 -- docs/mllib-collaborative-filtering.md | 2 +- docs/mllib-linear-methods.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/723aa75a/docs/mllib-collaborative-filtering.md -- diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index b3fd51d..1ad5212 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -119,7 +119,7 @@ All of MLlib's methods use Java-friendly types, so you can import and call them way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. A self-contained application example -that is equivalent to the provided example in Scala is given bellow: +that is equivalent to the provided example in Scala is given below: Refer to the [`ALS` Java docs](api/java/org/apache/spark/mllib/recommendation/ALS.html) for details on the API. http://git-wip-us.apache.org/repos/asf/spark/blob/723aa75a/docs/mllib-linear-methods.md -- diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index a3e1620..0c76e6e 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -230,7 +230,7 @@ All of MLlib's methods use Java-friendly types, so you can import and call them way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. A self-contained application example -that is equivalent to the provided example in Scala is given bellow: +that is equivalent to the provided example in Scala is given below: Refer to the [`SVMWithSGD` Java docs](api/java/org/apache/spark/mllib/classification/SVMWithSGD.html) and [`SVMModel` Java docs](api/java/org/apache/spark/mllib/classification/SVMModel.html) for details on the API. @@ -612,7 +612,7 @@ All of MLlib's methods use Java-friendly types, so you can import and call them way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. The corresponding Java example to -the Scala snippet provided, is presented bellow: +the Scala snippet provided, is presented below: Refer to the [`LinearRegressionWithSGD` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionWithSGD.html) and [`LinearRegressionModel` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionModel.html) for details on the API. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10412] [SQL] report memory usage for tungsten sql physical operator
Repository: spark Updated Branches: refs/heads/master 3b364ff0a -> 6a2359ff1 [SPARK-10412] [SQL] report memory usage for tungsten sql physical operator https://issues.apache.org/jira/browse/SPARK-10412 some screenshots: ### aggregate: ![screen shot 2015-10-12 at 2 23 11 pm](https://cloud.githubusercontent.com/assets/3182036/10439534/618320a4-70ef-11e5-94d8-62ea7f2d1531.png) ### join ![screen shot 2015-10-12 at 2 23 29 pm](https://cloud.githubusercontent.com/assets/3182036/10439537/6724797c-70ef-11e5-8f75-0cf5cbd42048.png) Author: Wenchen FanAuthor: Wenchen Fan Closes #8931 from cloud-fan/viz. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6a2359ff Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6a2359ff Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6a2359ff Branch: refs/heads/master Commit: 6a2359ff1f7ad2233af2c530313d6ec2ecf70d19 Parents: 3b364ff Author: Wenchen Fan Authored: Thu Oct 15 14:50:58 2015 -0700 Committer: Andrew Or Committed: Thu Oct 15 14:50:58 2015 -0700 -- .../execution/aggregate/TungstenAggregate.scala | 10 ++- .../aggregate/TungstenAggregationIterator.scala | 10 ++- .../spark/sql/execution/metric/SQLMetrics.scala | 72 ++-- .../org/apache/spark/sql/execution/sort.scala | 16 + .../spark/sql/execution/ui/ExecutionPage.scala | 2 +- .../spark/sql/execution/ui/SQLListener.scala| 9 ++- .../spark/sql/execution/ui/SparkPlanGraph.scala | 4 +- .../TungstenAggregationIteratorSuite.scala | 3 +- .../sql/execution/metric/SQLMetricsSuite.scala | 13 +++- .../sql/execution/ui/SQLListenerSuite.scala | 20 +++--- 10 files changed, 116 insertions(+), 43 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6a2359ff/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index c342940..0d3a4b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -49,7 +49,9 @@ case class TungstenAggregate( override private[sql] lazy val metrics = Map( "numInputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of input rows"), -"numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) +"numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"), +"dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), +"spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")) override def outputsUnsafeRows: Boolean = true @@ -79,6 +81,8 @@ case class TungstenAggregate( protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { val numInputRows = longMetric("numInputRows") val numOutputRows = longMetric("numOutputRows") +val dataSize = longMetric("dataSize") +val spillSize = longMetric("spillSize") /** * Set up the underlying unsafe data structures used before computing the parent partition. @@ -97,7 +101,9 @@ case class TungstenAggregate( child.output, testFallbackStartsAt, numInputRows, -numOutputRows) +numOutputRows, +dataSize, +spillSize) } /** Compute a partition using the iterator already set up previously. */ http://git-wip-us.apache.org/repos/asf/spark/blob/6a2359ff/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index fe708a5..7cd0f7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -87,7 +87,9 @@ class TungstenAggregationIterator( originalInputAttributes: Seq[Attribute], testFallbackStartsAt: Option[Int], numInputRows: LongSQLMetric, -numOutputRows: LongSQLMetric) +numOutputRows: LongSQLMetric, +dataSize: LongSQLMetric, +spillSize: LongSQLMetric) extends Iterator[UnsafeRow] with Logging { // The parent
spark git commit: [SPARK-10887] [SQL] Build HashedRelation outside of HashJoinNode.
Repository: spark Updated Branches: refs/heads/master 2a6f614cd -> 82d275f27 [SPARK-10887] [SQL] Build HashedRelation outside of HashJoinNode. This PR refactors `HashJoinNode` to take a existing `HashedRelation`. So, we can reuse this node for both `ShuffledHashJoin` and `BroadcastHashJoin`. https://issues.apache.org/jira/browse/SPARK-10887 Author: Yin HuaiCloses #8953 from yhuai/SPARK-10887. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/82d275f2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/82d275f2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/82d275f2 Branch: refs/heads/master Commit: 82d275f27c3e9211ce69c5c8685a0fe90c0be26f Parents: 2a6f614 Author: Yin Huai Authored: Thu Oct 8 11:56:44 2015 -0700 Committer: Andrew Or Committed: Thu Oct 8 11:56:44 2015 -0700 -- .../codegen/GenerateMutableProjection.scala | 2 + .../codegen/GenerateSafeProjection.scala| 4 +- .../execution/local/BinaryHashJoinNode.scala| 76 + .../execution/local/BroadcastHashJoinNode.scala | 59 ++ .../sql/execution/local/HashJoinNode.scala | 67 +++ .../sql/execution/local/HashJoinNodeSuite.scala | 85 +--- .../sql/execution/local/LocalNodeTest.scala | 20 - 7 files changed, 262 insertions(+), 51 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/82d275f2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index d82d191..e8ee647 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -27,6 +27,8 @@ abstract class BaseMutableProjection extends MutableProjection /** * Generates byte code that produces a [[MutableRow]] object that can update itself based on a new * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. + * It exposes a `target` method, which is used to set the row that will be updated. + * The internal [[MutableRow]] object created internally is used only when `target` is not used. */ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => MutableProjection] { http://git-wip-us.apache.org/repos/asf/spark/blob/82d275f2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index ea09e02..9873630 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.types._ /** - * Generates byte code that produces a [[MutableRow]] object that can update itself based on a new - * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. + * Generates byte code that produces a [[MutableRow]] object (not an [[UnsafeRow]]) that can update + * itself based on a new input [[InternalRow]] for a fixed set of [[Expression Expressions]]. */ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] { http://git-wip-us.apache.org/repos/asf/spark/blob/82d275f2/sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala new file mode 100644 index 000..52dcb9e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala @@ -0,0 +1,76 @@ +/* +* 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
spark git commit: [SPARK-7527] [CORE] Fix createNullValue to return the correct null values and REPL mode detection
Repository: spark Updated Branches: refs/heads/branch-1.4 e7c4346d0 -> e2ff49198 [SPARK-7527] [CORE] Fix createNullValue to return the correct null values and REPL mode detection The root cause of SPARK-7527 is `createNullValue` returns an incompatible value `Byte(0)` for `char` and `boolean`. This PR fixes it and corrects the class name of the main class, and also adds an unit test to demonstrate it. Author: zsxwingCloses #6735 from zsxwing/SPARK-7527 and squashes the following commits: bbdb271 [zsxwing] Use pattern match in createNullValue b0a0e7e [zsxwing] Remove the noisy in the test output 903e269 [zsxwing] Remove the code for Utils.isInInterpreter == false 5f92dc1 [zsxwing] Fix createNullValue to return the correct null values and REPL mode detection Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e2ff4919 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e2ff4919 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e2ff4919 Branch: refs/heads/branch-1.4 Commit: e2ff4919834fac5fa92c4fbb7e24078e04b81262 Parents: e7c4346 Author: zsxwing Authored: Wed Jun 10 13:22:52 2015 -0700 Committer: Andrew Or Committed: Thu Oct 8 11:27:24 2015 -0700 -- .../org/apache/spark/util/ClosureCleaner.scala | 40 -- .../scala/org/apache/spark/util/Utils.scala | 9 +--- .../apache/spark/util/ClosureCleanerSuite.scala | 44 3 files changed, 64 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e2ff4919/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 6f2966b..305de4c 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -109,7 +109,14 @@ private[spark] object ClosureCleaner extends Logging { private def createNullValue(cls: Class[_]): AnyRef = { if (cls.isPrimitive) { - new java.lang.Byte(0: Byte) // Should be convertible to any primitive type + cls match { +case java.lang.Boolean.TYPE => new java.lang.Boolean(false) +case java.lang.Character.TYPE => new java.lang.Character('\0') +case java.lang.Void.TYPE => + // This should not happen because `Foo(void x) {}` does not compile. + throw new IllegalStateException("Unexpected void parameter in constructor") +case _ => new java.lang.Byte(0: Byte) + } } else { null } @@ -319,28 +326,17 @@ private[spark] object ClosureCleaner extends Logging { private def instantiateClass( cls: Class[_], enclosingObject: AnyRef): AnyRef = { -if (!Utils.isInInterpreter) { - // This is a bona fide closure class, whose constructor has no effects - // other than to set its fields, so use its constructor - val cons = cls.getConstructors()(0) - val params = cons.getParameterTypes.map(createNullValue).toArray - if (enclosingObject != null) { -params(0) = enclosingObject // First param is always enclosing object - } - return cons.newInstance(params: _*).asInstanceOf[AnyRef] -} else { - // Use reflection to instantiate object without calling constructor - val rf = sun.reflect.ReflectionFactory.getReflectionFactory() - val parentCtor = classOf[java.lang.Object].getDeclaredConstructor() - val newCtor = rf.newConstructorForSerialization(cls, parentCtor) - val obj = newCtor.newInstance().asInstanceOf[AnyRef] - if (enclosingObject != null) { -val field = cls.getDeclaredField("$outer") -field.setAccessible(true) -field.set(obj, enclosingObject) - } - obj +// Use reflection to instantiate object without calling constructor +val rf = sun.reflect.ReflectionFactory.getReflectionFactory() +val parentCtor = classOf[java.lang.Object].getDeclaredConstructor() +val newCtor = rf.newConstructorForSerialization(cls, parentCtor) +val obj = newCtor.newInstance().asInstanceOf[AnyRef] +if (enclosingObject != null) { + val field = cls.getDeclaredField("$outer") + field.setAccessible(true) + field.set(obj, enclosingObject) } +obj } } http://git-wip-us.apache.org/repos/asf/spark/blob/e2ff4919/core/src/main/scala/org/apache/spark/util/Utils.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 615944e..6abc3ac
spark git commit: [SPARK-10851] [SPARKR] Exception not failing R applications (in yarn cluster mode)
Repository: spark Updated Branches: refs/heads/master 16fd2a2f4 -> c7b29ae64 [SPARK-10851] [SPARKR] Exception not failing R applications (in yarn cluster mode) The YARN backend doesn't like when user code calls System.exit, since it cannot know the exit status and thus cannot set an appropriate final status for the application. This PR remove the usage of system.exit to exit the RRunner. Instead, when the R process running an SparkR script returns an exit code other than 0, throws SparkUserAppException which will be caught by ApplicationMaster and ApplicationMaster knows it failed. For other failures, throws SparkException. Author: Sun RuiCloses #8938 from sun-rui/SPARK-10851. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c7b29ae6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c7b29ae6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c7b29ae6 Branch: refs/heads/master Commit: c7b29ae6418368a1266b960ba8776317fd867313 Parents: 16fd2a2 Author: Sun Rui Authored: Wed Sep 30 11:03:08 2015 -0700 Committer: Andrew Or Committed: Wed Sep 30 11:03:08 2015 -0700 -- core/src/main/scala/org/apache/spark/deploy/RRunner.scala | 10 +++--- 1 file changed, 7 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c7b29ae6/core/src/main/scala/org/apache/spark/deploy/RRunner.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala index 05b954c..58cc1f9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.api.r.{RBackend, RUtils} +import org.apache.spark.{SparkException, SparkUserAppException} import org.apache.spark.util.RedirectThread /** @@ -84,12 +85,15 @@ object RRunner { } finally { sparkRBackend.close() } - System.exit(returnCode) + if (returnCode != 0) { +throw new SparkUserAppException(returnCode) + } } else { + val errorMessage = s"SparkR backend did not initialize in $backendTimeout seconds" // scalastyle:off println - System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds") + System.err.println(errorMessage) // scalastyle:on println - System.exit(-1) + throw new SparkException(errorMessage) } } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10825] [CORE] [TESTS] Fix race conditions in StandaloneDynamicAllocationSuite
Repository: spark Updated Branches: refs/heads/branch-1.5 9b3014bc4 -> d54493279 [SPARK-10825] [CORE] [TESTS] Fix race conditions in StandaloneDynamicAllocationSuite Fix the following issues in StandaloneDynamicAllocationSuite: 1. It should not assume master and workers start in order 2. It should not assume master and workers get ready at once 3. It should not assume the application is already registered with master after creating SparkContext 4. It should not access Master.app and idToApp which are not thread safe The changes includes: * Use `eventually` to wait until master and workers are ready to fix 1 and 2 * Use `eventually` to wait until the application is registered with master to fix 3 * Use `askWithRetry[MasterStateResponse](RequestMasterState)` to get the application info to fix 4 Author: zsxwingCloses #8914 from zsxwing/fix-StandaloneDynamicAllocationSuite. (cherry picked from commit dba95ea03216e6b8e623db4a36e1018c6ed95538) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d5449327 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d5449327 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d5449327 Branch: refs/heads/branch-1.5 Commit: d5449327941ed00bf1aaa07ee02113d1522ad514 Parents: 9b3014b Author: zsxwing Authored: Tue Sep 29 11:53:28 2015 -0700 Committer: Andrew Or Committed: Tue Sep 29 11:53:38 2015 -0700 -- .../StandaloneDynamicAllocationSuite.scala | 305 --- 1 file changed, 192 insertions(+), 113 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d5449327/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 1f2a0f0..2e2fa22 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -17,10 +17,15 @@ package org.apache.spark.deploy +import scala.concurrent.duration._ + import org.mockito.Mockito.{mock, when} import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.master.ApplicationInfo import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} @@ -56,6 +61,10 @@ class StandaloneDynamicAllocationSuite } master = makeMaster() workers = makeWorkers(10, 2048) +// Wait until all workers register with master successfully +eventually(timeout(60.seconds), interval(10.millis)) { + assert(getMasterState.workers.size === numWorkers) +} } override def afterAll(): Unit = { @@ -73,167 +82,208 @@ class StandaloneDynamicAllocationSuite test("dynamic allocation default behavior") { sc = new SparkContext(appConf) val appId = sc.applicationId -assert(master.apps.size === 1) -assert(master.apps.head.id === appId) -assert(master.apps.head.executors.size === 2) -assert(master.apps.head.getExecutorLimit === Int.MaxValue) +eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.executors.size === 2) + assert(apps.head.getExecutorLimit === Int.MaxValue) +} // kill all executors assert(killAllExecutors(sc)) -assert(master.apps.head.executors.size === 0) -assert(master.apps.head.getExecutorLimit === 0) +var apps = getApplications() +assert(apps.head.executors.size === 0) +assert(apps.head.getExecutorLimit === 0) // request 1 assert(sc.requestExecutors(1)) -assert(master.apps.head.executors.size === 1) -assert(master.apps.head.getExecutorLimit === 1) +apps = getApplications() +assert(apps.head.executors.size === 1) +assert(apps.head.getExecutorLimit === 1) // request 1 more assert(sc.requestExecutors(1)) -assert(master.apps.head.executors.size === 2) -assert(master.apps.head.getExecutorLimit === 2) +apps = getApplications() +assert(apps.head.executors.size === 2) +assert(apps.head.getExecutorLimit === 2) // request 1 more; this one won't go through assert(sc.requestExecutors(1)) -assert(master.apps.head.executors.size === 2) -
spark git commit: [SPARK-10825] [CORE] [TESTS] Fix race conditions in StandaloneDynamicAllocationSuite
Repository: spark Updated Branches: refs/heads/master 9b9fe5f7b -> dba95ea03 [SPARK-10825] [CORE] [TESTS] Fix race conditions in StandaloneDynamicAllocationSuite Fix the following issues in StandaloneDynamicAllocationSuite: 1. It should not assume master and workers start in order 2. It should not assume master and workers get ready at once 3. It should not assume the application is already registered with master after creating SparkContext 4. It should not access Master.app and idToApp which are not thread safe The changes includes: * Use `eventually` to wait until master and workers are ready to fix 1 and 2 * Use `eventually` to wait until the application is registered with master to fix 3 * Use `askWithRetry[MasterStateResponse](RequestMasterState)` to get the application info to fix 4 Author: zsxwingCloses #8914 from zsxwing/fix-StandaloneDynamicAllocationSuite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dba95ea0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dba95ea0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dba95ea0 Branch: refs/heads/master Commit: dba95ea03216e6b8e623db4a36e1018c6ed95538 Parents: 9b9fe5f Author: zsxwing Authored: Tue Sep 29 11:53:28 2015 -0700 Committer: Andrew Or Committed: Tue Sep 29 11:53:28 2015 -0700 -- .../StandaloneDynamicAllocationSuite.scala | 305 --- 1 file changed, 192 insertions(+), 113 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/dba95ea0/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 1f2a0f0..2e2fa22 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -17,10 +17,15 @@ package org.apache.spark.deploy +import scala.concurrent.duration._ + import org.mockito.Mockito.{mock, when} import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.master.ApplicationInfo import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} @@ -56,6 +61,10 @@ class StandaloneDynamicAllocationSuite } master = makeMaster() workers = makeWorkers(10, 2048) +// Wait until all workers register with master successfully +eventually(timeout(60.seconds), interval(10.millis)) { + assert(getMasterState.workers.size === numWorkers) +} } override def afterAll(): Unit = { @@ -73,167 +82,208 @@ class StandaloneDynamicAllocationSuite test("dynamic allocation default behavior") { sc = new SparkContext(appConf) val appId = sc.applicationId -assert(master.apps.size === 1) -assert(master.apps.head.id === appId) -assert(master.apps.head.executors.size === 2) -assert(master.apps.head.getExecutorLimit === Int.MaxValue) +eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.executors.size === 2) + assert(apps.head.getExecutorLimit === Int.MaxValue) +} // kill all executors assert(killAllExecutors(sc)) -assert(master.apps.head.executors.size === 0) -assert(master.apps.head.getExecutorLimit === 0) +var apps = getApplications() +assert(apps.head.executors.size === 0) +assert(apps.head.getExecutorLimit === 0) // request 1 assert(sc.requestExecutors(1)) -assert(master.apps.head.executors.size === 1) -assert(master.apps.head.getExecutorLimit === 1) +apps = getApplications() +assert(apps.head.executors.size === 1) +assert(apps.head.getExecutorLimit === 1) // request 1 more assert(sc.requestExecutors(1)) -assert(master.apps.head.executors.size === 2) -assert(master.apps.head.getExecutorLimit === 2) +apps = getApplications() +assert(apps.head.executors.size === 2) +assert(apps.head.getExecutorLimit === 2) // request 1 more; this one won't go through assert(sc.requestExecutors(1)) -assert(master.apps.head.executors.size === 2) -assert(master.apps.head.getExecutorLimit === 3) +apps = getApplications() +assert(apps.head.executors.size === 2) +
spark git commit: [SPARK-10871] include number of executor failures in error msg
Repository: spark Updated Branches: refs/heads/master dba95ea03 -> b7ad54ec7 [SPARK-10871] include number of executor failures in error msg Author: Ryan WilliamsCloses #8939 from ryan-williams/errmsg. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b7ad54ec Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b7ad54ec Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b7ad54ec Branch: refs/heads/master Commit: b7ad54ec793af1c84973b402f5cceb88307f7996 Parents: dba95ea Author: Ryan Williams Authored: Tue Sep 29 13:19:46 2015 -0700 Committer: Andrew Or Committed: Tue Sep 29 13:19:46 2015 -0700 -- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b7ad54ec/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala -- diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 93621b4..07a0a45 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -345,7 +345,7 @@ private[spark] class ApplicationMaster( if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, -"Max number of executor failures reached") +s"Max number of executor failures ($maxNumExecutorFailures) reached") } else { logDebug("Sending progress") allocator.allocateResources() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10871] include number of executor failures in error msg
Repository: spark Updated Branches: refs/heads/branch-1.5 d54493279 -> 3b2387368 [SPARK-10871] include number of executor failures in error msg Author: Ryan WilliamsCloses #8939 from ryan-williams/errmsg. (cherry picked from commit b7ad54ec793af1c84973b402f5cceb88307f7996) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3b238736 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3b238736 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3b238736 Branch: refs/heads/branch-1.5 Commit: 3b23873684fba02803af49e9b56063b38fb61eca Parents: d544932 Author: Ryan Williams Authored: Tue Sep 29 13:19:46 2015 -0700 Committer: Andrew Or Committed: Tue Sep 29 13:19:52 2015 -0700 -- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3b238736/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala -- diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 991b5ce..8e909e9 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -345,7 +345,7 @@ private[spark] class ApplicationMaster( if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, -"Max number of executor failures reached") +s"Max number of executor failures ($maxNumExecutorFailures) reached") } else { logDebug("Sending progress") allocator.allocateResources() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array (round 2)
Repository: spark Updated Branches: refs/heads/master 084e4e126 -> 83f6f54d1 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array (round 2) This patch reverts most of the changes in a previous fix #8827. The real cause of the issue is that in `TungstenAggregate`'s prepare method we only reserve 1 page, but later when we switch to sort-based aggregation we try to acquire 1 page AND a pointer array. The longer-term fix should be to reserve also the pointer array, but for now ***we will simply not track the pointer array***. (Note that elsewhere we already don't track the pointer array, e.g. [here](https://github.com/apache/spark/blob/a18208047f06a4244703c17023bb20cbe1f59d73/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java#L88)) Note: This patch reuses the unit test added in #8827 so it doesn't show up in the diff. Author: Andrew Or <and...@databricks.com> Closes #8888 from andrewor14/dont-track-pointer-array. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/83f6f54d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/83f6f54d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/83f6f54d Branch: refs/heads/master Commit: 83f6f54d12a418f5158ee7ee985b54eef8cc1cf0 Parents: 084e4e1 Author: Andrew Or <and...@databricks.com> Authored: Wed Sep 23 19:34:31 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Wed Sep 23 19:34:31 2015 -0700 -- .../unsafe/sort/UnsafeExternalSorter.java | 51 +--- .../sql/execution/UnsafeKVExternalSorter.java | 9 +--- .../UnsafeFixedWidthAggregationMapSuite.scala | 8 +-- 3 files changed, 16 insertions(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/83f6f54d/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java -- diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 14b6aaf..0a311d2 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -159,16 +159,15 @@ public final class UnsafeExternalSorter { /** * Allocates new sort data structures. Called when creating the sorter and after each spill. */ - public void initializeForWriting() throws IOException { + private void initializeForWriting() throws IOException { +// Note: Do not track memory for the pointer array for now because of SPARK-10474. +// In more detail, in TungstenAggregate we only reserve a page, but when we fall back to +// sort-based aggregation we try to acquire a page AND a pointer array, which inevitably +// fails if all other memory is already occupied. It should be safe to not track the array +// because its memory footprint is frequently much smaller than that of a page. This is a +// temporary hack that we should address in 1.6.0. +// TODO: track the pointer array memory! this.writeMetrics = new ShuffleWriteMetrics(); -final long pointerArrayMemory = - UnsafeInMemorySorter.getMemoryRequirementsForPointerArray(initialSize); -final long memoryAcquired = shuffleMemoryManager.tryToAcquire(pointerArrayMemory); -if (memoryAcquired != pointerArrayMemory) { - shuffleMemoryManager.release(memoryAcquired); - throw new IOException("Could not acquire " + pointerArrayMemory + " bytes of memory"); -} - this.inMemSorter = new UnsafeInMemorySorter(taskMemoryManager, recordComparator, prefixComparator, initialSize); this.isInMemSorterExternal = false; @@ -187,14 +186,6 @@ public final class UnsafeExternalSorter { * Sort and spill the current records in response to memory pressure. */ public void spill() throws IOException { -spill(true); - } - - /** - * Sort and spill the current records in response to memory pressure. - * @param shouldInitializeForWriting whether to allocate memory for writing after the spill - */ - public void spill(boolean shouldInitializeForWriting) throws IOException { assert(inMemSorter != null); logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", Thread.currentThread().getId(), @@ -225,9 +216,7 @@ public final class UnsafeExternalSorter { // written to disk. This also counts the space needed to store the sorter's pointer array. taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); -if (shouldInitializeForWriting) { - initi
spark git commit: [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array (round 2)
Repository: spark Updated Branches: refs/heads/branch-1.5 7564c2493 -> 1f47e68f5 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array (round 2) This patch reverts most of the changes in a previous fix #8827. The real cause of the issue is that in `TungstenAggregate`'s prepare method we only reserve 1 page, but later when we switch to sort-based aggregation we try to acquire 1 page AND a pointer array. The longer-term fix should be to reserve also the pointer array, but for now ***we will simply not track the pointer array***. (Note that elsewhere we already don't track the pointer array, e.g. [here](https://github.com/apache/spark/blob/a18208047f06a4244703c17023bb20cbe1f59d73/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java#L88)) Note: This patch reuses the unit test added in #8827 so it doesn't show up in the diff. Author: Andrew Or <and...@databricks.com> Closes #8888 from andrewor14/dont-track-pointer-array. (cherry picked from commit 83f6f54d12a418f5158ee7ee985b54eef8cc1cf0) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1f47e68f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1f47e68f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1f47e68f Branch: refs/heads/branch-1.5 Commit: 1f47e68f56398e4f39b3b00650aea6f71e832115 Parents: 7564c24 Author: Andrew Or <and...@databricks.com> Authored: Wed Sep 23 19:34:31 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Wed Sep 23 19:34:47 2015 -0700 -- .../unsafe/sort/UnsafeExternalSorter.java | 51 +--- .../sql/execution/UnsafeKVExternalSorter.java | 9 +--- .../UnsafeFixedWidthAggregationMapSuite.scala | 8 +-- 3 files changed, 16 insertions(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1f47e68f/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java -- diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 14b6aaf..0a311d2 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -159,16 +159,15 @@ public final class UnsafeExternalSorter { /** * Allocates new sort data structures. Called when creating the sorter and after each spill. */ - public void initializeForWriting() throws IOException { + private void initializeForWriting() throws IOException { +// Note: Do not track memory for the pointer array for now because of SPARK-10474. +// In more detail, in TungstenAggregate we only reserve a page, but when we fall back to +// sort-based aggregation we try to acquire a page AND a pointer array, which inevitably +// fails if all other memory is already occupied. It should be safe to not track the array +// because its memory footprint is frequently much smaller than that of a page. This is a +// temporary hack that we should address in 1.6.0. +// TODO: track the pointer array memory! this.writeMetrics = new ShuffleWriteMetrics(); -final long pointerArrayMemory = - UnsafeInMemorySorter.getMemoryRequirementsForPointerArray(initialSize); -final long memoryAcquired = shuffleMemoryManager.tryToAcquire(pointerArrayMemory); -if (memoryAcquired != pointerArrayMemory) { - shuffleMemoryManager.release(memoryAcquired); - throw new IOException("Could not acquire " + pointerArrayMemory + " bytes of memory"); -} - this.inMemSorter = new UnsafeInMemorySorter(taskMemoryManager, recordComparator, prefixComparator, initialSize); this.isInMemSorterExternal = false; @@ -187,14 +186,6 @@ public final class UnsafeExternalSorter { * Sort and spill the current records in response to memory pressure. */ public void spill() throws IOException { -spill(true); - } - - /** - * Sort and spill the current records in response to memory pressure. - * @param shouldInitializeForWriting whether to allocate memory for writing after the spill - */ - public void spill(boolean shouldInitializeForWriting) throws IOException { assert(inMemSorter != null); logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", Thread.currentThread().getId(), @@ -225,9 +216,7 @@ public final class UnsafeExternalSorter { // written to disk. This also counts the space needed to store
spark git commit: [Minor] style fix for previous commit f24316e
Repository: spark Updated Branches: refs/heads/master f24316e6d -> fd61b0048 [Minor] style fix for previous commit f24316e Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fd61b004 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fd61b004 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fd61b004 Branch: refs/heads/master Commit: fd61b004877ba4d51c95cd0e08f53bffdf106395 Parents: f24316e Author: Andrew OrAuthored: Tue Sep 22 00:05:30 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:05:30 2015 -0700 -- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + 1 file changed, 1 insertion(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fd61b004/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 967fec9..bf3aeb4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -265,6 +265,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val tachyonFolderName = externalBlockStoreFolderName def isLocal: Boolean = (master == "local" || master.startsWith("local[")) + /** * @return true if context is stopped or in the midst of stopping. */ - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10695] [DOCUMENTATION] [MESOS] Fixing incorrect value informati…
Repository: spark Updated Branches: refs/heads/master f3b727c80 -> 0bd0e5bed [SPARK-10695] [DOCUMENTATION] [MESOS] Fixing incorrect value informati⦠â¦on for spark.mesos.constraints parameter. Author: Akash MishraCloses #8816 from SleepyThread/constraint-fix. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0bd0e5be Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0bd0e5be Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0bd0e5be Branch: refs/heads/master Commit: 0bd0e5bed2176b119b3ada590993e153757ea09b Parents: f3b727c Author: Akash Mishra Authored: Tue Sep 22 00:14:27 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:14:27 2015 -0700 -- docs/running-on-mesos.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0bd0e5be/docs/running-on-mesos.md -- diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 460a66f..ec5a44d 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -189,10 +189,10 @@ using `conf.set("spark.cores.max", "10")` (for example). You may also make use of `spark.mesos.constraints` to set attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. {% highlight scala %} -conf.set("spark.mesos.constraints", "tachyon=true;us-east-1=false") +conf.set("spark.mesos.constraints", "tachyon:true;us-east-1:false") {% endhighlight %} -For example, Let's say `spark.mesos.constraints` is set to `tachyon=true;us-east-1=false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. +For example, Let's say `spark.mesos.constraints` is set to `tachyon:true;us-east-1:false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. # Mesos Docker Support - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8567] [SQL] Increase the timeout of o.a.s.sql.hive.HiveSparkSubmitSuite to 5 minutes.
Repository: spark Updated Branches: refs/heads/branch-1.5 d0e6e5312 -> 03215e3e8 [SPARK-8567] [SQL] Increase the timeout of o.a.s.sql.hive.HiveSparkSubmitSuite to 5 minutes. https://issues.apache.org/jira/browse/SPARK-8567 Looks like "SPARK-8368: includes jars passed in through --jars" is pretty flaky now. Based on some history runs, the time spent on a successful run may be from 1.5 minutes to almost 3 minutes. Let's try to increase the timeout and see if we can fix this test. https://amplab.cs.berkeley.edu/jenkins/job/Spark-1.5-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.0,label=spark-test/385/testReport/junit/org.apache.spark.sql.hive/HiveSparkSubmitSuite/SPARK_8368__includes_jars_passed_in_through___jars/history/?start=25 Author: Yin HuaiCloses #8850 from yhuai/SPARK-8567-anotherTry. (cherry picked from commit 4da32bc0e747fefe847bffe493785d4d16069c04) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/03215e3e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/03215e3e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/03215e3e Branch: refs/heads/branch-1.5 Commit: 03215e3e89640b76a343c36b3fbafb89a2a98dc5 Parents: d0e6e53 Author: Yin Huai Authored: Tue Sep 22 00:07:30 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:07:37 2015 -0700 -- .../scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/03215e3e/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index dc2d85f..1d5ee22 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -139,7 +139,7 @@ class HiveSparkSubmitSuite new ProcessOutputCapturer(process.getErrorStream, captureOutput("stderr")).start() try { - val exitCode = failAfter(180.seconds) { process.waitFor() } + val exitCode = failAfter(300.seconds) { process.waitFor() } if (exitCode != 0) { // include logs in output. Note that logging is async and may not have completed // at the time this exception is raised - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10649] [STREAMING] Prevent inheriting job group and irrelevant job description in streaming jobs
Repository: spark Updated Branches: refs/heads/branch-1.5 f83b6e625 -> d0e6e5312 [SPARK-10649] [STREAMING] Prevent inheriting job group and irrelevant job description in streaming jobs **Note that this PR only for branch 1.5. See #8781 for the solution for Spark master.** The job group, and job descriptions information is passed through thread local properties, and get inherited by child threads. In case of spark streaming, the streaming jobs inherit these properties from the thread that called streamingContext.start(). This may not make sense. 1. Job group: This is mainly used for cancelling a group of jobs together. It does not make sense to cancel streaming jobs like this, as the effect will be unpredictable. And its not a valid usecase any way, to cancel a streaming context, call streamingContext.stop() 2. Job description: This is used to pass on nice text descriptions for jobs to show up in the UI. The job description of the thread that calls streamingContext.start() is not useful for all the streaming jobs, as it does not make sense for all of the streaming jobs to have the same description, and the description may or may not be related to streaming. The solution in this PR is meant for the Spark branch 1.5, where local properties are inherited by cloning the properties only when the Spark config `spark.localProperties.clone` is set to `true` (see #8781 for the PR for Spark master branch). Similar to the approach taken by #8721, StreamingContext sets that configuration to true, which makes sure that all subsequent child threads get a cloned copy of the threadlocal properties. This allows the job group and job description to be explicitly removed in the thread that starts the streaming scheduler, so that all the subsequent child threads does not inherit them. Also, the starting is done in a new child thread, so that setting the job group and description for streaming, does not change those properties in the thread that called streamingContext.start(). Author: Tathagata DasCloses #8856 from tdas/SPARK-10649-1.5. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d0e6e531 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d0e6e531 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d0e6e531 Branch: refs/heads/branch-1.5 Commit: d0e6e5312b8b51617b1e7d6f32be50fbe3d48b2a Parents: f83b6e6 Author: Tathagata Das Authored: Tue Sep 22 00:06:18 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:06:18 2015 -0700 -- .../org/apache/spark/util/ThreadUtils.scala | 59 .../apache/spark/util/ThreadUtilsSuite.scala| 24 +++- .../spark/streaming/StreamingContext.scala | 21 ++- .../spark/streaming/StreamingContextSuite.scala | 32 +++ 4 files changed, 132 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d0e6e531/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index ca5624a..22e291a 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -21,6 +21,7 @@ package org.apache.spark.util import java.util.concurrent._ import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} +import scala.util.control.NonFatal import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} @@ -86,4 +87,62 @@ private[spark] object ThreadUtils { val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() Executors.newSingleThreadScheduledExecutor(threadFactory) } + + /** + * Run a piece of code in a new thread and return the result. Exception in the new thread is + * thrown in the caller thread with an adjusted stack trace that removes references to this + * method for clarity. The exception stack traces will be like the following + * + * SomeException: exception-message + * at CallerClass.body-method (sourcefile.scala) + * at ... run in separate thread using org.apache.spark.util.ThreadUtils ... () + * at CallerClass.caller-method (sourcefile.scala) + * ... + */ + def runInNewThread[T]( + threadName: String, + isDaemon: Boolean = true)(body: => T): T = { +@volatile var exception: Option[Throwable] = None +@volatile var result: T = null.asInstanceOf[T] + +val thread = new Thread(threadName) { + override def run(): Unit = { +try { + result = body +} catch { +
spark git commit: [SPARK-10458] [SPARK CORE] Added isStopped() method in SparkContext
Repository: spark Updated Branches: refs/heads/master 1fcefef06 -> f24316e6d [SPARK-10458] [SPARK CORE] Added isStopped() method in SparkContext Added isStopped() method in SparkContext Author: Madhusudanan KandasamyCloses #8749 from kmadhugit/SPARK-10458. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f24316e6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f24316e6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f24316e6 Branch: refs/heads/master Commit: f24316e6d928c263cbf3872edd97982059c3db22 Parents: 1fcefef Author: Madhusudanan Kandasamy Authored: Tue Sep 22 00:03:48 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:03:48 2015 -0700 -- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 1 file changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f24316e6/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ebd8e94..967fec9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -265,6 +265,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val tachyonFolderName = externalBlockStoreFolderName def isLocal: Boolean = (master == "local" || master.startsWith("local[")) + /** + * @return true if context is stopped or in the midst of stopping. + */ + def isStopped: Boolean = stopped.get() // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SQL] [MINOR] map -> foreach.
Repository: spark Updated Branches: refs/heads/branch-1.5 03215e3e8 -> a2b0fee7b [SQL] [MINOR] map -> foreach. DataFrame.explain should use foreach to print the explain content. Author: Reynold XinCloses #8862 from rxin/map-foreach. (cherry picked from commit f3b727c801408b1cd50e5d9463f2fe0fce654a16) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a2b0fee7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a2b0fee7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a2b0fee7 Branch: refs/heads/branch-1.5 Commit: a2b0fee7b10be27727afa599fd460ba77f620ebd Parents: 03215e3 Author: Reynold Xin Authored: Tue Sep 22 00:09:29 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:09:36 2015 -0700 -- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a2b0fee7/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 246eaa8..49b40a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -320,9 +320,8 @@ class DataFrame private[sql]( * @since 1.3.0 */ def explain(extended: Boolean): Unit = { -ExplainCommand( - queryExecution.logical, - extended = extended).queryExecution.executedPlan.executeCollect().map { +val explain = ExplainCommand(queryExecution.logical, extended = extended) +explain.queryExecution.executedPlan.executeCollect().foreach { // scalastyle:off println r => println(r.getString(0)) // scalastyle:on println - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SQL] [MINOR] map -> foreach.
Repository: spark Updated Branches: refs/heads/master 4da32bc0e -> f3b727c80 [SQL] [MINOR] map -> foreach. DataFrame.explain should use foreach to print the explain content. Author: Reynold XinCloses #8862 from rxin/map-foreach. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f3b727c8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f3b727c8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f3b727c8 Branch: refs/heads/master Commit: f3b727c801408b1cd50e5d9463f2fe0fce654a16 Parents: 4da32bc Author: Reynold Xin Authored: Tue Sep 22 00:09:29 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:09:29 2015 -0700 -- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f3b727c8/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index ba94d77..a11140b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -320,9 +320,8 @@ class DataFrame private[sql]( * @since 1.3.0 */ def explain(extended: Boolean): Unit = { -ExplainCommand( - queryExecution.logical, - extended = extended).queryExecution.executedPlan.executeCollect().map { +val explain = ExplainCommand(queryExecution.logical, extended = extended) +explain.queryExecution.executedPlan.executeCollect().foreach { // scalastyle:off println r => println(r.getString(0)) // scalastyle:on println - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10695] [DOCUMENTATION] [MESOS] Fixing incorrect value informati…
Repository: spark Updated Branches: refs/heads/branch-1.5 a2b0fee7b -> 646155e6e [SPARK-10695] [DOCUMENTATION] [MESOS] Fixing incorrect value informati⦠â¦on for spark.mesos.constraints parameter. Author: Akash MishraCloses #8816 from SleepyThread/constraint-fix. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/646155e6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/646155e6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/646155e6 Branch: refs/heads/branch-1.5 Commit: 646155e6ecafc1e0a7c8e6d1457fe64004bcf0a4 Parents: a2b0fee Author: Akash Mishra Authored: Tue Sep 22 00:14:27 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:15:43 2015 -0700 -- docs/running-on-mesos.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/646155e6/docs/running-on-mesos.md -- diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 477d5a5..16cee01 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -187,10 +187,10 @@ using `conf.set("spark.cores.max", "10")` (for example). You may also make use of `spark.mesos.constraints` to set attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. {% highlight scala %} -conf.set("spark.mesos.constraints", "tachyon=true;us-east-1=false") +conf.set("spark.mesos.constraints", "tachyon:true;us-east-1:false") {% endhighlight %} -For example, Let's say `spark.mesos.constraints` is set to `tachyon=true;us-east-1=false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. +For example, Let's say `spark.mesos.constraints` is set to `tachyon:true;us-east-1:false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. # Mesos Docker Support - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8567] [SQL] Increase the timeout of o.a.s.sql.hive.HiveSparkSubmitSuite to 5 minutes.
Repository: spark Updated Branches: refs/heads/master fd61b0048 -> 4da32bc0e [SPARK-8567] [SQL] Increase the timeout of o.a.s.sql.hive.HiveSparkSubmitSuite to 5 minutes. https://issues.apache.org/jira/browse/SPARK-8567 Looks like "SPARK-8368: includes jars passed in through --jars" is pretty flaky now. Based on some history runs, the time spent on a successful run may be from 1.5 minutes to almost 3 minutes. Let's try to increase the timeout and see if we can fix this test. https://amplab.cs.berkeley.edu/jenkins/job/Spark-1.5-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.0,label=spark-test/385/testReport/junit/org.apache.spark.sql.hive/HiveSparkSubmitSuite/SPARK_8368__includes_jars_passed_in_through___jars/history/?start=25 Author: Yin HuaiCloses #8850 from yhuai/SPARK-8567-anotherTry. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4da32bc0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4da32bc0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4da32bc0 Branch: refs/heads/master Commit: 4da32bc0e747fefe847bffe493785d4d16069c04 Parents: fd61b00 Author: Yin Huai Authored: Tue Sep 22 00:07:30 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 00:07:30 2015 -0700 -- .../scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4da32bc0/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 97df249..5f1660b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -139,7 +139,7 @@ class HiveSparkSubmitSuite new ProcessOutputCapturer(process.getErrorStream, captureOutput("stderr")).start() try { - val exitCode = failAfter(180.seconds) { process.waitFor() } + val exitCode = failAfter(300.seconds) { process.waitFor() } if (exitCode != 0) { // include logs in output. Note that logging is async and may not have completed // at the time this exception is raised - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10640] History server fails to parse TaskCommitDenied
Repository: spark Updated Branches: refs/heads/branch-1.5 118ebd405 -> 26187ab74 [SPARK-10640] History server fails to parse TaskCommitDenied ... simply because the code is missing! Author: Andrew Or <and...@databricks.com> Closes #8828 from andrewor14/task-end-reason-json. Conflicts: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/26187ab7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/26187ab7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/26187ab7 Branch: refs/heads/branch-1.5 Commit: 26187ab744687f57f295700141b5c0844949faad Parents: 118ebd4 Author: Andrew Or <and...@databricks.com> Authored: Tue Sep 22 16:35:43 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Tue Sep 22 17:35:05 2015 -0700 -- .../scala/org/apache/spark/TaskEndReason.scala | 6 +- .../scala/org/apache/spark/util/JsonProtocol.scala | 13 + .../org/apache/spark/util/JsonProtocolSuite.scala | 17 + 3 files changed, 35 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/26187ab7/core/src/main/scala/org/apache/spark/TaskEndReason.scala -- diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 7a690df..b50354c 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,13 +17,17 @@ package org.apache.spark -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} +import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils +// == +// NOTE: new task end reasons MUST be accompanied with serialization logic in util.JsonProtocol! +// == + /** * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry http://git-wip-us.apache.org/repos/asf/spark/blob/26187ab7/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f742c39..311bb59 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -364,6 +364,10 @@ private[spark] object JsonProtocol { ("Metrics" -> metrics) case ExecutorLostFailure(executorId) => ("Executor ID" -> executorId) + case taskCommitDenied: TaskCommitDenied => +("Job ID" -> taskCommitDenied.jobID) ~ +("Partition ID" -> taskCommitDenied.partitionID) ~ +("Attempt Number" -> taskCommitDenied.attemptNumber) case _ => Utils.emptyJson } ("Reason" -> reason) ~ json @@ -769,6 +773,7 @@ private[spark] object JsonProtocol { val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) val taskResultLost = Utils.getFormattedClassName(TaskResultLost) val taskKilled = Utils.getFormattedClassName(TaskKilled) +val taskCommitDenied = Utils.getFormattedClassName(TaskCommitDenied) val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) val unknownReason = Utils.getFormattedClassName(UnknownReason) @@ -793,6 +798,14 @@ private[spark] object JsonProtocol { ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics, None) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled + case `taskCommitDenied` => +// Unfortunately, the `TaskCommitDenied` message was introduced in 1.3.0 but the JSON +// de/serialization logic was not added until 1.5.1. To provide backward compatibility +// for reading those logs, we need to provide default values for all the fields. +val jobId = Utils.jsonOption(json \ "Job ID").map(_.extract[Int]).getOrElse(-1) +val partitionId = Utils.jsonOption(json \ "Partition ID").map(_.extract[Int]).getOrElse(-1) +val attemptNo = Utils.js
spark git commit: [SPARK-10640] History server fails to parse TaskCommitDenied
Repository: spark Updated Branches: refs/heads/master a96ba40f7 -> 61d4c07f4 [SPARK-10640] History server fails to parse TaskCommitDenied ... simply because the code is missing! Author: Andrew Or <and...@databricks.com> Closes #8828 from andrewor14/task-end-reason-json. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/61d4c07f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/61d4c07f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/61d4c07f Branch: refs/heads/master Commit: 61d4c07f4becb42f054e588be56ed13239644410 Parents: a96ba40 Author: Andrew Or <and...@databricks.com> Authored: Tue Sep 22 16:35:43 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Tue Sep 22 16:35:43 2015 -0700 -- .../scala/org/apache/spark/TaskEndReason.scala | 6 +- .../scala/org/apache/spark/util/JsonProtocol.scala | 13 + .../org/apache/spark/util/JsonProtocolSuite.scala | 17 + 3 files changed, 35 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/61d4c07f/core/src/main/scala/org/apache/spark/TaskEndReason.scala -- diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 7137246..9335c5f 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,13 +17,17 @@ package org.apache.spark -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} +import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils +// == +// NOTE: new task end reasons MUST be accompanied with serialization logic in util.JsonProtocol! +// == + /** * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry http://git-wip-us.apache.org/repos/asf/spark/blob/61d4c07f/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 99614a7..40729fa 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -362,6 +362,10 @@ private[spark] object JsonProtocol { ("Stack Trace" -> stackTrace) ~ ("Full Stack Trace" -> exceptionFailure.fullStackTrace) ~ ("Metrics" -> metrics) + case taskCommitDenied: TaskCommitDenied => +("Job ID" -> taskCommitDenied.jobID) ~ +("Partition ID" -> taskCommitDenied.partitionID) ~ +("Attempt Number" -> taskCommitDenied.attemptNumber) case ExecutorLostFailure(executorId, isNormalExit) => ("Executor ID" -> executorId) ~ ("Normal Exit" -> isNormalExit) @@ -770,6 +774,7 @@ private[spark] object JsonProtocol { val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) val taskResultLost = Utils.getFormattedClassName(TaskResultLost) val taskKilled = Utils.getFormattedClassName(TaskKilled) +val taskCommitDenied = Utils.getFormattedClassName(TaskCommitDenied) val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) val unknownReason = Utils.getFormattedClassName(UnknownReason) @@ -794,6 +799,14 @@ private[spark] object JsonProtocol { ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics, None) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled + case `taskCommitDenied` => +// Unfortunately, the `TaskCommitDenied` message was introduced in 1.3.0 but the JSON +// de/serialization logic was not added until 1.5.1. To provide backward compatibility +// for reading those logs, we need to provide default values for all the fields. +val jobId = Utils.jsonOption(json \ "Job ID").map(_.extract[Int]).getOrElse(-1) +val partitionId = Utils.jsonOption(json \ "Partition ID").map(_.extract[Int]).getOrElse(-1) +val attemptNo = Utils.jsonOption(json \ "Attempt Number").map(_.e
spark git commit: [SPARK-10640] History server fails to parse TaskCommitDenied
Repository: spark Updated Branches: refs/heads/branch-1.5 3339916ef -> 5ffd0841e [SPARK-10640] History server fails to parse TaskCommitDenied ... simply because the code is missing! Author: Andrew Or <and...@databricks.com> Closes #8828 from andrewor14/task-end-reason-json. Conflicts: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5ffd0841 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5ffd0841 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5ffd0841 Branch: refs/heads/branch-1.5 Commit: 5ffd0841e016301807b0a008af7c3346e9f59e7a Parents: 3339916 Author: Andrew Or <and...@databricks.com> Authored: Tue Sep 22 16:35:43 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Tue Sep 22 16:52:47 2015 -0700 -- .../scala/org/apache/spark/TaskEndReason.scala | 6 +- .../org/apache/spark/util/JsonProtocol.scala| 13 .../apache/spark/util/JsonProtocolSuite.scala | 22 3 files changed, 40 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5ffd0841/core/src/main/scala/org/apache/spark/TaskEndReason.scala -- diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 7a690df..b50354c 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,13 +17,17 @@ package org.apache.spark -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} +import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils +// == +// NOTE: new task end reasons MUST be accompanied with serialization logic in util.JsonProtocol! +// == + /** * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry http://git-wip-us.apache.org/repos/asf/spark/blob/5ffd0841/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f742c39..311bb59 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -364,6 +364,10 @@ private[spark] object JsonProtocol { ("Metrics" -> metrics) case ExecutorLostFailure(executorId) => ("Executor ID" -> executorId) + case taskCommitDenied: TaskCommitDenied => +("Job ID" -> taskCommitDenied.jobID) ~ +("Partition ID" -> taskCommitDenied.partitionID) ~ +("Attempt Number" -> taskCommitDenied.attemptNumber) case _ => Utils.emptyJson } ("Reason" -> reason) ~ json @@ -769,6 +773,7 @@ private[spark] object JsonProtocol { val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) val taskResultLost = Utils.getFormattedClassName(TaskResultLost) val taskKilled = Utils.getFormattedClassName(TaskKilled) +val taskCommitDenied = Utils.getFormattedClassName(TaskCommitDenied) val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) val unknownReason = Utils.getFormattedClassName(UnknownReason) @@ -793,6 +798,14 @@ private[spark] object JsonProtocol { ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics, None) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled + case `taskCommitDenied` => +// Unfortunately, the `TaskCommitDenied` message was introduced in 1.3.0 but the JSON +// de/serialization logic was not added until 1.5.1. To provide backward compatibility +// for reading those logs, we need to provide default values for all the fields. +val jobId = Utils.jsonOption(json \ "Job ID").map(_.extract[Int]).getOrElse(-1) +val partitionId = Utils.jsonOption(json \ "Partition ID").map(_.extract[Int]).getOrElse(-1) +val attemptNo = Utils.js
spark git commit: Revert "[SPARK-10640] History server fails to parse TaskCommitDenied"
Repository: spark Updated Branches: refs/heads/branch-1.5 5ffd0841e -> 118ebd405 Revert "[SPARK-10640] History server fails to parse TaskCommitDenied" This reverts commit 5ffd0841e016301807b0a008af7c3346e9f59e7a. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/118ebd40 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/118ebd40 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/118ebd40 Branch: refs/heads/branch-1.5 Commit: 118ebd405a34acedb32e9f3d1cf7b5a835e17dbb Parents: 5ffd084 Author: Andrew OrAuthored: Tue Sep 22 17:10:58 2015 -0700 Committer: Andrew Or Committed: Tue Sep 22 17:10:58 2015 -0700 -- .../scala/org/apache/spark/TaskEndReason.scala | 6 +- .../org/apache/spark/util/JsonProtocol.scala| 13 .../apache/spark/util/JsonProtocolSuite.scala | 22 3 files changed, 1 insertion(+), 40 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/118ebd40/core/src/main/scala/org/apache/spark/TaskEndReason.scala -- diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index b50354c..7a690df 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,17 +17,13 @@ package org.apache.spark -import java.io.{ObjectInputStream, ObjectOutputStream} +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils -// == -// NOTE: new task end reasons MUST be accompanied with serialization logic in util.JsonProtocol! -// == - /** * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry http://git-wip-us.apache.org/repos/asf/spark/blob/118ebd40/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 311bb59..f742c39 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -364,10 +364,6 @@ private[spark] object JsonProtocol { ("Metrics" -> metrics) case ExecutorLostFailure(executorId) => ("Executor ID" -> executorId) - case taskCommitDenied: TaskCommitDenied => -("Job ID" -> taskCommitDenied.jobID) ~ -("Partition ID" -> taskCommitDenied.partitionID) ~ -("Attempt Number" -> taskCommitDenied.attemptNumber) case _ => Utils.emptyJson } ("Reason" -> reason) ~ json @@ -773,7 +769,6 @@ private[spark] object JsonProtocol { val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) val taskResultLost = Utils.getFormattedClassName(TaskResultLost) val taskKilled = Utils.getFormattedClassName(TaskKilled) -val taskCommitDenied = Utils.getFormattedClassName(TaskCommitDenied) val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) val unknownReason = Utils.getFormattedClassName(UnknownReason) @@ -798,14 +793,6 @@ private[spark] object JsonProtocol { ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics, None) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled - case `taskCommitDenied` => -// Unfortunately, the `TaskCommitDenied` message was introduced in 1.3.0 but the JSON -// de/serialization logic was not added until 1.5.1. To provide backward compatibility -// for reading those logs, we need to provide default values for all the fields. -val jobId = Utils.jsonOption(json \ "Job ID").map(_.extract[Int]).getOrElse(-1) -val partitionId = Utils.jsonOption(json \ "Partition ID").map(_.extract[Int]).getOrElse(-1) -val attemptNo = Utils.jsonOption(json \ "Attempt Number").map(_.extract[Int]).getOrElse(-1) -TaskCommitDenied(jobId, partitionId, attemptNo) case `executorLostFailure` => val executorId = Utils.jsonOption(json \ "Executor ID").map(_.extract[String]) ExecutorLostFailure(executorId.getOrElse("Unknown"))
spark git commit: [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array
Repository: spark Updated Branches: refs/heads/branch-1.5 b3f1e6533 -> 49355d0e0 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array When `TungstenAggregation` hits memory pressure, it switches from hash-based to sort-based aggregation in-place. However, in the process we try to allocate the pointer array for writing to the new `UnsafeExternalSorter` *before* actually freeing the memory from the hash map. This lead to the following exception: ``` java.io.IOException: Could not acquire 65536 bytes of memory at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.initializeForWriting(UnsafeExternalSorter.java:169) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:220) at org.apache.spark.sql.execution.UnsafeKVExternalSorter.(UnsafeKVExternalSorter.java:126) at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:257) at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:435) ``` Author: Andrew Or <and...@databricks.com> Closes #8827 from andrewor14/allocate-pointer-array. (cherry picked from commit 7ff8d68cc19299e16dedfd819b9e96480fa6cf44) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/49355d0e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/49355d0e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/49355d0e Branch: refs/heads/branch-1.5 Commit: 49355d0e032cfe82b907e6cb45c0b894387ba46b Parents: b3f1e65 Author: Andrew Or <and...@databricks.com> Authored: Fri Sep 18 23:58:25 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Fri Sep 18 23:58:36 2015 -0700 -- .../unsafe/sort/UnsafeExternalSorter.java | 14 +- .../sql/execution/UnsafeKVExternalSorter.java | 8 +++- .../UnsafeFixedWidthAggregationMapSuite.scala | 49 +++- 3 files changed, 66 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/49355d0e/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java -- diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index fc364e0..14b6aaf 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -159,7 +159,7 @@ public final class UnsafeExternalSorter { /** * Allocates new sort data structures. Called when creating the sorter and after each spill. */ - private void initializeForWriting() throws IOException { + public void initializeForWriting() throws IOException { this.writeMetrics = new ShuffleWriteMetrics(); final long pointerArrayMemory = UnsafeInMemorySorter.getMemoryRequirementsForPointerArray(initialSize); @@ -187,6 +187,14 @@ public final class UnsafeExternalSorter { * Sort and spill the current records in response to memory pressure. */ public void spill() throws IOException { +spill(true); + } + + /** + * Sort and spill the current records in response to memory pressure. + * @param shouldInitializeForWriting whether to allocate memory for writing after the spill + */ + public void spill(boolean shouldInitializeForWriting) throws IOException { assert(inMemSorter != null); logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", Thread.currentThread().getId(), @@ -217,7 +225,9 @@ public final class UnsafeExternalSorter { // written to disk. This also counts the space needed to store the sorter's pointer array. taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); -initializeForWriting(); +if (shouldInitializeForWriting) { + initializeForWriting(); +} } /** http://git-wip-us.apache.org/repos/asf/spark/blob/49355d0e/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java -- diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 7db6b7f..b81f67a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/
spark git commit: [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array
Repository: spark Updated Branches: refs/heads/master 22be2ae14 -> 7ff8d68cc [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array When `TungstenAggregation` hits memory pressure, it switches from hash-based to sort-based aggregation in-place. However, in the process we try to allocate the pointer array for writing to the new `UnsafeExternalSorter` *before* actually freeing the memory from the hash map. This lead to the following exception: ``` java.io.IOException: Could not acquire 65536 bytes of memory at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.initializeForWriting(UnsafeExternalSorter.java:169) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:220) at org.apache.spark.sql.execution.UnsafeKVExternalSorter.(UnsafeKVExternalSorter.java:126) at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:257) at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:435) ``` Author: Andrew Or <and...@databricks.com> Closes #8827 from andrewor14/allocate-pointer-array. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7ff8d68c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7ff8d68c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7ff8d68c Branch: refs/heads/master Commit: 7ff8d68cc19299e16dedfd819b9e96480fa6cf44 Parents: 22be2ae Author: Andrew Or <and...@databricks.com> Authored: Fri Sep 18 23:58:25 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Fri Sep 18 23:58:25 2015 -0700 -- .../unsafe/sort/UnsafeExternalSorter.java | 14 +- .../sql/execution/UnsafeKVExternalSorter.java | 8 +++- .../UnsafeFixedWidthAggregationMapSuite.scala | 49 +++- 3 files changed, 66 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7ff8d68c/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java -- diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index fc364e0..14b6aaf 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -159,7 +159,7 @@ public final class UnsafeExternalSorter { /** * Allocates new sort data structures. Called when creating the sorter and after each spill. */ - private void initializeForWriting() throws IOException { + public void initializeForWriting() throws IOException { this.writeMetrics = new ShuffleWriteMetrics(); final long pointerArrayMemory = UnsafeInMemorySorter.getMemoryRequirementsForPointerArray(initialSize); @@ -187,6 +187,14 @@ public final class UnsafeExternalSorter { * Sort and spill the current records in response to memory pressure. */ public void spill() throws IOException { +spill(true); + } + + /** + * Sort and spill the current records in response to memory pressure. + * @param shouldInitializeForWriting whether to allocate memory for writing after the spill + */ + public void spill(boolean shouldInitializeForWriting) throws IOException { assert(inMemSorter != null); logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", Thread.currentThread().getId(), @@ -217,7 +225,9 @@ public final class UnsafeExternalSorter { // written to disk. This also counts the space needed to store the sorter's pointer array. taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); -initializeForWriting(); +if (shouldInitializeForWriting) { + initializeForWriting(); +} } /** http://git-wip-us.apache.org/repos/asf/spark/blob/7ff8d68c/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java -- diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 7db6b7f..b81f67a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -85,6 +85,7 @@ public final class UnsafeKVExternalSorter { // We will use th
spark git commit: [SPARK-10548] [SPARK-10563] [SQL] Fix concurrent SQL executions
Repository: spark Updated Branches: refs/heads/master be52faa7c -> b6e998634 [SPARK-10548] [SPARK-10563] [SQL] Fix concurrent SQL executions *Note: this is for master branch only.* The fix for branch-1.5 is at #8721. The query execution ID is currently passed from a thread to its children, which is not the intended behavior. This led to `IllegalArgumentException: spark.sql.execution.id is already set` when running queries in parallel, e.g.: ``` (1 to 100).par.foreach { _ => sc.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count() } ``` The cause is `SparkContext`'s local properties are inherited by default. This patch adds a way to exclude keys we don't want to be inherited, and makes SQL go through that code path. Author: Andrew Or <and...@databricks.com> Closes #8710 from andrewor14/concurrent-sql-executions. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b6e99863 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b6e99863 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b6e99863 Branch: refs/heads/master Commit: b6e998634e05db0bb6267173e7b28f885c808c16 Parents: be52faa Author: Andrew Or <and...@databricks.com> Authored: Tue Sep 15 16:45:47 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Tue Sep 15 16:45:47 2015 -0700 -- .../scala/org/apache/spark/SparkContext.scala | 9 +- .../scala/org/apache/spark/ThreadingSuite.scala | 65 +--- .../spark/sql/execution/SQLExecutionSuite.scala | 101 +++ 3 files changed, 132 insertions(+), 43 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b6e99863/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dee6091..a2f34ea 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -33,6 +33,7 @@ import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} import scala.util.control.NonFatal +import org.apache.commons.lang.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, @@ -347,8 +348,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack - private val localProperties = new InheritableThreadLocal[Properties] { -override protected def childValue(parent: Properties): Properties = new Properties(parent) + protected[spark] val localProperties = new InheritableThreadLocal[Properties] { +override protected def childValue(parent: Properties): Properties = { + // Note: make a clone such that changes in the parent properties aren't reflected in + // the those of the children threads, which has confusing semantics (SPARK-10563). + SerializationUtils.clone(parent).asInstanceOf[Properties] +} override protected def initialValue(): Properties = new Properties() } http://git-wip-us.apache.org/repos/asf/spark/blob/b6e99863/core/src/test/scala/org/apache/spark/ThreadingSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index a96a4ce..54c131c 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -147,7 +147,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { }.start() } sem.acquire(2) -throwable.foreach { t => throw t } +throwable.foreach { t => throw improveStackTrace(t) } if (ThreadingSuiteState.failed.get()) { logError("Waited 1 second without seeing runningThreads = 4 (it was " + ThreadingSuiteState.runningThreads.get() + "); failing test") @@ -178,7 +178,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { threads.foreach(_.start()) sem.acquire(5) -throwable.foreach { t => throw t } +throwable.foreach { t => throw improveStackTrace(t) } assert(sc.getLocalProperty("test") === null) } @@ -207,58 +207,41 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { threads.foreach(_.start()) s
spark git commit: [SPARK-10548] [SPARK-10563] [SQL] Fix concurrent SQL executions / branch-1.5
Repository: spark Updated Branches: refs/heads/branch-1.5 7286c2ba6 -> 997be78c3 [SPARK-10548] [SPARK-10563] [SQL] Fix concurrent SQL executions / branch-1.5 *Note: this is for branch-1.5 only* This is the same as #8710 but affects only SQL. The more general fix for SPARK-10563 is considered risky to backport into a maintenance release, so it is disabled by default and enabled only in SQL. Author: Andrew Or <and...@databricks.com> Closes #8721 from andrewor14/concurrent-sql-executions-1.5 and squashes the following commits: 3b9b462 [Andrew Or] Merge branch 'branch-1.5' of github.com:apache/spark into concurrent-sql-executions-1.5 4435db7 [Andrew Or] Clone properties only for SQL for backward compatibility 0b7e5ab [Andrew Or] Clone parent local properties on inherit Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/997be78c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/997be78c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/997be78c Branch: refs/heads/branch-1.5 Commit: 997be78c3a291f86e348d626ae89745ead625251 Parents: 7286c2b Author: Andrew Or <and...@databricks.com> Authored: Tue Sep 15 16:46:34 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Tue Sep 15 16:46:34 2015 -0700 -- .../scala/org/apache/spark/SparkContext.scala | 13 ++- .../scala/org/apache/spark/ThreadingSuite.scala | 66 +--- .../scala/org/apache/spark/sql/SQLContext.scala | 6 ++ .../spark/sql/execution/SQLExecutionSuite.scala | 101 +++ 4 files changed, 143 insertions(+), 43 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/997be78c/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index bfeec7c..8a12f7e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -33,6 +33,7 @@ import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} import scala.util.control.NonFatal +import org.apache.commons.lang.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, @@ -347,8 +348,16 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack - private val localProperties = new InheritableThreadLocal[Properties] { -override protected def childValue(parent: Properties): Properties = new Properties(parent) + protected[spark] val localProperties = new InheritableThreadLocal[Properties] { +override protected def childValue(parent: Properties): Properties = { + // Note: make a clone such that changes in the parent properties aren't reflected in + // the those of the children threads, which has confusing semantics (SPARK-10563). + if (conf.get("spark.localProperties.clone", "false").toBoolean) { +SerializationUtils.clone(parent).asInstanceOf[Properties] + } else { +new Properties(parent) + } +} override protected def initialValue(): Properties = new Properties() } http://git-wip-us.apache.org/repos/asf/spark/blob/997be78c/core/src/test/scala/org/apache/spark/ThreadingSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index a96a4ce..6176c15 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -147,7 +147,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { }.start() } sem.acquire(2) -throwable.foreach { t => throw t } +throwable.foreach { t => throw improveStackTrace(t) } if (ThreadingSuiteState.failed.get()) { logError("Waited 1 second without seeing runningThreads = 4 (it was " + ThreadingSuiteState.runningThreads.get() + "); failing test") @@ -178,7 +178,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { threads.foreach(_.start()) sem.acquire(5) -throwable.foreach { t => throw t } +throwable.foreach { t => throw improveStackTrace(t) } assert(sc.getLocalProperty("test") === null) } @@ -2
spark git commit: [SPARK-10613] [SPARK-10624] [SQL] Reduce LocalNode tests dependency on SQLContext
Repository: spark Updated Branches: refs/heads/master 38700ea40 -> 35a19f335 [SPARK-10613] [SPARK-10624] [SQL] Reduce LocalNode tests dependency on SQLContext Instead of relying on `DataFrames` to verify our answers, we can just use simple arrays. This significantly simplifies the test logic for `LocalNode`s and reduces a lot of code duplicated from `SparkPlanTest`. This also fixes an additional issue [SPARK-10624](https://issues.apache.org/jira/browse/SPARK-10624) where the output of `TakeOrderedAndProjectNode` is not actually ordered. Author: Andrew Or <and...@databricks.com> Closes #8764 from andrewor14/sql-local-tests-cleanup. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/35a19f33 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/35a19f33 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/35a19f33 Branch: refs/heads/master Commit: 35a19f3357d2ec017cfefb90f1018403e9617de4 Parents: 38700ea Author: Andrew Or <and...@databricks.com> Authored: Tue Sep 15 17:24:32 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Tue Sep 15 17:24:32 2015 -0700 -- .../spark/sql/execution/local/LocalNode.scala | 8 +- .../spark/sql/execution/local/SampleNode.scala | 16 +- .../local/TakeOrderedAndProjectNode.scala | 2 +- .../spark/sql/execution/SparkPlanTest.scala | 2 +- .../spark/sql/execution/local/DummyNode.scala | 68 .../sql/execution/local/ExpandNodeSuite.scala | 54 ++-- .../sql/execution/local/FilterNodeSuite.scala | 34 +- .../sql/execution/local/HashJoinNodeSuite.scala | 141 - .../execution/local/IntersectNodeSuite.scala| 24 +- .../sql/execution/local/LimitNodeSuite.scala| 28 +- .../sql/execution/local/LocalNodeSuite.scala| 73 + .../sql/execution/local/LocalNodeTest.scala | 165 +++--- .../local/NestedLoopJoinNodeSuite.scala | 316 +++ .../sql/execution/local/ProjectNodeSuite.scala | 39 ++- .../sql/execution/local/SampleNodeSuite.scala | 35 +- .../local/TakeOrderedAndProjectNodeSuite.scala | 50 ++- .../sql/execution/local/UnionNodeSuite.scala| 49 +-- 17 files changed, 468 insertions(+), 636 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/35a19f33/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala index 569cff5..f96b62a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{SQLConf, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.types.StructType /** @@ -33,18 +33,14 @@ import org.apache.spark.sql.types.StructType * Before consuming the iterator, open function must be called. * After consuming the iterator, close function must be called. */ -abstract class LocalNode(conf: SQLConf) extends TreeNode[LocalNode] with Logging { +abstract class LocalNode(conf: SQLConf) extends QueryPlan[LocalNode] with Logging { protected val codegenEnabled: Boolean = conf.codegenEnabled protected val unsafeEnabled: Boolean = conf.unsafeEnabled - lazy val schema: StructType = StructType.fromAttributes(output) - private[this] lazy val isTesting: Boolean = sys.props.contains("spark.testing") - def output: Seq[Attribute] - /** * Called before open(). Prepare can be used to reserve memory needed. It must NOT consume * any input data. http://git-wip-us.apache.org/repos/asf/spark/blob/35a19f33/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SampleNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SampleNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SampleNode.scala index abf3df1..7937008 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SampleNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SampleNode.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.execution.local -import java.util.Random - import org.apache.spark.sql.SQLConf import org.apache.spark.s
spark git commit: [SPARK-10612] [SQL] Add prepare to LocalNode.
Repository: spark Updated Branches: refs/heads/master b6e998634 -> a63cdc769 [SPARK-10612] [SQL] Add prepare to LocalNode. The idea is that we should separate the function call that does memory reservation (i.e. prepare) from the function call that consumes the input (e.g. open()), so all operators can be a chance to reserve memory before they are all consumed. Author: Reynold XinCloses #8761 from rxin/SPARK-10612. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a63cdc76 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a63cdc76 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a63cdc76 Branch: refs/heads/master Commit: a63cdc769f511e98b38c3318bcc732c9a6c76c22 Parents: b6e9986 Author: Reynold Xin Authored: Tue Sep 15 16:53:27 2015 -0700 Committer: Andrew Or Committed: Tue Sep 15 16:53:27 2015 -0700 -- .../org/apache/spark/sql/execution/local/LocalNode.scala | 8 1 file changed, 8 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a63cdc76/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala index 9840080..569cff5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala @@ -46,6 +46,14 @@ abstract class LocalNode(conf: SQLConf) extends TreeNode[LocalNode] with Logging def output: Seq[Attribute] /** + * Called before open(). Prepare can be used to reserve memory needed. It must NOT consume + * any input data. + * + * Implementations of this must also call the `prepare()` function of its children. + */ + def prepare(): Unit = children.foreach(_.prepare()) + + /** * Initializes the iterator state. Must be called before calling `next()`. * * Implementations of this must also call the `open()` function of its children. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10575] [SPARK CORE] Wrapped RDD.takeSample with Scope
Repository: spark Updated Branches: refs/heads/master a63cdc769 -> 99ecfa594 [SPARK-10575] [SPARK CORE] Wrapped RDD.takeSample with Scope Remove return statements in RDD.takeSample and wrap it withScope Author: vinodkcAuthor: vinodkc Author: Vinod K C Closes #8730 from vinodkc/fix_takesample_return. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/99ecfa59 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/99ecfa59 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/99ecfa59 Branch: refs/heads/master Commit: 99ecfa5945aedaa71765ecf5cce59964ae52eebe Parents: a63cdc7 Author: vinodkc Authored: Tue Sep 15 17:01:10 2015 -0700 Committer: Andrew Or Committed: Tue Sep 15 17:01:39 2015 -0700 -- .../main/scala/org/apache/spark/rdd/RDD.scala | 68 +--- 1 file changed, 31 insertions(+), 37 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/99ecfa59/core/src/main/scala/org/apache/spark/rdd/RDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 7dd2bc5..a56e542 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -469,50 +469,44 @@ abstract class RDD[T: ClassTag]( * @param seed seed for the random number generator * @return sample of specified size in an array */ - // TODO: rewrite this without return statements so we can wrap it in a scope def takeSample( withReplacement: Boolean, num: Int, - seed: Long = Utils.random.nextLong): Array[T] = { + seed: Long = Utils.random.nextLong): Array[T] = withScope { val numStDev = 10.0 -if (num < 0) { - throw new IllegalArgumentException("Negative number of elements requested") -} else if (num == 0) { - return new Array[T](0) -} - -val initialCount = this.count() -if (initialCount == 0) { - return new Array[T](0) -} - -val maxSampleSize = Int.MaxValue - (numStDev * math.sqrt(Int.MaxValue)).toInt -if (num > maxSampleSize) { - throw new IllegalArgumentException("Cannot support a sample size > Int.MaxValue - " + -s"$numStDev * math.sqrt(Int.MaxValue)") -} - -val rand = new Random(seed) -if (!withReplacement && num >= initialCount) { - return Utils.randomizeInPlace(this.collect(), rand) -} - -val fraction = SamplingUtils.computeFractionForSampleSize(num, initialCount, - withReplacement) - -var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() +require(num >= 0, "Negative number of elements requested") +require(num <= (Int.MaxValue - (numStDev * math.sqrt(Int.MaxValue)).toInt), + "Cannot support a sample size > Int.MaxValue - " + + s"$numStDev * math.sqrt(Int.MaxValue)") -// If the first sample didn't turn out large enough, keep trying to take samples; -// this shouldn't happen often because we use a big multiplier for the initial size -var numIters = 0 -while (samples.length < num) { - logWarning(s"Needed to re-sample due to insufficient sample size. Repeat #$numIters") - samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() - numIters += 1 +if (num == 0) { + new Array[T](0) +} else { + val initialCount = this.count() + if (initialCount == 0) { +new Array[T](0) + } else { +val rand = new Random(seed) +if (!withReplacement && num >= initialCount) { + Utils.randomizeInPlace(this.collect(), rand) +} else { + val fraction = SamplingUtils.computeFractionForSampleSize(num, initialCount, +withReplacement) + var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() + + // If the first sample didn't turn out large enough, keep trying to take samples; + // this shouldn't happen often because we use a big multiplier for the initial size + var numIters = 0 + while (samples.length < num) { +logWarning(s"Needed to re-sample due to insufficient sample size. Repeat #$numIters") +samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() +numIters += 1 + } + Utils.randomizeInPlace(samples, rand).take(num) +} + } } - -Utils.randomizeInPlace(samples, rand).take(num) } /** - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
spark git commit: [SPARK-10576] [BUILD] Move .java files out of src/main/scala
Repository: spark Updated Branches: refs/heads/master 16b6d1861 -> 4e2242bb4 [SPARK-10576] [BUILD] Move .java files out of src/main/scala Move .java files in `src/main/scala` to `src/main/java` root, except for `package-info.java` (to stay next to package.scala) Author: Sean OwenCloses #8736 from srowen/SPARK-10576. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4e2242bb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4e2242bb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4e2242bb Branch: refs/heads/master Commit: 4e2242bb41dda922573046c00c5142745632f95f Parents: 16b6d18 Author: Sean Owen Authored: Mon Sep 14 15:03:51 2015 -0700 Committer: Andrew Or Committed: Mon Sep 14 15:03:51 2015 -0700 -- .../apache/spark/annotation/AlphaComponent.java | 33 + .../apache/spark/annotation/DeveloperApi.java | 35 ++ .../apache/spark/annotation/Experimental.java | 36 ++ .../org/apache/spark/annotation/Private.java| 41 +++ .../apache/spark/annotation/AlphaComponent.java | 33 - .../apache/spark/annotation/DeveloperApi.java | 35 -- .../apache/spark/annotation/Experimental.java | 36 -- .../org/apache/spark/annotation/Private.java| 41 --- .../org/apache/spark/graphx/TripletFields.java | 72 .../spark/graphx/impl/EdgeActiveness.java | 34 + .../org/apache/spark/graphx/TripletFields.java | 72 .../spark/graphx/impl/EdgeActiveness.java | 34 - .../spark/sql/types/SQLUserDefinedType.java | 45 .../spark/sql/types/SQLUserDefinedType.java | 45 .../spark/streaming/StreamingContextState.java | 45 .../spark/streaming/StreamingContextState.java | 45 16 files changed, 341 insertions(+), 341 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4e2242bb/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java -- diff --git a/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java new file mode 100644 index 000..db7b25c --- /dev/null +++ b/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java @@ -0,0 +1,33 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * A new component of Spark which may have unstable API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: AlphaComponent ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, +ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface AlphaComponent {} http://git-wip-us.apache.org/repos/asf/spark/blob/4e2242bb/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java -- diff --git a/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java new file mode 100644 index 000..0ecef6d --- /dev/null +++ b/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java @@ -0,0 +1,35 @@ +/* + * 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
spark git commit: [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test (round 2)
Repository: spark Updated Branches: refs/heads/branch-1.5 eb0cb25bb -> 5db51f911 [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test (round 2) This is a follow-up patch to #8723. I missed one case there. Author: Andrew Or <and...@databricks.com> Closes #8727 from andrewor14/fix-threading-suite. (cherry picked from commit 7b6c856367b9c36348e80e83959150da9656c4dd) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5db51f91 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5db51f91 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5db51f91 Branch: refs/heads/branch-1.5 Commit: 5db51f91131e867fd27cb6b0457a2698925cd920 Parents: eb0cb25 Author: Andrew Or <and...@databricks.com> Authored: Mon Sep 14 15:09:43 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Mon Sep 14 15:09:50 2015 -0700 -- .../scala/org/apache/spark/ThreadingSuite.scala | 23 +--- 1 file changed, 15 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5db51f91/core/src/test/scala/org/apache/spark/ThreadingSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index cda2b24..a96a4ce 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -147,12 +147,12 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { }.start() } sem.acquire(2) +throwable.foreach { t => throw t } if (ThreadingSuiteState.failed.get()) { logError("Waited 1 second without seeing runningThreads = 4 (it was " + ThreadingSuiteState.runningThreads.get() + "); failing test") fail("One or more threads didn't see runningThreads = 4") } -throwable.foreach { t => throw t } } test("set local properties in different thread") { @@ -178,8 +178,8 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { threads.foreach(_.start()) sem.acquire(5) -assert(sc.getLocalProperty("test") === null) throwable.foreach { t => throw t } +assert(sc.getLocalProperty("test") === null) } test("set and get local properties in parent-children thread") { @@ -207,15 +207,16 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { threads.foreach(_.start()) sem.acquire(5) +throwable.foreach { t => throw t } assert(sc.getLocalProperty("test") === "parent") assert(sc.getLocalProperty("Foo") === null) -throwable.foreach { t => throw t } } test("mutations to local properties should not affect submitted jobs (SPARK-6629)") { val jobStarted = new Semaphore(0) val jobEnded = new Semaphore(0) @volatile var jobResult: JobResult = null +var throwable: Option[Throwable] = None sc = new SparkContext("local", "test") sc.setJobGroup("originalJobGroupId", "description") @@ -232,14 +233,19 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { // Create a new thread which will inherit the current thread's properties val thread = new Thread() { override def run(): Unit = { -assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "originalJobGroupId") -// Sleeps for a total of 10 seconds, but allows cancellation to interrupt the task try { - sc.parallelize(1 to 100).foreach { x => -Thread.sleep(100) + assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "originalJobGroupId") + // Sleeps for a total of 10 seconds, but allows cancellation to interrupt the task + try { +sc.parallelize(1 to 100).foreach { x => + Thread.sleep(100) +} + } catch { +case s: SparkException => // ignored so that we don't print noise in test logs } } catch { - case s: SparkException => // ignored so that we don't print noise in test logs + case t: Throwable => +throwable = Some(t) } } } @@ -252,6 +258,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { // modification of the properties object
spark git commit: [SPARK-10549] scala 2.11 spark on yarn with security - Repl doesn't work
Repository: spark Updated Branches: refs/heads/master 4e2242bb4 -> ffbbc2c58 [SPARK-10549] scala 2.11 spark on yarn with security - Repl doesn't work Make this lazy so that it can set the yarn mode before creating the securityManager. Author: Tom GravesAuthor: Thomas Graves Closes #8719 from tgravescs/SPARK-10549. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ffbbc2c5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ffbbc2c5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ffbbc2c5 Branch: refs/heads/master Commit: ffbbc2c58b9bf1e2abc2ea797feada6821ab4de8 Parents: 4e2242b Author: Tom Graves Authored: Mon Sep 14 15:05:19 2015 -0700 Committer: Andrew Or Committed: Mon Sep 14 15:05:19 2015 -0700 -- repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ffbbc2c5/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala -- diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index be31eb2..627148d 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -35,7 +35,8 @@ object Main extends Logging { s.processArguments(List("-Yrepl-class-based", "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", "-classpath", getAddedJars.mkString(File.pathSeparator)), true) - val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf)) + // the creation of SecurityManager has to be lazy so SPARK_YARN_MODE is set if needed + lazy val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf)) var sparkContext: SparkContext = _ var sqlContext: SQLContext = _ var interp = new SparkILoop // this is a public var because tests reset it. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10543] [CORE] Peak Execution Memory Quantile should be Per-task Basis
Repository: spark Updated Branches: refs/heads/master ffbbc2c58 -> fd1e8cddf [SPARK-10543] [CORE] Peak Execution Memory Quantile should be Per-task Basis Read `PEAK_EXECUTION_MEMORY` using `update` to get per task partial value instead of cumulative value. I tested with this workload: ```scala val size = 1000 val repetitions = 10 val data = sc.parallelize(1 to size, 5).map(x => (util.Random.nextInt(size / repetitions),util.Random.nextDouble)).toDF("key", "value") val res = data.toDF.groupBy("key").agg(sum("value")).count ``` Before: ![image](https://cloud.githubusercontent.com/assets/4317392/9828197/07dd6874-58b8-11e5-9bd9-6ba927c38b26.png) After: ![image](https://cloud.githubusercontent.com/assets/4317392/9828151/a5ddff30-58b7-11e5-8d31-eda5dc4eae79.png) Tasks view: ![image](https://cloud.githubusercontent.com/assets/4317392/9828199/17dc2b84-58b8-11e5-92a8-be89ce4d29d1.png) cc andrewor14 I appreciate if you can give feedback on this since I think you introduced display of this metric. Author: Forest Fang <forest.f...@outlook.com> Closes #8726 from saurfang/stagepage. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fd1e8cdd Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fd1e8cdd Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fd1e8cdd Branch: refs/heads/master Commit: fd1e8cddf2635c55fec2ac6e1f1c221c9685af0f Parents: ffbbc2c Author: Forest Fang <forest.f...@outlook.com> Authored: Mon Sep 14 15:07:13 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Mon Sep 14 15:07:13 2015 -0700 -- .../org/apache/spark/ui/jobs/StagePage.scala| 2 +- .../org/apache/spark/ui/StagePageSuite.scala| 29 +++- 2 files changed, 23 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fd1e8cdd/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 4adc659..2b71f55 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -368,7 +368,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val peakExecutionMemory = validTasks.map { case TaskUIData(info, _, _) => info.accumulables .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.value.toLong } + .map { acc => acc.update.getOrElse("0").toLong } .getOrElse(0L) .toDouble } http://git-wip-us.apache.org/repos/asf/spark/blob/fd1e8cdd/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 3388c6d..86699e7 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -23,7 +23,7 @@ import scala.xml.Node import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite, Success} +import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.ui.jobs.{JobProgressListener, StagePage, StagesTab} @@ -47,6 +47,14 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { assert(html3.contains(targetString)) } + test("SPARK-10543: peak execution memory should be per-task rather than cumulative") { +val unsafeConf = "spark.sql.unsafe.enabled" +val conf = new SparkConf(false).set(unsafeConf, "true") +val html = renderStagePage(conf).toString().toLowerCase +// verify min/25/50/75/max show task value not cumulative values +assert(html.contains("10.0 b" * 5)) + } + /** * Render a stage page started with the given conf and return the HTML. * This also runs a dummy stage to populate the page with useful content. @@ -67,12 +75,19 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { // Simulate a stage in job progress listener val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") -val taskInfo = new TaskInfo(0, 0, 0, 0, "0", "localhost", TaskLocality.ANY, false) -jobListener.onStage
spark git commit: [SPARK-10543] [CORE] Peak Execution Memory Quantile should be Per-task Basis
Repository: spark Updated Branches: refs/heads/branch-1.5 0e1c9d9ff -> eb0cb25bb [SPARK-10543] [CORE] Peak Execution Memory Quantile should be Per-task Basis Read `PEAK_EXECUTION_MEMORY` using `update` to get per task partial value instead of cumulative value. I tested with this workload: ```scala val size = 1000 val repetitions = 10 val data = sc.parallelize(1 to size, 5).map(x => (util.Random.nextInt(size / repetitions),util.Random.nextDouble)).toDF("key", "value") val res = data.toDF.groupBy("key").agg(sum("value")).count ``` Before: ![image](https://cloud.githubusercontent.com/assets/4317392/9828197/07dd6874-58b8-11e5-9bd9-6ba927c38b26.png) After: ![image](https://cloud.githubusercontent.com/assets/4317392/9828151/a5ddff30-58b7-11e5-8d31-eda5dc4eae79.png) Tasks view: ![image](https://cloud.githubusercontent.com/assets/4317392/9828199/17dc2b84-58b8-11e5-92a8-be89ce4d29d1.png) cc andrewor14 I appreciate if you can give feedback on this since I think you introduced display of this metric. Author: Forest Fang <forest.f...@outlook.com> Closes #8726 from saurfang/stagepage. (cherry picked from commit fd1e8cddf2635c55fec2ac6e1f1c221c9685af0f) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/eb0cb25b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/eb0cb25b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/eb0cb25b Branch: refs/heads/branch-1.5 Commit: eb0cb25bb81a5aa271d2a0266e5a31b36d1fc071 Parents: 0e1c9d9 Author: Forest Fang <forest.f...@outlook.com> Authored: Mon Sep 14 15:07:13 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Mon Sep 14 15:07:24 2015 -0700 -- .../org/apache/spark/ui/jobs/StagePage.scala| 2 +- .../org/apache/spark/ui/StagePageSuite.scala| 29 +++- 2 files changed, 23 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/eb0cb25b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala -- diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 4adc659..2b71f55 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -368,7 +368,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val peakExecutionMemory = validTasks.map { case TaskUIData(info, _, _) => info.accumulables .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.value.toLong } + .map { acc => acc.update.getOrElse("0").toLong } .getOrElse(0L) .toDouble } http://git-wip-us.apache.org/repos/asf/spark/blob/eb0cb25b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 3388c6d..86699e7 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -23,7 +23,7 @@ import scala.xml.Node import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite, Success} +import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.ui.jobs.{JobProgressListener, StagePage, StagesTab} @@ -47,6 +47,14 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { assert(html3.contains(targetString)) } + test("SPARK-10543: peak execution memory should be per-task rather than cumulative") { +val unsafeConf = "spark.sql.unsafe.enabled" +val conf = new SparkConf(false).set(unsafeConf, "true") +val html = renderStagePage(conf).toString().toLowerCase +// verify min/25/50/75/max show task value not cumulative values +assert(html.contains("10.0 b" * 5)) + } + /** * Render a stage page started with the given conf and return the HTML. * This also runs a dummy stage to populate the page with useful content. @@ -67,12 +75,19 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { // Simulate a stage in job progress listener val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details"
spark git commit: [SPARK-9996] [SPARK-9997] [SQL] Add local expand and NestedLoopJoin operators
Repository: spark Updated Branches: refs/heads/master 64f04154e -> 217e49644 [SPARK-9996] [SPARK-9997] [SQL] Add local expand and NestedLoopJoin operators This PR is in conflict with #8535 and #8573. Will update this one when they are merged. Author: zsxwingCloses #8642 from zsxwing/expand-nest-join. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/217e4964 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/217e4964 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/217e4964 Branch: refs/heads/master Commit: 217e496f4e07b894b1bca768a0cbbe799ea0 Parents: 64f0415 Author: zsxwing Authored: Mon Sep 14 15:00:27 2015 -0700 Committer: Andrew Or Committed: Mon Sep 14 15:00:27 2015 -0700 -- .../spark/sql/execution/local/ExpandNode.scala | 60 + .../spark/sql/execution/local/LocalNode.scala | 55 - .../execution/local/NestedLoopJoinNode.scala| 156 .../sql/execution/local/ExpandNodeSuite.scala | 51 .../sql/execution/local/HashJoinNodeSuite.scala | 14 -- .../sql/execution/local/LocalNodeTest.scala | 14 ++ .../local/NestedLoopJoinNodeSuite.scala | 239 +++ 7 files changed, 574 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/217e4964/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ExpandNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ExpandNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ExpandNode.scala new file mode 100644 index 000..2aff156 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ExpandNode.scala @@ -0,0 +1,60 @@ +/* +* 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.sql.execution.local + +import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Projection} + +case class ExpandNode( +conf: SQLConf, +projections: Seq[Seq[Expression]], +output: Seq[Attribute], +child: LocalNode) extends UnaryLocalNode(conf) { + + assert(projections.size > 0) + + private[this] var result: InternalRow = _ + private[this] var idx: Int = _ + private[this] var input: InternalRow = _ + private[this] var groups: Array[Projection] = _ + + override def open(): Unit = { +child.open() +groups = projections.map(ee => newProjection(ee, child.output)).toArray +idx = groups.length + } + + override def next(): Boolean = { +if (idx >= groups.length) { + if (child.next()) { +input = child.fetch() +idx = 0 + } else { +return false + } +} +result = groups(idx)(input) +idx += 1 +true + } + + override def fetch(): InternalRow = result + + override def close(): Unit = child.close() +} http://git-wip-us.apache.org/repos/asf/spark/blob/217e4964/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala index e540ef8..9840080 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala @@ -23,7 +23,7 @@ import org.apache.spark.Logging import org.apache.spark.sql.{SQLConf, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types.StructType
spark git commit: [SPARK-10549] scala 2.11 spark on yarn with security - Repl doesn't work
Repository: spark Updated Branches: refs/heads/branch-1.5 a0d564a10 -> 0e1c9d9ff [SPARK-10549] scala 2.11 spark on yarn with security - Repl doesn't work Make this lazy so that it can set the yarn mode before creating the securityManager. Author: Tom GravesAuthor: Thomas Graves Closes #8719 from tgravescs/SPARK-10549. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0e1c9d9f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0e1c9d9f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0e1c9d9f Branch: refs/heads/branch-1.5 Commit: 0e1c9d9ff7f9c8f9ae40179c19abbd1d211d142e Parents: a0d564a Author: Tom Graves Authored: Mon Sep 14 15:05:19 2015 -0700 Committer: Andrew Or Committed: Mon Sep 14 15:05:52 2015 -0700 -- repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0e1c9d9f/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala -- diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index be31eb2..627148d 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -35,7 +35,8 @@ object Main extends Logging { s.processArguments(List("-Yrepl-class-based", "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", "-classpath", getAddedJars.mkString(File.pathSeparator)), true) - val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf)) + // the creation of SecurityManager has to be lazy so SPARK_YARN_MODE is set if needed + lazy val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf)) var sparkContext: SparkContext = _ var sqlContext: SQLContext = _ var interp = new SparkILoop // this is a public var because tests reset it. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10594] [YARN] Remove reference to --num-executors, add --properties-file
Repository: spark Updated Branches: refs/heads/master 217e49644 -> 16b6d1861 [SPARK-10594] [YARN] Remove reference to --num-executors, add --properties-file `ApplicationMaster` no longer has the `--num-executors` flag, and had an undocumented `--properties-file` configuration option. cc srowen Author: Erick TryzelaarCloses #8754 from erickt/master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/16b6d186 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/16b6d186 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/16b6d186 Branch: refs/heads/master Commit: 16b6d18613e150c7038c613992d80a7828413e66 Parents: 217e496 Author: Erick Tryzelaar Authored: Mon Sep 14 15:02:38 2015 -0700 Committer: Andrew Or Committed: Mon Sep 14 15:02:38 2015 -0700 -- .../org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/16b6d186/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala -- diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index b084124..17d9943 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -105,9 +105,9 @@ class ApplicationMasterArguments(val args: Array[String]) { | place on the PYTHONPATH for Python apps. | --args ARGS Arguments to be passed to your application's main class. | Multiple invocations are possible, each will be passed in order. - | --num-executors NUMNumber of executors to start (Default: 2) | --executor-cores NUM Number of cores for the executors (Default: 1) | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) + | --properties-file FILE Path to a custom Spark properties file. """.stripMargin) // scalastyle:on println System.exit(exitCode) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test (round 2)
Repository: spark Updated Branches: refs/heads/master fd1e8cddf -> 7b6c85636 [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test (round 2) This is a follow-up patch to #8723. I missed one case there. Author: Andrew Or <and...@databricks.com> Closes #8727 from andrewor14/fix-threading-suite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7b6c8563 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7b6c8563 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7b6c8563 Branch: refs/heads/master Commit: 7b6c856367b9c36348e80e83959150da9656c4dd Parents: fd1e8cd Author: Andrew Or <and...@databricks.com> Authored: Mon Sep 14 15:09:43 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Mon Sep 14 15:09:43 2015 -0700 -- .../scala/org/apache/spark/ThreadingSuite.scala | 23 +--- 1 file changed, 15 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7b6c8563/core/src/test/scala/org/apache/spark/ThreadingSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index cda2b24..a96a4ce 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -147,12 +147,12 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { }.start() } sem.acquire(2) +throwable.foreach { t => throw t } if (ThreadingSuiteState.failed.get()) { logError("Waited 1 second without seeing runningThreads = 4 (it was " + ThreadingSuiteState.runningThreads.get() + "); failing test") fail("One or more threads didn't see runningThreads = 4") } -throwable.foreach { t => throw t } } test("set local properties in different thread") { @@ -178,8 +178,8 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { threads.foreach(_.start()) sem.acquire(5) -assert(sc.getLocalProperty("test") === null) throwable.foreach { t => throw t } +assert(sc.getLocalProperty("test") === null) } test("set and get local properties in parent-children thread") { @@ -207,15 +207,16 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { threads.foreach(_.start()) sem.acquire(5) +throwable.foreach { t => throw t } assert(sc.getLocalProperty("test") === "parent") assert(sc.getLocalProperty("Foo") === null) -throwable.foreach { t => throw t } } test("mutations to local properties should not affect submitted jobs (SPARK-6629)") { val jobStarted = new Semaphore(0) val jobEnded = new Semaphore(0) @volatile var jobResult: JobResult = null +var throwable: Option[Throwable] = None sc = new SparkContext("local", "test") sc.setJobGroup("originalJobGroupId", "description") @@ -232,14 +233,19 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { // Create a new thread which will inherit the current thread's properties val thread = new Thread() { override def run(): Unit = { -assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "originalJobGroupId") -// Sleeps for a total of 10 seconds, but allows cancellation to interrupt the task try { - sc.parallelize(1 to 100).foreach { x => -Thread.sleep(100) + assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "originalJobGroupId") + // Sleeps for a total of 10 seconds, but allows cancellation to interrupt the task + try { +sc.parallelize(1 to 100).foreach { x => + Thread.sleep(100) +} + } catch { +case s: SparkException => // ignored so that we don't print noise in test logs } } catch { - case s: SparkException => // ignored so that we don't print noise in test logs + case t: Throwable => +throwable = Some(t) } } } @@ -252,6 +258,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { // modification of the properties object should not affect the properties of running jobs sc.cancelJobGroup("originalJobGroupId") jobEnded.tryAcquire(10, TimeUnit.SECONDS) +throwable.foreach { t => throw
spark git commit: [SPARK-9990] [SQL] Local hash join follow-ups
Repository: spark Updated Branches: refs/heads/master e626ac5f5 -> c2af42b5f [SPARK-9990] [SQL] Local hash join follow-ups 1. Hide `LocalNodeIterator` behind the `LocalNode#asIterator` method 2. Add tests for this Author: Andrew Or <and...@databricks.com> Closes #8708 from andrewor14/local-hash-join-follow-up. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c2af42b5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c2af42b5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c2af42b5 Branch: refs/heads/master Commit: c2af42b5f32287ff595ad027a8191d4b75702d8d Parents: e626ac5 Author: Andrew Or <and...@databricks.com> Authored: Fri Sep 11 15:01:37 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Fri Sep 11 15:01:37 2015 -0700 -- .../sql/execution/joins/HashedRelation.scala| 7 +- .../sql/execution/local/HashJoinNode.scala | 3 +- .../spark/sql/execution/local/LocalNode.scala | 4 +- .../sql/execution/local/LocalNodeSuite.scala| 116 +++ 4 files changed, 125 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c2af42b5/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 0cff21c..bc255b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -25,7 +25,8 @@ import org.apache.spark.shuffle.ShuffleMemoryManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkSqlSerializer -import org.apache.spark.sql.execution.metric.LongSQLMetric +import org.apache.spark.sql.execution.local.LocalNode +import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.unsafe.memory.{MemoryLocation, ExecutorMemoryManager, MemoryAllocator, TaskMemoryManager} @@ -113,6 +114,10 @@ final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[InternalR private[execution] object HashedRelation { + def apply(localNode: LocalNode, keyGenerator: Projection): HashedRelation = { +apply(localNode.asIterator, SQLMetrics.nullLongMetric, keyGenerator) + } + def apply( input: Iterator[InternalRow], numInputRows: LongSQLMetric, http://git-wip-us.apache.org/repos/asf/spark/blob/c2af42b5/sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala index a3e68d6..e7b24e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala @@ -75,8 +75,7 @@ case class HashJoinNode( override def open(): Unit = { buildNode.open() -hashed = HashedRelation.apply( - new LocalNodeIterator(buildNode), SQLMetrics.nullLongMetric, buildSideKeyGenerator) +hashed = HashedRelation(buildNode, buildSideKeyGenerator) streamedNode.open() joinRow = new JoinedRow resultProjection = { http://git-wip-us.apache.org/repos/asf/spark/blob/c2af42b5/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala index a2c275d..e540ef8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala @@ -77,7 +77,7 @@ abstract class LocalNode(conf: SQLConf) extends TreeNode[LocalNode] with Logging /** * Returns the content of the iterator from the beginning to the end in the form of a Scala Seq. */ - def collect(): Seq[Row] = { + final def collect(): Seq[Row] = { val converter = CatalystTypeConverters.createToScalaConverter(StructType.fromAttributes(output)) val result = new scala.collection.mutable.ArrayBuffer[Row] open() @@ -140,7 +140,7 @@ abstract class BinaryLocalNode(c
spark git commit: [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test
Repository: spark Updated Branches: refs/heads/master c2af42b5f -> d74c6a143 [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test This commit ensures if an assertion fails within a thread, it will ultimately fail the test. Otherwise we end up potentially masking real bugs by not propagating assertion failures properly. Author: Andrew Or <and...@databricks.com> Closes #8723 from andrewor14/fix-threading-suite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d74c6a14 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d74c6a14 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d74c6a14 Branch: refs/heads/master Commit: d74c6a143cbd060c25bf14a8d306841b3ec55d03 Parents: c2af42b Author: Andrew Or <and...@databricks.com> Authored: Fri Sep 11 15:02:59 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Fri Sep 11 15:02:59 2015 -0700 -- .../scala/org/apache/spark/ThreadingSuite.scala | 68 +--- 1 file changed, 45 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d74c6a14/core/src/test/scala/org/apache/spark/ThreadingSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 48509f0..cda2b24 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -119,23 +119,30 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { val nums = sc.parallelize(1 to 2, 2) val sem = new Semaphore(0) ThreadingSuiteState.clear() +var throwable: Option[Throwable] = None for (i <- 0 until 2) { new Thread { override def run() { - val ans = nums.map(number => { -val running = ThreadingSuiteState.runningThreads -running.getAndIncrement() -val time = System.currentTimeMillis() -while (running.get() != 4 && System.currentTimeMillis() < time + 1000) { - Thread.sleep(100) -} -if (running.get() != 4) { - ThreadingSuiteState.failed.set(true) -} -number - }).collect() - assert(ans.toList === List(1, 2)) - sem.release() + try { +val ans = nums.map(number => { + val running = ThreadingSuiteState.runningThreads + running.getAndIncrement() + val time = System.currentTimeMillis() + while (running.get() != 4 && System.currentTimeMillis() < time + 1000) { +Thread.sleep(100) + } + if (running.get() != 4) { +ThreadingSuiteState.failed.set(true) + } + number +}).collect() +assert(ans.toList === List(1, 2)) + } catch { +case t: Throwable => + throwable = Some(t) + } finally { +sem.release() + } } }.start() } @@ -145,18 +152,25 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { ThreadingSuiteState.runningThreads.get() + "); failing test") fail("One or more threads didn't see runningThreads = 4") } +throwable.foreach { t => throw t } } test("set local properties in different thread") { sc = new SparkContext("local", "test") val sem = new Semaphore(0) - +var throwable: Option[Throwable] = None val threads = (1 to 5).map { i => new Thread() { override def run() { - sc.setLocalProperty("test", i.toString) - assert(sc.getLocalProperty("test") === i.toString) - sem.release() + try { +sc.setLocalProperty("test", i.toString) +assert(sc.getLocalProperty("test") === i.toString) + } catch { +case t: Throwable => + throwable = Some(t) + } finally { +sem.release() + } } } } @@ -165,20 +179,27 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { sem.acquire(5) assert(sc.getLocalProperty("test") === null) +throwable.foreach { t => throw t } } test("set and get local properties in parent-children thread") { sc = new SparkContext("local", "test") sc.setLocalProperty("test", "parent") val s
spark git commit: [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test
Repository: spark Updated Branches: refs/heads/branch-1.5 7f10bd620 -> fcb24387a [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the test This commit ensures if an assertion fails within a thread, it will ultimately fail the test. Otherwise we end up potentially masking real bugs by not propagating assertion failures properly. Author: Andrew Or <and...@databricks.com> Closes #8723 from andrewor14/fix-threading-suite. (cherry picked from commit d74c6a143cbd060c25bf14a8d306841b3ec55d03) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fcb24387 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fcb24387 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fcb24387 Branch: refs/heads/branch-1.5 Commit: fcb24387add76529707f27b86b57f79de1110f24 Parents: 7f10bd6 Author: Andrew Or <and...@databricks.com> Authored: Fri Sep 11 15:02:59 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Fri Sep 11 15:03:07 2015 -0700 -- .../scala/org/apache/spark/ThreadingSuite.scala | 68 +--- 1 file changed, 45 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/fcb24387/core/src/test/scala/org/apache/spark/ThreadingSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 48509f0..cda2b24 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -119,23 +119,30 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { val nums = sc.parallelize(1 to 2, 2) val sem = new Semaphore(0) ThreadingSuiteState.clear() +var throwable: Option[Throwable] = None for (i <- 0 until 2) { new Thread { override def run() { - val ans = nums.map(number => { -val running = ThreadingSuiteState.runningThreads -running.getAndIncrement() -val time = System.currentTimeMillis() -while (running.get() != 4 && System.currentTimeMillis() < time + 1000) { - Thread.sleep(100) -} -if (running.get() != 4) { - ThreadingSuiteState.failed.set(true) -} -number - }).collect() - assert(ans.toList === List(1, 2)) - sem.release() + try { +val ans = nums.map(number => { + val running = ThreadingSuiteState.runningThreads + running.getAndIncrement() + val time = System.currentTimeMillis() + while (running.get() != 4 && System.currentTimeMillis() < time + 1000) { +Thread.sleep(100) + } + if (running.get() != 4) { +ThreadingSuiteState.failed.set(true) + } + number +}).collect() +assert(ans.toList === List(1, 2)) + } catch { +case t: Throwable => + throwable = Some(t) + } finally { +sem.release() + } } }.start() } @@ -145,18 +152,25 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { ThreadingSuiteState.runningThreads.get() + "); failing test") fail("One or more threads didn't see runningThreads = 4") } +throwable.foreach { t => throw t } } test("set local properties in different thread") { sc = new SparkContext("local", "test") val sem = new Semaphore(0) - +var throwable: Option[Throwable] = None val threads = (1 to 5).map { i => new Thread() { override def run() { - sc.setLocalProperty("test", i.toString) - assert(sc.getLocalProperty("test") === i.toString) - sem.release() + try { +sc.setLocalProperty("test", i.toString) +assert(sc.getLocalProperty("test") === i.toString) + } catch { +case t: Throwable => + throwable = Some(t) + } finally { +sem.release() + } } } } @@ -165,20 +179,27 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { sem.acquire(5) assert(sc.getLocalProperty("test") === null) +throwable.foreach { t => throw t } } test("set and get local properties in parent-children thread") { sc = new
spark git commit: [SPARK-9992] [SPARK-9994] [SPARK-9998] [SQL] Implement the local TopK, sample and intersect operators
Repository: spark Updated Branches: refs/heads/master 1eede3b25 -> e626ac5f5 [SPARK-9992] [SPARK-9994] [SPARK-9998] [SQL] Implement the local TopK, sample and intersect operators This PR is in conflict with #8535. I will update this one when #8535 gets merged. Author: zsxwingCloses #8573 from zsxwing/more-local-operators. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e626ac5f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e626ac5f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e626ac5f Branch: refs/heads/master Commit: e626ac5f5c27dcc74113070f2fec03682bcd12bd Parents: 1eede3b Author: zsxwing Authored: Fri Sep 11 15:00:13 2015 -0700 Committer: Andrew Or Committed: Fri Sep 11 15:00:13 2015 -0700 -- .../spark/sql/execution/basicOperators.scala| 2 +- .../sql/execution/local/IntersectNode.scala | 63 +++ .../spark/sql/execution/local/LocalNode.scala | 5 ++ .../spark/sql/execution/local/SampleNode.scala | 82 .../local/TakeOrderedAndProjectNode.scala | 73 + .../execution/local/IntersectNodeSuite.scala| 35 + .../sql/execution/local/SampleNodeSuite.scala | 40 ++ .../local/TakeOrderedAndProjectNodeSuite.scala | 54 + 8 files changed, 353 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e626ac5f/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 3f68b05..bf6d44c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -138,7 +138,7 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { * will be ub - lb. * @param withReplacement Whether to sample with replacement. * @param seed the random seed - * @param child the QueryPlan + * @param child the SparkPlan */ @DeveloperApi case class Sample( http://git-wip-us.apache.org/repos/asf/spark/blob/e626ac5f/sql/core/src/main/scala/org/apache/spark/sql/execution/local/IntersectNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/IntersectNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/IntersectNode.scala new file mode 100644 index 000..740d485 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/IntersectNode.scala @@ -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 +* (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.sql.execution.local + +import scala.collection.mutable + +import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class IntersectNode(conf: SQLConf, left: LocalNode, right: LocalNode) + extends BinaryLocalNode(conf) { + + override def output: Seq[Attribute] = left.output + + private[this] var leftRows: mutable.HashSet[InternalRow] = _ + + private[this] var currentRow: InternalRow = _ + + override def open(): Unit = { +left.open() +leftRows = mutable.HashSet[InternalRow]() +while (left.next()) { + leftRows += left.fetch().copy() +} +left.close() +right.open() + } + + override def next(): Boolean = { +currentRow = null +while (currentRow == null && right.next()) { + currentRow = right.fetch() + if (!leftRows.contains(currentRow)) { +currentRow = null + } +} +currentRow != null + } + + override def fetch(): InternalRow = currentRow + + override def close(): Unit = { +left.close() +right.close() + } + +}
spark git commit: [SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.cores
Repository: spark Updated Branches: refs/heads/master af3bc59d1 -> f0562e8cd [SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.cores This is a regression introduced in #4960, this commit fixes it and adds a test. tnachen andrewor14 please review, this should be an easy one. Author: Iulian Dragos <jagua...@gmail.com> Closes #8653 from dragos/issue/mesos/fine-grained-maxExecutorCores. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f0562e8c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f0562e8c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f0562e8c Branch: refs/heads/master Commit: f0562e8cdbab7ce40f3186da98595312252f8b5c Parents: af3bc59 Author: Iulian Dragos <jagua...@gmail.com> Authored: Thu Sep 10 12:00:21 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Thu Sep 10 12:00:21 2015 -0700 -- .../cluster/mesos/MesosSchedulerBackend.scala | 3 +- .../mesos/MesosSchedulerBackendSuite.scala | 33 +++- 2 files changed, 33 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f0562e8c/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 18da6d2..8edf700 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -32,7 +32,6 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils - /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks @@ -127,7 +126,7 @@ private[spark] class MesosSchedulerBackend( } val builder = MesosExecutorInfo.newBuilder() val (resourcesAfterCpu, usedCpuResources) = - partitionResources(availableResources, "cpus", scheduler.CPUS_PER_TASK) + partitionResources(availableResources, "cpus", mesosExecutorCores) val (resourcesAfterMem, usedMemResources) = partitionResources(resourcesAfterCpu.asJava, "mem", calculateTotalMemory(sc)) http://git-wip-us.apache.org/repos/asf/spark/blob/f0562e8c/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 319b317..c4dc560 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -42,6 +42,38 @@ import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSui class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { + test("Use configured mesosExecutor.cores for ExecutorInfo") { +val mesosExecutorCores = 3 +val conf = new SparkConf +conf.set("spark.mesos.mesosExecutor.cores", mesosExecutorCores.toString) + +val listenerBus = mock[LiveListenerBus] +listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + +val sc = mock[SparkContext] +when(sc.getSparkHome()).thenReturn(Option("/spark-home")) + +when(sc.conf).thenReturn(conf) +when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) +when(sc.executorMemory).thenReturn(100) +when(sc.listenerBus).thenReturn(listenerBus) +val taskScheduler = mock[TaskSchedulerImpl] +when(taskScheduler.CPUS_PER_TASK).thenReturn(2) + +val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + +val resources = Arrays.asList( + mesosSchedulerBackend.createResource("cpus", 4), + mesosSchedulerBackend.createResource("mem", 1024)) +// uri is null. +val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") +val executorResources = executorInfo.getResourcesList +val cpus = executorResources.asScala.find(_.getName.equals("cpus
spark git commit: [SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.cores
Repository: spark Updated Branches: refs/heads/branch-1.5 bff05aaa0 -> 8cf16191f [SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.cores This is a regression introduced in #4960, this commit fixes it and adds a test. tnachen andrewor14 please review, this should be an easy one. Author: Iulian Dragos <jagua...@gmail.com> Closes #8653 from dragos/issue/mesos/fine-grained-maxExecutorCores. (cherry picked from commit f0562e8cdbab7ce40f3186da98595312252f8b5c) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8cf16191 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8cf16191 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8cf16191 Branch: refs/heads/branch-1.5 Commit: 8cf16191f3e3b0562f22d44b0381bea35ba511d7 Parents: bff05aa Author: Iulian Dragos <jagua...@gmail.com> Authored: Thu Sep 10 12:00:21 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Thu Sep 10 12:00:30 2015 -0700 -- .../cluster/mesos/MesosSchedulerBackend.scala | 3 +- .../mesos/MesosSchedulerBackendSuite.scala | 33 +++- 2 files changed, 33 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8cf16191/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 5c20606..033454e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -32,7 +32,6 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils - /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks @@ -127,7 +126,7 @@ private[spark] class MesosSchedulerBackend( } val builder = MesosExecutorInfo.newBuilder() val (resourcesAfterCpu, usedCpuResources) = - partitionResources(availableResources, "cpus", scheduler.CPUS_PER_TASK) + partitionResources(availableResources, "cpus", mesosExecutorCores) val (resourcesAfterMem, usedMemResources) = partitionResources(resourcesAfterCpu, "mem", calculateTotalMemory(sc)) http://git-wip-us.apache.org/repos/asf/spark/blob/8cf16191/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 5ed30f6..3291783 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -41,6 +41,38 @@ import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSui class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { + test("Use configured mesosExecutor.cores for ExecutorInfo") { +val mesosExecutorCores = 3 +val conf = new SparkConf +conf.set("spark.mesos.mesosExecutor.cores", mesosExecutorCores.toString) + +val listenerBus = mock[LiveListenerBus] +listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + +val sc = mock[SparkContext] +when(sc.getSparkHome()).thenReturn(Option("/spark-home")) + +when(sc.conf).thenReturn(conf) +when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) +when(sc.executorMemory).thenReturn(100) +when(sc.listenerBus).thenReturn(listenerBus) +val taskScheduler = mock[TaskSchedulerImpl] +when(taskScheduler.CPUS_PER_TASK).thenReturn(2) + +val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + +val resources = Arrays.asList( + mesosSchedulerBackend.createResource("cpus", 4), + mesosSchedulerBackend.createResource("mem", 1024)) +// uri is null. +val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") +val
spark git commit: [SPARK-10514] [MESOS] waiting for min no of total cores acquired by Spark by implementing the sufficientResourcesRegistered method
Repository: spark Updated Branches: refs/heads/master f0562e8cd -> a5ef2d060 [SPARK-10514] [MESOS] waiting for min no of total cores acquired by Spark by implementing the sufficientResourcesRegistered method spark.scheduler.minRegisteredResourcesRatio configuration parameter works for YARN mode but not for Mesos Coarse grained mode. If the parameter specified default value of 0 will be set for spark.scheduler.minRegisteredResourcesRatio in base class and this method will always return true. There are no existing test for YARN mode too. Hence not added test for the same. Author: Akash MishraCloses #8672 from SleepyThread/master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a5ef2d06 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a5ef2d06 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a5ef2d06 Branch: refs/heads/master Commit: a5ef2d0600d5e23ca05fabc1005bb81e5ada0727 Parents: f0562e8 Author: Akash Mishra Authored: Thu Sep 10 12:03:11 2015 -0700 Committer: Andrew Or Committed: Thu Sep 10 12:04:02 2015 -0700 -- .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 4 docs/configuration.md | 5 +++-- 2 files changed, 7 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a5ef2d06/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 65df887..65cb501 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -222,6 +222,10 @@ private[spark] class CoarseMesosSchedulerBackend( markRegistered() } + override def sufficientResourcesRegistered(): Boolean = { +totalCoresAcquired >= maxCores * minRegisteredRatio + } + override def disconnected(d: SchedulerDriver) {} override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} http://git-wip-us.apache.org/repos/asf/spark/blob/a5ef2d06/docs/configuration.md -- diff --git a/docs/configuration.md b/docs/configuration.md index 0b1a273..1a701f1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1112,10 +1112,11 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.minRegisteredResourcesRatio - 0.8 for YARN mode; 0.0 otherwise + 0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained mode The minimum ratio of registered resources (registered resources / total expected resources) -(resources are executors in yarn mode, CPU cores in standalone mode) +(resources are executors in yarn mode, CPU cores in standalone mode and Mesos coarsed-grained + mode ['spark.cores.max' value is total expected resources for Mesos coarse-grained mode] ) to wait for before scheduling begins. Specified as a double between 0.0 and 1.0. Regardless of whether the minimum ratio of resources has been reached, the maximum amount of time it will wait before scheduling begins is controlled by config - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10466] [SQL] UnsafeRow SerDe exception with data spill
Repository: spark Updated Branches: refs/heads/master 49da38e5f -> e04811137 [SPARK-10466] [SQL] UnsafeRow SerDe exception with data spill Data Spill with UnsafeRow causes assert failure. ``` java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:165) at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2.writeKey(UnsafeRowSerializer.scala:75) at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:180) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:688) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:687) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:687) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:683) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.util.collection.ExternalSorter.writePartitionedFile(ExternalSorter.scala:683) at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:80) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:88) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) ``` To reproduce that with code (thanks andrewor14): ```scala bin/spark-shell --master local --conf spark.shuffle.memoryFraction=0.005 --conf spark.shuffle.sort.bypassMergeThreshold=0 sc.parallelize(1 to 2 * 1000 * 1000, 10) .map { i => (i, i) }.toDF("a", "b").groupBy("b").avg().count() ``` Author: Cheng Hao <hao.ch...@intel.com> Closes #8635 from chenghao-intel/unsafe_spill. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e0481113 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e0481113 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e0481113 Branch: refs/heads/master Commit: e04811137680f937669cdcc78771227aeb7cd849 Parents: 49da38e Author: Cheng Hao <hao.ch...@intel.com> Authored: Thu Sep 10 11:48:43 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Thu Sep 10 11:48:43 2015 -0700 -- .../spark/util/collection/ExternalSorter.scala | 6 ++ .../sql/execution/UnsafeRowSerializer.scala | 2 +- .../execution/UnsafeRowSerializerSuite.scala| 64 ++-- 3 files changed, 67 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e0481113/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 19287ed..138c05d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -188,6 +188,12 @@ private[spark] class ExternalSorter[K, V, C]( private val spills = new ArrayBuffer[SpilledFile] + /** + * Number of files this sorter has spilled so far. + * Exposed for testing. + */ + private[spark] def numSpills: Int = spills.size + override def insertAll(records: Iterator[Product2[K, V]]): Unit = { // TODO: stop combining if we find that the reduction factor isn't high val shouldCombine = aggregator.isDefined http://git-wip-us.apache.org/repos/asf/spark/blob/e0481113/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala index 5c18558..e060c06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala @@ -72,7 +72,7 @@ private class UnsafeRowSerializerInstance(numFields: Int) extends SerializerInst override def writeKey[T: ClassTag](key: T): SerializationStream = {
spark git commit: [SPARK-10466] [SQL] UnsafeRow SerDe exception with data spill
Repository: spark Updated Branches: refs/heads/branch-1.5 5e06d41a4 -> bc70043c8 [SPARK-10466] [SQL] UnsafeRow SerDe exception with data spill Data Spill with UnsafeRow causes assert failure. ``` java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:165) at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2.writeKey(UnsafeRowSerializer.scala:75) at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:180) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:688) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:687) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:687) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:683) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.util.collection.ExternalSorter.writePartitionedFile(ExternalSorter.scala:683) at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:80) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:88) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) ``` To reproduce that with code (thanks andrewor14): ```scala bin/spark-shell --master local --conf spark.shuffle.memoryFraction=0.005 --conf spark.shuffle.sort.bypassMergeThreshold=0 sc.parallelize(1 to 2 * 1000 * 1000, 10) .map { i => (i, i) }.toDF("a", "b").groupBy("b").avg().count() ``` Author: Cheng Hao <hao.ch...@intel.com> Closes #8635 from chenghao-intel/unsafe_spill. (cherry picked from commit e04811137680f937669cdcc78771227aeb7cd849) Signed-off-by: Andrew Or <and...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bc70043c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bc70043c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bc70043c Branch: refs/heads/branch-1.5 Commit: bc70043c8ebdc985ae4a02092b2750c22460d657 Parents: 5e06d41 Author: Cheng Hao <hao.ch...@intel.com> Authored: Thu Sep 10 11:48:43 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Thu Sep 10 11:48:50 2015 -0700 -- .../spark/util/collection/ExternalSorter.scala | 6 ++ .../sql/execution/UnsafeRowSerializer.scala | 2 +- .../execution/UnsafeRowSerializerSuite.scala| 64 ++-- 3 files changed, 67 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bc70043c/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 19287ed..138c05d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -188,6 +188,12 @@ private[spark] class ExternalSorter[K, V, C]( private val spills = new ArrayBuffer[SpilledFile] + /** + * Number of files this sorter has spilled so far. + * Exposed for testing. + */ + private[spark] def numSpills: Int = spills.size + override def insertAll(records: Iterator[Product2[K, V]]): Unit = { // TODO: stop combining if we find that the reduction factor isn't high val shouldCombine = aggregator.isDefined http://git-wip-us.apache.org/repos/asf/spark/blob/bc70043c/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala index 5c18558..e060c06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala @@ -72,7 +72,7 @@ private class UnsafeR
spark git commit: [SPARK-10469] [DOC] Try and document the three options
Repository: spark Updated Branches: refs/heads/master e04811137 -> a76bde9da [SPARK-10469] [DOC] Try and document the three options >From JIRA: Add documentation for tungsten-sort. >From the mailing list "I saw a new "spark.shuffle.manager=tungsten-sort" >implemented in https://issues.apache.org/jira/browse/SPARK-7081, but it can't be found its corresponding description in http://people.apache.org/~pwendell/spark-releases/spark-1.5.0-rc3-docs/configuration.html(Currenlty there are only 'sort' and 'hash' two options)." Author: Holden KarauCloses #8638 from holdenk/SPARK-10469-document-tungsten-sort. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a76bde9d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a76bde9d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a76bde9d Branch: refs/heads/master Commit: a76bde9dae54c4641e21f3c1ceb4870e3dc91881 Parents: e048111 Author: Holden Karau Authored: Thu Sep 10 11:49:53 2015 -0700 Committer: Andrew Or Committed: Thu Sep 10 11:49:53 2015 -0700 -- docs/configuration.md | 9 ++--- 1 file changed, 6 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a76bde9d/docs/configuration.md -- diff --git a/docs/configuration.md b/docs/configuration.md index e287591..0b1a273 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -447,9 +447,12 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.manager sort -Implementation to use for shuffling data. There are two implementations available: -sort and hash. Sort-based shuffle is more memory-efficient and is -the default option starting in 1.2. +Implementation to use for shuffling data. There are three implementations available: +sort, hash and the new (1.5+) tungsten-sort. +Sort-based shuffle is more memory-efficient and is the default option starting in 1.2. +Tungsten-sort is similar to the sort based shuffle, with a direct binary cache-friendly +implementation with a fall back to regular sort based shuffle if its requirements are not +met. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8167] Make tasks that fail from YARN preemption not fail job
Repository: spark Updated Branches: refs/heads/master a76bde9da -> af3bc59d1 [SPARK-8167] Make tasks that fail from YARN preemption not fail job The architecture is that, in YARN mode, if the driver detects that an executor has disconnected, it asks the ApplicationMaster why the executor died. If the ApplicationMaster is aware that the executor died because of preemption, all tasks associated with that executor are not marked as failed. The executor is still removed from the driver's list of available executors, however. There's a few open questions: 1. Should standalone mode have a similar "get executor loss reason" as well? I localized this change as much as possible to affect only YARN, but there could be a valid case to differentiate executor losses in standalone mode as well. 2. I make a pretty strong assumption in YarnAllocator that getExecutorLossReason(executorId) will only be called once per executor id; I do this so that I can remove the metadata from the in-memory map to avoid object accumulation. It's not clear if I'm being overly zealous to save space, however. cc vanzin specifically for review because it collided with some earlier YARN scheduling work. cc JoshRosen because it's similar to output commit coordination we did in the past cc andrewor14 for our discussion on how to get executor exit codes and loss reasons Author: mcheah <mch...@palantir.com> Closes #8007 from mccheah/feature/preemption-handling. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/af3bc59d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/af3bc59d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/af3bc59d Branch: refs/heads/master Commit: af3bc59d1f5d9d952c2d7ad1af599c49f1dbdaf0 Parents: a76bde9 Author: mcheah <mch...@palantir.com> Authored: Thu Sep 10 11:58:54 2015 -0700 Committer: Andrew Or <and...@databricks.com> Committed: Thu Sep 10 11:58:54 2015 -0700 -- .../scala/org/apache/spark/TaskEndReason.scala | 18 +++- .../spark/scheduler/ExecutorLossReason.scala| 14 +-- .../scala/org/apache/spark/scheduler/Pool.scala | 4 +- .../apache/spark/scheduler/Schedulable.scala| 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 9 +- .../apache/spark/scheduler/TaskSetManager.scala | 21 +++-- .../cluster/CoarseGrainedClusterMessage.scala | 8 +- .../cluster/CoarseGrainedSchedulerBackend.scala | 24 +++-- .../cluster/SparkDeploySchedulerBackend.scala | 6 +- .../cluster/YarnSchedulerBackend.scala | 77 ++-- .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala| 9 +- .../spark/scheduler/TaskSetManagerSuite.scala | 33 +-- .../apache/spark/util/JsonProtocolSuite.scala | 10 ++- .../spark/deploy/yarn/ApplicationMaster.scala | 7 ++ .../spark/deploy/yarn/YarnAllocator.scala | 92 +++- 17 files changed, 261 insertions(+), 79 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/af3bc59d/core/src/main/scala/org/apache/spark/TaskEndReason.scala -- diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 934d00d..2ae878b 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -48,6 +48,8 @@ case object Success extends TaskEndReason sealed trait TaskFailedReason extends TaskEndReason { /** Error message displayed in the web UI. */ def toErrorString: String + + def shouldEventuallyFailJob: Boolean = true } /** @@ -194,6 +196,12 @@ case object TaskKilled extends TaskFailedReason { case class TaskCommitDenied(jobID: Int, partitionID: Int, attemptID: Int) extends TaskFailedReason { override def toErrorString: String = s"TaskCommitDenied (Driver denied task commit)" + s" for job: $jobID, partition: $partitionID, attempt: $attemptID" + /** + * If a task failed because its attempt to commit was denied, do not count this failure + * towards failing the stage. This is intended to prevent spurious stage failures in cases + * where many speculative tasks are launched and denied to commit. + */ + override def shouldEventuallyFailJob: Boolean = false } /** @@ -202,8 +210,14 @@ case class TaskCommitDenied(jobID: Int, partitionID: Int, attemptID: Int) extend * the task crashed the JVM. */ @DeveloperApi -case class ExecutorLostFailure(execId: String) extends TaskFailedReason { - override def toErrorString: String = s"ExecutorLostFailure (executor ${execId} lost)" +case class Exec
spark git commit: [SPARK-10469] [DOC] Try and document the three options
Repository: spark Updated Branches: refs/heads/branch-1.5 bc70043c8 -> bff05aaa0 [SPARK-10469] [DOC] Try and document the three options >From JIRA: Add documentation for tungsten-sort. >From the mailing list "I saw a new "spark.shuffle.manager=tungsten-sort" >implemented in https://issues.apache.org/jira/browse/SPARK-7081, but it can't be found its corresponding description in http://people.apache.org/~pwendell/spark-releases/spark-1.5.0-rc3-docs/configuration.html(Currenlty there are only 'sort' and 'hash' two options)." Author: Holden KarauCloses #8638 from holdenk/SPARK-10469-document-tungsten-sort. (cherry picked from commit a76bde9dae54c4641e21f3c1ceb4870e3dc91881) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bff05aaa Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bff05aaa Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bff05aaa Branch: refs/heads/branch-1.5 Commit: bff05aaa099cc02465f258e44d581bd92370c09b Parents: bc70043 Author: Holden Karau Authored: Thu Sep 10 11:49:53 2015 -0700 Committer: Andrew Or Committed: Thu Sep 10 11:50:02 2015 -0700 -- docs/configuration.md | 9 ++--- 1 file changed, 6 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bff05aaa/docs/configuration.md -- diff --git a/docs/configuration.md b/docs/configuration.md index 353efdb..e1fa321 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -458,9 +458,12 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.manager sort -Implementation to use for shuffling data. There are two implementations available: -sort and hash. Sort-based shuffle is more memory-efficient and is -the default option starting in 1.2. +Implementation to use for shuffling data. There are three implementations available: +sort, hash and the new (1.5+) tungsten-sort. +Sort-based shuffle is more memory-efficient and is the default option starting in 1.2. +Tungsten-sort is similar to the sort based shuffle, with a direct binary cache-friendly +implementation with a fall back to regular sort based shuffle if its requirements are not +met. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9990] [SQL] Create local hash join operator
Repository: spark Updated Branches: refs/heads/master a5ef2d060 -> d88abb7e2 [SPARK-9990] [SQL] Create local hash join operator This PR includes the following changes: - Add SQLConf to LocalNode - Add HashJoinNode - Add ConvertToUnsafeNode and ConvertToSafeNode.scala to test unsafe hash join. Author: zsxwingCloses #8535 from zsxwing/SPARK-9990. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d88abb7e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d88abb7e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d88abb7e Branch: refs/heads/master Commit: d88abb7e212fb55f9b0398a0f76a753c86b85cf1 Parents: a5ef2d0 Author: zsxwing Authored: Thu Sep 10 12:06:49 2015 -0700 Committer: Andrew Or Committed: Thu Sep 10 12:06:49 2015 -0700 -- .../sql/execution/joins/HashedRelation.scala| 4 +- .../sql/execution/local/ConvertToSafeNode.scala | 40 ++ .../execution/local/ConvertToUnsafeNode.scala | 40 ++ .../spark/sql/execution/local/FilterNode.scala | 4 +- .../sql/execution/local/HashJoinNode.scala | 137 +++ .../spark/sql/execution/local/LimitNode.scala | 3 +- .../spark/sql/execution/local/LocalNode.scala | 83 ++- .../spark/sql/execution/local/ProjectNode.scala | 4 +- .../spark/sql/execution/local/SeqScanNode.scala | 4 +- .../spark/sql/execution/local/UnionNode.scala | 3 +- .../sql/execution/local/FilterNodeSuite.scala | 4 +- .../sql/execution/local/HashJoinNodeSuite.scala | 130 ++ .../sql/execution/local/LimitNodeSuite.scala| 4 +- .../sql/execution/local/LocalNodeTest.scala | 9 +- .../sql/execution/local/ProjectNodeSuite.scala | 4 +- .../sql/execution/local/UnionNodeSuite.scala| 6 +- 16 files changed, 455 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d88abb7e/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 6c0196c..0cff21c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -38,7 +38,7 @@ import org.apache.spark.{SparkConf, SparkEnv} * Interface for a hashed relation by some key. Use [[HashedRelation.apply]] to create a concrete * object. */ -private[joins] sealed trait HashedRelation { +private[execution] sealed trait HashedRelation { def get(key: InternalRow): Seq[InternalRow] // This is a helper method to implement Externalizable, and is used by @@ -111,7 +111,7 @@ final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[InternalR // TODO(rxin): a version of [[HashedRelation]] backed by arrays for consecutive integer keys. -private[joins] object HashedRelation { +private[execution] object HashedRelation { def apply( input: Iterator[InternalRow], http://git-wip-us.apache.org/repos/asf/spark/blob/d88abb7e/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ConvertToSafeNode.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ConvertToSafeNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ConvertToSafeNode.scala new file mode 100644 index 000..b31c5a8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ConvertToSafeNode.scala @@ -0,0 +1,40 @@ +/* +* 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.sql.execution.local + +import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute,
spark git commit: Revert "[SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.cores"
Repository: spark Updated Branches: refs/heads/branch-1.5 8cf16191f -> 89d351b5a Revert "[SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.cores" This reverts commit 8cf16191f3e3b0562f22d44b0381bea35ba511d7. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/89d351b5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/89d351b5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/89d351b5 Branch: refs/heads/branch-1.5 Commit: 89d351b5a43f9ae8db1d02953be3eec7de7944a5 Parents: 8cf1619 Author: Andrew OrAuthored: Thu Sep 10 14:35:52 2015 -0700 Committer: Andrew Or Committed: Thu Sep 10 14:35:52 2015 -0700 -- .../cluster/mesos/MesosSchedulerBackend.scala | 3 +- .../mesos/MesosSchedulerBackendSuite.scala | 33 +--- 2 files changed, 3 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/89d351b5/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 033454e..5c20606 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -32,6 +32,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils + /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks @@ -126,7 +127,7 @@ private[spark] class MesosSchedulerBackend( } val builder = MesosExecutorInfo.newBuilder() val (resourcesAfterCpu, usedCpuResources) = - partitionResources(availableResources, "cpus", mesosExecutorCores) + partitionResources(availableResources, "cpus", scheduler.CPUS_PER_TASK) val (resourcesAfterMem, usedMemResources) = partitionResources(resourcesAfterCpu, "mem", calculateTotalMemory(sc)) http://git-wip-us.apache.org/repos/asf/spark/blob/89d351b5/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 3291783..5ed30f6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -41,38 +41,6 @@ import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSui class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { - test("Use configured mesosExecutor.cores for ExecutorInfo") { -val mesosExecutorCores = 3 -val conf = new SparkConf -conf.set("spark.mesos.mesosExecutor.cores", mesosExecutorCores.toString) - -val listenerBus = mock[LiveListenerBus] -listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - -val sc = mock[SparkContext] -when(sc.getSparkHome()).thenReturn(Option("/spark-home")) - -when(sc.conf).thenReturn(conf) -when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) -when(sc.executorMemory).thenReturn(100) -when(sc.listenerBus).thenReturn(listenerBus) -val taskScheduler = mock[TaskSchedulerImpl] -when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - -val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") - -val resources = Arrays.asList( - mesosSchedulerBackend.createResource("cpus", 4), - mesosSchedulerBackend.createResource("mem", 1024)) -// uri is null. -val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") -val executorResources = executorInfo.getResourcesList -val cpus = executorResources.asScala.find(_.getName.equals("cpus")).get.getScalar.getValue - -assert(cpus === mesosExecutorCores) - } - test("check spark-class location correctly") { val conf = new SparkConf conf.set("spark.mesos.executor.home" , "/mesos-home") @@ -294,6 +262,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi
spark git commit: [SPARK-5945] Spark should not retry a stage infinitely on a FetchFailedException
Repository: spark Updated Branches: refs/heads/master 44948a2e9 -> 4bd85d06e [SPARK-5945] Spark should not retry a stage infinitely on a FetchFailedException The ```Stage``` class now tracks whether there were a sufficient number of consecutive failures of that stage to trigger an abort. To avoid an infinite loop of stage retries, we abort the job completely after 4 consecutive stage failures for one stage. We still allow more than 4 consecutive stage failures if there is an intervening successful attempt for the stage, so that in very long-lived applications, where a stage may get reused many times, we don't abort the job after failures that have been recovered from successfully. I've added test cases to exercise the most obvious scenarios. Author: Ilya GanelinCloses #5636 from ilganeli/SPARK-5945. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4bd85d06 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4bd85d06 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4bd85d06 Branch: refs/heads/master Commit: 4bd85d06e0334c49be18c4612b04d013b37f189c Parents: 44948a2 Author: Ilya Ganelin Authored: Wed Sep 2 22:07:50 2015 -0700 Committer: Andrew Or Committed: Wed Sep 2 22:08:24 2015 -0700 -- .../apache/spark/scheduler/DAGScheduler.scala | 13 +- .../org/apache/spark/scheduler/Stage.scala | 30 +- .../spark/scheduler/DAGSchedulerSuite.scala | 282 ++- 3 files changed, 320 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4bd85d06/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index daf9b0f..d673cb0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1101,7 +1101,6 @@ class DAGScheduler( s" ${task.stageAttemptId} and there is a more recent attempt for that stage " + s"(attempt ID ${failedStage.latestInfo.attemptId}) running") } else { - // It is likely that we receive multiple FetchFailed for a single stage (because we have // multiple tasks running concurrently on different executors). In that case, it is // possible the fetch failure has already been handled by the scheduler. @@ -1117,6 +1116,11 @@ class DAGScheduler( if (disallowStageRetryForTest) { abortStage(failedStage, "Fetch failure will not retry stage due to testing config", None) + } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) { +abortStage(failedStage, s"$failedStage (${failedStage.name}) " + + s"has failed the maximum allowable number of " + + s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " + + s"Most recent failure reason: ${failureMessage}", None) } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. @@ -1240,10 +1244,17 @@ class DAGScheduler( if (errorMessage.isEmpty) { logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stage.latestInfo.completionTime = Some(clock.getTimeMillis()) + + // Clear failure count for this stage, now that it's succeeded. + // We only limit consecutive failures of stage attempts,so that if a stage is + // re-used many times in a long-running job, unrelated failures don't eventually cause the + // stage to be aborted. + stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) } + outputCommitCoordinator.stageEnd(stage.id) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) runningStages -= stage http://git-wip-us.apache.org/repos/asf/spark/blob/4bd85d06/core/src/main/scala/org/apache/spark/scheduler/Stage.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 1cf0685..c086535 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -46,7 +46,7 @@ import org.apache.spark.util.CallSite * be
spark git commit: [SPARK-10379] preserve first page in UnsafeShuffleExternalSorter
Repository: spark Updated Branches: refs/heads/branch-1.5 2fce5d880 -> b846a9dc3 [SPARK-10379] preserve first page in UnsafeShuffleExternalSorter Author: Davies LiuCloses #8543 from davies/preserve_page. (cherry picked from commit 62b4690d6b3016f41292b640ac28644ef31e299d) Signed-off-by: Andrew Or Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b846a9dc Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b846a9dc Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b846a9dc Branch: refs/heads/branch-1.5 Commit: b846a9dc3f74af235111b6313900016c6ccac1b9 Parents: 2fce5d8 Author: Davies Liu Authored: Wed Sep 2 22:15:54 2015 -0700 Committer: Andrew Or Committed: Wed Sep 2 22:16:05 2015 -0700 -- .../spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java | 4 .../org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala | 2 +- .../apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java | 5 +++-- 3 files changed, 8 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b846a9dc/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java index 3d1ef0c..e73ba39 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java @@ -122,6 +122,10 @@ final class UnsafeShuffleExternalSorter { this.maxRecordSizeBytes = pageSizeBytes - 4; this.writeMetrics = writeMetrics; initializeForWriting(); + +// preserve first page to ensure that we have at least one page to work with. Otherwise, +// other operators in the same task may starve this sorter (SPARK-9709). +acquireNewPageIfNecessary(pageSizeBytes); } /** http://git-wip-us.apache.org/repos/asf/spark/blob/b846a9dc/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala index 1f2213d..417ff52 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala @@ -41,7 +41,7 @@ private[spark] class MapPartitionsWithPreparationRDD[U: ClassTag, T: ClassTag, M // In certain join operations, prepare can be called on the same partition multiple times. // In this case, we need to ensure that each call to compute gets a separate prepare argument. - private[this] var preparedArguments: ArrayBuffer[M] = new ArrayBuffer[M] + private[this] val preparedArguments: ArrayBuffer[M] = new ArrayBuffer[M] /** * Prepare a partition for a single call to compute. http://git-wip-us.apache.org/repos/asf/spark/blob/b846a9dc/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java -- diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index 94650be..a266b0c 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -530,8 +530,9 @@ public class UnsafeShuffleWriterSuite { for (int i = 0; i < numRecordsPerPage * 10; i++) { writer.insertRecordIntoSorter(new Tuple2
spark git commit: [SPARK-10379] preserve first page in UnsafeShuffleExternalSorter
Repository: spark Updated Branches: refs/heads/master 3ddb9b323 -> 62b4690d6 [SPARK-10379] preserve first page in UnsafeShuffleExternalSorter Author: Davies LiuCloses #8543 from davies/preserve_page. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/62b4690d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/62b4690d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/62b4690d Branch: refs/heads/master Commit: 62b4690d6b3016f41292b640ac28644ef31e299d Parents: 3ddb9b3 Author: Davies Liu Authored: Wed Sep 2 22:15:54 2015 -0700 Committer: Andrew Or Committed: Wed Sep 2 22:15:54 2015 -0700 -- .../spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java | 4 .../org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala | 2 +- .../apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java | 5 +++-- 3 files changed, 8 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/62b4690d/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java index 3d1ef0c..e73ba39 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java @@ -122,6 +122,10 @@ final class UnsafeShuffleExternalSorter { this.maxRecordSizeBytes = pageSizeBytes - 4; this.writeMetrics = writeMetrics; initializeForWriting(); + +// preserve first page to ensure that we have at least one page to work with. Otherwise, +// other operators in the same task may starve this sorter (SPARK-9709). +acquireNewPageIfNecessary(pageSizeBytes); } /** http://git-wip-us.apache.org/repos/asf/spark/blob/62b4690d/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala index 1f2213d..417ff52 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala @@ -41,7 +41,7 @@ private[spark] class MapPartitionsWithPreparationRDD[U: ClassTag, T: ClassTag, M // In certain join operations, prepare can be called on the same partition multiple times. // In this case, we need to ensure that each call to compute gets a separate prepare argument. - private[this] var preparedArguments: ArrayBuffer[M] = new ArrayBuffer[M] + private[this] val preparedArguments: ArrayBuffer[M] = new ArrayBuffer[M] /** * Prepare a partition for a single call to compute. http://git-wip-us.apache.org/repos/asf/spark/blob/62b4690d/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java -- diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java index 94650be..a266b0c 100644 --- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java @@ -530,8 +530,9 @@ public class UnsafeShuffleWriterSuite { for (int i = 0; i < numRecordsPerPage * 10; i++) { writer.insertRecordIntoSorter(new Tuple2
spark git commit: [SPARK-10247] [CORE] improve readability of a test case in DAGSchedulerSuite
Repository: spark Updated Branches: refs/heads/master f6c447f87 -> 3ddb9b323 [SPARK-10247] [CORE] improve readability of a test case in DAGSchedulerSuite This is pretty minor, just trying to improve the readability of `DAGSchedulerSuite`, I figure every bit helps. Before whenever I read this test, I never knew what "should work" and "should be ignored" really meant -- this adds some asserts & updates comments to make it more clear. Also some reformatting per a suggestion from markhamstra on https://github.com/apache/spark/pull/7699 Author: Imran RashidCloses #8434 from squito/SPARK-10247. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3ddb9b32 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3ddb9b32 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3ddb9b32 Branch: refs/heads/master Commit: 3ddb9b32335154e47890a0c761e0dfea3ccaac7b Parents: f6c447f Author: Imran Rashid Authored: Wed Sep 2 22:14:50 2015 -0700 Committer: Andrew Or Committed: Wed Sep 2 22:14:50 2015 -0700 -- .../spark/scheduler/DAGSchedulerSuite.scala | 57 1 file changed, 47 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3ddb9b32/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 62957c6..80f64de 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -926,27 +926,64 @@ class DAGSchedulerSuite val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) + // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch runEvent(ExecutorLost("exec-hostA")) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) + +// now start completing some tasks in the shuffle map stage, under different hosts +// and epochs, and make sure scheduler updates its state correctly val taskSet = taskSets(0) +val shuffleStage = scheduler.stageIdToStage(taskSet.stageId).asInstanceOf[ShuffleMapStage] +assert(shuffleStage.numAvailableOutputs === 0) + // should be ignored for being too old -runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", - reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) -// should work because it's a non-failed host -runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", - reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) +runEvent(CompletionEvent( + taskSet.tasks(0), + Success, + makeMapStatus("hostA", reduceRdd.partitions.size), + null, + createFakeTaskInfo(), + null)) +assert(shuffleStage.numAvailableOutputs === 0) + +// should work because it's a non-failed host (so the available map outputs will increase) +runEvent(CompletionEvent( + taskSet.tasks(0), + Success, + makeMapStatus("hostB", reduceRdd.partitions.size), + null, + createFakeTaskInfo(), + null)) +assert(shuffleStage.numAvailableOutputs === 1) + // should be ignored for being too old -runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", - reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) -// should work because it's a new epoch +runEvent(CompletionEvent( + taskSet.tasks(0), + Success, + makeMapStatus("hostA", reduceRdd.partitions.size), + null, + createFakeTaskInfo(), + null)) +assert(shuffleStage.numAvailableOutputs === 1) + +// should work because it's a new epoch, which will increase the number of available map +// outputs, and also finish the stage taskSet.tasks(1).epoch = newEpoch -runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", - reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) +runEvent(CompletionEvent( + taskSet.tasks(1), + Success, + makeMapStatus("hostA", reduceRdd.partitions.size), + null, + createFakeTaskInfo(), + null)) +assert(shuffleStage.numAvailableOutputs === 2) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) + +// finish the next stage normally, which completes the job complete(taskSets(1),
spark git commit: Removed code duplication in ShuffleBlockFetcherIterator
Repository: spark Updated Branches: refs/heads/master 0985d2c30 -> f6c447f87 Removed code duplication in ShuffleBlockFetcherIterator Added fetchUpToMaxBytes() to prevent having to update both code blocks when a change is made. Author: Evan RacahCloses #8514 from eracah/master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f6c447f8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f6c447f8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f6c447f8 Branch: refs/heads/master Commit: f6c447f87592286a6f58aee5e0b2dc8dcb470d0c Parents: 0985d2c Author: Evan Racah Authored: Wed Sep 2 22:13:18 2015 -0700 Committer: Andrew Or Committed: Wed Sep 2 22:13:37 2015 -0700 -- .../storage/ShuffleBlockFetcherIterator.scala | 18 ++ 1 file changed, 10 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f6c447f8/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index a759ceb..0d0448f 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -260,10 +260,7 @@ final class ShuffleBlockFetcherIterator( fetchRequests ++= Utils.randomize(remoteRequests) // Send out initial requests for blocks, up to our maxBytesInFlight -while (fetchRequests.nonEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) -} +fetchUpToMaxBytes() val numFetches = remoteRequests.size - fetchRequests.size logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) @@ -296,10 +293,7 @@ final class ShuffleBlockFetcherIterator( case _ => } // Send fetch requests up to maxBytesInFlight -while (fetchRequests.nonEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) -} +fetchUpToMaxBytes() result match { case FailureFetchResult(blockId, address, e) => @@ -315,6 +309,14 @@ final class ShuffleBlockFetcherIterator( } } + private def fetchUpToMaxBytes(): Unit = { +// Send fetch requests up to maxBytesInFlight +while (fetchRequests.nonEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) +} + } + private def throwFetchFailedException(blockId: BlockId, address: BlockManagerId, e: Throwable) = { blockId match { case ShuffleBlockId(shufId, mapId, reduceId) => - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8707] RDD#toDebugString fails if any cached RDD has invalid partitions
Repository: spark Updated Branches: refs/heads/master 4bd85d06e -> 0985d2c30 [SPARK-8707] RDD#toDebugString fails if any cached RDD has invalid partitions Added numPartitions(evaluate: Boolean) to RDD. With "evaluate=true" the method is same with "partitions.length". With "evaluate=false", it checks checked-out or already evaluated partitions in the RDD to get number of partition. If it's not those cases, returns -1. RDDInfo.partitionNum calls numPartition only when it's accessed. Author: navis.ryuCloses #7127 from navis/SPARK-8707. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0985d2c3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0985d2c3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0985d2c3 Branch: refs/heads/master Commit: 0985d2c30e031f80892987f7c3581d15dd210303 Parents: 4bd85d0 Author: navis.ryu Authored: Wed Sep 2 22:11:11 2015 -0700 Committer: Andrew Or Committed: Wed Sep 2 22:11:51 2015 -0700 -- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0985d2c3/core/src/main/scala/org/apache/spark/SparkContext.scala -- diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7388870..cbfe8bf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1516,8 +1516,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { +getRDDStorageInfo(_ => true) + } + + private[spark] def getRDDStorageInfo(filter: RDD[_] => Boolean): Array[RDDInfo] = { assertNotStopped() -val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray +val rddInfos = persistentRdds.values.filter(filter).map(RDDInfo.fromRdd).toArray StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) rddInfos.filter(_.isCached) } http://git-wip-us.apache.org/repos/asf/spark/blob/0985d2c3/core/src/main/scala/org/apache/spark/rdd/RDD.scala -- diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 081c721..7dd2bc5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1666,7 +1666,7 @@ abstract class RDD[T: ClassTag]( import Utils.bytesToString val persistence = if (storageLevel != StorageLevel.NONE) storageLevel.description else "" - val storageInfo = rdd.context.getRDDStorageInfo.filter(_.id == rdd.id).map(info => + val storageInfo = rdd.context.getRDDStorageInfo(_.id == rdd.id).map(info => "CachedPartitions: %d; MemorySize: %s; ExternalBlockStoreSize: %s; DiskSize: %s".format( info.numCachedPartitions, bytesToString(info.memSize), bytesToString(info.externalBlockStoreSize), bytesToString(info.diskSize))) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode.
Repository: spark Updated Branches: refs/heads/branch-1.5 16414dae0 - a3ed2c31e [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode. Currently the spark applications can be queued to the Mesos cluster dispatcher, but when multiple jobs are in queue we don't handle removing jobs from the buffer correctly while iterating and causes null pointer exception. This patch copies the buffer before iterating them, so exceptions aren't thrown when the jobs are removed. Author: Timothy Chen tnac...@gmail.com Closes #8322 from tnachen/fix_cluster_mode. (cherry picked from commit 73431d8afb41b93888d2642a1ce2d011f03fb740) Signed-off-by: Andrew Or and...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a3ed2c31 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a3ed2c31 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a3ed2c31 Branch: refs/heads/branch-1.5 Commit: a3ed2c31e60b11c09f815b42c0cd894be3150c67 Parents: 16414da Author: Timothy Chen tnac...@gmail.com Authored: Wed Aug 19 19:43:26 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Wed Aug 19 19:43:34 2015 -0700 -- .../cluster/mesos/MesosClusterScheduler.scala| 19 +++ 1 file changed, 11 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a3ed2c31/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 64ec2b8..1206f18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -507,14 +507,16 @@ private[spark] class MesosClusterScheduler( val driversToRetry = pendingRetryDrivers.filter { d = d.retryState.get.nextRetry.before(currentTime) } + scheduleTasks( -driversToRetry, +copyBuffer(driversToRetry), removeFromPendingRetryDrivers, currentOffers, tasks) + // Then we walk through the queued drivers and try to schedule them. scheduleTasks( -queuedDrivers, +copyBuffer(queuedDrivers), removeFromQueuedDrivers, currentOffers, tasks) @@ -527,13 +529,14 @@ private[spark] class MesosClusterScheduler( .foreach(o = driver.declineOffer(o.getId)) } + private def copyBuffer( + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { +val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) +buffer.copyToBuffer(newBuffer) +newBuffer + } + def getSchedulerState(): MesosClusterSchedulerState = { -def copyBuffer( -buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { - val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) - buffer.copyToBuffer(newBuffer) - newBuffer -} stateLock.synchronized { new MesosClusterSchedulerState( frameworkId, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode.
Repository: spark Updated Branches: refs/heads/master affc8a887 - 73431d8af [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode. Currently the spark applications can be queued to the Mesos cluster dispatcher, but when multiple jobs are in queue we don't handle removing jobs from the buffer correctly while iterating and causes null pointer exception. This patch copies the buffer before iterating them, so exceptions aren't thrown when the jobs are removed. Author: Timothy Chen tnac...@gmail.com Closes #8322 from tnachen/fix_cluster_mode. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/73431d8a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/73431d8a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/73431d8a Branch: refs/heads/master Commit: 73431d8afb41b93888d2642a1ce2d011f03fb740 Parents: affc8a8 Author: Timothy Chen tnac...@gmail.com Authored: Wed Aug 19 19:43:26 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Wed Aug 19 19:43:26 2015 -0700 -- .../cluster/mesos/MesosClusterScheduler.scala| 19 +++ 1 file changed, 11 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/73431d8a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 64ec2b8..1206f18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -507,14 +507,16 @@ private[spark] class MesosClusterScheduler( val driversToRetry = pendingRetryDrivers.filter { d = d.retryState.get.nextRetry.before(currentTime) } + scheduleTasks( -driversToRetry, +copyBuffer(driversToRetry), removeFromPendingRetryDrivers, currentOffers, tasks) + // Then we walk through the queued drivers and try to schedule them. scheduleTasks( -queuedDrivers, +copyBuffer(queuedDrivers), removeFromQueuedDrivers, currentOffers, tasks) @@ -527,13 +529,14 @@ private[spark] class MesosClusterScheduler( .foreach(o = driver.declineOffer(o.getId)) } + private def copyBuffer( + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { +val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) +buffer.copyToBuffer(newBuffer) +newBuffer + } + def getSchedulerState(): MesosClusterSchedulerState = { -def copyBuffer( -buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { - val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) - buffer.copyToBuffer(newBuffer) - newBuffer -} stateLock.synchronized { new MesosClusterSchedulerState( frameworkId, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application
Repository: spark Updated Branches: refs/heads/branch-1.5 1ce0b01f4 - 130e06ef1 [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application Detailed exception log can be seen in [SPARK-9877](https://issues.apache.org/jira/browse/SPARK-9877), the problem is when creating `StandaloneRestServer`, `self` (`masterEndpoint`) is null. So this fix is creating `StandaloneRestServer` when `self` is available. Author: jerryshao ss...@hortonworks.com Closes #8127 from jerryshao/SPARK-9877. (cherry picked from commit 9407baa2a7c26f527f2d043715d313d75bd765bb) Signed-off-by: Andrew Or and...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/130e06ef Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/130e06ef Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/130e06ef Branch: refs/heads/branch-1.5 Commit: 130e06ef1324e859f96220c84bc04ee7146ec0a1 Parents: 1ce0b01 Author: jerryshao ss...@hortonworks.com Authored: Fri Aug 14 13:44:38 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Fri Aug 14 13:44:53 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 16 1 file changed, 8 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/130e06ef/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9217202..26904d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -127,14 +127,8 @@ private[deploy] class Master( // Alternative application submission gateway that is stable across Spark versions private val restServerEnabled = conf.getBoolean(spark.master.rest.enabled, true) - private val restServer = -if (restServerEnabled) { - val port = conf.getInt(spark.master.rest.port, 6066) - Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) -} else { - None -} - private val restServerBoundPort = restServer.map(_.start()) + private var restServer: Option[StandaloneRestServer] = None + private var restServerBoundPort: Option[Int] = None override def onStart(): Unit = { logInfo(Starting Spark master at + masterUrl) @@ -148,6 +142,12 @@ private[deploy] class Master( } }, 0, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) +if (restServerEnabled) { + val port = conf.getInt(spark.master.rest.port, 6066) + restServer = Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) +} +restServerBoundPort = restServer.map(_.start()) + masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() applicationMetricsSystem.start() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators
Repository: spark Updated Branches: refs/heads/master 33bae585d - 6518ef630 [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators In these tests, we use a custom listener and we assert on fields in the stage / task completion events. However, these events are posted in a separate thread so they're not guaranteed to be posted in time. This commit fixes this flakiness through a job end registration callback. Author: Andrew Or and...@databricks.com Closes #8176 from andrewor14/fix-accumulator-suite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6518ef63 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6518ef63 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6518ef63 Branch: refs/heads/master Commit: 6518ef63037aa56b541927f99ad26744f91098ce Parents: 33bae58 Author: Andrew Or and...@databricks.com Authored: Fri Aug 14 13:42:53 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Fri Aug 14 13:42:53 2015 -0700 -- .../org/apache/spark/AccumulatorSuite.scala | 153 +++ 1 file changed, 92 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6518ef63/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 0eb2293..5b84acf 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -182,26 +182,30 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc = new SparkContext(local, test) sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator -sc.parallelize(1 to 100, numPartitions).mapPartitions { iter = +val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter = TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 iter -}.count() -val stageInfos = listener.getCompletedStageInfos -val taskInfos = listener.getCompletedTaskInfos -assert(stageInfos.size === 1) -assert(taskInfos.size === numPartitions) -// The accumulator values should be merged in the stage -val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) -assert(stageAccum.value.toLong === numPartitions) -// The accumulator should be updated locally on each task -val taskAccumValues = taskInfos.map { taskInfo = - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - taskAccum.value.toLong } -// Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions -assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +// Register asserts in job completion callback to avoid flakiness +listener.registerJobCompletionCallback { _ = + val stageInfos = listener.getCompletedStageInfos + val taskInfos = listener.getCompletedTaskInfos + assert(stageInfos.size === 1) + assert(taskInfos.size === numPartitions) + // The accumulator values should be merged in the stage + val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) + assert(stageAccum.value.toLong === numPartitions) + // The accumulator should be updated locally on each task + val taskAccumValues = taskInfos.map { taskInfo = +val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) +assert(taskAccum.update.isDefined) +assert(taskAccum.update.get.toLong === 1) +taskAccum.value.toLong + } + // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions + assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +} +rdd.count() } test(internal accumulators in multiple stages) { @@ -211,7 +215,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Each stage creates its own set of internal accumulators so the // values for the same metric should not be mixed up across stages -sc.parallelize(1 to 100, numPartitions) +val rdd = sc.parallelize(1 to 100, numPartitions) .map { i = (i, i) } .mapPartitions { iter = TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 @@ -227,16 +231,20 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex TaskContext.get().internalMetricsToAccumulators
spark git commit: [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators
Repository: spark Updated Branches: refs/heads/branch-1.5 ff3e9561d - 1ce0b01f4 [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators In these tests, we use a custom listener and we assert on fields in the stage / task completion events. However, these events are posted in a separate thread so they're not guaranteed to be posted in time. This commit fixes this flakiness through a job end registration callback. Author: Andrew Or and...@databricks.com Closes #8176 from andrewor14/fix-accumulator-suite. (cherry picked from commit 6518ef63037aa56b541927f99ad26744f91098ce) Signed-off-by: Andrew Or and...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1ce0b01f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1ce0b01f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1ce0b01f Branch: refs/heads/branch-1.5 Commit: 1ce0b01f427e5840c2d3e7e0046fe18fed945336 Parents: ff3e956 Author: Andrew Or and...@databricks.com Authored: Fri Aug 14 13:42:53 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Fri Aug 14 13:43:01 2015 -0700 -- .../org/apache/spark/AccumulatorSuite.scala | 153 +++ 1 file changed, 92 insertions(+), 61 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1ce0b01f/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 0eb2293..5b84acf 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -182,26 +182,30 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc = new SparkContext(local, test) sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator -sc.parallelize(1 to 100, numPartitions).mapPartitions { iter = +val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter = TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 iter -}.count() -val stageInfos = listener.getCompletedStageInfos -val taskInfos = listener.getCompletedTaskInfos -assert(stageInfos.size === 1) -assert(taskInfos.size === numPartitions) -// The accumulator values should be merged in the stage -val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) -assert(stageAccum.value.toLong === numPartitions) -// The accumulator should be updated locally on each task -val taskAccumValues = taskInfos.map { taskInfo = - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - taskAccum.value.toLong } -// Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions -assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +// Register asserts in job completion callback to avoid flakiness +listener.registerJobCompletionCallback { _ = + val stageInfos = listener.getCompletedStageInfos + val taskInfos = listener.getCompletedTaskInfos + assert(stageInfos.size === 1) + assert(taskInfos.size === numPartitions) + // The accumulator values should be merged in the stage + val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) + assert(stageAccum.value.toLong === numPartitions) + // The accumulator should be updated locally on each task + val taskAccumValues = taskInfos.map { taskInfo = +val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) +assert(taskAccum.update.isDefined) +assert(taskAccum.update.get.toLong === 1) +taskAccum.value.toLong + } + // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions + assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) +} +rdd.count() } test(internal accumulators in multiple stages) { @@ -211,7 +215,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.addSparkListener(listener) // Each stage creates its own set of internal accumulators so the // values for the same metric should not be mixed up across stages -sc.parallelize(1 to 100, numPartitions) +val rdd = sc.parallelize(1 to 100, numPartitions) .map { i = (i, i) } .mapPartitions { iter = TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 @@ -227,16 +231,20 @@ class
spark git commit: [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application
Repository: spark Updated Branches: refs/heads/master 6518ef630 - 9407baa2a [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application Detailed exception log can be seen in [SPARK-9877](https://issues.apache.org/jira/browse/SPARK-9877), the problem is when creating `StandaloneRestServer`, `self` (`masterEndpoint`) is null. So this fix is creating `StandaloneRestServer` when `self` is available. Author: jerryshao ss...@hortonworks.com Closes #8127 from jerryshao/SPARK-9877. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9407baa2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9407baa2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9407baa2 Branch: refs/heads/master Commit: 9407baa2a7c26f527f2d043715d313d75bd765bb Parents: 6518ef6 Author: jerryshao ss...@hortonworks.com Authored: Fri Aug 14 13:44:38 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Fri Aug 14 13:44:38 2015 -0700 -- .../org/apache/spark/deploy/master/Master.scala | 16 1 file changed, 8 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9407baa2/core/src/main/scala/org/apache/spark/deploy/master/Master.scala -- diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9217202..26904d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -127,14 +127,8 @@ private[deploy] class Master( // Alternative application submission gateway that is stable across Spark versions private val restServerEnabled = conf.getBoolean(spark.master.rest.enabled, true) - private val restServer = -if (restServerEnabled) { - val port = conf.getInt(spark.master.rest.port, 6066) - Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) -} else { - None -} - private val restServerBoundPort = restServer.map(_.start()) + private var restServer: Option[StandaloneRestServer] = None + private var restServerBoundPort: Option[Int] = None override def onStart(): Unit = { logInfo(Starting Spark master at + masterUrl) @@ -148,6 +142,12 @@ private[deploy] class Master( } }, 0, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) +if (restServerEnabled) { + val port = conf.getInt(spark.master.rest.port, 6066) + restServer = Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) +} +restServerBoundPort = restServer.map(_.start()) + masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() applicationMetricsSystem.start() - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9809] Task crashes because the internal accumulators are not properly initialized
Repository: spark Updated Branches: refs/heads/branch-1.5 d92568ae5 - ff3e9561d [SPARK-9809] Task crashes because the internal accumulators are not properly initialized When a stage failed and another stage was resubmitted with only part of partitions to compute, all the tasks failed with error message: java.util.NoSuchElementException: key not found: peakExecutionMemory. This is because the internal accumulators are not properly initialized for this stage while other codes assume the internal accumulators always exist. Author: Carson Wang carson.w...@intel.com Closes #8090 from carsonwang/SPARK-9809. (cherry picked from commit 33bae585d4cb25aed2ac32e0d1248f78cc65318b) Signed-off-by: Andrew Or and...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ff3e9561 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ff3e9561 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ff3e9561 Branch: refs/heads/branch-1.5 Commit: ff3e9561d63348076b77b3d16ca1a720461e87ea Parents: d92568a Author: Carson Wang carson.w...@intel.com Authored: Fri Aug 14 13:38:25 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Fri Aug 14 13:38:34 2015 -0700 -- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ff3e9561/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7ab5ccf..f1c63d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -790,9 +790,10 @@ class DAGScheduler( } } +// Create internal accumulators if the stage has no accumulators initialized. // Reset internal accumulators only if this stage is not partially submitted // Otherwise, we may override existing accumulator values from some tasks -if (allPartitions == partitionsToCompute) { +if (stage.internalAccumulators.isEmpty || allPartitions == partitionsToCompute) { stage.resetInternalAccumulators() } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9809] Task crashes because the internal accumulators are not properly initialized
Repository: spark Updated Branches: refs/heads/master ffa05c84f - 33bae585d [SPARK-9809] Task crashes because the internal accumulators are not properly initialized When a stage failed and another stage was resubmitted with only part of partitions to compute, all the tasks failed with error message: java.util.NoSuchElementException: key not found: peakExecutionMemory. This is because the internal accumulators are not properly initialized for this stage while other codes assume the internal accumulators always exist. Author: Carson Wang carson.w...@intel.com Closes #8090 from carsonwang/SPARK-9809. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/33bae585 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/33bae585 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/33bae585 Branch: refs/heads/master Commit: 33bae585d4cb25aed2ac32e0d1248f78cc65318b Parents: ffa05c8 Author: Carson Wang carson.w...@intel.com Authored: Fri Aug 14 13:38:25 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Fri Aug 14 13:38:25 2015 -0700 -- core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/33bae585/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7ab5ccf..f1c63d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -790,9 +790,10 @@ class DAGScheduler( } } +// Create internal accumulators if the stage has no accumulators initialized. // Reset internal accumulators only if this stage is not partially submitted // Otherwise, we may override existing accumulator values from some tasks -if (allPartitions == partitionsToCompute) { +if (stage.internalAccumulators.isEmpty || allPartitions == partitionsToCompute) { stage.resetInternalAccumulators() } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9649] Fix flaky test MasterSuite - randomize ports
Repository: spark Updated Branches: refs/heads/branch-1.4 8ce86b23f - 041e720ec [SPARK-9649] Fix flaky test MasterSuite - randomize ports ``` Error Message Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries! Stacktrace java.net.BindException: Failed to bind to: /127.0.0.1:7093: Service 'sparkMaster' failed after 16 retries! at org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272) at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393) at akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389) at scala.util.Success$$anonfun$map$1.apply(Try.scala:206) at scala.util.Try$.apply(Try.scala:161) ``` Author: Andrew Or and...@databricks.com Closes #7968 from andrewor14/fix-master-flaky-test and squashes the following commits: fcc42ef [Andrew Or] Randomize port Conflicts: core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/041e720e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/041e720e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/041e720e Branch: refs/heads/branch-1.4 Commit: 041e720ecb98197dbeed40c8eb35fff1052ffd0a Parents: 8ce86b2 Author: Andrew Or and...@databricks.com Authored: Wed Aug 5 14:12:22 2015 -0700 Committer: Andrew Or and...@databricks.com Committed: Thu Aug 13 11:40:36 2015 -0700 -- .../test/scala/org/apache/spark/deploy/master/MasterSuite.scala | 3 ++- pom.xml | 1 + project/SparkBuild.scala | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/041e720e/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala -- diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 3eabc1a..d331807 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -81,6 +81,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually { conf.set(spark.deploy.recoveryMode, CUSTOM) conf.set(spark.deploy.recoveryMode.factory, classOf[CustomRecoveryModeFactory].getCanonicalName) +conf.set(spark.master.rest.enabled, false) val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts @@ -135,7 +136,7 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually { ) val (actorSystem, port, uiPort, restPort) = - Master.startSystemAndActor(127.0.0.1, 7077, 8080, conf) + Master.startSystemAndActor(127.0.0.1, 0, 0, conf) try { Await.result(actorSystem.actorSelection(/user/Master).resolveOne(10 seconds), 10 seconds) http://git-wip-us.apache.org/repos/asf/spark/blob/041e720e/pom.xml -- diff --git a/pom.xml b/pom.xml index 6c102da..be8aebd 100644 --- a/pom.xml +++ b/pom.xml @@ -1256,6 +1256,7 @@ spark.test.home${spark.test.home}/spark.test.home spark.testing1/spark.testing spark.ui.enabledfalse/spark.ui.enabled + spark.master.rest.enabledfalse/spark.master.rest.enabled spark.ui.showConsoleProgressfalse/spark.ui.showConsoleProgress spark.driver.allowMultipleContextstrue/spark.driver.allowMultipleContexts spark.unsafe.exceptionOnMemoryLeaktrue/spark.unsafe.exceptionOnMemoryLeak http://git-wip-us.apache.org/repos/asf/spark/blob/041e720e/project/SparkBuild.scala -- diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4b00a66..9b53be4 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -514,6 +514,7 @@ object TestSettings { javaOptions in Test += -Dspark.testing=1, javaOptions in Test += -Dspark.port.maxRetries=100, javaOptions in Test += -Dspark.ui.enabled=false, +javaOptions in Test += -Dspark.master.rest.enabled=false, javaOptions in Test += -Dspark.ui.showConsoleProgress=false, javaOptions in Test += -Dspark.driver.allowMultipleContexts=true, javaOptions in Test += -Dspark.unsafe.exceptionOnMemoryLeak=true, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org