(spark) branch master updated: [SPARK-42204][CORE] Add option to disable redundant logging of TaskMetrics internal accumulators in event logs
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new f9a8ca54e7ad [SPARK-42204][CORE] Add option to disable redundant logging of TaskMetrics internal accumulators in event logs f9a8ca54e7ad is described below commit f9a8ca54e7adb025aca709169c8d76ff1b2f7cd7 Author: Josh Rosen AuthorDate: Fri Sep 6 15:57:50 2024 -0700 [SPARK-42204][CORE] Add option to disable redundant logging of TaskMetrics internal accumulators in event logs ### What changes were proposed in this pull request? This PR adds an off-by-default option to JsonProtocol to have it exclude certain redundant accumulator information from Spark event logs in order to save space and processing time. Several event logs types contain both TaskMetrics and Accumulables, but there is redundancy in how the TaskMetrics data is stored: - TaskMetrics are stored in a map called "Task Metrics" which maps from metric names to metric values. - An "Accumulables" field contains information on accumulator updates from the task, but this field includes updates from the TaskMetrics internal accumulators (both the value from the task, plus a running "sum-so-far" from all of the tasks completed up to that point). The redundant task metrics accumulables are not actually used by the Spark History Server: I verified this by reading AppStatusListener and SQLAppStatusListener. I believe that this redundancy was introduced back in [SPARK-10620](https://issues.apache.org/jira/browse/SPARK-10620) when Spark 1.x's separate TaskMetrics implementation was replaced by the current accumulator-based version. In this PR, I add logic to exclude TaskMetrics internal accumulators when writing this field (if a new flag is enabled). The new `spark.eventLog.includeTaskMetricsAccumulators` configuration (default `false`, meaning "keep the redundant information") can be set to `true` to exclude these redundant internal accumulator updates. For now, I am merging this off-by-default, but in a followup PR for Spark 4.0.0 we might consider a change-of-default to `true` (in which case the flag would serve as an "escape-hatch" for users who want to restore the old behavior. Although I think it's somewhat unlikely that third-party non-Spark consumers of the event logs would be relying on this redundant information, this is changing a longstanding user-facing data format and thus needs a flag. ### Why are the changes needed? This change reduces the size of Spark event logs, especially for logs from applications that run many tasks. It should also have slight benefits on event log read and write speed (although I haven't tried to quantify this). ### Does this PR introduce _any_ user-facing change? No user-facing changes in Spark History Server. This flag's effects could be considered a user-facing change from the perspective of third-party code which does its own direct processing of Spark event logs, hence the config. However, in this PR (by itself) the flag is off-by-default. Out-of-the-box user-facing changes will be discussed / proposed in a separate flag-flip PR. ### How was this patch tested? New unit tests in `JsonProtocolSuite`. Manual tests of event log size in `spark-shell` with a job that runs `spark.parallelize(1 to 1000, 1000).count()`. For this toy query, this PR's change shrunk the uncompressed event log size by ~15%. The relative size reduction will be even greater once other issues like https://issues.apache.org/jira/browse/SPARK-42206 or https://issues.apache.org/jira/browse/SPARK-42203 are fixed. The relative reduction will be smaller for tasks with many SQL metrics because those accumulables canno [...] Closes #39763 from JoshRosen/SPARK-42204-remove-redundant-logging-of-taskmetrics-internal-accumulators-in-jsonprotocol. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../org/apache/spark/internal/config/package.scala | 12 +++ .../spark/scheduler/EventLoggingListener.scala | 8 +- .../scala/org/apache/spark/util/JsonProtocol.scala | 108 - .../org/apache/spark/util/JsonProtocolSuite.scala | 104 ++-- 4 files changed, 194 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 0e19143411e9..47019c04aada 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -271,6 +271,
(spark) branch master updated: [SPARK-48628][CORE] Add task peak on/off heap memory metrics
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 9b9a7a7478d1 [SPARK-48628][CORE] Add task peak on/off heap memory metrics 9b9a7a7478d1 is described below commit 9b9a7a7478d17222ab92a346aa5d4de31d71349f Author: Ziqi Liu AuthorDate: Wed Aug 21 08:41:42 2024 -0700 [SPARK-48628][CORE] Add task peak on/off heap memory metrics ### What changes were proposed in this pull request? This PR is trying to revive https://github.com/apache/spark/pull/47192, which was [reverted](https://github.com/apache/spark/pull/47747) due to regression in `ExternalAppendOnlyUnsafeRowArrayBenchmark`. **Root cause** We eventually decided to aggregate peak memory usage from all consumers on each `acquireExecutionMemory` invocation. (see [this discussion](https://github.com/apache/spark/pull/47192#discussion_r1681934753)), which is O(n) complexity where `n` is the number of consumers. `ExternalAppendOnlyUnsafeRowArrayBenchmark` is implemented in a way that all iterations are run in a single task context, therefore the number of consumers is exploding. Notice that `TaskMemoryManager.consumers` is never cleaned up the whole lifecycle, and `TaskMemoryManager.acquireExecutionMemory` is a very frequent operation, doing a linear complexity(in terms of number of consumers) operation here might not be a good choice. This benchmark might be a corner case, but it's still possible to have a large number of consumers in a large query plan. I fallback to the previous implementation: maintain current execution memory with an extra lock. cc Ngone51 Benchmark result [ExternalAppendOnlyUnsafeRowArrayBenchmark-results](https://github.com/liuzqt/spark/actions/runs/10415213026) [ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results](https://github.com/liuzqt/spark/actions/runs/10414246805) ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? NO Closes #47776 from liuzqt/SPARK-48628. Authored-by: Ziqi Liu Signed-off-by: Josh Rosen --- .../org/apache/spark/memory/TaskMemoryManager.java | 61 ++ .../org/apache/spark/InternalAccumulator.scala | 2 + .../scala/org/apache/spark/executor/Executor.scala | 2 + .../org/apache/spark/executor/TaskMetrics.scala| 21 +++ .../scala/org/apache/spark/util/JsonProtocol.scala | 6 ++ .../apache/spark/memory/MemoryManagerSuite.scala | 18 ++ .../org/apache/spark/util/JsonProtocolSuite.scala | 72 ++ ...ndOnlyUnsafeRowArrayBenchmark-jdk21-results.txt | 30 - ...alAppendOnlyUnsafeRowArrayBenchmark-results.txt | 30 - 9 files changed, 187 insertions(+), 55 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index fe798e40a6ad..df224bc902bf 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -122,6 +122,30 @@ public class TaskMemoryManager { */ private volatile long acquiredButNotUsed = 0L; + /** + * Current off heap memory usage by this task. + */ + private long currentOffHeapMemory = 0L; + + private final Object offHeapMemoryLock = new Object(); + + /* + * Current on heap memory usage by this task. + */ + private long currentOnHeapMemory = 0L; + + private final Object onHeapMemoryLock = new Object(); + + /** + * Peak off heap memory usage by this task. + */ + private volatile long peakOffHeapMemory = 0L; + + /** + * Peak on heap memory usage by this task. + */ + private volatile long peakOnHeapMemory = 0L; + /** * Construct a new TaskMemoryManager. */ @@ -202,6 +226,19 @@ public class TaskMemoryManager { logger.debug("Task {} acquired {} for {}", taskAttemptId, Utils.bytesToString(got), requestingConsumer); } + + if (mode == MemoryMode.OFF_HEAP) { +synchronized (offHeapMemoryLock) { + currentOffHeapMemory += got; + peakOffHeapMemory = Math.max(peakOffHeapMemory, currentOffHeapMemory); +} + } else { +synchronized (onHeapMemoryLock) { + currentOnHeapMemory += got; + peakOnHeapMemory = Math.max(peakOnHeapMemory, currentOnHeapMemory); +} + } + return got; } } @@ -269,6 +306,15 @@ public class TaskMemoryManager { consumer); } memoryManager.releaseExecutionMemory(size, taskAttemptId, consumer.getMode()); +if (c
(spark) branch master updated: [SPARK-48716] Add jobGroupId to SparkListenerSQLExecutionStart
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 4c99c4df7f9c [SPARK-48716] Add jobGroupId to SparkListenerSQLExecutionStart 4c99c4df7f9c is described below commit 4c99c4df7f9cda9b35bc7401c92da22ea683861b Author: Lingkai Kong AuthorDate: Tue Jul 9 11:29:52 2024 -0700 [SPARK-48716] Add jobGroupId to SparkListenerSQLExecutionStart ### What changes were proposed in this pull request? Add jobGroupId to SparkListenerSQLExecutionStart ### Why are the changes needed? JobGroupId can be used to combine jobs within the same group. This is going to be useful in the listener so it makes the job grouping easy to do ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit Test ### Was this patch authored or co-authored using generative AI tooling? No Closes #47092 from gjxdxh/gjxdxh/SPARK-48716. Authored-by: Lingkai Kong Signed-off-by: Josh Rosen --- .../apache/spark/sql/execution/SQLExecution.scala | 5 ++-- .../sql/execution/ui/SQLAppStatusListener.scala| 2 +- .../spark/sql/execution/ui/SQLListener.scala | 3 +- .../spark/sql/execution/SQLExecutionSuite.scala| 33 ++ .../execution/ui/SQLAppStatusListenerSuite.scala | 2 +- 5 files changed, 40 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index f4be03c90be7..7c03bad90ebb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal -import org.apache.spark.{ErrorMessageFormat, JobArtifactSet, SparkEnv, SparkException, SparkThrowable, SparkThrowableHelper} +import org.apache.spark.{ErrorMessageFormat, JobArtifactSet, SparkContext, SparkEnv, SparkException, SparkThrowable, SparkThrowableHelper} import org.apache.spark.SparkContext.{SPARK_JOB_DESCRIPTION, SPARK_JOB_INTERRUPT_ON_CANCEL} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{SPARK_DRIVER_PREFIX, SPARK_EXECUTOR_PREFIX} @@ -128,7 +128,8 @@ object SQLExecution extends Logging { sparkPlanInfo = SparkPlanInfo.EMPTY, time = System.currentTimeMillis(), modifiedConfigs = redactedConfigs, - jobTags = sc.getJobTags() + jobTags = sc.getJobTags(), + jobGroupId = Option(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID)) ) try { body match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index bf33ba2c96f1..dcbf328c71e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -343,7 +343,7 @@ class SQLAppStatusListener( private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { val SparkListenerSQLExecutionStart(executionId, rootExecutionId, description, details, - physicalPlanDescription, sparkPlanInfo, time, modifiedConfigs, _) = event + physicalPlanDescription, sparkPlanInfo, time, modifiedConfigs, _, _) = event val planGraph = SparkPlanGraph(sparkPlanInfo) val sqlPlanMetrics = planGraph.allNodes.flatMap { node => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 3a22dd23548f..416b9547b046 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -54,7 +54,8 @@ case class SparkListenerSQLExecutionStart( sparkPlanInfo: SparkPlanInfo, time: Long, modifiedConfigs: Map[String, String] = Map.empty, -jobTags: Set[String] = Set.empty) +jobTags: Set[String] = Set.empty, +jobGroupId: Option[String] = None) extends SparkListenerEvent @DeveloperApi diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala index b8a109919f8f..94d33731b6de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala @@ -227,6 +22
(spark) branch master updated: [SPARK-48541][CORE] Add a new exit code for executors killed by TaskReaper
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 54587638685b [SPARK-48541][CORE] Add a new exit code for executors killed by TaskReaper 54587638685b is described below commit 54587638685bd633cb3840a23afd5a809d796d47 Author: Bo Zhang AuthorDate: Wed Jun 19 11:03:24 2024 -0700 [SPARK-48541][CORE] Add a new exit code for executors killed by TaskReaper ### What changes were proposed in this pull request? This change adds a new exit code, 57, for executors killed by TaskReaper. ### Why are the changes needed? This is to better monitor the cases when executors are killed by TaskReaper. ### Does this PR introduce _any_ user-facing change? Yes. The exit code for executors killed by TaskReaper will change. ### How was this patch tested? Updated unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46883 from bozhang2820/spark-48541. Authored-by: Bo Zhang Signed-off-by: Josh Rosen --- .../main/scala/org/apache/spark/executor/Executor.scala | 7 --- .../org/apache/spark/executor/ExecutorExitCode.scala| 6 ++ .../spark/util/SparkUncaughtExceptionHandler.scala | 3 +++ .../scala/org/apache/spark/JobCancellationSuite.scala | 17 +++-- .../spark/util/SparkUncaughtExceptionHandlerSuite.scala | 5 + 5 files changed, 33 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4e5d151468d8..7317d3c47c08 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -1041,9 +1041,8 @@ private[spark] class Executor( } else { // In non-local-mode, the exception thrown here will bubble up to the uncaught exception // handler and cause the executor JVM to exit. -throw SparkException.internalError( - s"Killing executor JVM because killed task $taskId could not be stopped within " + -s"$killTimeoutMs ms.", category = "EXECUTOR") +throw new KilledByTaskReaperException(s"Killing executor JVM because killed task " + + s"$taskId could not be stopped within $killTimeoutMs ms.") } } } finally { @@ -1328,3 +1327,5 @@ private[spark] object Executor { } } } + +class KilledByTaskReaperException(message: String) extends SparkException(message) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 99858f785600..5300598ef53e 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -45,6 +45,10 @@ object ExecutorExitCode { */ val HEARTBEAT_FAILURE = 56 + /** The default uncaught exception handler was reached and the exception was thrown by + * TaskReaper. */ + val KILLED_BY_TASK_REAPER = 57 + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -59,6 +63,8 @@ object ExecutorExitCode { "ExternalBlockStore failed to create a local temporary directory." case HEARTBEAT_FAILURE => "Unable to send heartbeats to driver." + case KILLED_BY_TASK_REAPER => +"Executor killed by TaskReaper." case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { diff --git a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala index 730b762a3948..c1ea4f929101 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala @@ -17,6 +17,7 @@ package org.apache.spark.util +import org.apache.spark.executor.{ExecutorExitCode, KilledByTaskReaperException} import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.THREAD @@ -56,6 +57,8 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException: // SPARK-24294: This is defensive code, in case that SparkFatalException is // misused and uncaught. System.exit(SparkExitCode.OOM) + case _: KilledByTaskReaperException if exitOnUncaughtException => +
(spark) branch master updated: [SPARK-48544][SQL] Reduce memory pressure of empty TreeNode BitSets
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5a2f374a208f [SPARK-48544][SQL] Reduce memory pressure of empty TreeNode BitSets 5a2f374a208f is described below commit 5a2f374a208f9580ea8d0183d75df6cd2bee8e1f Author: Nick Young AuthorDate: Mon Jun 10 11:16:11 2024 -0700 [SPARK-48544][SQL] Reduce memory pressure of empty TreeNode BitSets ### What changes were proposed in this pull request? - Changed the `ineffectiveRules` variable of the `TreeNode` class to initialize lazily. This will reduce unnecessary driver memory pressure. ### Why are the changes needed? - Plans with large expression or operator trees are known to cause driver memory pressure; this is one step in alleviating that issue. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing UT covers behavior. Outwards facing behavior does not change. ### Was this patch authored or co-authored using generative AI tooling? No Closes #46919 from n-young-db/ineffective-rules-lazy. Authored-by: Nick Young Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 11 +-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index dd39f3182bfb..23d26854a767 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -120,7 +120,14 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] * ineffective for subsequent apply calls on this tree because query plan structures are * immutable. */ - private val ineffectiveRules: BitSet = new BitSet(RuleIdCollection.NumRules) + private[this] var _ineffectiveRules: BitSet = null + private def ineffectiveRules: BitSet = { +if (_ineffectiveRules eq null) { + _ineffectiveRules = new BitSet(RuleIdCollection.NumRules) +} +_ineffectiveRules + } + private def isIneffectiveRulesEmpty = _ineffectiveRules eq null /** * @return a sequence of tree pattern enums in a TreeNode T. It does not include propagated @@ -149,7 +156,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] * UnknownId, it returns false. */ protected def isRuleIneffective(ruleId : RuleId): Boolean = { -if (ruleId eq UnknownRuleId) { +if (isIneffectiveRulesEmpty || (ruleId eq UnknownRuleId)) { return false } ineffectiveRules.get(ruleId.id) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42205][CORE] Don't log accumulator values in stage / task start and getting result event logs
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new bb59b489204 [SPARK-42205][CORE] Don't log accumulator values in stage / task start and getting result event logs bb59b489204 is described below commit bb59b4892042bd4ad0c18f2a7fb2380c292d9e3e Author: Josh Rosen AuthorDate: Fri Sep 29 11:27:51 2023 -0700 [SPARK-42205][CORE] Don't log accumulator values in stage / task start and getting result event logs ### What changes were proposed in this pull request? This PR modifies JsonProtocol in order to skip logging of accumulator values in the logs for SparkListenerTaskStart, SparkListenerStageSubmitted, and SparkListenerTaskGettingResult events. These events contain mutable TaskInfo and StageInfo objects, which in turn contain Accumulables fields. When a task or stage is submitted, Accumulables is initially empty. When the task or stage finishes, this field is updated with values from the task. If a task or stage finishes _before_ the start event has been logged by the event logging listener then the start event will contain the Accumulable values from the task or stage end event. This PR updates JsonProtocol to log an empty Accumulables value for stage and task start events. I considered and rejected an alternative approach where the listener event itself would contain an immutable snapshot of the TaskInfo or StageInfo, as this will increase memory pressure on the driver during periods of heavy event logging. Those accumulables values in the start events are not used: I confirmed this by checking AppStatusListener and SQLAppStatusListener code. I have deliberately chosen to **not** drop the field for _job_ start events because it is technically possible (but rare) for a job to reference stages that are completed at the time that the job is submitted (a state can technically belong to multiple jobs) and in that case it seems consistent to have the StageInfo accurately reflect all of the information about the already-completed stage. ### Why are the changes needed? This information isn't used by the History Server and contributes to wasteful bloat in event log sizes. In one real-world log, I found that ~10% of the uncompressed log size was due to these redundant Accumulable fields in stage and task start events. I don't think that we need to worry about backwards-compatibility here because the old behavior was non-deterministic: whether or not a start event log contained accumulator updates was a function of the relative speed of task completion and the processing rate of the event logging listener; it seems unlikely that any third-party event log consumers would be relying on such an inconsistently present value when they could instead rely on the values in the corresponding end events. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New and updated tests in JsonProtocolSuite. Closes #39767 from JoshRosen/SPARK-42205-dont-log-accumulables-in-jsonprotocol-start-events. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/util/JsonProtocol.scala | 44 +++-- .../org/apache/spark/util/JsonProtocolSuite.scala | 183 + 2 files changed, 109 insertions(+), 118 deletions(-) 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 8654b658809..6525bd321e6 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -115,7 +115,8 @@ private[spark] object JsonProtocol extends JsonUtils { g.writeStartObject() g.writeStringField("Event", SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.stageSubmitted) g.writeFieldName("Stage Info") -stageInfoToJson(stageSubmitted.stageInfo, g) +// SPARK-42205: don't log accumulables in start events: +stageInfoToJson(stageSubmitted.stageInfo, g, includeAccumulables = false) Option(stageSubmitted.properties).foreach { properties => g.writeFieldName("Properties") propertiesToJson(properties, g) @@ -127,7 +128,7 @@ private[spark] object JsonProtocol extends JsonUtils { g.writeStartObject() g.writeStringField("Event", SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.stageCompleted) g.writeFieldName("Stage Info") -stageInfoToJson(stageCompleted.stageInfo, g) +stageInfoToJson(stageCompleted.stageInfo, g, includeAccumulables = true) g.writeEndObject() } @@ -137,7 +138,8 @@ private[spark] object JsonProt
[spark] branch master updated: [SPARK-44818] Fix race for pending task kill issued before taskThread is initialized
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c34ec411244 [SPARK-44818] Fix race for pending task kill issued before taskThread is initialized c34ec411244 is described below commit c34ec41124446164e9cfdd34101f25c6aa0ae235 Author: Anish Shrigondekar AuthorDate: Mon Aug 21 13:25:28 2023 -0700 [SPARK-44818] Fix race for pending task kill issued before taskThread is initialized ### What changes were proposed in this pull request? Fix race for pending task kill issued before taskThread is initialized ### Why are the changes needed? We see that there is a race for tasks that are interrupted through stage cancellation and that may be added to the TaskSet, but don't yet have taskThread initialized. Basically, we try to kill ongoing task attempts to handle stage cancellation ``` logInfo("Cancelling stage " + stageId) // Kill all running tasks for the stage. killAllTaskAttempts(stageId, interruptThread, reason = "Stage cancelled: " + reason) // Cancel all attempts for the stage. ``` However, there is a chance that taskThread is not initialized yet and we only set the reasonIfKilled. ``` def kill(interruptThread: Boolean, reason: String): Unit = { require(reason != null) _reasonIfKilled = reason if (context != null) { context.markInterrupted(reason) } if (interruptThread && taskThread != null) { taskThread.interrupt(). <--- never hit } ``` Then within the task execution thread itself, we try to call kill again since the reasonIfKilled is set. However, this time we pass interruptThread as false explicitly since we don't know the status of the previous call. ``` taskThread = Thread.currentThread() if (_reasonIfKilled != null) { kill(interruptThread = false, _reasonIfKilled) <-- only context will be set, } ``` The TaskReaper has also finished its previous and only attempt at task interruption since we don't try for multiple times in this case. Eventually, the task is not interrupted even once and it gets blocked on some I/O or wait calls which might not finish within the reaper timeout, leading to the JVM being killed. ``` taskRunner.kill(interruptThread = interruptThread, reason = reason) ``` The change tries to fix this issue by checking for the presence of `reasonIfKilled` on the context and issuing a `TaskKilledException` before we execute `runTask` thereby preventing execution of the actual task and freeing up the slot and also preventing future issues with the reaper. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing unit tests ``` [info] JobCancellationSuite: ... [info] Run completed in 35 seconds, 781 milliseconds. [info] Total number of tests run: 13 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 13, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` Closes #42504 from anishshri-db/task/SPARK-44818. Authored-by: Anish Shrigondekar Signed-off-by: Josh Rosen --- core/src/main/scala/org/apache/spark/TaskContext.scala | 5 + 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 450c00928c9..0f8a10d734b 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -158,6 +158,11 @@ abstract class TaskContext extends Serializable { /** Runs a task with this context, ensuring failure and completion listeners get triggered. */ private[spark] def runTaskWithListeners[T](task: Task[T]): T = { try { + // SPARK-44818 - Its possible that taskThread has not been initialized when kill is initially + // called with interruptThread=true. We do set the reason and eventually will set it on the + // context too within run(). If that's the case, kill the thread before it starts executing + // the actual task. + killTaskIfInterrupted() task.runTask(this) } catch { case e: Throwable => - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43300][CORE] NonFateSharingCache wrapper for Guava Cache
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new d53ddbe00fe [SPARK-43300][CORE] NonFateSharingCache wrapper for Guava Cache d53ddbe00fe is described below commit d53ddbe00fe73a703f870b0297278f3870148fc4 Author: Ziqi Liu AuthorDate: Mon May 15 18:47:29 2023 -0700 [SPARK-43300][CORE] NonFateSharingCache wrapper for Guava Cache ### What changes were proposed in this pull request? Create `NonFateSharingCache` to wrap around Guava cache with a KeyLock to synchronize all requests with the same key, so they will run individually and fail as if they come one at a time. Wrap cache in `CodeGenerator` with `NonFateSharingCache` to protect it from unexpected cascade failure due to cancellation from irrelevant queries that loading the same key. Feel free to use this in other places where we used Guava cache and don't want fate-sharing behavior. Also, instead of implementing Guava Cache and LoadingCache interface, I define a subset of it so that we can control at compile time what cache operations are allowed and make sure all cache loading action go through our narrow waist code path with key lock. Feel free to add new APIs when needed. ### Why are the changes needed? Guava cache is widely used in spark, however, it suffers from fate-sharing behavior: If there are multiple requests trying to access the same key in the cache at the same time when the key is not in the cache, Guava cache will block all requests and create the object only once. If the creation fails, all requests will fail immediately without retry. So we might see task failure due to irrelevant failure in other queries due to fate sharing. This fate sharing behavior leads to unexpected results in some situation(for example, in code gen). ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT Closes #40982 from liuzqt/SPARK-43300. Authored-by: Ziqi Liu Signed-off-by: Josh Rosen --- .../apache/spark/util/NonFateSharingCache.scala| 78 .../spark/util/NonFateSharingCacheSuite.scala | 140 + .../expressions/codegen/CodeGenerator.scala| 10 +- 3 files changed, 225 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala b/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala new file mode 100644 index 000..d9847313304 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/NonFateSharingCache.scala @@ -0,0 +1,78 @@ +/* + * 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.util + +import java.util.concurrent.Callable + +import com.google.common.cache.Cache +import com.google.common.cache.LoadingCache + +/** + * SPARK-43300: Guava cache fate-sharing behavior might lead to unexpected cascade failure: + * when multiple threads access the same key in the cache at the same time when the key is not in + * the cache, Guava cache will block all requests and load the data only once. If the loading fails, + * all requests will fail immediately without retry. Therefore individual failure will also fail + * other irrelevant queries who are waiting for the same key. Given that spark can cancel tasks at + * arbitrary times for many different reasons, fate sharing means that a task which gets canceled + * while populating a cache entry can cause spurious failures in tasks from unrelated jobs -- even + * though those tasks would have successfully populated the cache if they had been allowed to try. + * + * This util Cache wrapper with KeyLock to synchronize threads looking for the same key + * so that they should run individually and fail as if they had arrived one at a time. + * + * There are so many ways to add cache entries in Guava Cache, instead of implementing Guava Cache + * and LoadingCache interface, we expose a subset of APIs so that we can control
[spark] branch master updated: [SPARK-40261][CORE] Exclude DirectTaskResult metadata when calculating result size
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 5a4b075f95f [SPARK-40261][CORE] Exclude DirectTaskResult metadata when calculating result size 5a4b075f95f is described below commit 5a4b075f95f4cb305ba96d6de34d3c004e15f241 Author: Ziqi Liu AuthorDate: Wed Aug 31 17:38:35 2022 -0700 [SPARK-40261][CORE] Exclude DirectTaskResult metadata when calculating result size ### What changes were proposed in this pull request? When calculating driver result size, only counting actual result value while excluding other metadata (e.g., accumUpdates) in the serialized result task object. ### Why are the changes needed? metadata should not be counted because they will be discarded by the driver immediately after being processed, and will lead to unexpected exception when running jobs with tons of task but actually return small results. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test Closes #37713 from liuzqt/SPARK-40261. Lead-authored-by: Ziqi Liu Co-authored-by: liuzqt Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/scheduler/TaskResultGetter.scala | 2 +- .../org/apache/spark/scheduler/TaskResultGetterSuite.scala | 13 + 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 2dabee39131..cfc1f79fab2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -63,7 +63,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul try { val (result, size) = serializer.get().deserialize[TaskResult[_]](serializedData) match { case directResult: DirectTaskResult[_] => - if (!taskSetManager.canFetchMoreResults(serializedData.limit())) { + if (!taskSetManager.canFetchMoreResults(directResult.valueBytes.limit())) { // kill the task so that it will not become zombie task scheduler.handleFailedTask(taskSetManager, tid, TaskState.KILLED, TaskKilled( "Tasks result size has exceeded maxResultSize")) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index ea44a2d948c..1583d3b96ee 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -35,6 +35,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.TestUtils.JavaSourceFromString +import org.apache.spark.internal.config.MAX_RESULT_SIZE import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.storage.TaskResultBlockId import org.apache.spark.util.{MutableURLClassLoader, RpcUtils, ThreadUtils, Utils} @@ -297,6 +298,18 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local assert(unknownFailure.findFirstMatchIn(message).isDefined) } + test("SPARK-40261: task result metadata should not be counted into result size") { +val conf = new SparkConf().set(MAX_RESULT_SIZE.key, "1M") +sc = new SparkContext("local", "test", conf) +val rdd = sc.parallelize(1 to 1, 1) +// This will trigger 10k task but return empty result. The total serialized return tasks +// size(including accumUpdates metadata) would be ~10M in total in this example, but the result +// value itself is pretty small(empty arrays) +// Even setting MAX_RESULT_SIZE to a small value(1M here), it should not throw exception +// because the actual result is small +assert(rdd.filter(_ < 0).collect().isEmpty) + } + } private class UndeserializableException extends Exception { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-40235][CORE] Use interruptible lock instead of synchronized in Executor.updateDependencies()
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 295dd57c13c [SPARK-40235][CORE] Use interruptible lock instead of synchronized in Executor.updateDependencies() 295dd57c13c is described below commit 295dd57c13caaa9f9e78cd46dfda4e17ced7c449 Author: Josh Rosen AuthorDate: Mon Aug 29 16:47:38 2022 -0700 [SPARK-40235][CORE] Use interruptible lock instead of synchronized in Executor.updateDependencies() ### What changes were proposed in this pull request? This patch modifies the synchronization in `Executor.updateDependencies()` in order to allow tasks to be interrupted while they are blocked and waiting on other tasks to finish downloading dependencies. This synchronization was added years ago in https://github.com/mesos/spark/commit/7b9e96c99206c0679d9925e0161fde738a5c7c3a in order to prevent concurrently-launching tasks from performing concurrent dependency updates. If one task is downloading dependencies, all other newly-launched tasks will block until the original dependency download is complete. Let's say that a Spark task launches, becomes blocked on a `updateDependencies()` call, then is cancelled while it is blocked. Although Spark will send a `Thread.interrupt()` to the canceled task, the task will continue waiting because threads blocked on a `synchronized` won't throw an InterruptedException in response to the interrupt. As a result, the blocked thread will continue to wait until the other thread exits the synchronized block. This PR aims to fix this problem by replacing the `synchronized` with a `ReentrantLock`, which has a `lockInterruptibly` method. ### Why are the changes needed? In a real-world scenario, we hit a case where a task was canceled right after being launched while another task was blocked in a slow library download. The slow library download took so long that the TaskReaper killed the executor because the canceled task could not exit in a timely fashion. This patch's fix prevents this issue. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test case. Closes #37681 from JoshRosen/SPARK-40235-update-dependencies-lock. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/executor/Executor.scala | 22 +++-- .../org/apache/spark/executor/ExecutorSuite.scala | 53 ++ 2 files changed, 72 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index ab2bd1b7801..db507bd176b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -25,6 +25,7 @@ import java.nio.ByteBuffer import java.util.{Locale, Properties} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.locks.ReentrantLock import javax.annotation.concurrent.GuardedBy import javax.ws.rs.core.UriBuilder @@ -85,6 +86,11 @@ private[spark] class Executor( private[executor] val conf = env.conf + // SPARK-40235: updateDependencies() uses a ReentrantLock instead of the `synchronized` keyword + // so that tasks can exit quickly if they are interrupted while waiting on another task to + // finish downloading dependencies. + private val updateDependenciesLock = new ReentrantLock() + // No ip or host:port - just hostname Utils.checkHost(executorHostname) // must not have port specified. @@ -978,13 +984,19 @@ private[spark] class Executor( /** * Download any missing dependencies if we receive a new set of files and JARs from the * SparkContext. Also adds any new JARs we fetched to the class loader. + * Visible for testing. */ - private def updateDependencies( + private[executor] def updateDependencies( newFiles: Map[String, Long], newJars: Map[String, Long], - newArchives: Map[String, Long]): Unit = { + newArchives: Map[String, Long], + testStartLatch: Option[CountDownLatch] = None, + testEndLatch: Option[CountDownLatch] = None): Unit = { lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) -synchronized { +updateDependenciesLock.lockInterruptibly() +try { + // For testing, so we can simulate a slow file download: + testStartLatch.foreach(_.countDown()) // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo(s"Fetching $name with timestamp $timestamp") @@ -1027,6 +1039,10 @
[spark] branch master updated: [SPARK-40211][CORE][SQL] Allow customize initial partitions number in take() behavior
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1178bcecc83 [SPARK-40211][CORE][SQL] Allow customize initial partitions number in take() behavior 1178bcecc83 is described below commit 1178bcecc83925674cad4364537a39eba03e423e Author: Ziqi Liu AuthorDate: Fri Aug 26 17:18:09 2022 -0700 [SPARK-40211][CORE][SQL] Allow customize initial partitions number in take() behavior ### What changes were proposed in this pull request? [SPARK-40211](https://issues.apache.org/jira/browse/SPARK-40211) add a `initialNumPartitions` config parameter to allow customizing initial partitions to try in `take()` ### Why are the changes needed? Currently, the initial partitions to try to hardcode to `1`, which might cause unnecessary overhead. By setting this new configuration to a high value we could effectively mitigate the “run multiple jobs” overhead in take behavior. We could also set it to higher-than-1-but-still-small values (like, say, 10) to achieve a middle-ground trade-off. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Unit test Closes #37661 from liuzqt/SPARK-40211. Authored-by: Ziqi Liu Signed-off-by: Josh Rosen --- .../org/apache/spark/internal/config/package.scala | 7 .../org/apache/spark/rdd/AsyncRDDActions.scala | 17 + core/src/main/scala/org/apache/spark/rdd/RDD.scala | 8 ++--- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 39 +++- .../org/apache/spark/sql/internal/SQLConf.scala| 12 +++ .../org/apache/spark/sql/execution/SparkPlan.scala | 12 +++ .../org/apache/spark/sql/ConfigBehaviorSuite.scala | 41 ++ 7 files changed, 118 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 9d1a56843ca..07d3d3e0778 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1956,6 +1956,13 @@ package object config { .intConf .createWithDefault(10) + private[spark] val RDD_LIMIT_INITIAL_NUM_PARTITIONS = +ConfigBuilder("spark.rdd.limit.initialNumPartitions") + .version("3.4.0") + .intConf + .checkValue(_ > 0, "value should be positive") + .createWithDefault(1) + private[spark] val RDD_LIMIT_SCALE_UP_FACTOR = ConfigBuilder("spark.rdd.limit.scaleUpFactor") .version("2.1.0") diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index d6379156ccf..9f89c82db31 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -25,6 +25,7 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, JobSubmitter} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{RDD_LIMIT_INITIAL_NUM_PARTITIONS, RDD_LIMIT_SCALE_UP_FACTOR} import org.apache.spark.util.ThreadUtils /** @@ -72,6 +73,8 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi val results = new ArrayBuffer[T] val totalParts = self.partitions.length +val scaleUpFactor = Math.max(self.conf.get(RDD_LIMIT_SCALE_UP_FACTOR), 2) + /* Recursively triggers jobs to scan partitions until either the requested number of elements are retrieved, or the partitions to scan are exhausted. @@ -84,18 +87,18 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi } else { // The number of partitions to try in this iteration. It is ok for this number to be // greater than totalParts because we actually cap it at totalParts in runJob. -var numPartsToTry = 1L +var numPartsToTry = self.conf.get(RDD_LIMIT_INITIAL_NUM_PARTITIONS) if (partsScanned > 0) { - // If we didn't find any rows after the previous iteration, quadruple and retry. - // Otherwise, interpolate the number of partitions we need to try, but overestimate it - // by 50%. We also cap the estimation in the end. - if (results.size == 0) { -numPartsToTry = partsScanned * 4L + // If we didn't find any rows after the previous iteration, multiply by + // limitScaleUpFactor and retry. Otherwise, interpolate the number of partitions we need + // to try, but overestimate it by 50%. We also cap the estimation in the end. +
[spark] branch master updated (50c163578cf -> 6cd9d88e237)
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from 50c163578cf Revert "[SPARK-4][SQL] Update INSERTs without user-specified fields to not automatically add default values" add 6cd9d88e237 [SPARK-40106] Task failure should always trigger task failure listeners No new revisions were added by this update. Summary of changes: .../main/scala/org/apache/spark/TaskContext.scala | 41 - .../scala/org/apache/spark/TaskContextImpl.scala | 59 ++- .../scala/org/apache/spark/scheduler/Task.scala| 53 ++ .../apache/spark/scheduler/TaskContextSuite.scala | 186 + 4 files changed, 295 insertions(+), 44 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-39983][CORE][SQL] Do not cache unserialized broadcast relations on the driver
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new e17d8ecabca [SPARK-39983][CORE][SQL] Do not cache unserialized broadcast relations on the driver e17d8ecabca is described below commit e17d8ecabcad6e84428752b977120ff355a4007a Author: Alex Balikov AuthorDate: Wed Aug 10 17:13:03 2022 -0700 [SPARK-39983][CORE][SQL] Do not cache unserialized broadcast relations on the driver ### What changes were proposed in this pull request? This PR addresses the issue raised in https://issues.apache.org/jira/browse/SPARK-39983 - broadcast relations should not be cached on the driver as they are not needed and can cause significant memory pressure (in one case the relation was 60MB ) The PR adds a new SparkContext.broadcastInternal method with parameter serializedOnly allowing the caller to specify that the broadcasted object should be stored only in serialized form. The current behavior is to also cache an unserialized form of the object. The PR changes the broadcast implementation in TorrentBroadcast to honor the serializedOnly flag and not store the unserialized value, unless the execution is in a local mode (single process). In that case the broadcast cache is effectively shared between driver and executors and thus the unserialized value needs to be cached to satisfy the executor-side of the functionality. ### Why are the changes needed? The broadcast relations can be fairly large (observed 60MB one) and are not needed in unserialized form on the driver. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a new unit test to BroadcastSuite verifying the low-level broadcast functionality in respect to the serializedOnly flag. Added a new unit test to BroadcastExchangeSuite verifying that broadcasted relations are not cached on the driver. Closes #37413 from alex-balikov/SPARK-39983-broadcast-no-cache. Lead-authored-by: Alex Balikov Co-authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../main/scala/org/apache/spark/SparkContext.scala | 19 +- .../apache/spark/broadcast/BroadcastFactory.scala | 8 ++- .../apache/spark/broadcast/BroadcastManager.scala | 7 ++- .../apache/spark/broadcast/TorrentBroadcast.scala | 67 +- .../spark/broadcast/TorrentBroadcastFactory.scala | 8 ++- .../apache/spark/broadcast/BroadcastSuite.scala| 19 ++ .../execution/exchange/BroadcastExchangeExec.scala | 4 +- .../sql/execution/BroadcastExchangeSuite.scala | 29 +- 8 files changed, 136 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6cb4f04ac7f..f101dc8e083 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1511,16 +1511,31 @@ class SparkContext(config: SparkConf) extends Logging { /** * Broadcast a read-only variable to the cluster, returning a * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. - * The variable will be sent to each cluster only once. + * The variable will be sent to each executor only once. * * @param value value to broadcast to the Spark nodes * @return `Broadcast` object, a read-only variable cached on each machine */ def broadcast[T: ClassTag](value: T): Broadcast[T] = { +broadcastInternal(value, serializedOnly = false) + } + + /** + * Internal version of broadcast - broadcast a read-only variable to the cluster, returning a + * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. + * The variable will be sent to each executor only once. + * + * @param value value to broadcast to the Spark nodes + * @param serializedOnly if true, do not cache the unserialized value on the driver + * @return `Broadcast` object, a read-only variable cached on each machine + */ + private[spark] def broadcastInternal[T: ClassTag]( + value: T, + serializedOnly: Boolean): Broadcast[T] = { assertNotStopped() require(!classOf[RDD[_]].isAssignableFrom(classTag[T].runtimeClass), "Can not directly broadcast RDDs; instead, call collect() and broadcast the result.") -val bc = env.broadcastManager.newBroadcast[T](value, isLocal) +val bc = env.broadcastManager.newBroadcast[T](value, isLocal, serializedOnly) val callSite = getCallSite logInfo("Created broadcast " + bc.id + " from " + callSite.shortForm) cleaner.foreach(_.registerBroadcastForCleanup(bc)) diff --git a/core/src/main/sc
[spark] branch master updated: [SPARK-39636][CORE][UI] Fix multiple bugs in JsonProtocol, impacting off heap StorageLevels and Task/Executor ResourceRequests
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new a39fc8773b2 [SPARK-39636][CORE][UI] Fix multiple bugs in JsonProtocol, impacting off heap StorageLevels and Task/Executor ResourceRequests a39fc8773b2 is described below commit a39fc8773b2a4e9c58a1e5d0010e0c8396784c37 Author: Josh Rosen AuthorDate: Thu Jun 30 13:41:24 2022 -0700 [SPARK-39636][CORE][UI] Fix multiple bugs in JsonProtocol, impacting off heap StorageLevels and Task/Executor ResourceRequests ### What changes were proposed in this pull request? This PR fixes three longstanding bugs in Spark's `JsonProtocol`: - `TaskResourceRequest` loses precision for `amount` < 0.5. The `amount` is a floating point number which is either between 0 and 0.5 or is a positive integer, but the JSON read path assumes it is an integer. - `ExecutorResourceRequest` integer overflows for values larger than Int.MaxValue because the write path writes longs but the read path assumes integers. - Off heap StorageLevels are not handled properly: the `useOffHeap` field isn't included in the JSON, so this StorageLevel cannot be round-tripped through JSON. This could cause the History Server to display inaccurate "off heap memory used" stats on the executors page. I discovered these bugs while working on #36885. ### Why are the changes needed? JsonProtocol should be able to roundtrip events through JSON without loss of information. ### Does this PR introduce _any_ user-facing change? Yes: it fixes bugs that impact information shown in the History Server Web UI. The new StorageLevel JSON field will be visible to tools which process raw event log JSON. ### How was this patch tested? Updated existing unit tests to cover the changed logic. Closes #37027 from JoshRosen/jsonprotocol-bugfixes. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/util/JsonProtocol.scala | 17 +-- .../org/apache/spark/util/JsonProtocolSuite.scala | 53 -- 2 files changed, 64 insertions(+), 6 deletions(-) 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 0c15b13d5a1..f0755b04bef 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -512,6 +512,7 @@ private[spark] object JsonProtocol { def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ ("Use Memory" -> storageLevel.useMemory) ~ +("Use Off Heap" -> storageLevel.useOffHeap) ~ ("Deserialized" -> storageLevel.deserialized) ~ ("Replication" -> storageLevel.replication) } @@ -750,7 +751,7 @@ private[spark] object JsonProtocol { def executorResourceRequestFromJson(json: JValue): ExecutorResourceRequest = { val rName = (json \ "Resource Name").extract[String] -val amount = (json \ "Amount").extract[Int] +val amount = (json \ "Amount").extract[Long] val discoveryScript = (json \ "Discovery Script").extract[String] val vendor = (json \ "Vendor").extract[String] new ExecutorResourceRequest(rName, amount, discoveryScript, vendor) @@ -758,7 +759,7 @@ private[spark] object JsonProtocol { def taskResourceRequestFromJson(json: JValue): TaskResourceRequest = { val rName = (json \ "Resource Name").extract[String] -val amount = (json \ "Amount").extract[Int] +val amount = (json \ "Amount").extract[Double] new TaskResourceRequest(rName, amount) } @@ -1202,9 +1203,19 @@ private[spark] object JsonProtocol { def storageLevelFromJson(json: JValue): StorageLevel = { val useDisk = (json \ "Use Disk").extract[Boolean] val useMemory = (json \ "Use Memory").extract[Boolean] +// The "Use Off Heap" field was added in Spark 3.4.0 +val useOffHeap = jsonOption(json \ "Use Off Heap") match { + case Some(value) => value.extract[Boolean] + case None => false +} val deserialized = (json \ "Deserialized").extract[Boolean] val replication = (json \ "Replication").extract[Int] -StorageLevel(useDisk, useMemory, deserialized, replication) +StorageLevel( + useDisk = useDisk, + useMemory = useMemory, + useOffHeap = useOffHeap, + deserialized = deserialized, + replication = replication) } def blockStatusFromJson(json: JValue): BlockStatus = { diff --git a/core/src/test/scala/o
[spark] branch branch-3.2 updated: [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 7fd2e967a8a [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes 7fd2e967a8a is described below commit 7fd2e967a8a0d5ca05b0b313d08283b684e49e29 Author: Josh Rosen AuthorDate: Thu Jun 9 12:34:27 2022 -0700 [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes ### What changes were proposed in this pull request? This PR improves the error message that is thrown when trying to run `SHOW CREATE TABLE` on a Hive table with an unsupported serde. Currently this results in an error like ``` org.apache.spark.sql.AnalysisException: Failed to execute SHOW CREATE TABLE against table rcFileTable, which is created by Hive and uses the following unsupported serde configuration SERDE: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe INPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileInputFormat OUTPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileOutputFormat ``` This patch improves this error message by adding a suggestion to use `SHOW CREATE TABLE ... AS SERDE`: ``` org.apache.spark.sql.AnalysisException: Failed to execute SHOW CREATE TABLE against table rcFileTable, which is created by Hive and uses the following unsupported serde configuration SERDE: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe INPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileInputFormat OUTPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileOutputFormat Please use `SHOW CREATE TABLE rcFileTable AS SERDE` to show Hive DDL instead. ``` The suggestion's wording is consistent with other error messages thrown by SHOW CREATE TABLE. ### Why are the changes needed? The existing error message is confusing. ### Does this PR introduce _any_ user-facing change? Yes, it improves a user-facing error message. ### How was this patch tested? Manually tested with ``` CREATE TABLE rcFileTable(i INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' SHOW CREATE TABLE rcFileTable ``` to trigger the error. Confirmed that the `AS SERDE` suggestion actually works. Closes #36814 from JoshRosen/suggest-show-create-table-as-serde-in-error-message. Authored-by: Josh Rosen Signed-off-by: Josh Rosen (cherry picked from commit 8765eea1c08bc58a0cfc22b7cfbc0b5645cc81f9) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index e3102aa1e02..dcf4d23f562 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1973,7 +1973,8 @@ private[sql] object QueryCompilationErrors { new AnalysisException("Failed to execute SHOW CREATE TABLE against table " + s"${table.identifier}, which is created by Hive and uses the " + "following unsupported serde configuration\n" + -builder.toString() +builder.toString() + "\n" + +s"Please use `SHOW CREATE TABLE ${table.identifier} AS SERDE` to show Hive DDL instead." ) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.3 updated: [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.3 by this push: new ff048f1b69e [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes ff048f1b69e is described below commit ff048f1b69e5520c1fedbfd9869717f0b8919c0f Author: Josh Rosen AuthorDate: Thu Jun 9 12:34:27 2022 -0700 [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes ### What changes were proposed in this pull request? This PR improves the error message that is thrown when trying to run `SHOW CREATE TABLE` on a Hive table with an unsupported serde. Currently this results in an error like ``` org.apache.spark.sql.AnalysisException: Failed to execute SHOW CREATE TABLE against table rcFileTable, which is created by Hive and uses the following unsupported serde configuration SERDE: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe INPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileInputFormat OUTPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileOutputFormat ``` This patch improves this error message by adding a suggestion to use `SHOW CREATE TABLE ... AS SERDE`: ``` org.apache.spark.sql.AnalysisException: Failed to execute SHOW CREATE TABLE against table rcFileTable, which is created by Hive and uses the following unsupported serde configuration SERDE: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe INPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileInputFormat OUTPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileOutputFormat Please use `SHOW CREATE TABLE rcFileTable AS SERDE` to show Hive DDL instead. ``` The suggestion's wording is consistent with other error messages thrown by SHOW CREATE TABLE. ### Why are the changes needed? The existing error message is confusing. ### Does this PR introduce _any_ user-facing change? Yes, it improves a user-facing error message. ### How was this patch tested? Manually tested with ``` CREATE TABLE rcFileTable(i INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' SHOW CREATE TABLE rcFileTable ``` to trigger the error. Confirmed that the `AS SERDE` suggestion actually works. Closes #36814 from JoshRosen/suggest-show-create-table-as-serde-in-error-message. Authored-by: Josh Rosen Signed-off-by: Josh Rosen (cherry picked from commit 8765eea1c08bc58a0cfc22b7cfbc0b5645cc81f9) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index ebf40b4b5d0..d877bb5b2a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1989,7 +1989,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException("Failed to execute SHOW CREATE TABLE against table " + s"${table.identifier}, which is created by Hive and uses the " + "following unsupported serde configuration\n" + -builder.toString() +builder.toString() + "\n" + +s"Please use `SHOW CREATE TABLE ${table.identifier} AS SERDE` to show Hive DDL instead." ) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8765eea1c08 [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes 8765eea1c08 is described below commit 8765eea1c08bc58a0cfc22b7cfbc0b5645cc81f9 Author: Josh Rosen AuthorDate: Thu Jun 9 12:34:27 2022 -0700 [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes ### What changes were proposed in this pull request? This PR improves the error message that is thrown when trying to run `SHOW CREATE TABLE` on a Hive table with an unsupported serde. Currently this results in an error like ``` org.apache.spark.sql.AnalysisException: Failed to execute SHOW CREATE TABLE against table rcFileTable, which is created by Hive and uses the following unsupported serde configuration SERDE: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe INPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileInputFormat OUTPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileOutputFormat ``` This patch improves this error message by adding a suggestion to use `SHOW CREATE TABLE ... AS SERDE`: ``` org.apache.spark.sql.AnalysisException: Failed to execute SHOW CREATE TABLE against table rcFileTable, which is created by Hive and uses the following unsupported serde configuration SERDE: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe INPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileInputFormat OUTPUTFORMAT: org.apache.hadoop.hive.ql.io.RCFileOutputFormat Please use `SHOW CREATE TABLE rcFileTable AS SERDE` to show Hive DDL instead. ``` The suggestion's wording is consistent with other error messages thrown by SHOW CREATE TABLE. ### Why are the changes needed? The existing error message is confusing. ### Does this PR introduce _any_ user-facing change? Yes, it improves a user-facing error message. ### How was this patch tested? Manually tested with ``` CREATE TABLE rcFileTable(i INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' SHOW CREATE TABLE rcFileTable ``` to trigger the error. Confirmed that the `AS SERDE` suggestion actually works. Closes #36814 from JoshRosen/suggest-show-create-table-as-serde-in-error-message. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 68f4320ff67..2a8692efd0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1992,7 +1992,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException("Failed to execute SHOW CREATE TABLE against table " + s"${table.identifier}, which is created by Hive and uses the " + "following unsupported serde configuration\n" + -builder.toString() +builder.toString() + "\n" + +s"Please use `SHOW CREATE TABLE ${table.identifier} AS SERDE` to show Hive DDL instead." ) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.3 updated: [SPARK-39361] Don't use Log4J2's extended throwable conversion pattern in default logging configurations
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.3 by this push: new 4da8f3a76b1 [SPARK-39361] Don't use Log4J2's extended throwable conversion pattern in default logging configurations 4da8f3a76b1 is described below commit 4da8f3a76b196383e00664e4d1c863f5fe927474 Author: Josh Rosen AuthorDate: Thu Jun 2 09:28:34 2022 -0700 [SPARK-39361] Don't use Log4J2's extended throwable conversion pattern in default logging configurations ### What changes were proposed in this pull request? This PR addresses a performance problem in Log4J 2 related to exception logging: in certain scenarios I observed that Log4J2's default exception stacktrace logging can be ~10x slower than Log4J 1. The problem stems from a new log pattern format in Log4J2 called ["extended exception"](https://logging.apache.org/log4j/2.x/manual/layouts.html#PatternExtendedException), which enriches the regular stacktrace string with information on the name of the JAR files that contained the classes in each stack frame. Log4J queries the classloader to determine the source JAR for each class. This isn't cheap, but this information is cached and reused in future exception logging calls. In certain scenarios involving runtime-generated classes, this lookup will fail and the failed lookup result will _not_ be cached. As a result, expensive classloading operations will be performed every time such an exception is logged. In addition to being very slow, these operations take out a lock on the classloader [...] By default, if you do not specify an explicit exception format in your logging pattern then Log4J2 will add this "extended exception" pattern (see PatternLayout's alwaysWriteExceptions flag in Log4J's documentation, plus [the code implementing that flag](https://github.com/apache/logging-log4j2/blob/d6c8ab0863c551cdf0f8a5b1966ab45e3cddf572/log4j-core/src/main/java/org/apache/logging/log4j/core/pattern/PatternParser.java#L206-L209) in Log4J2). In this PR, I have updated Spark's default Log4J2 configurations so that each pattern layout includes an explicit %ex so that it uses the normal (non-extended) exception logging format. This is the workaround that is currently recommended on the Log4J JIRA. ### Why are the changes needed? Avoid performance regressions in Spark programs which use Spark's default Log4J 2 configuration and log many exceptions. Although it's true that any program logging exceptions at a high rate should probably just fix the source of the exceptions, I think it's still a good idea for us to try to fix this out-of-the-box performance difference so that users' existing workloads do not regress when upgrading to 3.3.0. ### Does this PR introduce _any_ user-facing change? Yes: it changes the default exception logging format so that it matches Log4J 1's default rather than Log4J 2's. The new format is consistent with behavior in previous Spark versions, but is different than the behavior in the current Spark 3.3.0-rc3. ### How was this patch tested? Existing tests. Closes #36747 from JoshRosen/disable-log4j2-extended-exception-pattern. Authored-by: Josh Rosen Signed-off-by: Josh Rosen (cherry picked from commit fd45c3656be6add7cf483ddfb7016b12f77d7c8e) Signed-off-by: Josh Rosen --- R/log4j2.properties | 2 +- common/kvstore/src/test/resources/log4j2.properties | 2 +- common/network-common/src/test/resources/log4j2.properties| 2 +- common/network-shuffle/src/test/resources/log4j2.properties | 2 +- conf/log4j2.properties.template | 8 +++- .../main/resources/org/apache/spark/log4j2-defaults.properties| 2 +- core/src/main/scala/org/apache/spark/TestUtils.scala | 2 +- .../main/scala/org/apache/spark/util/logging/DriverLogger.scala | 2 +- core/src/test/resources/log4j2.properties | 4 ++-- docs/configuration.md | 2 +- external/avro/src/test/resources/log4j2.properties| 2 +- .../docker-integration-tests/src/test/resources/log4j2.properties | 4 ++-- external/kafka-0-10-sql/src/test/resources/log4j2.properties | 2 +- .../src/test/resources/log4j2.properties | 2 +- external/kafka-0-10/src/test/resources/log4j2.properties | 2 +- external/kinesis-asl/src/main/resources/log4j2.properties | 4 ++-- external/kinesis-asl/src/test/resources/log4j2.properties | 2 +- graphx/src/test/resources/log4j2.properties
[spark] branch master updated: [SPARK-39361] Don't use Log4J2's extended throwable conversion pattern in default logging configurations
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new fd45c3656be [SPARK-39361] Don't use Log4J2's extended throwable conversion pattern in default logging configurations fd45c3656be is described below commit fd45c3656be6add7cf483ddfb7016b12f77d7c8e Author: Josh Rosen AuthorDate: Thu Jun 2 09:28:34 2022 -0700 [SPARK-39361] Don't use Log4J2's extended throwable conversion pattern in default logging configurations ### What changes were proposed in this pull request? This PR addresses a performance problem in Log4J 2 related to exception logging: in certain scenarios I observed that Log4J2's default exception stacktrace logging can be ~10x slower than Log4J 1. The problem stems from a new log pattern format in Log4J2 called ["extended exception"](https://logging.apache.org/log4j/2.x/manual/layouts.html#PatternExtendedException), which enriches the regular stacktrace string with information on the name of the JAR files that contained the classes in each stack frame. Log4J queries the classloader to determine the source JAR for each class. This isn't cheap, but this information is cached and reused in future exception logging calls. In certain scenarios involving runtime-generated classes, this lookup will fail and the failed lookup result will _not_ be cached. As a result, expensive classloading operations will be performed every time such an exception is logged. In addition to being very slow, these operations take out a lock on the classloader [...] By default, if you do not specify an explicit exception format in your logging pattern then Log4J2 will add this "extended exception" pattern (see PatternLayout's alwaysWriteExceptions flag in Log4J's documentation, plus [the code implementing that flag](https://github.com/apache/logging-log4j2/blob/d6c8ab0863c551cdf0f8a5b1966ab45e3cddf572/log4j-core/src/main/java/org/apache/logging/log4j/core/pattern/PatternParser.java#L206-L209) in Log4J2). In this PR, I have updated Spark's default Log4J2 configurations so that each pattern layout includes an explicit %ex so that it uses the normal (non-extended) exception logging format. This is the workaround that is currently recommended on the Log4J JIRA. ### Why are the changes needed? Avoid performance regressions in Spark programs which use Spark's default Log4J 2 configuration and log many exceptions. Although it's true that any program logging exceptions at a high rate should probably just fix the source of the exceptions, I think it's still a good idea for us to try to fix this out-of-the-box performance difference so that users' existing workloads do not regress when upgrading to 3.3.0. ### Does this PR introduce _any_ user-facing change? Yes: it changes the default exception logging format so that it matches Log4J 1's default rather than Log4J 2's. The new format is consistent with behavior in previous Spark versions, but is different than the behavior in the current Spark 3.3.0-rc3. ### How was this patch tested? Existing tests. Closes #36747 from JoshRosen/disable-log4j2-extended-exception-pattern. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- R/log4j2.properties | 2 +- common/kvstore/src/test/resources/log4j2.properties | 2 +- common/network-common/src/test/resources/log4j2.properties| 2 +- common/network-shuffle/src/test/resources/log4j2.properties | 2 +- conf/log4j2.properties.template | 8 +++- connector/avro/src/test/resources/log4j2.properties | 2 +- .../docker-integration-tests/src/test/resources/log4j2.properties | 4 ++-- connector/kafka-0-10-sql/src/test/resources/log4j2.properties | 2 +- .../src/test/resources/log4j2.properties | 2 +- connector/kafka-0-10/src/test/resources/log4j2.properties | 2 +- connector/kinesis-asl/src/main/resources/log4j2.properties| 4 ++-- connector/kinesis-asl/src/test/resources/log4j2.properties| 2 +- .../main/resources/org/apache/spark/log4j2-defaults.properties| 2 +- core/src/main/scala/org/apache/spark/TestUtils.scala | 2 +- .../main/scala/org/apache/spark/util/logging/DriverLogger.scala | 2 +- core/src/test/resources/log4j2.properties | 4 ++-- docs/configuration.md | 2 +- graphx/src/test/resources/log4j2.properties | 2 +- hadoop-cloud/src/test/resources/log4j2.properties | 4 ++-- launcher/src
[spark] 01/02: [SPARK-32911][CORE] Free memory in UnsafeExternalSorter.SpillableIterator.spill() when all records have been read
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git commit 02488e0df30a25fd235e2dd25e9b1b3404150125 Author: Tom van Bussel AuthorDate: Fri Sep 18 11:49:26 2020 + [SPARK-32911][CORE] Free memory in UnsafeExternalSorter.SpillableIterator.spill() when all records have been read ### What changes were proposed in this pull request? This PR changes `UnsafeExternalSorter.SpillableIterator` to free its memory (except for the page holding the last record) if it is forced to spill after all of its records have been read. It also makes sure that `lastPage` is freed if `loadNext` is never called the again. The latter was necessary to get my test case to succeed (otherwise it would complain about a leak). ### Why are the changes needed? No memory is freed after calling `UnsafeExternalSorter.SpillableIterator.spill()` when all records have been read, even though it is still holding onto some memory. This may cause a `SparkOutOfMemoryError` to be thrown, even though we could have just freed the memory instead. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? A test was added to `UnsafeExternalSorterSuite`. Closes #29787 from tomvanbussel/SPARK-32911. Authored-by: Tom van Bussel Signed-off-by: Wenchen Fan --- .../unsafe/sort/UnsafeExternalSorter.java | 31 +++--- .../unsafe/sort/UnsafeExternalSorterSuite.java | 30 + 2 files changed, 52 insertions(+), 9 deletions(-) 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 e4e369baf9d..c38327cae8c 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 @@ -553,7 +553,7 @@ public final class UnsafeExternalSorter extends MemoryConsumer { private UnsafeSorterIterator upstream; private MemoryBlock lastPage = null; private boolean loaded = false; -private int numRecords = 0; +private int numRecords; private Object currentBaseObject; private long currentBaseOffset; @@ -577,19 +577,25 @@ public final class UnsafeExternalSorter extends MemoryConsumer { public long spill() throws IOException { synchronized (this) { -if (inMemSorter == null || numRecords <= 0) { +if (inMemSorter == null) { return 0L; } long currentPageNumber = upstream.getCurrentPageNumber(); ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); -// Iterate over the records that have not been returned and spill them. -final UnsafeSorterSpillWriter spillWriter = - new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, numRecords); -spillIterator(upstream, spillWriter); -spillWriters.add(spillWriter); -upstream = spillWriter.getReader(serializerManager); +if (numRecords > 0) { + // Iterate over the records that have not been returned and spill them. + final UnsafeSorterSpillWriter spillWriter = new UnsafeSorterSpillWriter( + blockManager, fileBufferSizeBytes, writeMetrics, numRecords); + spillIterator(upstream, spillWriter); + spillWriters.add(spillWriter); + upstream = spillWriter.getReader(serializerManager); +} else { + // Nothing to spill as all records have been read already, but do not return yet, as the + // memory still has to be freed. + upstream = null; +} long released = 0L; synchronized (UnsafeExternalSorter.this) { @@ -605,6 +611,11 @@ public final class UnsafeExternalSorter extends MemoryConsumer { } } allocatedPages.clear(); + if (lastPage != null) { +// Add the last page back to the list of allocated pages to make sure it gets freed in +// case loadNext() never gets called again. +allocatedPages.add(lastPage); + } } // in-memory sorter will not be used after spilling @@ -627,11 +638,12 @@ public final class UnsafeExternalSorter extends MemoryConsumer { @Override public void loadNext() throws IOException { + assert upstream != null; MemoryBlock pageToFree = null; try { synchronized (this) { loaded = true; - // Just consumed the last record from in memory iterator + // Just consumed the last record from the in-memory iterator. if (lastPage != null) { // Do not free th
[spark] 02/02: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git commit c653d287b15db3e50fa206071a5435028879f15f Author: sandeepvinayak AuthorDate: Tue May 31 15:28:07 2022 -0700 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator ### What changes were proposed in this pull request? This PR fixes a deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator. ### Why are the changes needed? We are facing the deadlock issue b/w TaskMemoryManager and UnsafeExternalSorter.SpillableIterator during the join. It turns out that in UnsafeExternalSorter.SpillableIterator#spill() function, it tries to get lock on UnsafeExternalSorter`SpillableIterator` and UnsafeExternalSorter and call `freePage` to free all allocated pages except the last one which takes the lock on TaskMemoryManager. At the same time, there can be another `MemoryConsumer` using `UnsafeExternalSorter` as part of sorting can try to allocatePage needs to get lock on `TaskMemoryManager` which can cause spill to happen which requires lock on `UnsafeExternalSorter` again causing deadlock. There is a similar fix here as well: https://issues.apache.org/jira/browse/SPARK-27338 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #36680 from sandeepvinayak/SPARK-39283. Authored-by: sandeepvinayak Signed-off-by: Josh Rosen --- .../unsafe/sort/UnsafeExternalSorter.java | 160 + 1 file changed, 104 insertions(+), 56 deletions(-) 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 c38327cae8c..ac8170c9d97 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 @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.LinkedList; +import java.util.List; import java.util.Queue; import java.util.function.Supplier; @@ -298,16 +299,30 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * @return the number of bytes freed. */ private long freeMemory() { -updatePeakMemoryUsed(); +List pagesToFree = clearAndGetAllocatedPagesToFree(); long memoryFreed = 0; -for (MemoryBlock block : allocatedPages) { +for (MemoryBlock block : pagesToFree) { memoryFreed += block.size(); freePage(block); } +return memoryFreed; + } + + /** + * Clear the allocated pages and return the list of allocated pages to let + * the caller free the page. This is to prevent the deadlock by nested locks + * if the caller locks the UnsafeExternalSorter and call freePage which locks the + * TaskMemoryManager and cause nested locks. + * + * @return list of allocated pages to free + */ + private List clearAndGetAllocatedPagesToFree() { +updatePeakMemoryUsed(); +List pagesToFree = new LinkedList<>(allocatedPages); allocatedPages.clear(); currentPage = null; pageCursor = 0; -return memoryFreed; +return pagesToFree; } /** @@ -328,12 +343,27 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * Frees this sorter's in-memory data structures and cleans up its spill files. */ public void cleanupResources() { -synchronized (this) { - deleteSpillFiles(); - freeMemory(); - if (inMemSorter != null) { -inMemSorter.freeMemory(); -inMemSorter = null; +// To avoid deadlocks, we can't call methods that lock the TaskMemoryManager +// (such as various free() methods) while synchronizing on the UnsafeExternalSorter. +// Instead, we will manipulate UnsafeExternalSorter state inside the synchronized +// lock and perform the actual free() calls outside it. +UnsafeInMemorySorter inMemSorterToFree = null; +List pagesToFree = null; +try { + synchronized (this) { +deleteSpillFiles(); +pagesToFree = clearAndGetAllocatedPagesToFree(); +if (inMemSorter != null) { + inMemSorterToFree = inMemSorter; + inMemSorter = null; +} + } +} finally { + for (MemoryBlock pageToFree : pagesToFree) { +freePage(pageToFree); + } + if (inMemSorterToFree != null) { +inMemSorterToFree.freeMemory(); } } } @@ -576,58 +606,76 @@ public final class UnsafeExternalSorter extends MemoryConsumer { } public long spill() throws IOException { - sy
[spark] branch branch-3.0 updated (9b268122f68 -> c653d287b15)
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a change to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git from 9b268122f68 [SPARK-39293][SQL] Fix the accumulator of ArrayAggregate to handle complex types properly new 02488e0df30 [SPARK-32911][CORE] Free memory in UnsafeExternalSorter.SpillableIterator.spill() when all records have been read new c653d287b15 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../unsafe/sort/UnsafeExternalSorter.java | 157 ++--- .../unsafe/sort/UnsafeExternalSorterSuite.java | 30 2 files changed, 139 insertions(+), 48 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.1 updated: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.1 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.1 by this push: new 0908337a765 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator 0908337a765 is described below commit 0908337a765026a2e58a8b56e2eaa9882053aadc Author: sandeepvinayak AuthorDate: Tue May 31 15:28:07 2022 -0700 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator ### What changes were proposed in this pull request? This PR fixes a deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator. ### Why are the changes needed? We are facing the deadlock issue b/w TaskMemoryManager and UnsafeExternalSorter.SpillableIterator during the join. It turns out that in UnsafeExternalSorter.SpillableIterator#spill() function, it tries to get lock on UnsafeExternalSorter`SpillableIterator` and UnsafeExternalSorter and call `freePage` to free all allocated pages except the last one which takes the lock on TaskMemoryManager. At the same time, there can be another `MemoryConsumer` using `UnsafeExternalSorter` as part of sorting can try to allocatePage needs to get lock on `TaskMemoryManager` which can cause spill to happen which requires lock on `UnsafeExternalSorter` again causing deadlock. There is a similar fix here as well: https://issues.apache.org/jira/browse/SPARK-27338 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #36680 from sandeepvinayak/SPARK-39283. Authored-by: sandeepvinayak Signed-off-by: Josh Rosen (cherry picked from commit 8d0c035f102b005c2e85f03253f1c0c24f0a539f) Signed-off-by: Josh Rosen --- .../unsafe/sort/UnsafeExternalSorter.java | 160 + 1 file changed, 104 insertions(+), 56 deletions(-) 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 c38327cae8c..ac8170c9d97 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 @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.LinkedList; +import java.util.List; import java.util.Queue; import java.util.function.Supplier; @@ -298,16 +299,30 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * @return the number of bytes freed. */ private long freeMemory() { -updatePeakMemoryUsed(); +List pagesToFree = clearAndGetAllocatedPagesToFree(); long memoryFreed = 0; -for (MemoryBlock block : allocatedPages) { +for (MemoryBlock block : pagesToFree) { memoryFreed += block.size(); freePage(block); } +return memoryFreed; + } + + /** + * Clear the allocated pages and return the list of allocated pages to let + * the caller free the page. This is to prevent the deadlock by nested locks + * if the caller locks the UnsafeExternalSorter and call freePage which locks the + * TaskMemoryManager and cause nested locks. + * + * @return list of allocated pages to free + */ + private List clearAndGetAllocatedPagesToFree() { +updatePeakMemoryUsed(); +List pagesToFree = new LinkedList<>(allocatedPages); allocatedPages.clear(); currentPage = null; pageCursor = 0; -return memoryFreed; +return pagesToFree; } /** @@ -328,12 +343,27 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * Frees this sorter's in-memory data structures and cleans up its spill files. */ public void cleanupResources() { -synchronized (this) { - deleteSpillFiles(); - freeMemory(); - if (inMemSorter != null) { -inMemSorter.freeMemory(); -inMemSorter = null; +// To avoid deadlocks, we can't call methods that lock the TaskMemoryManager +// (such as various free() methods) while synchronizing on the UnsafeExternalSorter. +// Instead, we will manipulate UnsafeExternalSorter state inside the synchronized +// lock and perform the actual free() calls outside it. +UnsafeInMemorySorter inMemSorterToFree = null; +List pagesToFree = null; +try { + synchronized (this) { +deleteSpillFiles(); +pagesToFree = clearAndGetAllocatedPagesToFree(); +if (inMemSorter != null) { + inMemSorterToFree = inMemSorter; + inMemSorter = null; +} + } +} finally { + for (Memory
[spark] branch branch-3.2 updated: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 606830e9cae [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator 606830e9cae is described below commit 606830e9caefeec90bd0556f367395b1acbc827c Author: sandeepvinayak AuthorDate: Tue May 31 15:28:07 2022 -0700 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator ### What changes were proposed in this pull request? This PR fixes a deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator. ### Why are the changes needed? We are facing the deadlock issue b/w TaskMemoryManager and UnsafeExternalSorter.SpillableIterator during the join. It turns out that in UnsafeExternalSorter.SpillableIterator#spill() function, it tries to get lock on UnsafeExternalSorter`SpillableIterator` and UnsafeExternalSorter and call `freePage` to free all allocated pages except the last one which takes the lock on TaskMemoryManager. At the same time, there can be another `MemoryConsumer` using `UnsafeExternalSorter` as part of sorting can try to allocatePage needs to get lock on `TaskMemoryManager` which can cause spill to happen which requires lock on `UnsafeExternalSorter` again causing deadlock. There is a similar fix here as well: https://issues.apache.org/jira/browse/SPARK-27338 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #36680 from sandeepvinayak/SPARK-39283. Authored-by: sandeepvinayak Signed-off-by: Josh Rosen (cherry picked from commit 8d0c035f102b005c2e85f03253f1c0c24f0a539f) Signed-off-by: Josh Rosen --- .../unsafe/sort/UnsafeExternalSorter.java | 160 + 1 file changed, 104 insertions(+), 56 deletions(-) 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 c38327cae8c..ac8170c9d97 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 @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.LinkedList; +import java.util.List; import java.util.Queue; import java.util.function.Supplier; @@ -298,16 +299,30 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * @return the number of bytes freed. */ private long freeMemory() { -updatePeakMemoryUsed(); +List pagesToFree = clearAndGetAllocatedPagesToFree(); long memoryFreed = 0; -for (MemoryBlock block : allocatedPages) { +for (MemoryBlock block : pagesToFree) { memoryFreed += block.size(); freePage(block); } +return memoryFreed; + } + + /** + * Clear the allocated pages and return the list of allocated pages to let + * the caller free the page. This is to prevent the deadlock by nested locks + * if the caller locks the UnsafeExternalSorter and call freePage which locks the + * TaskMemoryManager and cause nested locks. + * + * @return list of allocated pages to free + */ + private List clearAndGetAllocatedPagesToFree() { +updatePeakMemoryUsed(); +List pagesToFree = new LinkedList<>(allocatedPages); allocatedPages.clear(); currentPage = null; pageCursor = 0; -return memoryFreed; +return pagesToFree; } /** @@ -328,12 +343,27 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * Frees this sorter's in-memory data structures and cleans up its spill files. */ public void cleanupResources() { -synchronized (this) { - deleteSpillFiles(); - freeMemory(); - if (inMemSorter != null) { -inMemSorter.freeMemory(); -inMemSorter = null; +// To avoid deadlocks, we can't call methods that lock the TaskMemoryManager +// (such as various free() methods) while synchronizing on the UnsafeExternalSorter. +// Instead, we will manipulate UnsafeExternalSorter state inside the synchronized +// lock and perform the actual free() calls outside it. +UnsafeInMemorySorter inMemSorterToFree = null; +List pagesToFree = null; +try { + synchronized (this) { +deleteSpillFiles(); +pagesToFree = clearAndGetAllocatedPagesToFree(); +if (inMemSorter != null) { + inMemSorterToFree = inMemSorter; + inMemSorter = null; +} + } +} finally { + for (Memory
[spark] branch branch-3.3 updated: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.3 by this push: new 1ad1c18fc28 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator 1ad1c18fc28 is described below commit 1ad1c18fc283acea6d18bc4c8753d3b6e50408ed Author: sandeepvinayak AuthorDate: Tue May 31 15:28:07 2022 -0700 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator ### What changes were proposed in this pull request? This PR fixes a deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator. ### Why are the changes needed? We are facing the deadlock issue b/w TaskMemoryManager and UnsafeExternalSorter.SpillableIterator during the join. It turns out that in UnsafeExternalSorter.SpillableIterator#spill() function, it tries to get lock on UnsafeExternalSorter`SpillableIterator` and UnsafeExternalSorter and call `freePage` to free all allocated pages except the last one which takes the lock on TaskMemoryManager. At the same time, there can be another `MemoryConsumer` using `UnsafeExternalSorter` as part of sorting can try to allocatePage needs to get lock on `TaskMemoryManager` which can cause spill to happen which requires lock on `UnsafeExternalSorter` again causing deadlock. There is a similar fix here as well: https://issues.apache.org/jira/browse/SPARK-27338 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #36680 from sandeepvinayak/SPARK-39283. Authored-by: sandeepvinayak Signed-off-by: Josh Rosen (cherry picked from commit 8d0c035f102b005c2e85f03253f1c0c24f0a539f) Signed-off-by: Josh Rosen --- .../unsafe/sort/UnsafeExternalSorter.java | 160 + 1 file changed, 104 insertions(+), 56 deletions(-) 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 c38327cae8c..ac8170c9d97 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 @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.LinkedList; +import java.util.List; import java.util.Queue; import java.util.function.Supplier; @@ -298,16 +299,30 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * @return the number of bytes freed. */ private long freeMemory() { -updatePeakMemoryUsed(); +List pagesToFree = clearAndGetAllocatedPagesToFree(); long memoryFreed = 0; -for (MemoryBlock block : allocatedPages) { +for (MemoryBlock block : pagesToFree) { memoryFreed += block.size(); freePage(block); } +return memoryFreed; + } + + /** + * Clear the allocated pages and return the list of allocated pages to let + * the caller free the page. This is to prevent the deadlock by nested locks + * if the caller locks the UnsafeExternalSorter and call freePage which locks the + * TaskMemoryManager and cause nested locks. + * + * @return list of allocated pages to free + */ + private List clearAndGetAllocatedPagesToFree() { +updatePeakMemoryUsed(); +List pagesToFree = new LinkedList<>(allocatedPages); allocatedPages.clear(); currentPage = null; pageCursor = 0; -return memoryFreed; +return pagesToFree; } /** @@ -328,12 +343,27 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * Frees this sorter's in-memory data structures and cleans up its spill files. */ public void cleanupResources() { -synchronized (this) { - deleteSpillFiles(); - freeMemory(); - if (inMemSorter != null) { -inMemSorter.freeMemory(); -inMemSorter = null; +// To avoid deadlocks, we can't call methods that lock the TaskMemoryManager +// (such as various free() methods) while synchronizing on the UnsafeExternalSorter. +// Instead, we will manipulate UnsafeExternalSorter state inside the synchronized +// lock and perform the actual free() calls outside it. +UnsafeInMemorySorter inMemSorterToFree = null; +List pagesToFree = null; +try { + synchronized (this) { +deleteSpillFiles(); +pagesToFree = clearAndGetAllocatedPagesToFree(); +if (inMemSorter != null) { + inMemSorterToFree = inMemSorter; + inMemSorter = null; +} + } +} finally { + for (Memory
[spark] branch master updated: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 8d0c035f102 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator 8d0c035f102 is described below commit 8d0c035f102b005c2e85f03253f1c0c24f0a539f Author: sandeepvinayak AuthorDate: Tue May 31 15:28:07 2022 -0700 [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator ### What changes were proposed in this pull request? This PR fixes a deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator. ### Why are the changes needed? We are facing the deadlock issue b/w TaskMemoryManager and UnsafeExternalSorter.SpillableIterator during the join. It turns out that in UnsafeExternalSorter.SpillableIterator#spill() function, it tries to get lock on UnsafeExternalSorter`SpillableIterator` and UnsafeExternalSorter and call `freePage` to free all allocated pages except the last one which takes the lock on TaskMemoryManager. At the same time, there can be another `MemoryConsumer` using `UnsafeExternalSorter` as part of sorting can try to allocatePage needs to get lock on `TaskMemoryManager` which can cause spill to happen which requires lock on `UnsafeExternalSorter` again causing deadlock. There is a similar fix here as well: https://issues.apache.org/jira/browse/SPARK-27338 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #36680 from sandeepvinayak/SPARK-39283. Authored-by: sandeepvinayak Signed-off-by: Josh Rosen --- .../unsafe/sort/UnsafeExternalSorter.java | 160 + 1 file changed, 104 insertions(+), 56 deletions(-) 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 d836cf3f0e3..d442b0ef0ef 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 @@ -22,6 +22,7 @@ import java.io.Closeable; import java.io.File; import java.io.IOException; import java.util.LinkedList; +import java.util.List; import java.util.Queue; import java.util.function.Supplier; @@ -300,16 +301,30 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * @return the number of bytes freed. */ private long freeMemory() { -updatePeakMemoryUsed(); +List pagesToFree = clearAndGetAllocatedPagesToFree(); long memoryFreed = 0; -for (MemoryBlock block : allocatedPages) { +for (MemoryBlock block : pagesToFree) { memoryFreed += block.size(); freePage(block); } +return memoryFreed; + } + + /** + * Clear the allocated pages and return the list of allocated pages to let + * the caller free the page. This is to prevent the deadlock by nested locks + * if the caller locks the UnsafeExternalSorter and call freePage which locks the + * TaskMemoryManager and cause nested locks. + * + * @return list of allocated pages to free + */ + private List clearAndGetAllocatedPagesToFree() { +updatePeakMemoryUsed(); +List pagesToFree = new LinkedList<>(allocatedPages); allocatedPages.clear(); currentPage = null; pageCursor = 0; -return memoryFreed; +return pagesToFree; } /** @@ -330,12 +345,27 @@ public final class UnsafeExternalSorter extends MemoryConsumer { * Frees this sorter's in-memory data structures and cleans up its spill files. */ public void cleanupResources() { -synchronized (this) { - deleteSpillFiles(); - freeMemory(); - if (inMemSorter != null) { -inMemSorter.freeMemory(); -inMemSorter = null; +// To avoid deadlocks, we can't call methods that lock the TaskMemoryManager +// (such as various free() methods) while synchronizing on the UnsafeExternalSorter. +// Instead, we will manipulate UnsafeExternalSorter state inside the synchronized +// lock and perform the actual free() calls outside it. +UnsafeInMemorySorter inMemSorterToFree = null; +List pagesToFree = null; +try { + synchronized (this) { +deleteSpillFiles(); +pagesToFree = clearAndGetAllocatedPagesToFree(); +if (inMemSorter != null) { + inMemSorterToFree = inMemSorter; + inMemSorter = null; +} + } +} finally { + for (MemoryBlock pageToFree : pagesToFree) { +freePage(pageToFree); + } + if (inMemSort
[spark] branch branch-3.0 updated: [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState()
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 3aaf722 [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() 3aaf722 is described below commit 3aaf722a0d6552733504c794f59d390c349dfa80 Author: Josh Rosen AuthorDate: Tue Jan 4 10:59:53 2022 -0800 [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() ### What changes were proposed in this pull request? This PR fixes a correctness issue in the CodeGenerator.addBufferedState() helper method (which is used by the SortMergeJoinExec operator). The addBufferedState() method generates code for buffering values that come from a row in an operator's input iterator, performing any necessary copying so that the buffered values remain correct after the input iterator advances to the next row. The current logic does not correctly handle UDTs: these fall through to the match statement's default branch, causing UDT values to be buffered without copying. This is problematic if the UDT's underlying SQL type is an array, map, struct, or string type (since those types require copying). Failing to copy values can lead to correctness issues or crashes. This patch's fix is simple: when the dataType is a UDT, use its underlying sqlType for determining whether values need to be copied. I used an existing helper function to perform this type unwrapping. ### Why are the changes needed? Fix a correctness issue. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I manually tested this change by re-running a workload which failed with a segfault prior to this patch. See JIRA for more details: https://issues.apache.org/jira/browse/SPARK-37784 So far I have been unable to come up with a CI-runnable regression test which would have failed prior to this change (my only working reproduction runs in a pre-production environment and does not fail in my development environment). Closes #35066 from JoshRosen/SPARK-37784. Authored-by: Josh Rosen Signed-off-by: Josh Rosen (cherry picked from commit eeef48fac412a57382b02ba3f39456d96379b5f5) Signed-off-by: Josh Rosen --- .../apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 74c6ea5..90ebe40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -330,7 +330,7 @@ class CodegenContext extends Logging { */ def addBufferedState(dataType: DataType, variableName: String, initCode: String): ExprCode = { val value = addMutableState(javaType(dataType), variableName) -val code = dataType match { +val code = UserDefinedType.sqlType(dataType) match { case StringType => code"$value = $initCode.clone();" case _: StructType | _: ArrayType | _: MapType => code"$value = $initCode.copy();" case _ => code"$value = $initCode;" - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.1 updated: [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState()
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.1 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.1 by this push: new 5cc8b39 [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() 5cc8b39 is described below commit 5cc8b397f1b01b5ee4a26e8e8540baf1e05c97a0 Author: Josh Rosen AuthorDate: Tue Jan 4 10:59:53 2022 -0800 [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() ### What changes were proposed in this pull request? This PR fixes a correctness issue in the CodeGenerator.addBufferedState() helper method (which is used by the SortMergeJoinExec operator). The addBufferedState() method generates code for buffering values that come from a row in an operator's input iterator, performing any necessary copying so that the buffered values remain correct after the input iterator advances to the next row. The current logic does not correctly handle UDTs: these fall through to the match statement's default branch, causing UDT values to be buffered without copying. This is problematic if the UDT's underlying SQL type is an array, map, struct, or string type (since those types require copying). Failing to copy values can lead to correctness issues or crashes. This patch's fix is simple: when the dataType is a UDT, use its underlying sqlType for determining whether values need to be copied. I used an existing helper function to perform this type unwrapping. ### Why are the changes needed? Fix a correctness issue. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I manually tested this change by re-running a workload which failed with a segfault prior to this patch. See JIRA for more details: https://issues.apache.org/jira/browse/SPARK-37784 So far I have been unable to come up with a CI-runnable regression test which would have failed prior to this change (my only working reproduction runs in a pre-production environment and does not fail in my development environment). Closes #35066 from JoshRosen/SPARK-37784. Authored-by: Josh Rosen Signed-off-by: Josh Rosen (cherry picked from commit eeef48fac412a57382b02ba3f39456d96379b5f5) Signed-off-by: Josh Rosen --- .../apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 6e6b946..4092436 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -334,7 +334,7 @@ class CodegenContext extends Logging { */ def addBufferedState(dataType: DataType, variableName: String, initCode: String): ExprCode = { val value = addMutableState(javaType(dataType), variableName) -val code = dataType match { +val code = UserDefinedType.sqlType(dataType) match { case StringType => code"$value = $initCode.clone();" case _: StructType | _: ArrayType | _: MapType => code"$value = $initCode.copy();" case _ => code"$value = $initCode;" - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.2 updated: [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState()
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 45b7b7e [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() 45b7b7e is described below commit 45b7b7e1682946ef86b42fd85d58bdd471996d0c Author: Josh Rosen AuthorDate: Tue Jan 4 10:59:53 2022 -0800 [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() ### What changes were proposed in this pull request? This PR fixes a correctness issue in the CodeGenerator.addBufferedState() helper method (which is used by the SortMergeJoinExec operator). The addBufferedState() method generates code for buffering values that come from a row in an operator's input iterator, performing any necessary copying so that the buffered values remain correct after the input iterator advances to the next row. The current logic does not correctly handle UDTs: these fall through to the match statement's default branch, causing UDT values to be buffered without copying. This is problematic if the UDT's underlying SQL type is an array, map, struct, or string type (since those types require copying). Failing to copy values can lead to correctness issues or crashes. This patch's fix is simple: when the dataType is a UDT, use its underlying sqlType for determining whether values need to be copied. I used an existing helper function to perform this type unwrapping. ### Why are the changes needed? Fix a correctness issue. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I manually tested this change by re-running a workload which failed with a segfault prior to this patch. See JIRA for more details: https://issues.apache.org/jira/browse/SPARK-37784 So far I have been unable to come up with a CI-runnable regression test which would have failed prior to this change (my only working reproduction runs in a pre-production environment and does not fail in my development environment). Closes #35066 from JoshRosen/SPARK-37784. Authored-by: Josh Rosen Signed-off-by: Josh Rosen (cherry picked from commit eeef48fac412a57382b02ba3f39456d96379b5f5) Signed-off-by: Josh Rosen --- .../apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index b8b5a40..132bb25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -347,7 +347,7 @@ class CodegenContext extends Logging { */ def addBufferedState(dataType: DataType, variableName: String, initCode: String): ExprCode = { val value = addMutableState(javaType(dataType), variableName) -val code = dataType match { +val code = UserDefinedType.sqlType(dataType) match { case StringType => code"$value = $initCode.clone();" case _: StructType | _: ArrayType | _: MapType => code"$value = $initCode.copy();" case _ => code"$value = $initCode;" - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState()
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new eeef48fa [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() eeef48fa is described below commit eeef48fac412a57382b02ba3f39456d96379b5f5 Author: Josh Rosen AuthorDate: Tue Jan 4 10:59:53 2022 -0800 [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState() ### What changes were proposed in this pull request? This PR fixes a correctness issue in the CodeGenerator.addBufferedState() helper method (which is used by the SortMergeJoinExec operator). The addBufferedState() method generates code for buffering values that come from a row in an operator's input iterator, performing any necessary copying so that the buffered values remain correct after the input iterator advances to the next row. The current logic does not correctly handle UDTs: these fall through to the match statement's default branch, causing UDT values to be buffered without copying. This is problematic if the UDT's underlying SQL type is an array, map, struct, or string type (since those types require copying). Failing to copy values can lead to correctness issues or crashes. This patch's fix is simple: when the dataType is a UDT, use its underlying sqlType for determining whether values need to be copied. I used an existing helper function to perform this type unwrapping. ### Why are the changes needed? Fix a correctness issue. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I manually tested this change by re-running a workload which failed with a segfault prior to this patch. See JIRA for more details: https://issues.apache.org/jira/browse/SPARK-37784 So far I have been unable to come up with a CI-runnable regression test which would have failed prior to this change (my only working reproduction runs in a pre-production environment and does not fail in my development environment). Closes #35066 from JoshRosen/SPARK-37784. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 720dfb2..c982a7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -348,7 +348,7 @@ class CodegenContext extends Logging { */ def addBufferedState(dataType: DataType, variableName: String, initCode: String): ExprCode = { val value = addMutableState(javaType(dataType), variableName) -val code = dataType match { +val code = UserDefinedType.sqlType(dataType) match { case StringType => code"$value = $initCode.clone();" case _: StructType | _: ArrayType | _: MapType => code"$value = $initCode.copy();" case _ => code"$value = $initCode;" - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-37379][SQL] Add tree pattern pruning to CTESubstitution rule
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 3b4eb1f [SPARK-37379][SQL] Add tree pattern pruning to CTESubstitution rule 3b4eb1f is described below commit 3b4eb1fbd8a351c29a12bfd94ec4cdbee803f416 Author: Josh Rosen AuthorDate: Fri Nov 19 15:24:52 2021 -0800 [SPARK-37379][SQL] Add tree pattern pruning to CTESubstitution rule ### What changes were proposed in this pull request? This PR adds tree pattern pruning to the `CTESubstitution` analyzer rule. The rule will now exit early if the tree does not contain an `UnresolvedWith` node. ### Why are the changes needed? Analysis is eagerly performed after every DataFrame transformation. If a user's program performs a long chain of _n_ transformations to construct a large query plan then this can lead to _O(n^2)_ performance costs from `CTESubstitution` because it is applied _n_ times and each application traverses the entire logical plan tree (which contains _O(n)_ nodes). In the case of chained `withColumn` calls (leading to stacked `Project` nodes) it's possible to see _O(n^3)_ slowdowns where _n_ [...] Very large DataFrame plans typically do not use CTEs because there is not a DataFrame syntax for them (although they might appear in the plan if `sql(someQueryWithCTE)` is used). As a result, this PR's proposed optimization to skip `CTESubstitution` can greatly reduce the analysis cost for such plans. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I believe that optimizer correctness is covered by existing tests. As a toy benchmark, I ran ``` import org.apache.spark.sql.DataFrame org.apache.spark.sql.catalyst.rules.RuleExecutor.resetMetrics() (1 to 600).foldLeft(spark.range(100).toDF)((df: DataFrame, i: Int) => df.withColumn(s"col$i", $"id" % i)) println(org.apache.spark.sql.catalyst.rules.RuleExecutor.dumpTimeSpent()) ``` on my laptop before and after this PR's changes (simulating a _O(n^3)_ case). Skipping `CTESubstitution` cut the running time from ~28.4 seconds to ~15.5 seconds. The bulk of the remaining time comes from `DeduplicateRelations`, for which I plan to submit a separate optimization PR. Closes #34658 from JoshRosen/CTESubstitution-tree-pattern-pruning. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala | 3 +++ .../spark/sql/catalyst/plans/logical/basicLogicalOperators.scala | 2 ++ .../main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala | 1 + 3 files changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index ec3d957..2e2d415 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -48,6 +48,9 @@ import org.apache.spark.sql.internal.SQLConf.{LEGACY_CTE_PRECEDENCE_POLICY, Lega */ object CTESubstitution extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { +if (!plan.containsPattern(UNRESOLVED_WITH)) { + return plan +} val isCommand = plan.find { case _: Command | _: ParsedStatement | _: InsertIntoDir => true case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index f1b954d..e8a632d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -626,6 +626,8 @@ object View { case class UnresolvedWith( child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { + final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_WITH) + override def output: Seq[Attribute] = child.output override def simpleString(maxFields: Int): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 6c1b64d..aad90ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -111,6
[spark] 01/01: hacky wip towards python udf profiling
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch python-udf-accumulator in repository https://gitbox.apache.org/repos/asf/spark.git commit 9213a85a40499fc7f0e24ea14c5051c45a022ef2 Author: Josh Rosen AuthorDate: Wed Oct 20 16:17:44 2021 -0700 hacky wip towards python udf profiling --- python/pyspark/profiler.py | 1 + python/pyspark/sql/udf.py | 15 --- python/pyspark/worker.py | 37 +++-- 3 files changed, 32 insertions(+), 21 deletions(-) diff --git a/python/pyspark/profiler.py b/python/pyspark/profiler.py index 99cfe71..8d8458c 100644 --- a/python/pyspark/profiler.py +++ b/python/pyspark/profiler.py @@ -142,6 +142,7 @@ class PStatsParam(AccumulatorParam): @staticmethod def addInPlace(value1, value2): +print("ACCUM UPDATE PARAM") if value1 is None: return value2 value1.add(value2) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 752ccca..164588f 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -24,6 +24,7 @@ from typing import Callable, Any, TYPE_CHECKING, Optional, cast, Union from py4j.java_gateway import JavaObject from pyspark import SparkContext +from pyspark.profiler import Profiler from pyspark.rdd import _prepare_for_python_RDD, PythonEvalType # type: ignore[attr-defined] from pyspark.sql.column import Column, _to_java_column, _to_seq from pyspark.sql.types import ( # type: ignore[attr-defined] @@ -44,9 +45,10 @@ __all__ = ["UDFRegistration"] def _wrap_function( sc: SparkContext, func: Callable[..., Any], -returnType: "DataTypeOrString" +returnType: "DataTypeOrString", +profiler: Optional[Profiler] = None ) -> JavaObject: -command = (func, returnType) +command = (func, returnType, profiler) pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command) return sc._jvm.PythonFunction( # type: ignore[attr-defined] bytearray(pickled_command), @@ -199,7 +201,14 @@ class UserDefinedFunction(object): spark = SparkSession.builder.getOrCreate() sc = spark.sparkContext -wrapped_func = _wrap_function(sc, self.func, self.returnType) +if sc.profiler_collector: +profiler = sc.profiler_collector.new_profiler(sc) + # TODO: better ID +sc.profiler_collector.add_profiler(0, profiler) +else: +profiler = None + +wrapped_func = _wrap_function(sc, self.func, self.returnType, profiler) jdt = spark._jsparkSession.parseDataType(self.returnType.json()) judf = ( sc._jvm.org.apache.spark.sql.execution.python # type: ignore[attr-defined] diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index ad6c003..8f13822 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -247,8 +247,9 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): num_arg = read_int(infile) arg_offsets = [read_int(infile) for i in range(num_arg)] chained_func = None +profiler = None for i in range(read_int(infile)): -f, return_type = read_command(pickleSer, infile) +f, return_type, profiler = read_command(pickleSer, infile) if chained_func is None: chained_func = f else: @@ -263,28 +264,29 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): # the last returnType will be the return type of UDF if eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF: -return arg_offsets, wrap_scalar_pandas_udf(func, return_type) +return arg_offsets, profiler, wrap_scalar_pandas_udf(func, return_type) elif eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF: -return arg_offsets, wrap_pandas_iter_udf(func, return_type) +return arg_offsets, profiler, wrap_pandas_iter_udf(func, return_type) elif eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF: -return arg_offsets, wrap_pandas_iter_udf(func, return_type) +return arg_offsets, profiler, wrap_pandas_iter_udf(func, return_type) elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: argspec = getfullargspec(chained_func) # signature was lost when wrapping it -return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec) +return arg_offsets, profiler, wrap_grouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: argspec = getfullargspec(chained_func) # signature was lost when wrapping it -return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type, argspec) +return arg_offsets, profiler, wrap_cogrouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.
[spark] branch python-udf-accumulator created (now 9213a85)
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a change to branch python-udf-accumulator in repository https://gitbox.apache.org/repos/asf/spark.git. at 9213a85 hacky wip towards python udf profiling This branch includes the following new commits: new 9213a85 hacky wip towards python udf profiling The 1 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-36933][CORE] Clean up TaskMemoryManager.acquireExecutionMemory()
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 1ef6c13 [SPARK-36933][CORE] Clean up TaskMemoryManager.acquireExecutionMemory() 1ef6c13 is described below commit 1ef6c13e37bfb64b0f9dd9b624b436064ea86593 Author: Tim Armstrong AuthorDate: Mon Oct 18 14:51:24 2021 -0700 [SPARK-36933][CORE] Clean up TaskMemoryManager.acquireExecutionMemory() ### What changes were proposed in this pull request? * Factor out a method `trySpillAndAcquire()` from `acquireExecutionMemory()` that handles the details of how to spill a `MemoryConsumer` and acquire the spilled memory. This logic was duplicated twice. * Combine the two loops (spill other consumers and self-spill) into a single loop that implements equivalent logic. I made self-spill the lowest priority consumer and this is exactly equivalent. * Consolidate comments a little to explain what the policy is trying to achieve and how at a high level * Add a couple more debug log messages to make it easier to follow ### Why are the changes needed? Reduce code duplication and better separate the policy decision of which MemoryConsumer to spill from the mechanism of requesting it to spill. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added some unit tests to verify the details of the spilling decisions in some scenarios that are not covered by current unit tests. Ran these on Spark master without the TaskMemoryManager changes to confirm that the behaviour is the same before and after my refactoring. The SPARK-35486 test also provides some coverage for the retry loop. Closes #34186 from timarmstrong/cleanup-task-memory-manager. Authored-by: Tim Armstrong Signed-off-by: Josh Rosen --- .../org/apache/spark/memory/TaskMemoryManager.java | 149 +++-- .../spark/memory/TaskMemoryManagerSuite.java | 79 +++ 2 files changed, 158 insertions(+), 70 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 7a1e8c4..e2e44a5 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -135,10 +135,10 @@ public class TaskMemoryManager { * * @return number of bytes successfully granted (<= N). */ - public long acquireExecutionMemory(long required, MemoryConsumer consumer) { + public long acquireExecutionMemory(long required, MemoryConsumer requestingConsumer) { assert(required >= 0); -assert(consumer != null); -MemoryMode mode = consumer.getMode(); +assert(requestingConsumer != null); +MemoryMode mode = requestingConsumer.getMode(); // If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap // memory here, then it may not make sense to spill since that would only end up freeing // off-heap memory. This is subject to change, though, so it may be risky to make this @@ -149,96 +149,105 @@ public class TaskMemoryManager { // Try to release memory from other consumers first, then we can reduce the frequency of // spilling, avoid to have too many spilled files. if (got < required) { -// Call spill() on other consumers to release memory -// Sort the consumers according their memory usage. So we avoid spilling the same consumer -// which is just spilled in last few times and re-spilling on it will produce many small -// spill files. +logger.debug("Task {} need to spill {} for {}", taskAttemptId, + Utils.bytesToString(required - got), requestingConsumer); +// We need to call spill() on consumers to free up more memory. We want to optimize for two +// things: +// * Minimize the number of spill calls, to reduce the number of spill files and avoid small +// spill files. +// * Avoid spilling more data than necessary - if we only need a little more memory, we may +// not want to spill as much data as possible. Many consumers spill more than the +// requested amount, so we can take that into account in our decisions. +// We use a heuristic that selects the smallest memory consumer with at least `required` +// bytes of memory in an attempt to balance these factors. It may work well if there are +// fewer larger requests, but can result in many small spills if there are many smaller +// requests. + +// Build a map of consumer in order of memory usage to prioritize spilling. Assign current +// consumer (if present) a nominal memory
[spark] branch branch-3.0 updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.0 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.0 by this push: new 1709265 [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler 1709265 is described below commit 1709265af1589ffa9e44d050bfa913aa0fd27dea Author: Josh Rosen AuthorDate: Thu Oct 14 14:34:24 2021 -0700 [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler ### What changes were proposed in this pull request? This PR fixes a longstanding issue where the `DAGScheduler'`s single-threaded event processing loop could become blocked by slow `RDD.getPartitions()` calls, preventing other events (like task completions and concurrent job submissions) from being processed in a timely manner. With this patch's change, Spark will now call `.partitions` on every RDD in the DAG before submitting a job to the scheduler, ensuring that the expensive `getPartitions()` calls occur outside of the scheduler event loop. Background The `RDD.partitions` method lazily computes an RDD's partitions by calling `RDD.getPartitions()`. The `getPartitions()` method is invoked only once per RDD and its result is cached in the `RDD.partitions_` private field. Sometimes the `getPartitions()` call can be expensive: for example, `HadoopRDD.getPartitions()` performs file listing operations. The `.partitions` method is invoked at many different places in Spark's code, including many existing call sites that are outside of the scheduler event loop. As a result, it's _often_ the case that an RDD's partitions will have been computed before the RDD is submitted to the DAGScheduler. For example, [`submitJob` calls `rdd.partitions.length`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.sc [...] However, there's still some cases where `partitions` gets evaluated for the first time inside of the `DAGScheduler` internals. For example, [`ShuffledRDD.getPartitions`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala#L92-L94) doesn't call `.partitions` on the RDD being shuffled, so a plan with a ShuffledRDD at the root won't necessarily result in `.partitions` having been called on all RDDs prior [...] Correctness: proving that we make no excess `.partitions` calls This PR adds code to traverse the DAG prior to job submission and call `.partitions` on every RDD encountered. I'd like to argue that this results in no _excess_ `.partitions` calls: in every case where the new code calls `.partitions` there is existing code which would have called `.partitions` at some point during a successful job execution: - Assume that this is the first time we are computing every RDD in the DAG. - Every RDD appears in some stage. - [`submitStage` will call `submitMissingTasks`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1438) on every stage root RDD. - [`submitStage` calls `getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1687-L1696) on every stage root RDD. - [`getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2995-L3043) visits the RDD and all of its parents RDDs that are computed in the same stage (via narrow dependencies) and calls `.partitions` on each RDD visited. - Therefore `.partitions` is invoked on every RDD in the DAG by the time the job has successfully completed. - Therefore this patch's change does not introduce any new calls to `.partitions` which would not have otherwise occurred (assuming the job succeeded). Ordering of `.partitions` calls I don't think the order in which `.partitions` calls occur matters for correctness: the DAGScheduler happens to invoke `.partitions` in a particular order today (defined by the DAG traversal order in internal scheduler methods), but there's many lots of out-of-order `.partition` calls occurring elsewhere in the codebase. Handling of exceptions in `.partitions` I've chosen **not** to add special error-handling for the new `.partitions` calls: if exceptions occur then they'll bubble up, unwrapped, to the user code submitting the Spark job.
[spark] branch branch-3.1 updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.1 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.1 by this push: new c43f355 [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler c43f355 is described below commit c43f35579bc2f95cbd3334a19ed334d8e706082d Author: Josh Rosen AuthorDate: Thu Oct 14 14:34:24 2021 -0700 [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler ### What changes were proposed in this pull request? This PR fixes a longstanding issue where the `DAGScheduler'`s single-threaded event processing loop could become blocked by slow `RDD.getPartitions()` calls, preventing other events (like task completions and concurrent job submissions) from being processed in a timely manner. With this patch's change, Spark will now call `.partitions` on every RDD in the DAG before submitting a job to the scheduler, ensuring that the expensive `getPartitions()` calls occur outside of the scheduler event loop. Background The `RDD.partitions` method lazily computes an RDD's partitions by calling `RDD.getPartitions()`. The `getPartitions()` method is invoked only once per RDD and its result is cached in the `RDD.partitions_` private field. Sometimes the `getPartitions()` call can be expensive: for example, `HadoopRDD.getPartitions()` performs file listing operations. The `.partitions` method is invoked at many different places in Spark's code, including many existing call sites that are outside of the scheduler event loop. As a result, it's _often_ the case that an RDD's partitions will have been computed before the RDD is submitted to the DAGScheduler. For example, [`submitJob` calls `rdd.partitions.length`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.sc [...] However, there's still some cases where `partitions` gets evaluated for the first time inside of the `DAGScheduler` internals. For example, [`ShuffledRDD.getPartitions`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala#L92-L94) doesn't call `.partitions` on the RDD being shuffled, so a plan with a ShuffledRDD at the root won't necessarily result in `.partitions` having been called on all RDDs prior [...] Correctness: proving that we make no excess `.partitions` calls This PR adds code to traverse the DAG prior to job submission and call `.partitions` on every RDD encountered. I'd like to argue that this results in no _excess_ `.partitions` calls: in every case where the new code calls `.partitions` there is existing code which would have called `.partitions` at some point during a successful job execution: - Assume that this is the first time we are computing every RDD in the DAG. - Every RDD appears in some stage. - [`submitStage` will call `submitMissingTasks`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1438) on every stage root RDD. - [`submitStage` calls `getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1687-L1696) on every stage root RDD. - [`getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2995-L3043) visits the RDD and all of its parents RDDs that are computed in the same stage (via narrow dependencies) and calls `.partitions` on each RDD visited. - Therefore `.partitions` is invoked on every RDD in the DAG by the time the job has successfully completed. - Therefore this patch's change does not introduce any new calls to `.partitions` which would not have otherwise occurred (assuming the job succeeded). Ordering of `.partitions` calls I don't think the order in which `.partitions` calls occur matters for correctness: the DAGScheduler happens to invoke `.partitions` in a particular order today (defined by the DAG traversal order in internal scheduler methods), but there's many lots of out-of-order `.partition` calls occurring elsewhere in the codebase. Handling of exceptions in `.partitions` I've chosen **not** to add special error-handling for the new `.partitions` calls: if exceptions occur then they'll bubble up, unwrapped, to the user code submitting the Spark job.
[spark] branch branch-3.2 updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 01ee46e [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler 01ee46e is described below commit 01ee46e03a7f5c6f8656690fae96f39a897b9799 Author: Josh Rosen AuthorDate: Thu Oct 14 14:34:24 2021 -0700 [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler ### What changes were proposed in this pull request? This PR fixes a longstanding issue where the `DAGScheduler'`s single-threaded event processing loop could become blocked by slow `RDD.getPartitions()` calls, preventing other events (like task completions and concurrent job submissions) from being processed in a timely manner. With this patch's change, Spark will now call `.partitions` on every RDD in the DAG before submitting a job to the scheduler, ensuring that the expensive `getPartitions()` calls occur outside of the scheduler event loop. Background The `RDD.partitions` method lazily computes an RDD's partitions by calling `RDD.getPartitions()`. The `getPartitions()` method is invoked only once per RDD and its result is cached in the `RDD.partitions_` private field. Sometimes the `getPartitions()` call can be expensive: for example, `HadoopRDD.getPartitions()` performs file listing operations. The `.partitions` method is invoked at many different places in Spark's code, including many existing call sites that are outside of the scheduler event loop. As a result, it's _often_ the case that an RDD's partitions will have been computed before the RDD is submitted to the DAGScheduler. For example, [`submitJob` calls `rdd.partitions.length`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.sc [...] However, there's still some cases where `partitions` gets evaluated for the first time inside of the `DAGScheduler` internals. For example, [`ShuffledRDD.getPartitions`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala#L92-L94) doesn't call `.partitions` on the RDD being shuffled, so a plan with a ShuffledRDD at the root won't necessarily result in `.partitions` having been called on all RDDs prior [...] Correctness: proving that we make no excess `.partitions` calls This PR adds code to traverse the DAG prior to job submission and call `.partitions` on every RDD encountered. I'd like to argue that this results in no _excess_ `.partitions` calls: in every case where the new code calls `.partitions` there is existing code which would have called `.partitions` at some point during a successful job execution: - Assume that this is the first time we are computing every RDD in the DAG. - Every RDD appears in some stage. - [`submitStage` will call `submitMissingTasks`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1438) on every stage root RDD. - [`submitStage` calls `getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1687-L1696) on every stage root RDD. - [`getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2995-L3043) visits the RDD and all of its parents RDDs that are computed in the same stage (via narrow dependencies) and calls `.partitions` on each RDD visited. - Therefore `.partitions` is invoked on every RDD in the DAG by the time the job has successfully completed. - Therefore this patch's change does not introduce any new calls to `.partitions` which would not have otherwise occurred (assuming the job succeeded). Ordering of `.partitions` calls I don't think the order in which `.partitions` calls occur matters for correctness: the DAGScheduler happens to invoke `.partitions` in a particular order today (defined by the DAG traversal order in internal scheduler methods), but there's many lots of out-of-order `.partition` calls occurring elsewhere in the codebase. Handling of exceptions in `.partitions` I've chosen **not** to add special error-handling for the new `.partitions` calls: if exceptions occur then they'll bubble up, unwrapped, to the user code submitting the Spark job.
[spark] branch master updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new c4e975e [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler c4e975e is described below commit c4e975e175c01f67ece7ae492a79554ad1b44106 Author: Josh Rosen AuthorDate: Thu Oct 14 14:34:24 2021 -0700 [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler ### What changes were proposed in this pull request? This PR fixes a longstanding issue where the `DAGScheduler'`s single-threaded event processing loop could become blocked by slow `RDD.getPartitions()` calls, preventing other events (like task completions and concurrent job submissions) from being processed in a timely manner. With this patch's change, Spark will now call `.partitions` on every RDD in the DAG before submitting a job to the scheduler, ensuring that the expensive `getPartitions()` calls occur outside of the scheduler event loop. Background The `RDD.partitions` method lazily computes an RDD's partitions by calling `RDD.getPartitions()`. The `getPartitions()` method is invoked only once per RDD and its result is cached in the `RDD.partitions_` private field. Sometimes the `getPartitions()` call can be expensive: for example, `HadoopRDD.getPartitions()` performs file listing operations. The `.partitions` method is invoked at many different places in Spark's code, including many existing call sites that are outside of the scheduler event loop. As a result, it's _often_ the case that an RDD's partitions will have been computed before the RDD is submitted to the DAGScheduler. For example, [`submitJob` calls `rdd.partitions.length`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.sc [...] However, there's still some cases where `partitions` gets evaluated for the first time inside of the `DAGScheduler` internals. For example, [`ShuffledRDD.getPartitions`](https://github.com/apache/spark/blob/3ba57f5edc5594ee676249cd309b8f0d8248462e/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala#L92-L94) doesn't call `.partitions` on the RDD being shuffled, so a plan with a ShuffledRDD at the root won't necessarily result in `.partitions` having been called on all RDDs prior [...] Correctness: proving that we make no excess `.partitions` calls This PR adds code to traverse the DAG prior to job submission and call `.partitions` on every RDD encountered. I'd like to argue that this results in no _excess_ `.partitions` calls: in every case where the new code calls `.partitions` there is existing code which would have called `.partitions` at some point during a successful job execution: - Assume that this is the first time we are computing every RDD in the DAG. - Every RDD appears in some stage. - [`submitStage` will call `submitMissingTasks`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1438) on every stage root RDD. - [`submitStage` calls `getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1687-L1696) on every stage root RDD. - [`getPreferredLocsInternal`](https://github.com/databricks/runtime/blob/1e83dfe4f685bad7f260621e77282b1b4cf9bca4/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2995-L3043) visits the RDD and all of its parents RDDs that are computed in the same stage (via narrow dependencies) and calls `.partitions` on each RDD visited. - Therefore `.partitions` is invoked on every RDD in the DAG by the time the job has successfully completed. - Therefore this patch's change does not introduce any new calls to `.partitions` which would not have otherwise occurred (assuming the job succeeded). Ordering of `.partitions` calls I don't think the order in which `.partitions` calls occur matters for correctness: the DAGScheduler happens to invoke `.partitions` in a particular order today (defined by the DAG traversal order in internal scheduler methods), but there's many lots of out-of-order `.partition` calls occurring elsewhere in the codebase. Handling of exceptions in `.partitions` I've chosen **not** to add special error-handling for the new `.partitions` calls: if exceptions occur then they'll bubble up, unwrapped, to the user code submitting the Spark job. It&
[spark] branch branch-3.2 updated: [SPARK-36774][CORE][TESTS] Move SparkSubmitTestUtils to core module and use it in SparkSubmitSuite
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-3.2 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-3.2 by this push: new 3502fda [SPARK-36774][CORE][TESTS] Move SparkSubmitTestUtils to core module and use it in SparkSubmitSuite 3502fda is described below commit 3502fda78395a92ef6320431cc9bb78569d7d796 Author: Josh Rosen AuthorDate: Thu Sep 16 14:28:47 2021 -0700 [SPARK-36774][CORE][TESTS] Move SparkSubmitTestUtils to core module and use it in SparkSubmitSuite ### What changes were proposed in this pull request? This PR refactors test code in order to improve the debugability of `SparkSubmitSuite`. The `sql/hive` module contains a `SparkSubmitTestUtils` helper class which launches `spark-submit` and captures its output in order to display better error messages when tests fail. This helper is currently used by `HiveSparkSubmitSuite` and `HiveExternalCatalogVersionsSuite`, but isn't used by `SparkSubmitSuite`. In this PR, I moved `SparkSubmitTestUtils` and `ProcessTestUtils` into the `core` module and updated `SparkSubmitSuite`, `BufferHolderSparkSubmitSuite`, and `WholestageCodegenSparkSubmitSuite` to use the relocated helper classes. This required me to change `SparkSubmitTestUtils` to make its timeouts configurable and to generalize its method for locating the `spark-submit` binary. ### Why are the changes needed? Previously, `SparkSubmitSuite` tests would fail with messages like: ``` [info] - launch simple application with spark-submit *** FAILED *** (1 second, 832 milliseconds) [info] Process returned with exit code 101. See the log4j logs for more detail. (SparkSubmitSuite.scala:1551) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472) ``` which require the Spark developer to hunt in log4j logs in order to view the logs from the failed `spark-submit` command. After this change, those tests will fail with detailed error messages that include the text of failed command plus timestamped logs captured from the failed proces: ``` [info] - launch simple application with spark-submit *** FAILED *** (2 seconds, 800 milliseconds) [info] spark-submit returned with exit code 101. [info] Command line: '/Users/joshrosen/oss-spark/bin/spark-submit' '--class' 'invalidClassName' '--name' 'testApp' '--master' 'local' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' 'file:/Users/joshrosen/oss-spark/target/tmp/spark-0a8a0c93-3aaf-435d-9cf3-b97abd318d91/testJar-1631768004882.jar' [info] [info] 2021-09-15 21:53:26.041 - stderr> SLF4J: Class path contains multiple SLF4J bindings. [info] 2021-09-15 21:53:26.042 - stderr> SLF4J: Found binding in [jar:file:/Users/joshrosen/oss-spark/assembly/target/scala-2.12/jars/slf4j-log4j12-1.7.30.jar!/org/slf4j/impl/StaticLoggerBinder.class] [info] 2021-09-15 21:53:26.042 - stderr> SLF4J: Found binding in [jar:file:/Users/joshrosen/.m2/repository/org/slf4j/slf4j-log4j12/1.7.30/slf4j-log4j12-1.7.30.jar!/org/slf4j/impl/StaticLoggerBinder.class] [info] 2021-09-15 21:53:26.042 - stderr> SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation. [info] 2021-09-15 21:53:26.042 - stderr> SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory] [info] 2021-09-15 21:53:26.619 - stderr> Error: Failed to load class invalidClassName. (SparkSubmitTestUtils.scala:97) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I manually ran the affected test suites. Closes #34013 from JoshRosen/SPARK-36774-move-SparkSubmitTestUtils-to-core. Authored-by: Josh Rosen Signed-off-by: Josh Rosen (cherry picked from commit 3ae6e6775beae8225f8cb7404bd1a2ea961dd339) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark}/ProcessTestUtils.scala | 2 +- .../org/apache/spark/deploy/SparkSubmitSuite.scala | 41 +- .../spark/deploy}/SparkSubmitTestUtils.scala | 18 +- .../codegen/BufferHolderSparkSubmitSuite.scala | 8 ++--- .../WholeStageCodegenSparkSubmitSuite.scala| 8 ++--- .../spark/sql/hive/thriftserver/CliSuite.scala | 2 +- .../thriftserver/HiveThriftServer2Suites.scala | 2 +- .../hive/HiveExternalCatalogVersionsSuite.scala| 6 +++- .../spark/sql/hive/HiveSpark
[spark] branch master updated (f1f2ec3 -> 3ae6e67)
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from f1f2ec3 [SPARK-36735][SQL][FOLLOWUP] Fix indentation of DynamicPartitionPruningSuite add 3ae6e67 [SPARK-36774][CORE][TESTS] Move SparkSubmitTestUtils to core module and use it in SparkSubmitSuite No new revisions were added by this update. Summary of changes: .../scala/org/apache/spark}/ProcessTestUtils.scala | 2 +- .../org/apache/spark/deploy/SparkSubmitSuite.scala | 41 +- .../spark/deploy}/SparkSubmitTestUtils.scala | 18 +- .../codegen/BufferHolderSparkSubmitSuite.scala | 8 ++--- .../WholeStageCodegenSparkSubmitSuite.scala| 8 ++--- .../spark/sql/hive/thriftserver/CliSuite.scala | 2 +- .../thriftserver/HiveThriftServer2Suites.scala | 2 +- .../hive/HiveExternalCatalogVersionsSuite.scala| 6 +++- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 5 +++ 9 files changed, 32 insertions(+), 60 deletions(-) rename {sql/core/src/test/scala/org/apache/spark/sql/test => core/src/test/scala/org/apache/spark}/ProcessTestUtils.scala (97%) rename {sql/hive/src/test/scala/org/apache/spark/sql/hive => core/src/test/scala/org/apache/spark/deploy}/SparkSubmitTestUtils.scala (89%) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (33e45ec -> 23bed0d)
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git. from 33e45ec [SPARK-28769][CORE] Improve warning message of BarrierExecutionMode when required slots > maximum slots add 23bed0d [SPARK-28702][SQL] Display useful error message (instead of NPE) for invalid Dataset operations No new revisions were added by this update. Summary of changes: .../src/main/scala/org/apache/spark/sql/Dataset.scala| 16 ++-- 1 file changed, 14 insertions(+), 2 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-2.4 updated: [SPARK-26038][BRANCH-2.4] Decimal toScalaBigInt/toJavaBigInteger for decimals not fitting in long
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.4 by this push: new a71e90a [SPARK-26038][BRANCH-2.4] Decimal toScalaBigInt/toJavaBigInteger for decimals not fitting in long a71e90a is described below commit a71e90a76a982dde09d3b60bb2cf4548c62f57a1 Author: Juliusz Sompolski AuthorDate: Fri Jun 21 07:56:49 2019 -0700 [SPARK-26038][BRANCH-2.4] Decimal toScalaBigInt/toJavaBigInteger for decimals not fitting in long This is a Spark 2.4.x backport of #23022. Original description follows below: ## What changes were proposed in this pull request? Fix Decimal `toScalaBigInt` and `toJavaBigInteger` used to only work for decimals not fitting long. ## How was this patch tested? Added test to DecimalSuite. Closes #24928 from JoshRosen/joshrosen/SPARK-26038-backport. Authored-by: Juliusz Sompolski Signed-off-by: Josh Rosen --- .../main/scala/org/apache/spark/sql/types/Decimal.scala | 16 ++-- .../scala/org/apache/spark/sql/types/DecimalSuite.scala | 11 +++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 9eed2eb..12182324 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -185,9 +185,21 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } - def toScalaBigInt: BigInt = BigInt(toLong) + def toScalaBigInt: BigInt = { +if (decimalVal.ne(null)) { + decimalVal.toBigInt() +} else { + BigInt(toLong) +} + } - def toJavaBigInteger: java.math.BigInteger = java.math.BigInteger.valueOf(toLong) + def toJavaBigInteger: java.math.BigInteger = { +if (decimalVal.ne(null)) { + decimalVal.underlying().toBigInteger() +} else { + java.math.BigInteger.valueOf(toLong) +} + } def toUnscaledLong: Long = { if (decimalVal.ne(null)) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index 10de90c..8abd762 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -228,4 +228,15 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { val decimal = Decimal.apply(bigInt) assert(decimal.toJavaBigDecimal.unscaledValue.toString === "9223372036854775808") } + + test("SPARK-26038: toScalaBigInt/toJavaBigInteger") { +// not fitting long +val decimal = Decimal("1234568790123456789012348790.1234879012345678901234568790") +assert(decimal.toScalaBigInt == scala.math.BigInt("1234568790123456789012348790")) +assert(decimal.toJavaBigInteger == new java.math.BigInteger("1234568790123456789012348790")) +// fitting long +val decimalLong = Decimal(123456789123456789L, 18, 9) +assert(decimalLong.toScalaBigInt == scala.math.BigInt("123456789")) +assert(decimalLong.toJavaBigInteger == new java.math.BigInteger("123456789")) + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-28112][TEST] Fix Kryo exception perf. bottleneck in tests due to absence of ML/MLlib classes
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new ec032ce [SPARK-28112][TEST] Fix Kryo exception perf. bottleneck in tests due to absence of ML/MLlib classes ec032ce is described below commit ec032cea4f91a5ee6ce51e2216de23104486a053 Author: Josh Rosen AuthorDate: Wed Jun 19 19:06:22 2019 -0700 [SPARK-28112][TEST] Fix Kryo exception perf. bottleneck in tests due to absence of ML/MLlib classes ## What changes were proposed in this pull request? In a nutshell, it looks like the absence of ML / MLlib classes on the classpath causes code in KryoSerializer to throw and catch ClassNotFoundExceptions whenever instantiating a new serializer in newInstance(). This isn't a performance problem in production (since MLlib is on the classpath there) but it's a huge issue in tests and appears to account for an enormous amount of test time We can address this problem by reducing the total number of ClassNotFoundExceptions by performing the class existence checks once and storing the results in KryoSerializer instances rather than repeating the checks on each newInstance() call. ## How was this patch tested? The existing tests. Authored-by: Josh Rosen Closes #24916 from gatorsmile/kryoException. Lead-authored-by: Josh Rosen Co-authored-by: gatorsmile Signed-off-by: Josh Rosen --- .../apache/spark/serializer/KryoSerializer.scala | 78 +- 1 file changed, 45 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 3969106..20774c8 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -212,40 +212,8 @@ class KryoSerializer(conf: SparkConf) // We can't load those class directly in order to avoid unnecessary jar dependencies. // We load them safely, ignore it if the class not found. -Seq( - "org.apache.spark.sql.catalyst.expressions.UnsafeRow", - "org.apache.spark.sql.catalyst.expressions.UnsafeArrayData", - "org.apache.spark.sql.catalyst.expressions.UnsafeMapData", - - "org.apache.spark.ml.attribute.Attribute", - "org.apache.spark.ml.attribute.AttributeGroup", - "org.apache.spark.ml.attribute.BinaryAttribute", - "org.apache.spark.ml.attribute.NominalAttribute", - "org.apache.spark.ml.attribute.NumericAttribute", - - "org.apache.spark.ml.feature.Instance", - "org.apache.spark.ml.feature.LabeledPoint", - "org.apache.spark.ml.feature.OffsetInstance", - "org.apache.spark.ml.linalg.DenseMatrix", - "org.apache.spark.ml.linalg.DenseVector", - "org.apache.spark.ml.linalg.Matrix", - "org.apache.spark.ml.linalg.SparseMatrix", - "org.apache.spark.ml.linalg.SparseVector", - "org.apache.spark.ml.linalg.Vector", - "org.apache.spark.ml.stat.distribution.MultivariateGaussian", - "org.apache.spark.ml.tree.impl.TreePoint", - "org.apache.spark.mllib.clustering.VectorWithNorm", - "org.apache.spark.mllib.linalg.DenseMatrix", - "org.apache.spark.mllib.linalg.DenseVector", - "org.apache.spark.mllib.linalg.Matrix", - "org.apache.spark.mllib.linalg.SparseMatrix", - "org.apache.spark.mllib.linalg.SparseVector", - "org.apache.spark.mllib.linalg.Vector", - "org.apache.spark.mllib.regression.LabeledPoint", - "org.apache.spark.mllib.stat.distribution.MultivariateGaussian" -).foreach { name => +KryoSerializer.loadableSparkClasses.foreach { clazz => try { -val clazz = Utils.classForName(name) kryo.register(clazz) } catch { case NonFatal(_) => // do nothing @@ -516,6 +484,50 @@ private[serializer] object KryoSerializer { } } ) + + // classForName() is expensive in case the class is not found, so we filter the list of + // SQL / ML / MLlib classes once and then re-use that filtered list in newInstance() calls. + private lazy val loadableSparkClasses: Seq[Class[_]] = { +Seq( + "org.apache.spark.sql.catalyst.expressions.UnsafeRow", + "org.apache.spark.sql.catalyst.expressions.UnsafeArrayData", + "org.apache.spark.sql.catalyst.expressions.UnsafeMapData", + + "org.apache.spark.ml.attribute.Attribute", + "org.apache.spark.m
[spark] branch branch-2.4 updated: [SPARK-26555][SQL][BRANCH-2.4] make ScalaReflection subtype checking thread safe
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch branch-2.4 in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/branch-2.4 by this push: new ba7f61e [SPARK-26555][SQL][BRANCH-2.4] make ScalaReflection subtype checking thread safe ba7f61e is described below commit ba7f61e25d58aa379f94a23b03503a25574529bc Author: mwlon AuthorDate: Wed Jun 19 19:03:35 2019 -0700 [SPARK-26555][SQL][BRANCH-2.4] make ScalaReflection subtype checking thread safe This is a Spark 2.4.x backport of #24085. Original description follows below: ## What changes were proposed in this pull request? Make ScalaReflection subtype checking thread safe by adding a lock. There is a thread safety bug in the <:< operator in all versions of scala (https://github.com/scala/bug/issues/10766). ## How was this patch tested? Existing tests and a new one for the new subtype checking function. Closes #24913 from JoshRosen/joshrosen/SPARK-26555-branch-2.4-backport. Authored-by: mwlon Signed-off-by: Josh Rosen --- .../spark/sql/catalyst/ScalaReflection.scala | 216 +++-- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 6 + 2 files changed, 124 insertions(+), 98 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index c27180e..1b186bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -40,6 +40,9 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} trait DefinedByConstructorParams +private[catalyst] object ScalaSubtypeLock + + /** * A default version of ScalaReflection that uses the runtime universe. */ @@ -68,19 +71,32 @@ object ScalaReflection extends ScalaReflection { */ def dataTypeFor[T : TypeTag]: DataType = dataTypeFor(localTypeOf[T]) + /** + * Synchronize to prevent concurrent usage of `<:<` operator. + * This operator is not thread safe in any current version of scala; i.e. + * (2.11.12, 2.12.8, 2.13.0-M5). + * + * See https://github.com/scala/bug/issues/10766 + */ + private[catalyst] def isSubtype(tpe1: `Type`, tpe2: `Type`): Boolean = { +ScalaSubtypeLock.synchronized { + tpe1 <:< tpe2 +} + } + private def dataTypeFor(tpe: `Type`): DataType = cleanUpReflectionObjects { tpe.dealias match { - case t if t <:< definitions.NullTpe => NullType - case t if t <:< definitions.IntTpe => IntegerType - case t if t <:< definitions.LongTpe => LongType - case t if t <:< definitions.DoubleTpe => DoubleType - case t if t <:< definitions.FloatTpe => FloatType - case t if t <:< definitions.ShortTpe => ShortType - case t if t <:< definitions.ByteTpe => ByteType - case t if t <:< definitions.BooleanTpe => BooleanType - case t if t <:< localTypeOf[Array[Byte]] => BinaryType - case t if t <:< localTypeOf[CalendarInterval] => CalendarIntervalType - case t if t <:< localTypeOf[Decimal] => DecimalType.SYSTEM_DEFAULT + case t if isSubtype(t, definitions.NullTpe) => NullType + case t if isSubtype(t, definitions.IntTpe) => IntegerType + case t if isSubtype(t, definitions.LongTpe) => LongType + case t if isSubtype(t, definitions.DoubleTpe) => DoubleType + case t if isSubtype(t, definitions.FloatTpe) => FloatType + case t if isSubtype(t, definitions.ShortTpe) => ShortType + case t if isSubtype(t, definitions.ByteTpe) => ByteType + case t if isSubtype(t, definitions.BooleanTpe) => BooleanType + case t if isSubtype(t, localTypeOf[Array[Byte]]) => BinaryType + case t if isSubtype(t, localTypeOf[CalendarInterval]) => CalendarIntervalType + case t if isSubtype(t, localTypeOf[Decimal]) => DecimalType.SYSTEM_DEFAULT case _ => val className = getClassNameFromType(tpe) className match { @@ -103,13 +119,13 @@ object ScalaReflection extends ScalaReflection { */ private def arrayClassFor(tpe: `Type`): ObjectType = cleanUpReflectionObjects { val cls = tpe.dealias match { - case t if t <:< definitions.IntTpe => classOf[Array[Int]] - case t if t <:< definitions.LongTpe => classOf[Array[Long]] - case t if t <:< definitions.DoubleTpe => classOf[Array[Double]] - case t if t <:< definitions.FloatTpe => classOf[Array[Float]] - case t if t <:< definitions.ShortTpe => classOf[Array[Short]] - case t if t <:< definitions.ByteTpe => classOf[A
[spark] branch master updated: [SPARK-28102][CORE] Avoid performance problems when lz4-java JNI libraries fail to initialize
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 6b27ad5 [SPARK-28102][CORE] Avoid performance problems when lz4-java JNI libraries fail to initialize 6b27ad5 is described below commit 6b27ad5ea11297c39ac216054f061af334387a59 Author: Josh Rosen AuthorDate: Wed Jun 19 15:26:26 2019 -0700 [SPARK-28102][CORE] Avoid performance problems when lz4-java JNI libraries fail to initialize ## What changes were proposed in this pull request? This PR fixes a performance problem in environments where `lz4-java`'s native JNI libraries fail to initialize. Spark's uses `lz4-java` for LZ4 compression. Under the hood, the `LZ4BlockInputStream` and `LZ4BlockOutputStream` constructors call `LZ4Factory.fastestInstance()`, which attempts to load JNI libraries and falls back on Java implementations in case the JNI library cannot be loaded or initialized. If the LZ4 JNI libraries are present on the library load path (`Native.isLoaded()`) but cannot be initialized (e.g. due to breakage caused by shading) then an exception will be thrown and caught, triggering fallback to `fastestJavaInstance()` (a non-JNI implementation). Unfortunately, the LZ4 library does not cache the fact that the JNI library failed during initialization, so every call to `LZ4Factory.fastestInstance()` re-attempts (and fails) to initialize the native code. These initialization attempts are performed in a `static synchronized` method, so exceptions from failures are thrown while holding shared monitors and this causes monitor-contention performance issues. Here's an example stack trace showing the problem: ```java java.lang.Throwable.fillInStackTrace(Native Method) java.lang.Throwable.fillInStackTrace(Throwable.java:783) => holding Monitor(java.lang.NoClassDefFoundError441628568}) java.lang.Throwable.(Throwable.java:265) java.lang.Error.(Error.java:70) java.lang.LinkageError.(LinkageError.java:55) java.lang.NoClassDefFoundError.(NoClassDefFoundError.java:59) shaded.net.jpountz.lz4.LZ4JNICompressor.compress(LZ4JNICompressor.java:36) shaded.net.jpountz.lz4.LZ4Factory.(LZ4Factory.java:200) shaded.net.jpountz.lz4.LZ4Factory.instance(LZ4Factory.java:51) shaded.net.jpountz.lz4.LZ4Factory.nativeInstance(LZ4Factory.java:84) => holding Monitor(java.lang.Class1475983836}) shaded.net.jpountz.lz4.LZ4Factory.fastestInstance(LZ4Factory.java:157) shaded.net.jpountz.lz4.LZ4BlockOutputStream.(LZ4BlockOutputStream.java:135) org.apache.spark.io.LZ4CompressionCodec.compressedOutputStream(CompressionCodec.scala:122) org.apache.spark.serializer.SerializerManager.wrapForCompression(SerializerManager.scala:156) org.apache.spark.serializer.SerializerManager.wrapStream(SerializerManager.scala:131) org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:120) org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:249) org.apache.spark.shuffle.sort.ShuffleExternalSorter.writeSortedFile(ShuffleExternalSorter.java:211) [...] ``` To avoid this problem, this PR modifies Spark's `LZ4CompressionCodec` to call `fastestInstance()` itself and cache the result (which is safe because these factories [are thread-safe](https://github.com/lz4/lz4-java/issues/82)). ## How was this patch tested? Existing unit tests. Closes #24905 from JoshRosen/lz4-factory-flags. Lead-authored-by: Josh Rosen Co-authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../org/apache/spark/io/CompressionCodec.scala | 28 +++--- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 065f05e..adbd59c 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -22,7 +22,8 @@ import java.util.Locale import com.github.luben.zstd.{ZstdInputStream, ZstdOutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} -import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} +import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream, LZ4Factory} +import net.jpountz.xxhash.XXHashFactory import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf @@ -118,14 +119,35 @@ private[spark] object CompressionCodec { @DeveloperApi class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { + // SPARK-28102: if the LZ4 JNI libraries fail to initialize then `fastestInstance()` calls fall + // back
[spark] branch master updated: [SPARK-27839][SQL] Change UTF8String.replace() to operate on UTF8 bytes
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new fc65e0f [SPARK-27839][SQL] Change UTF8String.replace() to operate on UTF8 bytes fc65e0f is described below commit fc65e0fe2c8a114feba47d8f7b63628a676dd24c Author: Josh Rosen AuthorDate: Wed Jun 19 15:21:26 2019 -0700 [SPARK-27839][SQL] Change UTF8String.replace() to operate on UTF8 bytes ## What changes were proposed in this pull request? This PR significantly improves the performance of `UTF8String.replace()` by performing direct replacement over UTF8 bytes instead of decoding those bytes into Java Strings. In cases where the search string is not found (i.e. no replacements are performed, a case which I expect to be common) this new implementation performs no object allocation or memory copying. My implementation is modeled after `commons-lang3`'s `StringUtils.replace()` method. As part of my implementation, I needed a StringBuilder / resizable buffer, so I moved `UTF8StringBuilder` from the `catalyst` package to `unsafe`. ## How was this patch tested? Copied tests from `StringExpressionSuite` to `UTF8StringSuite` and added a couple of new cases. To evaluate performance, I did some quick local benchmarking by running the following code in `spark-shell` (with Java 1.8.0_191): ```scala import org.apache.spark.unsafe.types.UTF8String def benchmark(text: String, search: String, replace: String) { val utf8Text = UTF8String.fromString(text) val utf8Search = UTF8String.fromString(search) val utf8Replace = UTF8String.fromString(replace) val start = System.currentTimeMillis var i = 0 while (i < 1000 * 1000 * 100) { utf8Text.replace(utf8Search, utf8Replace) i += 1 } val end = System.currentTimeMillis println(end - start) } benchmark("ABCDEFGH", "DEF", "") // replacement occurs benchmark("ABCDEFGH", "Z", "") // no replacement occurs ``` On my laptop this took ~54 / ~40 seconds seconds before this patch's changes and ~6.5 / ~3.8 seconds afterwards. Closes #24707 from JoshRosen/faster-string-replace. Authored-by: Josh Rosen Signed-off-by: Josh Rosen --- .../apache/spark/unsafe}/UTF8StringBuilder.java| 27 +-- .../org/apache/spark/unsafe/types/UTF8String.java | 26 --- .../apache/spark/unsafe/types/UTF8StringSuite.java | 38 ++ .../spark/sql/catalyst/expressions/Cast.scala | 1 + .../expressions/collectionOperations.scala | 1 + 5 files changed, 86 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/UTF8StringBuilder.java similarity index 80% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java rename to common/unsafe/src/main/java/org/apache/spark/unsafe/UTF8StringBuilder.java index f0f66ba..481ea89 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/UTF8StringBuilder.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions.codegen; +package org.apache.spark.unsafe; -import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.types.UTF8String; @@ -34,7 +33,18 @@ public class UTF8StringBuilder { public UTF8StringBuilder() { // Since initial buffer size is 16 in `StringBuilder`, we set the same size here -this.buffer = new byte[16]; +this(16); + } + + public UTF8StringBuilder(int initialSize) { +if (initialSize < 0) { + throw new IllegalArgumentException("Size must be non-negative"); +} +if (initialSize > ARRAY_MAX) { + throw new IllegalArgumentException( +"Size " + initialSize + " exceeded maximum size of " + ARRAY_MAX); +} +this.buffer = new byte[initialSize]; } // Grows the buffer by at least `neededSize` @@ -72,6 +82,17 @@ public class UTF8StringBuilder { append(UTF8String.fromString(value)); } + public void appendBytes(Object base, long offset, int length) { +grow(length); +Platform.copyMemory( + base, + offset, + buffer, + cursor, + length); +cursor += length; + } + public UTF8String build() { return UTF8String.fromBytes(buffer, 0, totalSize()); } diff --git a/common/unsaf
[spark] branch master updated: [SPARK-27684][SQL] Avoid conversion overhead for primitive types
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git The following commit(s) were added to refs/heads/master by this push: new 93db7b8 [SPARK-27684][SQL] Avoid conversion overhead for primitive types 93db7b8 is described below commit 93db7b870df634d84121ba2527057670d6250564 Author: Marco Gaido AuthorDate: Thu May 30 17:09:19 2019 -0700 [SPARK-27684][SQL] Avoid conversion overhead for primitive types ## What changes were proposed in this pull request? As outlined in the JIRA by JoshRosen, our conversion mechanism from catalyst types to scala ones is pretty inefficient for primitive data types. Indeed, in these cases, most of the times we are adding useless calls to `identity` function or anyway to functions which return the same value. Using the information we have when we generate the code, we can avoid most of these overheads. ## How was this patch tested? Here is a simple test which shows the benefit that this PR can bring: ``` test("SPARK-27684: perf evaluation") { val intLongUdf = ScalaUDF( (a: Int, b: Long) => a + b, LongType, Literal(1) :: Literal(1L) :: Nil, true :: true :: Nil, nullable = false) val plan = generateProject( MutableProjection.create(Alias(intLongUdf, s"udf")() :: Nil), intLongUdf) plan.initialize(0) var i = 0 val N = 1 val t0 = System.nanoTime() while(i < N) { plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) plan(EmptyRow).get(0, intLongUdf.dataType) i += 1 } val t1 = System.nanoTime() println(s"Avg time: ${(t1 - t0).toDouble / N} ns") } ``` The output before the patch is: ``` Avg time: 51.27083294 ns ``` after, we get: ``` Avg time: 11.85874227 ns ``` which is ~5X faster. Moreover a benchmark has been added for Scala UDF. The output after the patch can be seen in this PR, before the patch, the output was: ``` UDF with mixed input types Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 Intel(R) Core(TM) i7-4558U CPU 2.80GHz long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative long/nullable int/string to string wholestage off257 287 42 0,42569,5 1,0X long/nullable int/string to string wholestage on158 172 18 0,61579,0 1,6X Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 Intel(R) Core(TM) i7-4558U CPU 2.80GHz long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative long/nullable int/string to option wholestage off104 107 5 1,01037,9 1,0X long/nullable int/string to option wholestage on 80 92 12 1,2 804,0 1,3X Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 Intel(R) Core(TM) i7-4558U CPU 2.80GHz long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative long/nullable int to primitive wholestage off 71 76 7 1,4 712,1 1,0X long/nullable int to primitive wholestage on 64 71 6 1,6 636,2 1,1X =
[spark-website] branch asf-site updated: Update Josh Rosen's affiliation
This is an automated email from the ASF dual-hosted git repository. joshrosen pushed a commit to branch asf-site in repository https://gitbox.apache.org/repos/asf/spark-website.git The following commit(s) were added to refs/heads/asf-site by this push: new f90d6dd Update Josh Rosen's affiliation f90d6dd is described below commit f90d6dd26315847b907370227476ad99f638caf4 Author: Josh Rosen AuthorDate: Tue May 14 22:04:53 2019 -0700 Update Josh Rosen's affiliation --- committers.md| 2 +- site/committers.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/committers.md b/committers.md index bd48bd8..15e7185 100644 --- a/committers.md +++ b/committers.md @@ -59,7 +59,7 @@ navigation: |Anirudh Ramanathan|Google| |Imran Rashid|Cloudera| |Charles Reiss|University of Virginia| -|Josh Rosen|Databricks| +|Josh Rosen|Stripe| |Sandy Ryza|Remix| |Kousuke Saruta|NTT Data| |Saisai Shao|Tencent| diff --git a/site/committers.html b/site/committers.html index bcc7c21..792b03a 100644 --- a/site/committers.html +++ b/site/committers.html @@ -408,7 +408,7 @@ Josh Rosen - Databricks + Stripe Sandy Ryza - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-22997] Add additional defenses against use of freed MemoryBlocks
Repository: spark Updated Branches: refs/heads/branch-2.3 2db523959 -> 60d4d79bb [SPARK-22997] Add additional defenses against use of freed MemoryBlocks ## What changes were proposed in this pull request? This patch modifies Spark's `MemoryAllocator` implementations so that `free(MemoryBlock)` mutates the passed block to clear pointers (in the off-heap case) or null out references to backing `long[]` arrays (in the on-heap case). The goal of this change is to add an extra layer of defense against use-after-free bugs because currently it's hard to detect corruption caused by blind writes to freed memory blocks. ## How was this patch tested? New unit tests in `PlatformSuite`, including new tests for existing functionality because we did not have sufficient mutation coverage of the on-heap memory allocator's pooling logic. Author: Josh Rosen Closes #20191 from JoshRosen/SPARK-22997-add-defenses-against-use-after-free-bugs-in-memory-allocator. (cherry picked from commit f340b6b3066033d40b7e163fd5fb68e9820adfb1) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/60d4d79b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/60d4d79b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/60d4d79b Branch: refs/heads/branch-2.3 Commit: 60d4d79bb40f13c68773a0224f2003cdca28c138 Parents: 2db5239 Author: Josh Rosen Authored: Wed Jan 10 00:45:47 2018 -0800 Committer: Josh Rosen Committed: Wed Jan 10 00:46:27 2018 -0800 -- .../unsafe/memory/HeapMemoryAllocator.java | 35 ++ .../apache/spark/unsafe/memory/MemoryBlock.java | 21 +++- .../unsafe/memory/UnsafeMemoryAllocator.java| 11 + .../apache/spark/unsafe/PlatformUtilSuite.java | 50 +++- .../apache/spark/memory/TaskMemoryManager.java | 13 - .../spark/memory/TaskMemoryManagerSuite.java| 29 6 files changed, 146 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/60d4d79b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java -- diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index cc9cc42..3acfe36 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -31,8 +31,7 @@ import org.apache.spark.unsafe.Platform; public class HeapMemoryAllocator implements MemoryAllocator { @GuardedBy("this") - private final Map>> bufferPoolsBySize = -new HashMap<>(); + private final Map>> bufferPoolsBySize = new HashMap<>(); private static final int POOLING_THRESHOLD_BYTES = 1024 * 1024; @@ -49,13 +48,14 @@ public class HeapMemoryAllocator implements MemoryAllocator { public MemoryBlock allocate(long size) throws OutOfMemoryError { if (shouldPool(size)) { synchronized (this) { -final LinkedList> pool = bufferPoolsBySize.get(size); +final LinkedList> pool = bufferPoolsBySize.get(size); if (pool != null) { while (!pool.isEmpty()) { -final WeakReference blockReference = pool.pop(); -final MemoryBlock memory = blockReference.get(); -if (memory != null) { - assert (memory.size() == size); +final WeakReference arrayReference = pool.pop(); +final long[] array = arrayReference.get(); +if (array != null) { + assert (array.length * 8L >= size); + MemoryBlock memory = new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -76,18 +76,35 @@ public class HeapMemoryAllocator implements MemoryAllocator { @Override public void free(MemoryBlock memory) { +assert (memory.obj != null) : + "baseObject was null; are you trying to use the on-heap allocator to free off-heap memory?"; +assert (memory.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + "page has already been freed"; +assert ((memory.pageNumber == MemoryBlock.NO_PAGE_NUMBER) +|| (memory.pageNumber == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : + "TMM-allocated pages must first be freed via TMM.freePage(), not directly in allocator free()"; + final long size = memory.size(); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) {
spark git commit: [SPARK-22997] Add additional defenses against use of freed MemoryBlocks
Repository: spark Updated Branches: refs/heads/master 70bcc9d5a -> f340b6b30 [SPARK-22997] Add additional defenses against use of freed MemoryBlocks ## What changes were proposed in this pull request? This patch modifies Spark's `MemoryAllocator` implementations so that `free(MemoryBlock)` mutates the passed block to clear pointers (in the off-heap case) or null out references to backing `long[]` arrays (in the on-heap case). The goal of this change is to add an extra layer of defense against use-after-free bugs because currently it's hard to detect corruption caused by blind writes to freed memory blocks. ## How was this patch tested? New unit tests in `PlatformSuite`, including new tests for existing functionality because we did not have sufficient mutation coverage of the on-heap memory allocator's pooling logic. Author: Josh Rosen Closes #20191 from JoshRosen/SPARK-22997-add-defenses-against-use-after-free-bugs-in-memory-allocator. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f340b6b3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f340b6b3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f340b6b3 Branch: refs/heads/master Commit: f340b6b3066033d40b7e163fd5fb68e9820adfb1 Parents: 70bcc9d Author: Josh Rosen Authored: Wed Jan 10 00:45:47 2018 -0800 Committer: Josh Rosen Committed: Wed Jan 10 00:45:47 2018 -0800 -- .../unsafe/memory/HeapMemoryAllocator.java | 35 ++ .../apache/spark/unsafe/memory/MemoryBlock.java | 21 +++- .../unsafe/memory/UnsafeMemoryAllocator.java| 11 + .../apache/spark/unsafe/PlatformUtilSuite.java | 50 +++- .../apache/spark/memory/TaskMemoryManager.java | 13 - .../spark/memory/TaskMemoryManagerSuite.java| 29 6 files changed, 146 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f340b6b3/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java -- diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index cc9cc42..3acfe36 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -31,8 +31,7 @@ import org.apache.spark.unsafe.Platform; public class HeapMemoryAllocator implements MemoryAllocator { @GuardedBy("this") - private final Map>> bufferPoolsBySize = -new HashMap<>(); + private final Map>> bufferPoolsBySize = new HashMap<>(); private static final int POOLING_THRESHOLD_BYTES = 1024 * 1024; @@ -49,13 +48,14 @@ public class HeapMemoryAllocator implements MemoryAllocator { public MemoryBlock allocate(long size) throws OutOfMemoryError { if (shouldPool(size)) { synchronized (this) { -final LinkedList> pool = bufferPoolsBySize.get(size); +final LinkedList> pool = bufferPoolsBySize.get(size); if (pool != null) { while (!pool.isEmpty()) { -final WeakReference blockReference = pool.pop(); -final MemoryBlock memory = blockReference.get(); -if (memory != null) { - assert (memory.size() == size); +final WeakReference arrayReference = pool.pop(); +final long[] array = arrayReference.get(); +if (array != null) { + assert (array.length * 8L >= size); + MemoryBlock memory = new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -76,18 +76,35 @@ public class HeapMemoryAllocator implements MemoryAllocator { @Override public void free(MemoryBlock memory) { +assert (memory.obj != null) : + "baseObject was null; are you trying to use the on-heap allocator to free off-heap memory?"; +assert (memory.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + "page has already been freed"; +assert ((memory.pageNumber == MemoryBlock.NO_PAGE_NUMBER) +|| (memory.pageNumber == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : + "TMM-allocated pages must first be freed via TMM.freePage(), not directly in allocator free()"; + final long size = memory.size(); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_FREED_VALUE);
spark git commit: [SPARK-21444] Be more defensive when removing broadcasts in MapOutputTracker
Repository: spark Updated Branches: refs/heads/master e9faae135 -> 5952ad2b4 [SPARK-21444] Be more defensive when removing broadcasts in MapOutputTracker ## What changes were proposed in this pull request? In SPARK-21444, sitalkedia reported an issue where the `Broadcast.destroy()` call in `MapOutputTracker`'s `ShuffleStatus.invalidateSerializedMapOutputStatusCache()` was failing with an `IOException`, causing the DAGScheduler to crash and bring down the entire driver. This is a bug introduced by #17955. In the old code, we removed a broadcast variable by calling `BroadcastManager.unbroadcast` with `blocking=false`, but the new code simply calls `Broadcast.destroy()` which is capable of failing with an IOException in case certain blocking RPCs time out. The fix implemented here is to replace this with a call to `destroy(blocking = false)` and to wrap the entire operation in `Utils.tryLogNonFatalError`. ## How was this patch tested? I haven't written regression tests for this because it's really hard to inject mocks to simulate RPC failures here. Instead, this class of issue is probably best uncovered with more generalized error injection / network unreliability / fuzz testing tools. Author: Josh Rosen Closes #18662 from JoshRosen/SPARK-21444. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5952ad2b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5952ad2b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5952ad2b Branch: refs/heads/master Commit: 5952ad2b40c82c0ccb2ec16fa09071bf198ff99d Parents: e9faae1 Author: Josh Rosen Authored: Mon Jul 17 20:40:32 2017 -0700 Committer: Josh Rosen Committed: Mon Jul 17 20:40:32 2017 -0700 -- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5952ad2b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala -- diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 5d48bc7..7f760a5 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -194,7 +194,12 @@ private class ShuffleStatus(numPartitions: Int) { */ def invalidateSerializedMapOutputStatusCache(): Unit = synchronized { if (cachedSerializedBroadcast != null) { - cachedSerializedBroadcast.destroy() + // Prevent errors during broadcast cleanup from crashing the DAGScheduler (see SPARK-21444) + Utils.tryLogNonFatalError { +// Use `blocking = false` so that this operation doesn't hang while trying to send cleanup +// RPCs to dead executors. +cachedSerializedBroadcast.destroy(blocking = false) + } cachedSerializedBroadcast = null } cachedSerializedMapStatus = null - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-20715] Store MapStatuses only in MapOutputTracker, not ShuffleMapStage
g parent stages. Given this, this patch chooses to do away with tracking multiple locations for map outputs and instead stores only a single location. This change removes the main distinction between the `ShuffleMapTask` and `MapOutputTracker`'s copies of this state, paving the way for storing it only in the `MapOutputTracker`. ### Overview of other changes - Significantly simplified the cache / lock management inside of the `MapOutputTrackerMaster`: - The old code had several parallel `HashMap`s which had to be guarded by maps of `Object`s which were used as locks. This code was somewhat complicated to follow. - The new code uses a new `ShuffleStatus` class to group together all of the state associated with a particular shuffle, including cached serialized map statuses, significantly simplifying the logic. - Moved more code out of the shared `MapOutputTracker` abstract base class and into the `MapOutputTrackerMaster` and `MapOutputTrackerWorker` subclasses. This makes it easier to reason about which functionality needs to be supported only on the driver or executor. - Removed a bunch of code from the `DAGScheduler` which was used to synchronize information from the `MapOutputTracker` to `ShuffleMapStage`. - Added comments to clarify the role of `MapOutputTrackerMaster`'s `epoch` in invalidating executor-side shuffle map output caches. I will comment on these changes via inline GitHub review comments. /cc hvanhovell and rxin (whom I discussed this with offline), tgravescs (who recently worked on caching of serialized MapOutputStatuses), and kayousterhout and markhamstra (for scheduler changes). ## How was this patch tested? Existing tests. I purposely avoided making interface / API which would require significant updates or modifications to test code. Author: Josh Rosen Closes #17955 from JoshRosen/map-output-tracker-rewrite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3476390c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3476390c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3476390c Branch: refs/heads/master Commit: 3476390c6e5d0fcfff340410f57e114039b5fbd4 Parents: f48273c Author: Josh Rosen Authored: Sun Jun 11 18:34:12 2017 -0700 Committer: Josh Rosen Committed: Sun Jun 11 18:34:12 2017 -0700 -- .../org/apache/spark/MapOutputTracker.scala | 636 +++ .../org/apache/spark/executor/Executor.scala| 10 +- .../apache/spark/scheduler/DAGScheduler.scala | 51 +- .../spark/scheduler/ShuffleMapStage.scala | 76 +-- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../apache/spark/MapOutputTrackerSuite.scala| 6 +- .../scala/org/apache/spark/ShuffleSuite.scala | 3 +- .../spark/scheduler/BlacklistTrackerSuite.scala | 3 +- 8 files changed, 398 insertions(+), 389 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3476390c/core/src/main/scala/org/apache/spark/MapOutputTracker.scala -- diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 4ef6656..3e10b9e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -34,6 +34,156 @@ import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util._ +/** + * Helper class used by the [[MapOutputTrackerMaster]] to perform bookkeeping for a single + * ShuffleMapStage. + * + * This class maintains a mapping from mapIds to `MapStatus`. It also maintains a cache of + * serialized map statuses in order to speed up tasks' requests for map output statuses. + * + * All public methods of this class are thread-safe. + */ +private class ShuffleStatus(numPartitions: Int) { + + // All accesses to the following state must be guarded with `this.synchronized`. + + /** + * MapStatus for each partition. The index of the array is the map partition id. + * Each value in the array is the MapStatus for a partition, or null if the partition + * is not available. Even though in theory a task may run multiple times (due to speculation, + * stage retries, etc.), in practice the likelihood of a map output being available at multiple + * locations is so small that we choose to ignore that case and store only a single location + * for each output. + */ + private[this] val mapStatuses = new Array[MapStatus](numPartitions) + + /** + * The cached result of serializing the map statuses array. This cache is lazily populated when + * [[serializedMapStatus]] is called. The cache is invalida
spark git commit: HOTFIX: fix Scalastyle break introduced in 4d57981cfb18e7500cde6c03ae46c7c9b697d064
Repository: spark Updated Branches: refs/heads/master de953c214 -> 798a04fd7 HOTFIX: fix Scalastyle break introduced in 4d57981cfb18e7500cde6c03ae46c7c9b697d064 Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/798a04fd Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/798a04fd Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/798a04fd Branch: refs/heads/master Commit: 798a04fd7645224b26a05b0e17e565daeeff3b64 Parents: de953c2 Author: Josh Rosen Authored: Tue May 30 12:22:23 2017 -0700 Committer: Josh Rosen Committed: Tue May 30 12:22:23 2017 -0700 -- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/798a04fd/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index b98a705..1cd6fda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2747,7 +2747,7 @@ class Dataset[T] private[sql]( def createGlobalTempView(viewName: String): Unit = withPlan { createTempViewCommand(viewName, replace = false, global = true) } - + /** * Creates or replaces a global temporary view using the given name. The lifetime of this * temporary view is tied to this Spark application. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-20102] Fix nightly packaging and RC packaging scripts w/ two minor build fixes
Repository: spark Updated Branches: refs/heads/branch-2.1 b6d348eea -> 4056191d3 [SPARK-20102] Fix nightly packaging and RC packaging scripts w/ two minor build fixes ## What changes were proposed in this pull request? The master snapshot publisher builds are currently broken due to two minor build issues: 1. For unknown reasons, the LFTP `mkdir -p` command began throwing errors when the remote directory already exists. This change of behavior might have been caused by configuration changes in the ASF's SFTP server, but I'm not entirely sure of that. To work around this problem, this patch updates the script to ignore errors from the `lftp mkdir -p` commands. 2. The PySpark `setup.py` file references a non-existent `pyspark.ml.stat` module, causing Python packaging to fail by complaining about a missing directory. The fix is to simply drop that line from the setup script. ## How was this patch tested? The LFTP fix was tested by manually running the failing commands on AMPLab Jenkins against the ASF SFTP server. The PySpark fix was tested locally. Author: Josh Rosen Closes #17437 from JoshRosen/spark-20102. (cherry picked from commit 314cf51ded52834cfbaacf58d3d05a220965ca2a) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4056191d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4056191d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4056191d Branch: refs/heads/branch-2.1 Commit: 4056191d3d8b178b15cabdc170233be0cbe64345 Parents: b6d348e Author: Josh Rosen Authored: Mon Mar 27 10:23:28 2017 -0700 Committer: Josh Rosen Committed: Mon Mar 27 10:23:44 2017 -0700 -- dev/create-release/release-build.sh | 8 python/setup.py | 1 - 2 files changed, 4 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4056191d/dev/create-release/release-build.sh -- diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index b08577c..ab17f2f 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -249,7 +249,7 @@ if [[ "$1" == "package" ]]; then dest_dir="$REMOTE_PARENT_DIR/${DEST_DIR_NAME}-bin" echo "Copying release tarballs to $dest_dir" # Put to new directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mput -O $dest_dir 'spark-*' LFTP mput -O $dest_dir 'pyspark-*' LFTP mput -O $dest_dir 'SparkR_*' @@ -257,7 +257,7 @@ if [[ "$1" == "package" ]]; then LFTP "rm -r -f $REMOTE_PARENT_DIR/latest || exit 0" LFTP mv $dest_dir "$REMOTE_PARENT_DIR/latest" # Re-upload a second time and leave the files in the timestamped upload directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mput -O $dest_dir 'spark-*' LFTP mput -O $dest_dir 'pyspark-*' LFTP mput -O $dest_dir 'SparkR_*' @@ -275,13 +275,13 @@ if [[ "$1" == "docs" ]]; then PRODUCTION=1 RELEASE_VERSION="$SPARK_VERSION" jekyll build echo "Copying release documentation to $dest_dir" # Put to new directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mirror -R _site $dest_dir # Delete /latest directory and rename new upload to /latest LFTP "rm -r -f $REMOTE_PARENT_DIR/latest || exit 0" LFTP mv $dest_dir "$REMOTE_PARENT_DIR/latest" # Re-upload a second time and leave the files in the timestamped upload directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mirror -R _site $dest_dir cd .. exit 0 http://git-wip-us.apache.org/repos/asf/spark/blob/4056191d/python/setup.py -- diff --git a/python/setup.py b/python/setup.py index 47eab98..f500354 100644 --- a/python/setup.py +++ b/python/setup.py @@ -167,7 +167,6 @@ try: 'pyspark.ml', 'pyspark.ml.linalg', 'pyspark.ml.param', - 'pyspark.ml.stat', 'pyspark.sql', 'pyspark.streaming', 'pyspark.bin', - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-20102] Fix nightly packaging and RC packaging scripts w/ two minor build fixes
Repository: spark Updated Branches: refs/heads/master 0588dc7c0 -> 314cf51de [SPARK-20102] Fix nightly packaging and RC packaging scripts w/ two minor build fixes ## What changes were proposed in this pull request? The master snapshot publisher builds are currently broken due to two minor build issues: 1. For unknown reasons, the LFTP `mkdir -p` command began throwing errors when the remote directory already exists. This change of behavior might have been caused by configuration changes in the ASF's SFTP server, but I'm not entirely sure of that. To work around this problem, this patch updates the script to ignore errors from the `lftp mkdir -p` commands. 2. The PySpark `setup.py` file references a non-existent `pyspark.ml.stat` module, causing Python packaging to fail by complaining about a missing directory. The fix is to simply drop that line from the setup script. ## How was this patch tested? The LFTP fix was tested by manually running the failing commands on AMPLab Jenkins against the ASF SFTP server. The PySpark fix was tested locally. Author: Josh Rosen Closes #17437 from JoshRosen/spark-20102. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/314cf51d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/314cf51d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/314cf51d Branch: refs/heads/master Commit: 314cf51ded52834cfbaacf58d3d05a220965ca2a Parents: 0588dc7 Author: Josh Rosen Authored: Mon Mar 27 10:23:28 2017 -0700 Committer: Josh Rosen Committed: Mon Mar 27 10:23:28 2017 -0700 -- dev/create-release/release-build.sh | 8 python/setup.py | 1 - 2 files changed, 4 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/314cf51d/dev/create-release/release-build.sh -- diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index e1db997..7976d8a 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -246,7 +246,7 @@ if [[ "$1" == "package" ]]; then dest_dir="$REMOTE_PARENT_DIR/${DEST_DIR_NAME}-bin" echo "Copying release tarballs to $dest_dir" # Put to new directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mput -O $dest_dir 'spark-*' LFTP mput -O $dest_dir 'pyspark-*' LFTP mput -O $dest_dir 'SparkR_*' @@ -254,7 +254,7 @@ if [[ "$1" == "package" ]]; then LFTP "rm -r -f $REMOTE_PARENT_DIR/latest || exit 0" LFTP mv $dest_dir "$REMOTE_PARENT_DIR/latest" # Re-upload a second time and leave the files in the timestamped upload directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mput -O $dest_dir 'spark-*' LFTP mput -O $dest_dir 'pyspark-*' LFTP mput -O $dest_dir 'SparkR_*' @@ -271,13 +271,13 @@ if [[ "$1" == "docs" ]]; then PRODUCTION=1 RELEASE_VERSION="$SPARK_VERSION" jekyll build echo "Copying release documentation to $dest_dir" # Put to new directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mirror -R _site $dest_dir # Delete /latest directory and rename new upload to /latest LFTP "rm -r -f $REMOTE_PARENT_DIR/latest || exit 0" LFTP mv $dest_dir "$REMOTE_PARENT_DIR/latest" # Re-upload a second time and leave the files in the timestamped upload directory: - LFTP mkdir -p $dest_dir + LFTP mkdir -p $dest_dir || true LFTP mirror -R _site $dest_dir cd .. exit 0 http://git-wip-us.apache.org/repos/asf/spark/blob/314cf51d/python/setup.py -- diff --git a/python/setup.py b/python/setup.py index 47eab98..f500354 100644 --- a/python/setup.py +++ b/python/setup.py @@ -167,7 +167,6 @@ try: 'pyspark.ml', 'pyspark.ml.linalg', 'pyspark.ml.param', - 'pyspark.ml.stat', 'pyspark.sql', 'pyspark.streaming', 'pyspark.bin', - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-19529][BRANCH-1.6] Backport PR #16866 to branch-1.6
Repository: spark Updated Branches: refs/heads/branch-1.6 e78138a43 -> a50ef3d9a [SPARK-19529][BRANCH-1.6] Backport PR #16866 to branch-1.6 ## What changes were proposed in this pull request? This PR backports PR #16866 to branch-1.6 ## How was this patch tested? Existing tests. Author: Cheng Lian Closes #16917 from liancheng/spark-19529-1.6-backport. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a50ef3d9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a50ef3d9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a50ef3d9 Branch: refs/heads/branch-1.6 Commit: a50ef3d9a06fcbb8c5eca0762fdf0967f4aa7a88 Parents: e78138a Author: Cheng Lian Authored: Tue Feb 14 11:27:37 2017 -0800 Committer: Josh Rosen Committed: Tue Feb 14 11:27:37 2017 -0800 -- .../network/client/TransportClientFactory.java | 10 ++ .../spark/network/TransportClientFactorySuite.java | 6 -- .../network/shuffle/ExternalShuffleClient.java | 4 ++-- .../spark/network/shuffle/RetryingBlockFetcher.java | 3 ++- .../shuffle/mesos/MesosExternalShuffleClient.java | 3 ++- .../spark/network/sasl/SaslIntegrationSuite.java| 4 ++-- .../shuffle/ExternalShuffleIntegrationSuite.java| 2 +- .../shuffle/ExternalShuffleSecuritySuite.java | 7 --- .../network/shuffle/RetryingBlockFetcherSuite.java | 16 9 files changed, 31 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a50ef3d9/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java -- diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 61bafc8..5b438b7 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -120,7 +120,8 @@ public class TransportClientFactory implements Closeable { * * Concurrency: This method is safe to call from multiple threads. */ - public TransportClient createClient(String remoteHost, int remotePort) throws IOException { + public TransportClient createClient(String remoteHost, int remotePort) + throws IOException, InterruptedException { // Get connection from the connection pool first. // If it is not found or not active, create a new one. final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); @@ -176,13 +177,14 @@ public class TransportClientFactory implements Closeable { * As with {@link #createClient(String, int)}, this method is blocking. */ public TransportClient createUnmanagedClient(String remoteHost, int remotePort) - throws IOException { + throws IOException, InterruptedException { final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); return createClient(address); } /** Create a completely new {@link TransportClient} to the remote address. */ - private TransportClient createClient(InetSocketAddress address) throws IOException { + private TransportClient createClient(InetSocketAddress address) + throws IOException, InterruptedException { logger.debug("Creating new connection to " + address); Bootstrap bootstrap = new Bootstrap(); @@ -209,7 +211,7 @@ public class TransportClientFactory implements Closeable { // Connect to the remote server long preConnect = System.nanoTime(); ChannelFuture cf = bootstrap.connect(address); -if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { +if (!cf.await(conf.connectionTimeoutMs())) { throw new IOException( String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); } else if (cf.cause() != null) { http://git-wip-us.apache.org/repos/asf/spark/blob/a50ef3d9/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java -- diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index dac7d4a..6b77e34 100644 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -99,6 +99,8 @@ public class TransportClientFactorySuite { clients.add(client); } catch (IOException e) { failed.incrementAndGet(); +
spark git commit: [SPARK-18952][BACKPORT] Regex strings not properly escaped in codegen for aggregations
Repository: spark Updated Branches: refs/heads/branch-2.1 80a3e13e5 -> 3b6ac323b [SPARK-18952][BACKPORT] Regex strings not properly escaped in codegen for aggregations ## What changes were proposed in this pull request? Backport for #16361 to 2.1 branch. ## How was this patch tested? Unit tests Author: Burak Yavuz Closes #16518 from brkyvz/reg-break-2.1. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3b6ac323 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3b6ac323 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3b6ac323 Branch: refs/heads/branch-2.1 Commit: 3b6ac323b16f8f6d79ee7bac6e7a57f841897d96 Parents: 80a3e13 Author: Burak Yavuz Authored: Mon Jan 9 15:17:59 2017 -0800 Committer: Josh Rosen Committed: Mon Jan 9 15:17:59 2017 -0800 -- .../execution/aggregate/RowBasedHashMapGenerator.scala | 12 +++- .../aggregate/VectorizedHashMapGenerator.scala | 12 +++- .../org/apache/spark/sql/DataFrameAggregateSuite.scala | 9 + 3 files changed, 23 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3b6ac323/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala index a77e178..1b6e6d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala @@ -43,28 +43,30 @@ class RowBasedHashMapGenerator( extends HashMapGenerator (ctx, aggregateExpressions, generatedClassName, groupingKeySchema, bufferSchema) { - protected def initializeAggregateHashMap(): String = { + override protected def initializeAggregateHashMap(): String = { val generatedKeySchema: String = s"new org.apache.spark.sql.types.StructType()" + groupingKeySchema.map { key => + val keyName = ctx.addReferenceObj(key.name) key.dataType match { case d: DecimalType => - s""".add("${key.name}", org.apache.spark.sql.types.DataTypes.createDecimalType( + s""".add("$keyName", org.apache.spark.sql.types.DataTypes.createDecimalType( |${d.precision}, ${d.scale}))""".stripMargin case _ => - s""".add("${key.name}", org.apache.spark.sql.types.DataTypes.${key.dataType})""" + s""".add("$keyName", org.apache.spark.sql.types.DataTypes.${key.dataType})""" } }.mkString("\n").concat(";") val generatedValueSchema: String = s"new org.apache.spark.sql.types.StructType()" + bufferSchema.map { key => + val keyName = ctx.addReferenceObj(key.name) key.dataType match { case d: DecimalType => - s""".add("${key.name}", org.apache.spark.sql.types.DataTypes.createDecimalType( + s""".add("$keyName", org.apache.spark.sql.types.DataTypes.createDecimalType( |${d.precision}, ${d.scale}))""".stripMargin case _ => - s""".add("${key.name}", org.apache.spark.sql.types.DataTypes.${key.dataType})""" + s""".add("$keyName", org.apache.spark.sql.types.DataTypes.${key.dataType})""" } }.mkString("\n").concat(";") http://git-wip-us.apache.org/repos/asf/spark/blob/3b6ac323/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala index 7418df9..586328a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala @@ -48,28 +48,30 @@ class VectorizedHashMapGenerator( extends HashMapGenerator (ctx, aggregateExpressions, generatedClassName, groupingKeySchema, bufferSchema) { - protected def initializeAggregateHashMap(): String = { + override protected def initializeAggregateHashMap(): String = { val generatedSchema: String = s"new org.apache.spark.sql.types.StructType()" + (groupingKeySchema ++ bufferSchema).map { key => + val keyName = ctx.addReferenceObj(key.name) key.dataType mat
spark git commit: [SPARK-18952] Regex strings not properly escaped in codegen for aggregations
Repository: spark Updated Branches: refs/heads/master 15c2bd01b -> faabe69cc [SPARK-18952] Regex strings not properly escaped in codegen for aggregations ## What changes were proposed in this pull request? If I use the function regexp_extract, and then in my regex string, use `\`, i.e. escape character, this fails codegen, because the `\` character is not properly escaped when codegen'd. Example stack trace: ``` /* 059 */ private int maxSteps = 2; /* 060 */ private int numRows = 0; /* 061 */ private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add("date_format(window#325.start, -MM-dd HH:mm)", org.apache.spark.sql.types.DataTypes.StringType) /* 062 */ .add("regexp_extract(source#310.description, ([a-zA-Z]+)\[.*, 1)", org.apache.spark.sql.types.DataTypes.StringType); /* 063 */ private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add("sum", org.apache.spark.sql.types.DataTypes.LongType); /* 064 */ private Object emptyVBase; ... org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 62, Column 58: Invalid escape sequence at org.codehaus.janino.Scanner.scanLiteralCharacter(Scanner.java:918) at org.codehaus.janino.Scanner.produce(Scanner.java:604) at org.codehaus.janino.Parser.peekRead(Parser.java:3239) at org.codehaus.janino.Parser.parseArguments(Parser.java:3055) at org.codehaus.janino.Parser.parseSelector(Parser.java:2914) at org.codehaus.janino.Parser.parseUnaryExpression(Parser.java:2617) at org.codehaus.janino.Parser.parseMultiplicativeExpression(Parser.java:2573) at org.codehaus.janino.Parser.parseAdditiveExpression(Parser.java:2552) ``` In the codegend expression, the literal should use `\\` instead of `\` A similar problem was solved here: https://github.com/apache/spark/pull/15156. ## How was this patch tested? Regression test in `DataFrameAggregationSuite` Author: Burak Yavuz Closes #16361 from brkyvz/reg-break. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/faabe69c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/faabe69c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/faabe69c Branch: refs/heads/master Commit: faabe69cc081145f43f9c68db1a7a8c5c39684fb Parents: 15c2bd0 Author: Burak Yavuz Authored: Mon Jan 9 14:25:38 2017 -0800 Committer: Josh Rosen Committed: Mon Jan 9 14:25:38 2017 -0800 -- .../execution/aggregate/RowBasedHashMapGenerator.scala | 12 +++- .../aggregate/VectorizedHashMapGenerator.scala | 12 +++- .../org/apache/spark/sql/DataFrameAggregateSuite.scala | 9 + 3 files changed, 23 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/faabe69c/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala index a77e178..9316ebc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala @@ -43,28 +43,30 @@ class RowBasedHashMapGenerator( extends HashMapGenerator (ctx, aggregateExpressions, generatedClassName, groupingKeySchema, bufferSchema) { - protected def initializeAggregateHashMap(): String = { + override protected def initializeAggregateHashMap(): String = { val generatedKeySchema: String = s"new org.apache.spark.sql.types.StructType()" + groupingKeySchema.map { key => + val keyName = ctx.addReferenceMinorObj(key.name) key.dataType match { case d: DecimalType => - s""".add("${key.name}", org.apache.spark.sql.types.DataTypes.createDecimalType( + s""".add("$keyName", org.apache.spark.sql.types.DataTypes.createDecimalType( |${d.precision}, ${d.scale}))""".stripMargin case _ => - s""".add("${key.name}", org.apache.spark.sql.types.DataTypes.${key.dataType})""" + s""".add("$keyName", org.apache.spark.sql.types.DataTypes.${key.dataType})""" } }.mkString("\n").concat(";") val generatedValueSchema: String = s"new org.apache.spark.sql.types.StructType()" + bufferSchema.map { key => + val keyName = ctx.addReferenceMinorObj(key.name) key.dataType match { case d: DecimalType => -
spark git commit: [SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors
Repository: spark Updated Branches: refs/heads/branch-2.1 f07e989c0 -> 2971ae564 [SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen Closes #16189 from JoshRosen/cancellation. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2971ae56 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2971ae56 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2971ae56 Branch: refs/heads/branch-2.1 Commit: 2971ae564cb3e97aa5ecac7f411daed7d54248ad Parents: f07e989 Author: Josh Rosen Authored: Mon Dec 19 18:43:59 2016 -0800 Committer: Josh Rosen Committed: Tue Dec 20 11:53:20 2016 -0800 -- .../org/apache/spark/executor/Executor.scala| 169 ++- .../scala/org/apache/spark/util/Utils.scala | 56 +++--- .../org/apache/spark/JobCancellationSuite.scala | 77 + docs/configuration.md | 42 + 4 files changed, 316 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2971ae56/core/src/main/scala/org/apache/spark/executor/Executor.scala -- diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 9501dd9..3346f6d 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -84,6 +84,16 @@ private[spark] class Executor( // Start worker thread pool private val threadPool = ThreadUtils.newDaemonCachedThreadPool("Executor task launch worker") private val executorSource = new ExecutorSource(threadPool, executorId) + // Pool used for threads that supervise task killing / cancellation + private val taskReaperPool = ThreadUtils.newDaemonCachedThreadPool("Task reaper") + // For tasks which are in the process of being killed, this map holds the most recently created + // TaskReaper. All accesses to this map should be synchronized on the map itself (this isn't + // a ConcurrentHashMap because we use the synchronization for purposes other than simply guarding + // the integrity of the map's internal state). The purpose of this map is to prevent the creation + // of a separate TaskReaper for every killTask() of a given task. Instead, this map allows us to + // track whether an existing TaskReaper fulfills the role of a TaskReaper that we would otherwise + // create. The map key is a task id. + private val taskReaperForTask: HashMap[Long, TaskReaper] = HashMap[Long, TaskReaper]() if (!isLocal) { env.metricsSystem.registerSource(executorSource) @@ -93,6 +103,9 @@ private[spark] class Executor( // Whether to load classes in user jars before those in Spark jars private val userClassPathFirst = conf.getBoolean("spark.executor.userClassPathFirst", false) + // Whether to monitor killed / interrupted tasks + private val taskReaperEnabled = conf.getBoolean("spark.task.reaper.enabled", false) + // Create our ClassLoader // do this after SparkEnv creation
spark git commit: [SPARK-18553][CORE][BRANCH-1.6] Fix leak of TaskSetManager following executor loss
Repository: spark Updated Branches: refs/heads/branch-1.6 9136e2693 -> 8f25cb26f [SPARK-18553][CORE][BRANCH-1.6] Fix leak of TaskSetManager following executor loss ## What changes were proposed in this pull request? _This is the master branch-1.6 version of #15986; the original description follows:_ This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails. This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map. Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L338) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](https://github.com/apache /spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L527) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here. This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss. There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L523) in `removeExecutor`, so I'd appreciate a very careful review of these changes. ## How was this patch tested? I added a new unit test to `TaskSchedulerImplSuite`. /cc kayousterhout and markhamstra, who reviewed #15986. Author: Josh Rosen Closes #16070 from JoshRosen/fix-leak-following-total-executor-loss-1.6. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8f25cb26 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8f25cb26 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8f25cb26 Branch: refs/heads/branch-1.6 Commit: 8f25cb26f44bbb7466b6d1385d3fd857e4f6157e Parents: 9136e26 Author: Josh Rosen Authored: Thu Dec 1 10:42:27 2016 -0800 Committer: Josh Rosen Committed: Thu Dec 1 10:42:27 2016 -0800 -- .../spark/scheduler/TaskSchedulerImpl.scala | 75 .../StandaloneDynamicAllocationSuite.scala | 7 +- .../scheduler/TaskSchedulerImplSuite.scala | 66 + 3 files changed, 115 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8f25cb26/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index bdf19f9..6d1ba42 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -87,8 +87,8 @@ private[spark] class TaskSchedulerImpl( // Incrementing task IDs val nextTaskId = new AtomicLong(0) - // Number of tasks running on each executor - private val executorIdToTaskCount = new HashMap[String, Int] + // IDs of the tasks running on each executor + private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]] // The set of executors we have on each host; this is used to comp
spark git commit: [SPARK-18553][CORE] Fix leak of TaskSetManager following executor loss
Repository: spark Updated Branches: refs/heads/branch-2.1 68e8d243b -> 045ae299c [SPARK-18553][CORE] Fix leak of TaskSetManager following executor loss _This is the master branch version of #15986; the original description follows:_ This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails. This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map. Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L338) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](https://github.com/apache /spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L527) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here. This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss. There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L523) in `removeExecutor`, so I'd appreciate a very careful review of these changes. I added a new unit test to `TaskSchedulerImplSuite`. /cc kayousterhout and markhamstra, who reviewed #15986. Author: Josh Rosen Closes #16045 from JoshRosen/fix-leak-following-total-executor-loss-master. (cherry picked from commit 9a02f6821265ff67ba3f7b095cd1afaebd25a898) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/045ae299 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/045ae299 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/045ae299 Branch: refs/heads/branch-2.1 Commit: 045ae299c358e3b991e4e0cd0eb660cd501fdc4d Parents: 68e8d24 Author: Josh Rosen Authored: Tue Nov 29 16:27:25 2016 -0800 Committer: Josh Rosen Committed: Tue Nov 29 16:28:20 2016 -0800 -- .../spark/scheduler/TaskSchedulerImpl.scala | 82 .../StandaloneDynamicAllocationSuite.scala | 7 +- .../scheduler/TaskSchedulerImplSuite.scala | 72 + 3 files changed, 125 insertions(+), 36 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/045ae299/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 3e3f1ad..67446da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -93,10 +93,12 @@ private[spark] class TaskSchedulerImpl( // Incrementing task IDs val nextTaskId = new AtomicLong(0) - // Number of tasks running on each executor - private val executorIdToTaskCount = new HashMap[String, Int] + // IDs of the tasks running on each executor + private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]] - def runningTasksByExecutors(): Map[String, Int] = executorIdT
spark git commit: [SPARK-18553][CORE] Fix leak of TaskSetManager following executor loss
Repository: spark Updated Branches: refs/heads/master 360063521 -> 9a02f6821 [SPARK-18553][CORE] Fix leak of TaskSetManager following executor loss ## What changes were proposed in this pull request? _This is the master branch version of #15986; the original description follows:_ This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails. This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map. Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L338) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](https://github.com/apache /spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L527) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here. This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss. There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L523) in `removeExecutor`, so I'd appreciate a very careful review of these changes. ## How was this patch tested? I added a new unit test to `TaskSchedulerImplSuite`. /cc kayousterhout and markhamstra, who reviewed #15986. Author: Josh Rosen Closes #16045 from JoshRosen/fix-leak-following-total-executor-loss-master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9a02f682 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9a02f682 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9a02f682 Branch: refs/heads/master Commit: 9a02f6821265ff67ba3f7b095cd1afaebd25a898 Parents: 3600635 Author: Josh Rosen Authored: Tue Nov 29 16:27:25 2016 -0800 Committer: Josh Rosen Committed: Tue Nov 29 16:27:25 2016 -0800 -- .../spark/scheduler/TaskSchedulerImpl.scala | 82 .../StandaloneDynamicAllocationSuite.scala | 7 +- .../scheduler/TaskSchedulerImplSuite.scala | 68 3 files changed, 121 insertions(+), 36 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9a02f682/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 3e3f1ad..67446da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -93,10 +93,12 @@ private[spark] class TaskSchedulerImpl( // Incrementing task IDs val nextTaskId = new AtomicLong(0) - // Number of tasks running on each executor - private val executorIdToTaskCount = new HashMap[String, Int] + // IDs of the tasks running on each executor + private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]] - def runningTasksByExecutors(): Map[String, Int] = executorIdToTaskCount.toMap +
spark git commit: [SPARK-18553][CORE][BRANCH-2.0] Fix leak of TaskSetManager following executor loss
Repository: spark Updated Branches: refs/heads/branch-2.0 f158045fd -> 9ff03fa23 [SPARK-18553][CORE][BRANCH-2.0] Fix leak of TaskSetManager following executor loss ## What changes were proposed in this pull request? This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails. This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map. Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L338) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](https://github.com/apache /spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L527) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here. This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss. There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L523) in `removeExecutor`, so I'd appreciate a very careful review of these changes. This PR is opened against branch-2.0, where I first observed this problem, but will also need to be fixed in master, branch-2.1, and branch-1.6 (which I'll do in followup PRs after this fix is reviewed and merged). ## How was this patch tested? I added a new unit test to `TaskSchedulerImplSuite`. You can check out this PR as of 25e455e711b978cd331ee0f484f70fde31307634 to see the failing test. cc kayousterhout, markhamstra, rxin for review. Author: Josh Rosen Closes #15986 from JoshRosen/fix-leak-following-total-executor-loss. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9ff03fa2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9ff03fa2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9ff03fa2 Branch: refs/heads/branch-2.0 Commit: 9ff03fa23e664bc0241914c7b5a7bda0c38eec15 Parents: f158045 Author: Josh Rosen Authored: Mon Nov 28 13:17:24 2016 -0800 Committer: Josh Rosen Committed: Mon Nov 28 13:17:24 2016 -0800 -- .../spark/scheduler/TaskSchedulerImpl.scala | 80 .../StandaloneDynamicAllocationSuite.scala | 7 +- .../scheduler/TaskSchedulerImplSuite.scala | 68 + 3 files changed, 120 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9ff03fa2/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d22321b..b2ef41e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -88,10 +88,12 @@ private[spark] class TaskSchedulerImpl( // Incrementing task IDs val nextTaskId = new AtomicLong(0) - // Number of tasks running on each executor - private val executorIdToTaskCou
spark git commit: [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed
Repository: spark Updated Branches: refs/heads/branch-2.1 951579382 -> 6a3cbbc03 [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed ## What changes were proposed in this pull request? This PR aims to provide a pip installable PySpark package. This does a bunch of work to copy the jars over and package them with the Python code (to prevent challenges from trying to use different versions of the Python code with different versions of the JAR). It does not currently publish to PyPI but that is the natural follow up (SPARK-18129). Done: - pip installable on conda [manual tested] - setup.py installed on a non-pip managed system (RHEL) with YARN [manual tested] - Automated testing of this (virtualenv) - packaging and signing with release-build* Possible follow up work: - release-build update to publish to PyPI (SPARK-18128) - figure out who owns the pyspark package name on prod PyPI (is it someone with in the project or should we ask PyPI or should we choose a different name to publish with like ApachePySpark?) - Windows support and or testing ( SPARK-18136 ) - investigate details of wheel caching and see if we can avoid cleaning the wheel cache during our test - consider how we want to number our dev/snapshot versions Explicitly out of scope: - Using pip installed PySpark to start a standalone cluster - Using pip installed PySpark for non-Python Spark programs *I've done some work to test release-build locally but as a non-committer I've just done local testing. ## How was this patch tested? Automated testing with virtualenv, manual testing with conda, a system wide install, and YARN integration. release-build changes tested locally as a non-committer (no testing of upload artifacts to Apache staging websites) Author: Holden Karau Author: Juliet Hougland Author: Juliet Hougland Closes #15659 from holdenk/SPARK-1267-pip-install-pyspark. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6a3cbbc0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6a3cbbc0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6a3cbbc0 Branch: refs/heads/branch-2.1 Commit: 6a3cbbc037fe631e1b89c46000373dc2ba86a5eb Parents: 9515793 Author: Holden Karau Authored: Wed Nov 16 14:22:15 2016 -0800 Committer: Josh Rosen Committed: Wed Nov 16 20:15:57 2016 -0800 -- .gitignore | 2 + bin/beeline | 2 +- bin/find-spark-home | 41 bin/load-spark-env.sh | 2 +- bin/pyspark | 6 +- bin/run-example | 2 +- bin/spark-class | 6 +- bin/spark-shell | 4 +- bin/spark-sql | 2 +- bin/spark-submit| 2 +- bin/sparkR | 2 +- dev/create-release/release-build.sh | 26 ++- dev/create-release/release-tag.sh | 11 +- dev/lint-python | 4 +- dev/make-distribution.sh| 16 +- dev/pip-sanity-check.py | 36 dev/run-pip-tests | 115 ++ dev/run-tests-jenkins.py| 1 + dev/run-tests.py| 7 + dev/sparktestsupport/__init__.py| 1 + docs/building-spark.md | 8 + docs/index.md | 4 +- .../spark/launcher/CommandBuilderUtils.java | 2 +- python/MANIFEST.in | 22 ++ python/README.md| 32 +++ python/pyspark/__init__.py | 1 + python/pyspark/find_spark_home.py | 74 +++ python/pyspark/java_gateway.py | 3 +- python/pyspark/version.py | 19 ++ python/setup.cfg| 22 ++ python/setup.py | 209 +++ 31 files changed, 660 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6a3cbbc0/.gitignore -- diff --git a/.gitignore b/.gitignore index 39d17e1..5634a43 100644 --- a/.gitignore +++ b/.gitignore @@ -57,6 +57,8 @@ project/plugins/project/build.properties project/plugins/src_managed/ project/plugins/target/ python/lib/pyspark.zip +python/deps +python/pyspark/python reports/ scalastyle-on-compile.generated.xml scalastyle-output.xml http://git-wip-us.apache.org/repos/asf/spark/blob/6a3cbbc0/bin/beeline -
spark git commit: [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed
Repository: spark Updated Branches: refs/heads/master bb6cdfd9a -> a36a76ac4 [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed ## What changes were proposed in this pull request? This PR aims to provide a pip installable PySpark package. This does a bunch of work to copy the jars over and package them with the Python code (to prevent challenges from trying to use different versions of the Python code with different versions of the JAR). It does not currently publish to PyPI but that is the natural follow up (SPARK-18129). Done: - pip installable on conda [manual tested] - setup.py installed on a non-pip managed system (RHEL) with YARN [manual tested] - Automated testing of this (virtualenv) - packaging and signing with release-build* Possible follow up work: - release-build update to publish to PyPI (SPARK-18128) - figure out who owns the pyspark package name on prod PyPI (is it someone with in the project or should we ask PyPI or should we choose a different name to publish with like ApachePySpark?) - Windows support and or testing ( SPARK-18136 ) - investigate details of wheel caching and see if we can avoid cleaning the wheel cache during our test - consider how we want to number our dev/snapshot versions Explicitly out of scope: - Using pip installed PySpark to start a standalone cluster - Using pip installed PySpark for non-Python Spark programs *I've done some work to test release-build locally but as a non-committer I've just done local testing. ## How was this patch tested? Automated testing with virtualenv, manual testing with conda, a system wide install, and YARN integration. release-build changes tested locally as a non-committer (no testing of upload artifacts to Apache staging websites) Author: Holden Karau Author: Juliet Hougland Author: Juliet Hougland Closes #15659 from holdenk/SPARK-1267-pip-install-pyspark. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a36a76ac Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a36a76ac Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a36a76ac Branch: refs/heads/master Commit: a36a76ac43c36a3b897a748bd9f138b629dbc684 Parents: bb6cdfd Author: Holden Karau Authored: Wed Nov 16 14:22:15 2016 -0800 Committer: Josh Rosen Committed: Wed Nov 16 14:22:15 2016 -0800 -- .gitignore | 2 + bin/beeline | 2 +- bin/find-spark-home | 41 bin/load-spark-env.sh | 2 +- bin/pyspark | 6 +- bin/run-example | 2 +- bin/spark-class | 6 +- bin/spark-shell | 4 +- bin/spark-sql | 2 +- bin/spark-submit| 2 +- bin/sparkR | 2 +- dev/create-release/release-build.sh | 26 ++- dev/create-release/release-tag.sh | 11 +- dev/lint-python | 4 +- dev/make-distribution.sh| 16 +- dev/pip-sanity-check.py | 36 dev/run-pip-tests | 115 ++ dev/run-tests-jenkins.py| 1 + dev/run-tests.py| 7 + dev/sparktestsupport/__init__.py| 1 + docs/building-spark.md | 8 + docs/index.md | 4 +- .../spark/launcher/CommandBuilderUtils.java | 2 +- python/MANIFEST.in | 22 ++ python/README.md| 32 +++ python/pyspark/__init__.py | 1 + python/pyspark/find_spark_home.py | 74 +++ python/pyspark/java_gateway.py | 3 +- python/pyspark/version.py | 19 ++ python/setup.cfg| 22 ++ python/setup.py | 209 +++ 31 files changed, 660 insertions(+), 24 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a36a76ac/.gitignore -- diff --git a/.gitignore b/.gitignore index 39d17e1..5634a43 100644 --- a/.gitignore +++ b/.gitignore @@ -57,6 +57,8 @@ project/plugins/project/build.properties project/plugins/src_managed/ project/plugins/target/ python/lib/pyspark.zip +python/deps +python/pyspark/python reports/ scalastyle-on-compile.generated.xml scalastyle-output.xml http://git-wip-us.apache.org/repos/asf/spark/blob/a36a76ac/bin/beeline -
spark git commit: [SPARK-18418] Fix flags for make_binary_release for hadoop profile
Repository: spark Updated Branches: refs/heads/branch-2.1 b2ba83d10 -> 6fae4241f [SPARK-18418] Fix flags for make_binary_release for hadoop profile ## What changes were proposed in this pull request? Fix the flags used to specify the hadoop version ## How was this patch tested? Manually tested as part of https://github.com/apache/spark/pull/15659 by having the build succeed. cc joshrosen Author: Holden Karau Closes #15860 from holdenk/minor-fix-release-build-script. (cherry picked from commit 1386fd28daf798bf152606f4da30a36223d75d18) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6fae4241 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6fae4241 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6fae4241 Branch: refs/heads/branch-2.1 Commit: 6fae4241f281638d52071102c7f0ee6c2c73a8c7 Parents: b2ba83d Author: Holden Karau Authored: Sat Nov 12 14:50:37 2016 -0800 Committer: Josh Rosen Committed: Sat Nov 12 14:51:08 2016 -0800 -- dev/create-release/release-build.sh | 8 1 file changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/6fae4241/dev/create-release/release-build.sh -- diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 96f9b57..81f0d63 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -187,10 +187,10 @@ if [[ "$1" == "package" ]]; then # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. FLAGS="-Psparkr -Phive -Phive-thriftserver -Pyarn -Pmesos" - make_binary_release "hadoop2.3" "-Phadoop2.3 $FLAGS" "3033" & - make_binary_release "hadoop2.4" "-Phadoop2.4 $FLAGS" "3034" & - make_binary_release "hadoop2.6" "-Phadoop2.6 $FLAGS" "3035" & - make_binary_release "hadoop2.7" "-Phadoop2.7 $FLAGS" "3036" & + make_binary_release "hadoop2.3" "-Phadoop-2.3 $FLAGS" "3033" & + make_binary_release "hadoop2.4" "-Phadoop-2.4 $FLAGS" "3034" & + make_binary_release "hadoop2.6" "-Phadoop-2.6 $FLAGS" "3035" & + make_binary_release "hadoop2.7" "-Phadoop-2.7 $FLAGS" "3036" & make_binary_release "hadoop2.4-without-hive" "-Psparkr -Phadoop-2.4 -Pyarn -Pmesos" "3037" & make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn -Pmesos" "3038" & wait - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-18418] Fix flags for make_binary_release for hadoop profile
Repository: spark Updated Branches: refs/heads/master 22cb3a060 -> 1386fd28d [SPARK-18418] Fix flags for make_binary_release for hadoop profile ## What changes were proposed in this pull request? Fix the flags used to specify the hadoop version ## How was this patch tested? Manually tested as part of https://github.com/apache/spark/pull/15659 by having the build succeed. cc joshrosen Author: Holden Karau Closes #15860 from holdenk/minor-fix-release-build-script. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1386fd28 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1386fd28 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1386fd28 Branch: refs/heads/master Commit: 1386fd28daf798bf152606f4da30a36223d75d18 Parents: 22cb3a0 Author: Holden Karau Authored: Sat Nov 12 14:50:37 2016 -0800 Committer: Josh Rosen Committed: Sat Nov 12 14:50:37 2016 -0800 -- dev/create-release/release-build.sh | 8 1 file changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1386fd28/dev/create-release/release-build.sh -- diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 96f9b57..81f0d63 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -187,10 +187,10 @@ if [[ "$1" == "package" ]]; then # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. FLAGS="-Psparkr -Phive -Phive-thriftserver -Pyarn -Pmesos" - make_binary_release "hadoop2.3" "-Phadoop2.3 $FLAGS" "3033" & - make_binary_release "hadoop2.4" "-Phadoop2.4 $FLAGS" "3034" & - make_binary_release "hadoop2.6" "-Phadoop2.6 $FLAGS" "3035" & - make_binary_release "hadoop2.7" "-Phadoop2.7 $FLAGS" "3036" & + make_binary_release "hadoop2.3" "-Phadoop-2.3 $FLAGS" "3033" & + make_binary_release "hadoop2.4" "-Phadoop-2.4 $FLAGS" "3034" & + make_binary_release "hadoop2.6" "-Phadoop-2.6 $FLAGS" "3035" & + make_binary_release "hadoop2.7" "-Phadoop-2.7 $FLAGS" "3036" & make_binary_release "hadoop2.4-without-hive" "-Psparkr -Phadoop-2.4 -Pyarn -Pmesos" "3037" & make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn -Pmesos" "3038" & wait - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer
Repository: spark Updated Branches: refs/heads/master 19cf20806 -> 3a710b94b [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer ## What changes were proposed in this pull request? When profiling heap dumps from the HistoryServer and live Spark web UIs, I found a large amount of memory being wasted on duplicated objects and strings. This patch's changes remove most of this duplication, resulting in over 40% memory savings for some benchmarks. - **Task metrics** (6441f0624dfcda9c7193a64bfb416a145b5aabdf): previously, every `TaskUIData` object would have its own instances of `InputMetricsUIData`, `OutputMetricsUIData`, `ShuffleReadMetrics`, and `ShuffleWriteMetrics`, but for many tasks these metrics are irrelevant because they're all zero. This patch changes how we construct these metrics in order to re-use a single immutable "empty" value for the cases where these metrics are empty. - **TaskInfo.accumulables** (ade86db901127bf13c0e0bdc3f09c933a093bb76): Previously, every `TaskInfo` object had its own empty `ListBuffer` for holding updates from named accumulators. Tasks which didn't use named accumulators still paid for the cost of allocating and storing this empty buffer. To avoid this overhead, I changed the `val` with a mutable buffer into a `var` which holds an immutable Scala list, allowing tasks which do not have named accumulator updates to share the same singleton `Nil` object. - **String.intern() in JSONProtocol** (7e05630e9a78c455db8c8c499f0590c864624e05): in the HistoryServer, executor hostnames and ids are deserialized from JSON, leading to massive duplication of these string objects. By calling `String.intern()` on the deserialized values we can remove all of this duplication. Since Spark now requires Java 7+ we don't have to worry about string interning exhausting the permgen (see http://java-performance.info/string-intern-in-java-6-7-8/). ## How was this patch tested? I ran ``` sc.parallelize(1 to 10, 10).count() ``` in `spark-shell` with event logging enabled, then loaded that event log in the HistoryServer, performed a full GC, and took a heap dump. According to YourKit, the changes in this patch reduced memory consumption by roughly 28 megabytes (or 770k Java objects): ![image](https://cloud.githubusercontent.com/assets/50748/19953276/4f3a28aa-a129-11e6-93df-d7fa91396f66.png) Here's a table illustrating the drop in objects due to deduplication (the drop is <100k for some objects because some events were dropped from the listener bus; this is a separate, existing bug that I'll address separately after CPU-profiling): ![image](https://cloud.githubusercontent.com/assets/50748/19953290/6a271290-a129-11e6-93ad-b825f1448886.png) Author: Josh Rosen Closes #15743 from JoshRosen/spark-ui-memory-usage. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3a710b94 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3a710b94 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3a710b94 Branch: refs/heads/master Commit: 3a710b94b0c853a2dd4c40dca446ecde4e7be959 Parents: 19cf208 Author: Josh Rosen Authored: Mon Nov 7 16:14:19 2016 -0800 Committer: Josh Rosen Committed: Mon Nov 7 16:14:19 2016 -0800 -- .../apache/spark/scheduler/DAGScheduler.scala | 3 +- .../org/apache/spark/scheduler/TaskInfo.scala | 10 ++- .../scala/org/apache/spark/ui/jobs/UIData.scala | 83 +++- .../org/apache/spark/util/JsonProtocol.scala| 10 +-- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 7 +- project/MimaExcludes.scala | 5 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- 8 files changed, 84 insertions(+), 38 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3a710b94/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 f251740..7fde34d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1089,7 +1089,8 @@ class DAGScheduler( // To avoid UI cruft, ignore cases where value wasn't updated if (acc.name.isDefined && !updates.isZero) { stage.latestInfo.accumulables(id) = acc.toInfo(None, Some(acc.value)) - event.taskInfo.accumulables += acc.toInfo(Some(updates.value), Some(acc.value)) + event.taskInfo.setAccumulables( +acc.toInfo(Some(updat
spark git commit: [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness
Repository: spark Updated Branches: refs/heads/branch-2.0 6cc6cb2a9 -> a65d40ab6 [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness We should upgrade to the latest release of MiMa (0.1.11) in order to include a fix for a bug which led to flakiness in the MiMa checks (https://github.com/typesafehub/migration-manager/issues/115). Author: Josh Rosen Closes #15571 from JoshRosen/SPARK-18034. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a65d40ab Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a65d40ab Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a65d40ab Branch: refs/heads/branch-2.0 Commit: a65d40ab63fecc993136a98b8a820d2a8893a9ba Parents: 6cc6cb2 Author: Josh Rosen Authored: Fri Oct 21 11:25:01 2016 -0700 Committer: Josh Rosen Committed: Fri Oct 21 11:28:49 2016 -0700 -- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a65d40ab/project/plugins.sbt -- diff --git a/project/plugins.sbt b/project/plugins.sbt index 8bebd7b..76597d2 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,7 +6,7 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.8.2") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") -addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.9") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.11") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness
Repository: spark Updated Branches: refs/heads/master 3a237512b -> b3b4b9542 [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness We should upgrade to the latest release of MiMa (0.1.11) in order to include a fix for a bug which led to flakiness in the MiMa checks (https://github.com/typesafehub/migration-manager/issues/115). Author: Josh Rosen Closes #15571 from JoshRosen/SPARK-18034. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b3b4b954 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b3b4b954 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b3b4b954 Branch: refs/heads/master Commit: b3b4b9542223de3495a7a7e0dd27634ddb9f929d Parents: 3a23751 Author: Josh Rosen Authored: Fri Oct 21 11:25:01 2016 -0700 Committer: Josh Rosen Committed: Fri Oct 21 11:25:01 2016 -0700 -- project/MimaExcludes.scala | 7 ++- project/plugins.sbt| 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b3b4b954/project/MimaExcludes.scala -- diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index facf034..350b144 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -81,7 +81,12 @@ object MimaExcludes { // [SPARK-17338][SQL] add global temp view ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), + + // [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness. + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.aggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.getAggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.org$apache$spark$ml$param$shared$HasAggregationDepth$_setter_$aggregationDepth_=") ) } http://git-wip-us.apache.org/repos/asf/spark/blob/b3b4b954/project/plugins.sbt -- diff --git a/project/plugins.sbt b/project/plugins.sbt index 8bebd7b..76597d2 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,7 +6,7 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.8.2") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") -addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.9") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.11") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17803][TESTS] Upgrade docker-client dependency
Repository: spark Updated Branches: refs/heads/branch-2.0 a2bf09588 -> e355ca8e8 [SPARK-17803][TESTS] Upgrade docker-client dependency [SPARK-17803: Docker integration tests don't run with "Docker for Mac"](https://issues.apache.org/jira/browse/SPARK-17803) ## What changes were proposed in this pull request? This PR upgrades the [docker-client](https://mvnrepository.com/artifact/com.spotify/docker-client) dependency from [3.6.6](https://mvnrepository.com/artifact/com.spotify/docker-client/3.6.6) to [5.0.2](https://mvnrepository.com/artifact/com.spotify/docker-client/5.0.2) to enable _Docker for Mac_ users to run the `docker-integration-tests` out of the box. The very latest docker-client version is [6.0.0](https://mvnrepository.com/artifact/com.spotify/docker-client/6.0.0) but that has one additional dependency and no usage yet. ## How was this patch tested? The code change was tested on Mac OS X Yosemite with both _Docker Toolbox_ as well as _Docker for Mac_ and on Linux Ubuntu 14.04. ``` $ build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -Phive -Phive-thriftserver -DskipTests clean package $ build/mvn -Pdocker-integration-tests -Pscala-2.11 -pl :spark-docker-integration-tests_2.11 clean compile test ``` Author: Christian Kadner Closes #15378 from ckadner/SPARK-17803_Docker_for_Mac. (cherry picked from commit 49d11d49983fbe270f4df4fb1e34b5fbe854c5ec) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e355ca8e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e355ca8e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e355ca8e Branch: refs/heads/branch-2.0 Commit: e355ca8e828629455228b6a346d64638ab639cfa Parents: a2bf095 Author: Christian Kadner Authored: Thu Oct 6 14:28:49 2016 -0700 Committer: Josh Rosen Committed: Thu Oct 6 14:29:21 2016 -0700 -- .../org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala | 1 + pom.xml| 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e355ca8e/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala -- diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index c36f4d5..609696b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import scala.util.control.NonFatal import com.spotify.docker.client._ +import com.spotify.docker.client.exceptions.ImageNotFoundException import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding} import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually http://git-wip-us.apache.org/repos/asf/spark/blob/e355ca8e/pom.xml -- diff --git a/pom.xml b/pom.xml index f2ffb39..bb4b8a0 100644 --- a/pom.xml +++ b/pom.xml @@ -744,7 +744,7 @@ com.spotify docker-client -3.6.6 +5.0.2 test - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17803][TESTS] Upgrade docker-client dependency
Repository: spark Updated Branches: refs/heads/master 9a48e60e6 -> 49d11d499 [SPARK-17803][TESTS] Upgrade docker-client dependency [SPARK-17803: Docker integration tests don't run with "Docker for Mac"](https://issues.apache.org/jira/browse/SPARK-17803) ## What changes were proposed in this pull request? This PR upgrades the [docker-client](https://mvnrepository.com/artifact/com.spotify/docker-client) dependency from [3.6.6](https://mvnrepository.com/artifact/com.spotify/docker-client/3.6.6) to [5.0.2](https://mvnrepository.com/artifact/com.spotify/docker-client/5.0.2) to enable _Docker for Mac_ users to run the `docker-integration-tests` out of the box. The very latest docker-client version is [6.0.0](https://mvnrepository.com/artifact/com.spotify/docker-client/6.0.0) but that has one additional dependency and no usage yet. ## How was this patch tested? The code change was tested on Mac OS X Yosemite with both _Docker Toolbox_ as well as _Docker for Mac_ and on Linux Ubuntu 14.04. ``` $ build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -Phive -Phive-thriftserver -DskipTests clean package $ build/mvn -Pdocker-integration-tests -Pscala-2.11 -pl :spark-docker-integration-tests_2.11 clean compile test ``` Author: Christian Kadner Closes #15378 from ckadner/SPARK-17803_Docker_for_Mac. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/49d11d49 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/49d11d49 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/49d11d49 Branch: refs/heads/master Commit: 49d11d49983fbe270f4df4fb1e34b5fbe854c5ec Parents: 9a48e60 Author: Christian Kadner Authored: Thu Oct 6 14:28:49 2016 -0700 Committer: Josh Rosen Committed: Thu Oct 6 14:28:49 2016 -0700 -- .../org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala | 1 + pom.xml| 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/49d11d49/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala -- diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index c36f4d5..609696b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import scala.util.control.NonFatal import com.spotify.docker.client._ +import com.spotify.docker.client.exceptions.ImageNotFoundException import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding} import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually http://git-wip-us.apache.org/repos/asf/spark/blob/49d11d49/pom.xml -- diff --git a/pom.xml b/pom.xml index 37976b0..7d13c51 100644 --- a/pom.xml +++ b/pom.xml @@ -744,7 +744,7 @@ com.spotify docker-client -3.6.6 +5.0.2 test - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17712][SQL] Fix invalid pushdown of data-independent filters beneath aggregates
Repository: spark Updated Branches: refs/heads/branch-2.0 7ffafa3bf -> f7839e47c [SPARK-17712][SQL] Fix invalid pushdown of data-independent filters beneath aggregates ## What changes were proposed in this pull request? This patch fixes a minor correctness issue impacting the pushdown of filters beneath aggregates. Specifically, if a filter condition references no grouping or aggregate columns (e.g. `WHERE false`) then it would be incorrectly pushed beneath an aggregate. Intuitively, the only case where you can push a filter beneath an aggregate is when that filter is deterministic and is defined over the grouping columns / expressions, since in that case the filter is acting to exclude entire groups from the query (like a `HAVING` clause). The existing code would only push deterministic filters beneath aggregates when all of the filter's references were grouping columns, but this logic missed the case where a filter has no references. For example, `WHERE false` is deterministic but is independent of the actual data. This patch fixes this minor bug by adding a new check to ensure that we don't push filters beneath aggregates when those filters don't reference any columns. ## How was this patch tested? New regression test in FilterPushdownSuite. Author: Josh Rosen Closes #15289 from JoshRosen/SPARK-17712. (cherry picked from commit 37eb9184f1e9f1c07142c66936671f4711ef407d) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f7839e47 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f7839e47 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f7839e47 Branch: refs/heads/branch-2.0 Commit: f7839e47c3bda86d61c3b2be72c168aab4a5674f Parents: 7ffafa3 Author: Josh Rosen Authored: Wed Sep 28 19:03:05 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 29 12:05:46 2016 -0700 -- .../spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- .../catalyst/optimizer/FilterPushdownSuite.scala | 17 + 2 files changed, 18 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f7839e47/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 35b122d..4c06038 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1071,7 +1071,7 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { val (pushDown, rest) = candidates.partition { cond => val replaced = replaceAlias(cond, aliasMap) -replaced.references.subsetOf(aggregate.child.outputSet) +cond.references.nonEmpty && replaced.references.subsetOf(aggregate.child.outputSet) } val stayUp = rest ++ containingNonDeterministic http://git-wip-us.apache.org/repos/asf/spark/blob/f7839e47/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala -- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 55836f9..019f132 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -687,6 +687,23 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("SPARK-17712: aggregate: don't push down filters that are data-independent") { +val originalQuery = LocalRelation.apply(testRelation.output, Seq.empty) + .select('a, 'b) + .groupBy('a)(count('a)) + .where(false) + +val optimized = Optimize.execute(originalQuery.analyze) + +val correctAnswer = testRelation + .select('a, 'b) + .groupBy('a)(count('a)) + .where(false) + .analyze + +comparePlans(optimized, correctAnswer) + } + test("broadcast hint") { val originalQuery = BroadcastHint(testRelation) .where('a === 2L && 'b + Rand(10).as("rnd") === 3) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-16343][SQL] Improve the PushDownPredicate rule to pushdown predicates correctly in non-deterministic condition.
Repository: spark Updated Branches: refs/heads/branch-2.0 ca8130050 -> 7ffafa3bf [SPARK-16343][SQL] Improve the PushDownPredicate rule to pushdown predicates correctly in non-deterministic condition. ## What changes were proposed in this pull request? Currently our Optimizer may reorder the predicates to run them more efficient, but in non-deterministic condition, change the order between deterministic parts and non-deterministic parts may change the number of input rows. For example: ```SELECT a FROM t WHERE rand() < 0.1 AND a = 1``` And ```SELECT a FROM t WHERE a = 1 AND rand() < 0.1``` may call rand() for different times and therefore the output rows differ. This PR improved this condition by checking whether the predicate is placed before any non-deterministic predicates. ## How was this patch tested? Expanded related testcases in FilterPushdownSuite. Author: èæå Closes #14012 from jiangxb1987/ppd. (cherry picked from commit f376c37268848dbb4b2fb57677e22ef2bf207b49) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7ffafa3b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7ffafa3b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7ffafa3b Branch: refs/heads/branch-2.0 Commit: 7ffafa3bfecb8bc92b79eddea1ca18166efd3385 Parents: ca81300 Author: èæå Authored: Thu Jul 14 00:21:27 2016 +0800 Committer: Josh Rosen Committed: Thu Sep 29 11:44:00 2016 -0700 -- .../sql/catalyst/optimizer/Optimizer.scala | 44 +--- .../optimizer/FilterPushdownSuite.scala | 8 ++-- 2 files changed, 33 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7ffafa3b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index d824c2e..35b122d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1031,19 +1031,23 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { project.copy(child = Filter(replaceAlias(condition, aliasMap), grandChild)) // Push [[Filter]] operators through [[Window]] operators. Parts of the predicate that can be -// pushed beneath must satisfy the following two conditions: +// pushed beneath must satisfy the following conditions: // 1. All the expressions are part of window partitioning key. The expressions can be compound. -// 2. Deterministic +// 2. Deterministic. +// 3. Placed before any non-deterministic predicates. case filter @ Filter(condition, w: Window) if w.partitionSpec.forall(_.isInstanceOf[AttributeReference]) => val partitionAttrs = AttributeSet(w.partitionSpec.flatMap(_.references)) - val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond => -cond.references.subsetOf(partitionAttrs) && cond.deterministic && - // This is for ensuring all the partitioning expressions have been converted to alias - // in Analyzer. Thus, we do not need to check if the expressions in conditions are - // the same as the expressions used in partitioning columns. - partitionAttrs.forall(_.isInstanceOf[Attribute]) + + val (candidates, containingNonDeterministic) = +splitConjunctivePredicates(condition).span(_.deterministic) + + val (pushDown, rest) = candidates.partition { cond => +cond.references.subsetOf(partitionAttrs) } + + val stayUp = rest ++ containingNonDeterministic + if (pushDown.nonEmpty) { val pushDownPredicate = pushDown.reduce(And) val newWindow = w.copy(child = Filter(pushDownPredicate, w.child)) @@ -1062,11 +1066,16 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { // For each filter, expand the alias and check if the filter can be evaluated using // attributes produced by the aggregate operator's child operator. - val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond => + val (candidates, containingNonDeterministic) = +splitConjunctivePredicates(condition).span(_.deterministic) + + val (pushDown, rest) = candidates.partition { cond => val replaced = replaceAlias(cond, aliasMap) -replaced.references.subsetOf(aggregate.child.outputSet) && replaced.deterministic +replaced.references.subsetOf(aggregate.child.outputSet
spark git commit: [SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in MemoryStore
Repository: spark Updated Branches: refs/heads/branch-2.0 98bbc4410 -> 2cd327ef5 [SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in MemoryStore ## What changes were proposed in this pull request? There is an assert in MemoryStore's putIteratorAsValues method which is used to check if unroll memory is not released too much. This assert looks wrong. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #14642 from viirya/fix-unroll-memory. (cherry picked from commit e7bce9e1876de6ee975ccc89351db58119674aef) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2cd327ef Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2cd327ef Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2cd327ef Branch: refs/heads/branch-2.0 Commit: 2cd327ef5e4c3f6b8468ebb2352479a1686b7888 Parents: 98bbc44 Author: Liang-Chi Hsieh Authored: Tue Sep 27 16:00:39 2016 -0700 Committer: Josh Rosen Committed: Tue Sep 27 16:01:04 2016 -0700 -- .../main/scala/org/apache/spark/storage/memory/MemoryStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2cd327ef/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 161434c..9b87c42 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -271,7 +271,7 @@ private[spark] class MemoryStore( blockId, Utils.bytesToString(size), Utils.bytesToString(maxMemory - blocksMemoryUsed))) Right(size) } else { -assert(currentUnrollMemoryForThisTask >= currentUnrollMemoryForThisTask, +assert(currentUnrollMemoryForThisTask >= unrollMemoryUsedByThisBlock, "released too much unroll memory") Left(new PartiallyUnrolledIterator( this, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in MemoryStore
Repository: spark Updated Branches: refs/heads/master 2f84a6866 -> e7bce9e18 [SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in MemoryStore ## What changes were proposed in this pull request? There is an assert in MemoryStore's putIteratorAsValues method which is used to check if unroll memory is not released too much. This assert looks wrong. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #14642 from viirya/fix-unroll-memory. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e7bce9e1 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e7bce9e1 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e7bce9e1 Branch: refs/heads/master Commit: e7bce9e1876de6ee975ccc89351db58119674aef Parents: 2f84a68 Author: Liang-Chi Hsieh Authored: Tue Sep 27 16:00:39 2016 -0700 Committer: Josh Rosen Committed: Tue Sep 27 16:00:39 2016 -0700 -- .../main/scala/org/apache/spark/storage/memory/MemoryStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e7bce9e1/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 205d469..095d324 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -273,7 +273,7 @@ private[spark] class MemoryStore( blockId, Utils.bytesToString(size), Utils.bytesToString(maxMemory - blocksMemoryUsed))) Right(size) } else { -assert(currentUnrollMemoryForThisTask >= currentUnrollMemoryForThisTask, +assert(currentUnrollMemoryForThisTask >= unrollMemoryUsedByThisBlock, "released too much unroll memory") Left(new PartiallyUnrolledIterator( this, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17618] Guard against invalid comparisons between UnsafeRow and other formats
Repository: spark Updated Branches: refs/heads/branch-2.0 f4594900d -> 98bbc4410 [SPARK-17618] Guard against invalid comparisons between UnsafeRow and other formats This patch ports changes from #15185 to Spark 2.x. In that patch, a correctness bug in Spark 1.6.x which was caused by an invalid `equals()` comparison between an `UnsafeRow` and another row of a different format. Spark 2.x is not affected by that specific correctness bug but it can still reap the error-prevention benefits of that patch's changes, which modify ``UnsafeRow.equals()` to throw an IllegalArgumentException if it is called with an object that is not an `UnsafeRow`. Author: Josh Rosen Closes #15265 from JoshRosen/SPARK-17618-master. (cherry picked from commit 2f84a686604b298537bfd4d087b41594d2aa7ec6) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/98bbc441 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/98bbc441 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/98bbc441 Branch: refs/heads/branch-2.0 Commit: 98bbc4410181741d903a703eac289408cb5b2c5e Parents: f459490 Author: Josh Rosen Authored: Tue Sep 27 14:14:27 2016 -0700 Committer: Josh Rosen Committed: Tue Sep 27 14:17:43 2016 -0700 -- .../org/apache/spark/sql/catalyst/expressions/UnsafeRow.java | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/98bbc441/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java -- diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index dd2f39e..9027652 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -31,6 +31,7 @@ import com.esotericsoftware.kryo.KryoSerializable; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; +import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; @@ -577,8 +578,12 @@ public final class UnsafeRow extends MutableRow implements Externalizable, KryoS return (sizeInBytes == o.sizeInBytes) && ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset, sizeInBytes); +} else if (!(other instanceof InternalRow)) { + return false; +} else { + throw new IllegalArgumentException( +"Cannot compare UnsafeRow to " + other.getClass().getName()); } -return false; } /** - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17618] Guard against invalid comparisons between UnsafeRow and other formats
Repository: spark Updated Branches: refs/heads/master 67c73052b -> 2f84a6866 [SPARK-17618] Guard against invalid comparisons between UnsafeRow and other formats This patch ports changes from #15185 to Spark 2.x. In that patch, a correctness bug in Spark 1.6.x which was caused by an invalid `equals()` comparison between an `UnsafeRow` and another row of a different format. Spark 2.x is not affected by that specific correctness bug but it can still reap the error-prevention benefits of that patch's changes, which modify ``UnsafeRow.equals()` to throw an IllegalArgumentException if it is called with an object that is not an `UnsafeRow`. Author: Josh Rosen Closes #15265 from JoshRosen/SPARK-17618-master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2f84a686 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2f84a686 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2f84a686 Branch: refs/heads/master Commit: 2f84a686604b298537bfd4d087b41594d2aa7ec6 Parents: 67c7305 Author: Josh Rosen Authored: Tue Sep 27 14:14:27 2016 -0700 Committer: Josh Rosen Committed: Tue Sep 27 14:14:27 2016 -0700 -- .../org/apache/spark/sql/catalyst/expressions/UnsafeRow.java | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2f84a686/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java -- diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index dd2f39e..9027652 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -31,6 +31,7 @@ import com.esotericsoftware.kryo.KryoSerializable; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; +import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; @@ -577,8 +578,12 @@ public final class UnsafeRow extends MutableRow implements Externalizable, KryoS return (sizeInBytes == o.sizeInBytes) && ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset, sizeInBytes); +} else if (!(other instanceof InternalRow)) { + return false; +} else { + throw new IllegalArgumentException( +"Cannot compare UnsafeRow to " + other.getClass().getName()); } -return false; } /** - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames
Repository: spark Updated Branches: refs/heads/branch-2.0 f14f47f07 -> 243bdb11d [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames Consider you have a bucket as `s3a://some-bucket` and under it you have files: ``` s3a://some-bucket/file1.parquet s3a://some-bucket/file2.parquet ``` Getting the parent path of `s3a://some-bucket/file1.parquet` yields `s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map. When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list! This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics. Unit test in `FileCatalogSuite`. Author: Burak Yavuz Closes #15169 from brkyvz/SPARK-17613. (cherry picked from commit 85d609cf25c1da2df3cd4f5d5aeaf3cbcf0d674c) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/243bdb11 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/243bdb11 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/243bdb11 Branch: refs/heads/branch-2.0 Commit: 243bdb11d89ee379acae1ea1ed78df10797e86d1 Parents: f14f47f Author: Burak Yavuz Authored: Thu Sep 22 13:05:41 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 22 13:06:15 2016 -0700 -- .../PartitioningAwareFileCatalog.scala | 10 - .../datasources/FileCatalogSuite.scala | 45 +++- 2 files changed, 53 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/243bdb11/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index cef9d4d..2130c27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -76,7 +76,15 @@ abstract class PartitioningAwareFileCatalog( paths.flatMap { path => // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). val fs = path.getFileSystem(hadoopConf) -val qualifiedPath = fs.makeQualified(path) +val qualifiedPathPre = fs.makeQualified(path) +val qualifiedPath: Path = if (qualifiedPathPre.isRoot && !qualifiedPathPre.isAbsolute) { + // SPARK-17613: Always append `Path.SEPARATOR` to the end of parent directories, + // because the `leafFile.getParent` would have returned an absolute path with the + // separator at the end. + new Path(qualifiedPathPre, Path.SEPARATOR) +} else { + qualifiedPathPre +} // There are three cases possible with each path // 1. The path is a directory and has children files in it. Then it must be present in http://git-wip-us.apache.org/repos/asf/spark/blob/243bdb11/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 0d9ea51..563f340 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.execution.datasources import java.io.File +import java.net.URI +import scala.collection.mutable import scala.language.reflectiveCalls -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test.SharedSQLContext @@ -67,4 +69,45 @@ class FileCatalogSuite extends SharedSQLContext { } } + + test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { +class MockCatalog( + override val paths: Seq[Path]) extends PartitioningAwareFileCatalog(spark, Map.empty, None) { + + override def refresh(): Unit = {} + + override def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = mutable.LinkedHashMap( +new Path("
spark git commit: [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames
Repository: spark Updated Branches: refs/heads/master 9f24a17c5 -> 85d609cf2 [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames ## What changes were proposed in this pull request? Consider you have a bucket as `s3a://some-bucket` and under it you have files: ``` s3a://some-bucket/file1.parquet s3a://some-bucket/file2.parquet ``` Getting the parent path of `s3a://some-bucket/file1.parquet` yields `s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map. When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list! This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics. ## How was this patch tested? Unit test in `FileCatalogSuite`. Author: Burak Yavuz Closes #15169 from brkyvz/SPARK-17613. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/85d609cf Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/85d609cf Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/85d609cf Branch: refs/heads/master Commit: 85d609cf25c1da2df3cd4f5d5aeaf3cbcf0d674c Parents: 9f24a17 Author: Burak Yavuz Authored: Thu Sep 22 13:05:41 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 22 13:05:41 2016 -0700 -- .../PartitioningAwareFileCatalog.scala | 10 - .../datasources/FileCatalogSuite.scala | 45 +++- 2 files changed, 53 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/85d609cf/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index d2d5b56..702ba97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -76,7 +76,15 @@ abstract class PartitioningAwareFileCatalog( paths.flatMap { path => // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). val fs = path.getFileSystem(hadoopConf) -val qualifiedPath = fs.makeQualified(path) +val qualifiedPathPre = fs.makeQualified(path) +val qualifiedPath: Path = if (qualifiedPathPre.isRoot && !qualifiedPathPre.isAbsolute) { + // SPARK-17613: Always append `Path.SEPARATOR` to the end of parent directories, + // because the `leafFile.getParent` would have returned an absolute path with the + // separator at the end. + new Path(qualifiedPathPre, Path.SEPARATOR) +} else { + qualifiedPathPre +} // There are three cases possible with each path // 1. The path is a directory and has children files in it. Then it must be present in http://git-wip-us.apache.org/repos/asf/spark/blob/85d609cf/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 5c8d322..fa3abd0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.execution.datasources import java.io.File +import java.net.URI +import scala.collection.mutable import scala.language.reflectiveCalls -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test.SharedSQLContext @@ -78,4 +80,45 @@ class FileCatalogSuite extends SharedSQLContext { assert(catalog1.listLeafFiles(catalog1.paths).isEmpty) } } + + test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { +class MockCatalog( + override val paths: Seq[Path]) extends PartitioningAwareFileCatalog(spark, Map.empty, None) { + + override def refresh(): Unit = {} + + override def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = mut
spark git commit: [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job (branch-1.6 backport)
Repository: spark Updated Branches: refs/heads/branch-1.6 ce0a222f5 -> 94524cef4 [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job (branch-1.6 backport) This patch is a branch-1.6 backport of #15037: ## What changes were proposed in this pull request? In Spark's `RDD.getOrCompute` we first try to read a local copy of a cached RDD block, then a remote copy, and only fall back to recomputing the block if no cached copy (local or remote) can be read. This logic works correctly in the case where no remote copies of the block exist, but if there _are_ remote copies and reads of those copies fail (due to network issues or internal Spark bugs) then the BlockManager will throw a `BlockFetchException` that will fail the task (and which could possibly fail the whole job if the read failures keep occurring). In the cases of TorrentBroadcast and task result fetching we really do want to fail the entire job in case no remote blocks can be fetched, but this logic is inappropriate for reads of cached RDD blocks because those can/should be recomputed in case cached blocks are unavailable. Therefore, I think that the `BlockManager.getRemoteBytes()` method should never throw on remote fetch errors and, instead, should handle failures by returning `None`. ## How was this patch tested? Block manager changes should be covered by modified tests in `BlockManagerSuite`: the old tests expected exceptions to be thrown on failed remote reads, while the modified tests now expect `None` to be returned from the `getRemote*` method. I also manually inspected all usages of `BlockManager.getRemoteValues()`, `getRemoteBytes()`, and `get()` to verify that they correctly pattern-match on the result and handle `None`. Note that these `None` branches are already exercised because the old `getRemoteBytes` returned `None` when no remote locations for the block could be found (which could occur if an executor died and its block manager de-registered with the master). Author: Josh Rosen Closes #15186 from JoshRosen/SPARK-17485-branch-1.6-backport. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/94524cef Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/94524cef Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/94524cef Branch: refs/heads/branch-1.6 Commit: 94524cef4cf367a0e73ebe0e919cc21f25f1043f Parents: ce0a222 Author: Josh Rosen Authored: Thu Sep 22 11:05:35 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 22 11:05:35 2016 -0700 -- .../spark/storage/BlockFetchException.scala | 24 .../org/apache/spark/storage/BlockManager.scala | 3 ++- .../spark/storage/BlockManagerSuite.scala | 7 +++--- 3 files changed, 5 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/94524cef/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala deleted file mode 100644 index f6e46ae..000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.storage - -import org.apache.spark.SparkException - -private[spark] -case class BlockFetchException(messages: String, throwable: Throwable) - extends SparkException(messages, throwable) http://git-wip-us.apache.org/repos/asf/spark/blob/94524cef/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 339ee144..1fc6f39 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.sca
spark git commit: [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published
Repository: spark Updated Branches: refs/heads/branch-1.6 8f88412c3 -> ce0a222f5 [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published This patch updates the `kinesis-asl-assembly` build to prevent that module from being published as part of Maven releases and snapshot builds. The `kinesis-asl-assembly` includes classes from the Kinesis Client Library (KCL) and Kinesis Producer Library (KPL), both of which are licensed under the Amazon Software License and are therefore prohibited from being distributed in Apache releases. Author: Josh Rosen Closes #15167 from JoshRosen/stop-publishing-kinesis-assembly. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ce0a222f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ce0a222f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ce0a222f Branch: refs/heads/branch-1.6 Commit: ce0a222f56ffaf85273d2935b3e6d02aa9f6fa48 Parents: 8f88412 Author: Josh Rosen Authored: Wed Sep 21 11:38:10 2016 -0700 Committer: Josh Rosen Committed: Wed Sep 21 11:42:48 2016 -0700 -- extras/kinesis-asl-assembly/pom.xml | 15 +++ 1 file changed, 15 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ce0a222f/extras/kinesis-asl-assembly/pom.xml -- diff --git a/extras/kinesis-asl-assembly/pom.xml b/extras/kinesis-asl-assembly/pom.xml index 98d6d8d..6528e4e 100644 --- a/extras/kinesis-asl-assembly/pom.xml +++ b/extras/kinesis-asl-assembly/pom.xml @@ -132,6 +132,21 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-deploy-plugin + +true + + + + org.apache.maven.plugins + maven-install-plugin + +true + + org.apache.maven.plugins maven-shade-plugin - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published
Repository: spark Updated Branches: refs/heads/branch-2.0 45bccdd9c -> cd0bd89d7 [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published This patch updates the `kinesis-asl-assembly` build to prevent that module from being published as part of Maven releases and snapshot builds. The `kinesis-asl-assembly` includes classes from the Kinesis Client Library (KCL) and Kinesis Producer Library (KPL), both of which are licensed under the Amazon Software License and are therefore prohibited from being distributed in Apache releases. Author: Josh Rosen Closes #15167 from JoshRosen/stop-publishing-kinesis-assembly. (cherry picked from commit d7ee12211a99efae6f7395e47089236838461d61) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cd0bd89d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cd0bd89d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cd0bd89d Branch: refs/heads/branch-2.0 Commit: cd0bd89d7852bab5adfee4b1b53c87efbf95176a Parents: 45bccdd Author: Josh Rosen Authored: Wed Sep 21 11:38:10 2016 -0700 Committer: Josh Rosen Committed: Wed Sep 21 11:38:55 2016 -0700 -- external/kinesis-asl-assembly/pom.xml | 15 +++ 1 file changed, 15 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/cd0bd89d/external/kinesis-asl-assembly/pom.xml -- diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 58c57c1..8fc6fd9 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -142,6 +142,21 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-deploy-plugin + +true + + + + org.apache.maven.plugins + maven-install-plugin + +true + + org.apache.maven.plugins maven-shade-plugin - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published
Repository: spark Updated Branches: refs/heads/master 248922fd4 -> d7ee12211 [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published This patch updates the `kinesis-asl-assembly` build to prevent that module from being published as part of Maven releases and snapshot builds. The `kinesis-asl-assembly` includes classes from the Kinesis Client Library (KCL) and Kinesis Producer Library (KPL), both of which are licensed under the Amazon Software License and are therefore prohibited from being distributed in Apache releases. Author: Josh Rosen Closes #15167 from JoshRosen/stop-publishing-kinesis-assembly. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d7ee1221 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d7ee1221 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d7ee1221 Branch: refs/heads/master Commit: d7ee12211a99efae6f7395e47089236838461d61 Parents: 248922f Author: Josh Rosen Authored: Wed Sep 21 11:38:10 2016 -0700 Committer: Josh Rosen Committed: Wed Sep 21 11:38:10 2016 -0700 -- external/kinesis-asl-assembly/pom.xml | 15 +++ 1 file changed, 15 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d7ee1221/external/kinesis-asl-assembly/pom.xml -- diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index df528b3..f7cb764 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -141,6 +141,21 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-deploy-plugin + +true + + + + org.apache.maven.plugins + maven-install-plugin + +true + + org.apache.maven.plugins maven-shade-plugin - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17160] Properly escape field names in code-generated error messages
Repository: spark Updated Branches: refs/heads/branch-2.0 c02bc926d -> 7026eb87e [SPARK-17160] Properly escape field names in code-generated error messages This patch addresses a corner-case escaping bug where field names which contain special characters were unsafely interpolated into error message string literals in generated Java code, leading to compilation errors. This patch addresses these issues by using `addReferenceObj` to store the error messages as string fields rather than inline string constants. Author: Josh Rosen Closes #15156 from JoshRosen/SPARK-17160. (cherry picked from commit e719b1c045ba185d242d21bbfcdee2c84dafc587) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7026eb87 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7026eb87 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7026eb87 Branch: refs/heads/branch-2.0 Commit: 7026eb87e7d7799d2818334a2e191dc46987975f Parents: c02bc92 Author: Josh Rosen Authored: Mon Sep 19 20:20:36 2016 -0700 Committer: Josh Rosen Committed: Mon Sep 19 20:21:25 2016 -0700 -- .../apache/spark/sql/catalyst/expressions/misc.scala | 12 +--- .../sql/catalyst/expressions/objects/objects.scala | 12 .../sql/catalyst/expressions/CodeGenerationSuite.scala | 13 - 3 files changed, 29 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7026eb87/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 3692075..92f8fb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -477,10 +477,13 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { protected override def nullSafeEval(input: Any): Any = input + private val outputPrefix = s"Result of ${child.simpleString} is " + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { +val outputPrefixField = ctx.addReferenceObj("outputPrefix", outputPrefix) nullSafeCodeGen(ctx, ev, c => s""" - | System.err.println("Result of ${child.simpleString} is " + $c); + | System.err.println($outputPrefixField + $c); | ${ev.value} = $c; """.stripMargin) } @@ -501,10 +504,12 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "assert_true" + private val errMsg = s"'${child.simpleString}' is not true!" + override def eval(input: InternalRow) : Any = { val v = child.eval(input) if (v == null || java.lang.Boolean.FALSE.equals(v)) { - throw new RuntimeException(s"'${child.simpleString}' is not true!") + throw new RuntimeException(errMsg) } else { null } @@ -512,9 +517,10 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = child.genCode(ctx) +val errMsgField = ctx.addReferenceObj("errMsg", errMsg) ExprCode(code = s"""${eval.code} |if (${eval.isNull} || !${eval.value}) { - | throw new RuntimeException("'${child.simpleString}' is not true."); + | throw new RuntimeException($errMsgField); |}""".stripMargin, isNull = "true", value = "null") } http://git-wip-us.apache.org/repos/asf/spark/blob/7026eb87/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 1cdda53..691edd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -763,7 +763,10 @@ case class GetExternalRowField( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") + private val errMsg = s"The ${index}th field
spark git commit: [SPARK-17160] Properly escape field names in code-generated error messages
Repository: spark Updated Branches: refs/heads/master d8104158a -> e719b1c04 [SPARK-17160] Properly escape field names in code-generated error messages This patch addresses a corner-case escaping bug where field names which contain special characters were unsafely interpolated into error message string literals in generated Java code, leading to compilation errors. This patch addresses these issues by using `addReferenceObj` to store the error messages as string fields rather than inline string constants. Author: Josh Rosen Closes #15156 from JoshRosen/SPARK-17160. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e719b1c0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e719b1c0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e719b1c0 Branch: refs/heads/master Commit: e719b1c045ba185d242d21bbfcdee2c84dafc587 Parents: d810415 Author: Josh Rosen Authored: Mon Sep 19 20:20:36 2016 -0700 Committer: Josh Rosen Committed: Mon Sep 19 20:20:36 2016 -0700 -- .../apache/spark/sql/catalyst/expressions/misc.scala | 12 +--- .../sql/catalyst/expressions/objects/objects.scala | 12 .../sql/catalyst/expressions/CodeGenerationSuite.scala | 13 - 3 files changed, 29 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e719b1c0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 3692075..92f8fb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -477,10 +477,13 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { protected override def nullSafeEval(input: Any): Any = input + private val outputPrefix = s"Result of ${child.simpleString} is " + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { +val outputPrefixField = ctx.addReferenceObj("outputPrefix", outputPrefix) nullSafeCodeGen(ctx, ev, c => s""" - | System.err.println("Result of ${child.simpleString} is " + $c); + | System.err.println($outputPrefixField + $c); | ${ev.value} = $c; """.stripMargin) } @@ -501,10 +504,12 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "assert_true" + private val errMsg = s"'${child.simpleString}' is not true!" + override def eval(input: InternalRow) : Any = { val v = child.eval(input) if (v == null || java.lang.Boolean.FALSE.equals(v)) { - throw new RuntimeException(s"'${child.simpleString}' is not true!") + throw new RuntimeException(errMsg) } else { null } @@ -512,9 +517,10 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = child.genCode(ctx) +val errMsgField = ctx.addReferenceObj("errMsg", errMsg) ExprCode(code = s"""${eval.code} |if (${eval.isNull} || !${eval.value}) { - | throw new RuntimeException("'${child.simpleString}' is not true."); + | throw new RuntimeException($errMsgField); |}""".stripMargin, isNull = "true", value = "null") } http://git-wip-us.apache.org/repos/asf/spark/blob/e719b1c0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 4da74a0..faf8fec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -938,7 +938,10 @@ case class GetExternalRowField( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") + private val errMsg = s"The ${index}th field '$fieldName' of input row cannot be null." + override def doGenCode(ctx: Cod
spark git commit: [SPARK-17473][SQL] fixing docker integration tests error due to different versions of jars.
Repository: spark Updated Branches: refs/heads/branch-2.0 c4660d607 -> f56035ba6 [SPARK-17473][SQL] fixing docker integration tests error due to different versions of jars. ## What changes were proposed in this pull request? Docker tests are using older version of jersey jars (1.19), which was used in older releases of spark. In 2.0 releases Spark was upgraded to use 2.x verison of Jersey. After upgrade to new versions, docker tests are failing with AbstractMethodError. Now that spark is upgraded to 2.x jersey version, using of shaded docker jars may not be required any more. Removed the exclusions/overrides of jersey related classes from pom file, and changed the docker-client to use regular jar instead of shaded one. ## How was this patch tested? Tested using existing docker-integration-tests Author: sureshthalamati Closes #15114 from sureshthalamati/docker_testfix-spark-17473. (cherry picked from commit cdea1d1343d02f0077e1f3c92ca46d04a3d30414) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f56035ba Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f56035ba Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f56035ba Branch: refs/heads/branch-2.0 Commit: f56035ba6c86fe93a45fd437f98f812431df0069 Parents: c4660d6 Author: sureshthalamati Authored: Mon Sep 19 09:56:16 2016 -0700 Committer: Josh Rosen Committed: Mon Sep 19 10:29:57 2016 -0700 -- external/docker-integration-tests/pom.xml | 68 -- pom.xml | 1 - 2 files changed, 69 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f56035ba/external/docker-integration-tests/pom.xml -- diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 18e14c7..0ca94e5 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -49,38 +49,7 @@ com.spotify docker-client - shaded test - - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - - - com.fasterxml.jackson.core - jackson-databind - - - org.glassfish.jersey.core - jersey-client - - - org.glassfish.jersey.connectors - jersey-apache-connector - - - org.glassfish.jersey.media - jersey-media-json-jackson - - org.apache.httpcomponents @@ -152,43 +121,6 @@ test - - - com.sun.jersey - jersey-server - 1.19 - test - - - com.sun.jersey - jersey-core - 1.19 - test - - - com.sun.jersey - jersey-servlet - 1.19 - test - - - com.sun.jersey - jersey-json - 1.19 - test - - - stax - stax-api - - - - -
spark git commit: [SPARK-17473][SQL] fixing docker integration tests error due to different versions of jars.
Repository: spark Updated Branches: refs/heads/master d720a4019 -> cdea1d134 [SPARK-17473][SQL] fixing docker integration tests error due to different versions of jars. ## What changes were proposed in this pull request? Docker tests are using older version of jersey jars (1.19), which was used in older releases of spark. In 2.0 releases Spark was upgraded to use 2.x verison of Jersey. After upgrade to new versions, docker tests are failing with AbstractMethodError. Now that spark is upgraded to 2.x jersey version, using of shaded docker jars may not be required any more. Removed the exclusions/overrides of jersey related classes from pom file, and changed the docker-client to use regular jar instead of shaded one. ## How was this patch tested? Tested using existing docker-integration-tests Author: sureshthalamati Closes #15114 from sureshthalamati/docker_testfix-spark-17473. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cdea1d13 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cdea1d13 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cdea1d13 Branch: refs/heads/master Commit: cdea1d1343d02f0077e1f3c92ca46d04a3d30414 Parents: d720a40 Author: sureshthalamati Authored: Mon Sep 19 09:56:16 2016 -0700 Committer: Josh Rosen Committed: Mon Sep 19 09:56:16 2016 -0700 -- external/docker-integration-tests/pom.xml | 68 -- pom.xml | 1 - 2 files changed, 69 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/cdea1d13/external/docker-integration-tests/pom.xml -- diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 7417199..57d553b 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -49,38 +49,7 @@ com.spotify docker-client - shaded test - - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - - - com.fasterxml.jackson.core - jackson-databind - - - org.glassfish.jersey.core - jersey-client - - - org.glassfish.jersey.connectors - jersey-apache-connector - - - org.glassfish.jersey.media - jersey-media-json-jackson - - org.apache.httpcomponents @@ -152,43 +121,6 @@ test - - - com.sun.jersey - jersey-server - 1.19 - test - - - com.sun.jersey - jersey-core - 1.19 - test - - - com.sun.jersey - jersey-servlet - 1.19 - test - - - com.sun.jersey - jersey-json - 1.19 - test - - - stax - stax-api - - - - -
spark git commit: [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes()
Repository: spark Updated Branches: refs/heads/master 86c2d393a -> 8faa5217b [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes() ## What changes were proposed in this pull request? `MemoryStore.putIteratorAsBytes()` may silently lose values when used with `KryoSerializer` because it does not properly close the serialization stream before attempting to deserialize the already-serialized values, which may cause values buffered in Kryo's internal buffers to not be read. This is the root cause behind a user-reported "wrong answer" bug in PySpark caching reported by bennoleslie on the Spark user mailing list in a thread titled "pyspark persist MEMORY_ONLY vs MEMORY_AND_DISK". Due to Spark 2.0's automatic use of KryoSerializer for "safe" types (such as byte arrays, primitives, etc.) this misuse of serializers manifested itself as silent data corruption rather than a StreamCorrupted error (which you might get from JavaSerializer). The minimal fix, implemented here, is to close the serialization stream before attempting to deserialize written values. In addition, this patch adds several additional assertions / precondition checks to prevent misuse of `PartiallySerializedBlock` and `ChunkedByteBufferOutputStream`. ## How was this patch tested? The original bug was masked by an invalid assert in the memory store test cases: the old assert compared two results record-by-record with `zip` but didn't first check that the lengths of the two collections were equal, causing missing records to go unnoticed. The updated test case reproduced this bug. In addition, I added a new `PartiallySerializedBlockSuite` to unit test that component. Author: Josh Rosen Closes #15043 from JoshRosen/partially-serialized-block-values-iterator-bugfix. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8faa5217 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8faa5217 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8faa5217 Branch: refs/heads/master Commit: 8faa5217b44e8d52eab7eb2d53d0652abaaf43cd Parents: 86c2d39 Author: Josh Rosen Authored: Sat Sep 17 11:46:15 2016 -0700 Committer: Josh Rosen Committed: Sat Sep 17 11:46:15 2016 -0700 -- .../scala/org/apache/spark/scheduler/Task.scala | 1 + .../spark/storage/memory/MemoryStore.scala | 89 ++-- .../spark/util/ByteBufferOutputStream.scala | 27 ++- .../util/io/ChunkedByteBufferOutputStream.scala | 12 +- .../apache/spark/storage/MemoryStoreSuite.scala | 34 ++- .../storage/PartiallySerializedBlockSuite.scala | 215 +++ .../PartiallyUnrolledIteratorSuite.scala| 2 +- .../io/ChunkedByteBufferOutputStreamSuite.scala | 8 + 8 files changed, 344 insertions(+), 44 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8faa5217/core/src/main/scala/org/apache/spark/scheduler/Task.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 35c4daf..1ed36bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -230,6 +230,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task) Utils.writeByteBuffer(taskBytes, out) +out.close() out.toByteBuffer } http://git-wip-us.apache.org/repos/asf/spark/blob/8faa5217/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index ec1b0f7..205d469 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -33,7 +33,7 @@ import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage.{BlockId, BlockInfoManager, StorageLevel} import org.apache.spark.unsafe.Platform -import org.apache.spark.util.{CompletionIterator, SizeEstimator, Utils} +import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} @@ -277,6 +277,7 @@ private[spark] class MemoryStore( "released too much unroll memory") Left(new PartiallyUnrolledIterator( this, + MemoryMode.ON_HEAP,
spark git commit: [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes()
Repository: spark Updated Branches: refs/heads/branch-2.0 a3bba372a -> bec077069 [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes() ## What changes were proposed in this pull request? `MemoryStore.putIteratorAsBytes()` may silently lose values when used with `KryoSerializer` because it does not properly close the serialization stream before attempting to deserialize the already-serialized values, which may cause values buffered in Kryo's internal buffers to not be read. This is the root cause behind a user-reported "wrong answer" bug in PySpark caching reported by bennoleslie on the Spark user mailing list in a thread titled "pyspark persist MEMORY_ONLY vs MEMORY_AND_DISK". Due to Spark 2.0's automatic use of KryoSerializer for "safe" types (such as byte arrays, primitives, etc.) this misuse of serializers manifested itself as silent data corruption rather than a StreamCorrupted error (which you might get from JavaSerializer). The minimal fix, implemented here, is to close the serialization stream before attempting to deserialize written values. In addition, this patch adds several additional assertions / precondition checks to prevent misuse of `PartiallySerializedBlock` and `ChunkedByteBufferOutputStream`. ## How was this patch tested? The original bug was masked by an invalid assert in the memory store test cases: the old assert compared two results record-by-record with `zip` but didn't first check that the lengths of the two collections were equal, causing missing records to go unnoticed. The updated test case reproduced this bug. In addition, I added a new `PartiallySerializedBlockSuite` to unit test that component. Author: Josh Rosen Closes #15043 from JoshRosen/partially-serialized-block-values-iterator-bugfix. (cherry picked from commit 8faa5217b44e8d52eab7eb2d53d0652abaaf43cd) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bec07706 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bec07706 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bec07706 Branch: refs/heads/branch-2.0 Commit: bec077069af0b3bc22092a0552baf855dfb344ad Parents: a3bba37 Author: Josh Rosen Authored: Sat Sep 17 11:46:15 2016 -0700 Committer: Josh Rosen Committed: Sat Sep 17 11:46:39 2016 -0700 -- .../scala/org/apache/spark/scheduler/Task.scala | 1 + .../spark/storage/memory/MemoryStore.scala | 89 ++-- .../spark/util/ByteBufferOutputStream.scala | 27 ++- .../util/io/ChunkedByteBufferOutputStream.scala | 12 +- .../apache/spark/storage/MemoryStoreSuite.scala | 34 ++- .../storage/PartiallySerializedBlockSuite.scala | 215 +++ .../PartiallyUnrolledIteratorSuite.scala| 2 +- .../io/ChunkedByteBufferOutputStreamSuite.scala | 8 + 8 files changed, 344 insertions(+), 44 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bec07706/core/src/main/scala/org/apache/spark/scheduler/Task.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 35c4daf..1ed36bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -230,6 +230,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task) Utils.writeByteBuffer(taskBytes, out) +out.close() out.toByteBuffer } http://git-wip-us.apache.org/repos/asf/spark/blob/bec07706/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 1230128..161434c 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -33,7 +33,7 @@ import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage.{BlockId, BlockInfoManager, StorageLevel} import org.apache.spark.unsafe.Platform -import org.apache.spark.util.{CompletionIterator, SizeEstimator, Utils} +import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} @@ -275,6 +275,7 @@ private[spark] class MemoryStore( "released too much unroll
spark git commit: [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions
Repository: spark Updated Branches: refs/heads/branch-2.0 0169c2edc -> 9c23f4408 [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions ## What changes were proposed in this pull request? If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed. This patch addresses this issue via multiple small changes: - The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup). - When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present. - Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls. This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix. For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484). ## How was this patch tested? Two new regression tests in BlockManagerSuite. Author: Josh Rosen Closes #15085 from JoshRosen/SPARK-17484. (cherry picked from commit 1202075c95eabba0ffebc170077df798f271a139) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9c23f440 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9c23f440 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9c23f440 Branch: refs/heads/branch-2.0 Commit: 9c23f4408d337f4af31ebfbcc78767df67d36aed Parents: 0169c2e Author: Josh Rosen Authored: Thu Sep 15 11:54:17 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 15 11:54:39 2016 -0700 -- .../org/apache/spark/storage/BlockManager.scala | 37 +++- .../spark/storage/BlockManagerSuite.scala | 34 ++ 2 files changed, 63 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9c23f440/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 48db97a..37dfbd6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -279,7 +279,12 @@ private[spark] class BlockManager( } else { getLocalBytes(blockId) match { case Some(buffer) => new BlockManagerManagedBuffer(blockInfoManager, blockId, buffer) -case None => throw new BlockNotFoundException(blockId.toString) +case None => + // If this block manager receives a request for a block that it doesn't have then it's + // likely that the master has outdated block statuses for this block. Therefore, we send + // an RPC so that this block is marked as being unavailable from this block manager. + reportBlockStatus(blockId, BlockStatus.empty) + throw new BlockNotFoundException(blockId.toString) } } } @@ -856,22 +861,38 @@ private[spark] class BlockManager( } val startTimeMs = System.currentTimeMillis -var blockWasSuccessfullyStored: Boolean = false +var exceptionWasThrown: Boolean = true val result: Option[T] = try { val res = putBody(putBlockInfo) - blockWasSuccessfullyStored = res.isEmpty - res -} finally { - if (blockWasSuccessfullyStored) { + exceptionWasThrown = false + if (res.isEmpty) { +// the block was successfully stored if (keepReadLock) { blockInfoManager.d
spark git commit: [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions
Repository: spark Updated Branches: refs/heads/master a6b818200 -> 1202075c9 [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions ## What changes were proposed in this pull request? If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed. This patch addresses this issue via multiple small changes: - The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup). - When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present. - Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls. This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix. For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484). ## How was this patch tested? Two new regression tests in BlockManagerSuite. Author: Josh Rosen Closes #15085 from JoshRosen/SPARK-17484. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1202075c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1202075c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1202075c Branch: refs/heads/master Commit: 1202075c95eabba0ffebc170077df798f271a139 Parents: a6b8182 Author: Josh Rosen Authored: Thu Sep 15 11:54:17 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 15 11:54:17 2016 -0700 -- .../org/apache/spark/storage/BlockManager.scala | 37 +++- .../spark/storage/BlockManagerSuite.scala | 34 ++ 2 files changed, 63 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1202075c/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c172ac2..aa29acf 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -283,7 +283,12 @@ private[spark] class BlockManager( } else { getLocalBytes(blockId) match { case Some(buffer) => new BlockManagerManagedBuffer(blockInfoManager, blockId, buffer) -case None => throw new BlockNotFoundException(blockId.toString) +case None => + // If this block manager receives a request for a block that it doesn't have then it's + // likely that the master has outdated block statuses for this block. Therefore, we send + // an RPC so that this block is marked as being unavailable from this block manager. + reportBlockStatus(blockId, BlockStatus.empty) + throw new BlockNotFoundException(blockId.toString) } } } @@ -859,22 +864,38 @@ private[spark] class BlockManager( } val startTimeMs = System.currentTimeMillis -var blockWasSuccessfullyStored: Boolean = false +var exceptionWasThrown: Boolean = true val result: Option[T] = try { val res = putBody(putBlockInfo) - blockWasSuccessfullyStored = res.isEmpty - res -} finally { - if (blockWasSuccessfullyStored) { + exceptionWasThrown = false + if (res.isEmpty) { +// the block was successfully stored if (keepReadLock) { blockInfoManager.downgradeLock(blockId) } else { blockInfoManager.unlock(blockId) } } else { -
spark git commit: [SPARK-17483] Refactoring in BlockManager status reporting and block removal
Repository: spark Updated Branches: refs/heads/branch-2.0 62ab53658 -> abb89c42e [SPARK-17483] Refactoring in BlockManager status reporting and block removal This patch makes three minor refactorings to the BlockManager: - Move the `if (info.tellMaster)` check out of `reportBlockStatus`; this fixes an issue where a debug logging message would incorrectly claim to have reported a block status to the master even though no message had been sent (in case `info.tellMaster == false`). This also makes it easier to write code which unconditionally sends block statuses to the master (which is necessary in another patch of mine). - Split `removeBlock()` into two methods, the existing method and an internal `removeBlockInternal()` method which is designed to be called by internal code that already holds a write lock on the block. This is also needed by a followup patch. - Instead of calling `getCurrentBlockStatus()` in `removeBlock()`, just pass `BlockStatus.empty`; the block status should always be empty following complete removal of a block. These changes were originally authored as part of a bug fix patch which is targeted at branch-2.0 and master; I've split them out here into their own separate PR in order to make them easier to review and so that the behavior-changing parts of my other patch can be isolated to their own PR. Author: Josh Rosen Closes #15036 from JoshRosen/cache-failure-race-conditions-refactorings-only. (cherry picked from commit 3d40896f410590c0be044b3fa7e5d32115fac05e) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/abb89c42 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/abb89c42 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/abb89c42 Branch: refs/heads/branch-2.0 Commit: abb89c42e760357e2d7eae4be344379c7f0d17f3 Parents: 62ab536 Author: Josh Rosen Authored: Mon Sep 12 13:09:33 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 15 11:51:43 2016 -0700 -- .../org/apache/spark/storage/BlockManager.scala | 87 ++-- 1 file changed, 42 insertions(+), 45 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/abb89c42/core/src/main/scala/org/apache/spark/storage/BlockManager.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index cd85d97..48db97a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -216,7 +216,7 @@ private[spark] class BlockManager( logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.") for ((blockId, info) <- blockInfoManager.entries) { val status = getCurrentBlockStatus(blockId, info) - if (!tryToReportBlockStatus(blockId, info, status)) { + if (info.tellMaster && !tryToReportBlockStatus(blockId, status)) { logError(s"Failed to report $blockId to master; giving up.") return } @@ -297,7 +297,7 @@ private[spark] class BlockManager( /** * Get the BlockStatus for the block identified by the given ID, if it exists. - * NOTE: This is mainly for testing, and it doesn't fetch information from external block store. + * NOTE: This is mainly for testing. */ def getStatus(blockId: BlockId): Option[BlockStatus] = { blockInfoManager.get(blockId).map { info => @@ -332,10 +332,9 @@ private[spark] class BlockManager( */ private def reportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Unit = { -val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) +val needReregister = !tryToReportBlockStatus(blockId, status, droppedMemorySize) if (needReregister) { logInfo(s"Got told to re-register updating block $blockId") // Re-registering will report our new block for free. @@ -351,17 +350,12 @@ private[spark] class BlockManager( */ private def tryToReportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { -if (info.tellMaster) { - val storageLevel = status.storageLevel - val inMemSize = Math.max(status.memSize, droppedMemorySize) - val onDiskSize = status.diskSize - master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) -} else { - true -} +val storageLevel = status.storageLevel +val inMemSize = Math.max(status.memSize, droppedMemorySize) +val onDiskSize = status.dis
spark git commit: [SPARK-17547] Ensure temp shuffle data file is cleaned up after error
Repository: spark Updated Branches: refs/heads/branch-1.6 a447cd888 -> 8646b84fb [SPARK-17547] Ensure temp shuffle data file is cleaned up after error SPARK-8029 (#9610) modified shuffle writers to first stage their data to a temporary file in the same directory as the final destination file and then to atomically rename this temporary file at the end of the write job. However, this change introduced the potential for the temporary output file to be leaked if an exception occurs during the write because the shuffle writers' existing error cleanup code doesn't handle deletion of the temp file. This patch avoids this potential cause of disk-space leaks by adding `finally` blocks to ensure that temp files are always deleted if they haven't been renamed. Author: Josh Rosen Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer. (cherry picked from commit 5b8f7377d54f83b93ef2bfc2a01ca65fae6d3032) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8646b84f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8646b84f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8646b84f Branch: refs/heads/branch-1.6 Commit: 8646b84fb8ed319e3a998f93de4821c723f7d419 Parents: a447cd8 Author: Josh Rosen Authored: Thu Sep 15 11:22:58 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 15 11:24:00 2016 -0700 -- .../sort/BypassMergeSortShuffleWriter.java | 10 ++- .../spark/shuffle/sort/UnsafeShuffleWriter.java | 18 +++-- .../shuffle/IndexShuffleBlockResolver.scala | 80 +++- .../spark/shuffle/sort/SortShuffleWriter.scala | 14 +++- 4 files changed, 73 insertions(+), 49 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8646b84f/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index a1a1fb0..80d24b9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -157,8 +157,14 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); File tmp = Utils.tempFileWith(output); -partitionLengths = writePartitionedFile(tmp); -shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +try { + partitionLengths = writePartitionedFile(tmp); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +} finally { + if (tmp.exists() && !tmp.delete()) { +logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } +} mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } http://git-wip-us.apache.org/repos/asf/spark/blob/8646b84f/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 744c300..d5e16fc 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -209,15 +209,21 @@ public class UnsafeShuffleWriter extends ShuffleWriter { final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); final File tmp = Utils.tempFileWith(output); try { - partitionLengths = mergeSpills(spills, tmp); -} finally { - for (SpillInfo spill : spills) { -if (spill.file.exists() && ! spill.file.delete()) { - logger.error("Error while deleting spill file {}", spill.file.getPath()); + try { +partitionLengths = mergeSpills(spills, tmp); + } finally { +for (SpillInfo spill : spills) { + if (spill.file.exists() && ! spill.file.delete()) { +logger.error("Error while deleting spill file {}", spill.file.getPath()); + } } } + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +} finally { + if (tmp.exists() && !tmp.delete()) { +logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } } -shuffleBlockResolver.wri
spark git commit: [SPARK-17547] Ensure temp shuffle data file is cleaned up after error
Repository: spark Updated Branches: refs/heads/branch-2.0 a09c258c9 -> e77a437d2 [SPARK-17547] Ensure temp shuffle data file is cleaned up after error SPARK-8029 (#9610) modified shuffle writers to first stage their data to a temporary file in the same directory as the final destination file and then to atomically rename this temporary file at the end of the write job. However, this change introduced the potential for the temporary output file to be leaked if an exception occurs during the write because the shuffle writers' existing error cleanup code doesn't handle deletion of the temp file. This patch avoids this potential cause of disk-space leaks by adding `finally` blocks to ensure that temp files are always deleted if they haven't been renamed. Author: Josh Rosen Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer. (cherry picked from commit 5b8f7377d54f83b93ef2bfc2a01ca65fae6d3032) Signed-off-by: Josh Rosen Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e77a437d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e77a437d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e77a437d Branch: refs/heads/branch-2.0 Commit: e77a437d292ecda66163a895427d62e4f72e2a25 Parents: a09c258 Author: Josh Rosen Authored: Thu Sep 15 11:22:58 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 15 11:23:17 2016 -0700 -- .../sort/BypassMergeSortShuffleWriter.java | 10 ++- .../spark/shuffle/sort/UnsafeShuffleWriter.java | 18 +++-- .../shuffle/IndexShuffleBlockResolver.scala | 80 +++- .../spark/shuffle/sort/SortShuffleWriter.scala | 14 +++- 4 files changed, 73 insertions(+), 49 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e77a437d/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 0e9defe..601dd6e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -156,8 +156,14 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); File tmp = Utils.tempFileWith(output); -partitionLengths = writePartitionedFile(tmp); -shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +try { + partitionLengths = writePartitionedFile(tmp); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +} finally { + if (tmp.exists() && !tmp.delete()) { +logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } +} mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } http://git-wip-us.apache.org/repos/asf/spark/blob/e77a437d/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 44e6aa7..c08a5d4 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -207,15 +207,21 @@ public class UnsafeShuffleWriter extends ShuffleWriter { final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); final File tmp = Utils.tempFileWith(output); try { - partitionLengths = mergeSpills(spills, tmp); -} finally { - for (SpillInfo spill : spills) { -if (spill.file.exists() && ! spill.file.delete()) { - logger.error("Error while deleting spill file {}", spill.file.getPath()); + try { +partitionLengths = mergeSpills(spills, tmp); + } finally { +for (SpillInfo spill : spills) { + if (spill.file.exists() && ! spill.file.delete()) { +logger.error("Error while deleting spill file {}", spill.file.getPath()); + } } } + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +} finally { + if (tmp.exists() && !tmp.delete()) { +logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } } -shuffleBlockResolver.wri
spark git commit: [SPARK-17547] Ensure temp shuffle data file is cleaned up after error
Repository: spark Updated Branches: refs/heads/master 0ad8eeb4d -> 5b8f7377d [SPARK-17547] Ensure temp shuffle data file is cleaned up after error SPARK-8029 (#9610) modified shuffle writers to first stage their data to a temporary file in the same directory as the final destination file and then to atomically rename this temporary file at the end of the write job. However, this change introduced the potential for the temporary output file to be leaked if an exception occurs during the write because the shuffle writers' existing error cleanup code doesn't handle deletion of the temp file. This patch avoids this potential cause of disk-space leaks by adding `finally` blocks to ensure that temp files are always deleted if they haven't been renamed. Author: Josh Rosen Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5b8f7377 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5b8f7377 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5b8f7377 Branch: refs/heads/master Commit: 5b8f7377d54f83b93ef2bfc2a01ca65fae6d3032 Parents: 0ad8eeb Author: Josh Rosen Authored: Thu Sep 15 11:22:58 2016 -0700 Committer: Josh Rosen Committed: Thu Sep 15 11:22:58 2016 -0700 -- .../sort/BypassMergeSortShuffleWriter.java | 10 ++- .../spark/shuffle/sort/UnsafeShuffleWriter.java | 18 +++-- .../shuffle/IndexShuffleBlockResolver.scala | 80 +++- .../spark/shuffle/sort/SortShuffleWriter.scala | 14 +++- 4 files changed, 73 insertions(+), 49 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5b8f7377/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 0fcc56d..4a15559 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -160,8 +160,14 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); File tmp = Utils.tempFileWith(output); -partitionLengths = writePartitionedFile(tmp); -shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +try { + partitionLengths = writePartitionedFile(tmp); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +} finally { + if (tmp.exists() && !tmp.delete()) { +logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } +} mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } http://git-wip-us.apache.org/repos/asf/spark/blob/5b8f7377/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 63d376b..f235c43 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -210,15 +210,21 @@ public class UnsafeShuffleWriter extends ShuffleWriter { final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); final File tmp = Utils.tempFileWith(output); try { - partitionLengths = mergeSpills(spills, tmp); -} finally { - for (SpillInfo spill : spills) { -if (spill.file.exists() && ! spill.file.delete()) { - logger.error("Error while deleting spill file {}", spill.file.getPath()); + try { +partitionLengths = mergeSpills(spills, tmp); + } finally { +for (SpillInfo spill : spills) { + if (spill.file.exists() && ! spill.file.delete()) { +logger.error("Error while deleting spill file {}", spill.file.getPath()); + } } } + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); +} finally { + if (tmp.exists() && !tmp.delete()) { +logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } } -shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); mapStatus = MapStatus$.MODULE$.apply(bloc
spark git commit: [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak
Repository: spark Updated Branches: refs/heads/branch-2.0 fffcec90b -> bb2bdb440 [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak The expression like `if (memoryMap(taskAttemptId) == 0) memoryMap.remove(taskAttemptId)` in method `releaseUnrollMemoryForThisTask` and `releasePendingUnrollMemoryForThisTask` should be called after release memory operation, whatever `memoryToRelease` is > 0 or not. If the memory of a task has been set to 0 when calling a `releaseUnrollMemoryForThisTask` or a `releasePendingUnrollMemoryForThisTask` method, the key in the memory map corresponding to that task will never be removed from the hash map. See the details in [SPARK-17465](https://issues.apache.org/jira/browse/SPARK-17465). Author: Xing SHI Closes #15022 from saturday-shi/SPARK-17465. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bb2bdb44 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bb2bdb44 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bb2bdb44 Branch: refs/heads/branch-2.0 Commit: bb2bdb44032d2e71832b3e0e771590fb2225e4f3 Parents: fffcec9 Author: Xing SHI Authored: Wed Sep 14 13:46:46 2016 -0700 Committer: Josh Rosen Committed: Wed Sep 14 14:00:57 2016 -0700 -- .../scala/org/apache/spark/storage/memory/MemoryStore.scala| 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bb2bdb44/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 5fd4e88..1230128 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -591,11 +591,11 @@ private[spark] class MemoryStore( val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) if (memoryToRelease > 0) { unrollMemoryMap(taskAttemptId) -= memoryToRelease - if (unrollMemoryMap(taskAttemptId) == 0) { -unrollMemoryMap.remove(taskAttemptId) - } memoryManager.releaseUnrollMemory(memoryToRelease, memoryMode) } +if (unrollMemoryMap(taskAttemptId) == 0) { + unrollMemoryMap.remove(taskAttemptId) +} } } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak
Repository: spark Updated Branches: refs/heads/master dbfc7aa4d -> bb3229436 [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak The expression like `if (memoryMap(taskAttemptId) == 0) memoryMap.remove(taskAttemptId)` in method `releaseUnrollMemoryForThisTask` and `releasePendingUnrollMemoryForThisTask` should be called after release memory operation, whatever `memoryToRelease` is > 0 or not. If the memory of a task has been set to 0 when calling a `releaseUnrollMemoryForThisTask` or a `releasePendingUnrollMemoryForThisTask` method, the key in the memory map corresponding to that task will never be removed from the hash map. See the details in [SPARK-17465](https://issues.apache.org/jira/browse/SPARK-17465). Author: Xing SHI Closes #15022 from saturday-shi/SPARK-17465. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bb322943 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bb322943 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bb322943 Branch: refs/heads/master Commit: bb322943623d14b85283705e74d913e31230387f Parents: dbfc7aa Author: Xing SHI Authored: Wed Sep 14 13:46:46 2016 -0700 Committer: Josh Rosen Committed: Wed Sep 14 13:59:57 2016 -0700 -- .../scala/org/apache/spark/storage/memory/MemoryStore.scala| 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bb322943/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index baa3fde..ec1b0f7 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -593,11 +593,11 @@ private[spark] class MemoryStore( val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) if (memoryToRelease > 0) { unrollMemoryMap(taskAttemptId) -= memoryToRelease - if (unrollMemoryMap(taskAttemptId) == 0) { -unrollMemoryMap.remove(taskAttemptId) - } memoryManager.releaseUnrollMemory(memoryToRelease, memoryMode) } +if (unrollMemoryMap(taskAttemptId) == 0) { + unrollMemoryMap.remove(taskAttemptId) +} } } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak
Repository: spark Updated Branches: refs/heads/branch-1.6 bf3f6d2f1 -> a447cd888 [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak ## What changes were proposed in this pull request? The expression like `if (memoryMap(taskAttemptId) == 0) memoryMap.remove(taskAttemptId)` in method `releaseUnrollMemoryForThisTask` and `releasePendingUnrollMemoryForThisTask` should be called after release memory operation, whatever `memoryToRelease` is > 0 or not. If the memory of a task has been set to 0 when calling a `releaseUnrollMemoryForThisTask` or a `releasePendingUnrollMemoryForThisTask` method, the key in the memory map corresponding to that task will never be removed from the hash map. See the details in [SPARK-17465](https://issues.apache.org/jira/browse/SPARK-17465). Author: Xing SHI Closes #15022 from saturday-shi/SPARK-17465. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a447cd88 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a447cd88 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a447cd88 Branch: refs/heads/branch-1.6 Commit: a447cd88897bc3d76eee0e8757e6545019704f30 Parents: bf3f6d2 Author: Xing SHI Authored: Wed Sep 14 13:46:46 2016 -0700 Committer: Josh Rosen Committed: Wed Sep 14 13:46:46 2016 -0700 -- .../main/scala/org/apache/spark/scheduler/Task.scala| 1 + .../scala/org/apache/spark/storage/MemoryStore.scala| 12 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a447cd88/core/src/main/scala/org/apache/spark/scheduler/Task.scala -- diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index c7b1199..2f4225e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -104,6 +104,7 @@ private[spark] abstract class Task[T]( Utils.tryLogNonFatalError { // Release memory used by this thread for unrolling blocks SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask() + SparkEnv.get.blockManager.memoryStore.releasePendingUnrollMemoryForThisTask() // Notify any tasks waiting for execution memory to be freed to wake up and try to // acquire memory again. This makes impossible the scenario where a task sleeps forever // because there are no other tasks left to notify it. Since this is safe to do but may http://git-wip-us.apache.org/repos/asf/spark/blob/a447cd88/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala -- diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index aed0da9..1113160 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -511,11 +511,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) if (memoryToRelease > 0) { unrollMemoryMap(taskAttemptId) -= memoryToRelease - if (unrollMemoryMap(taskAttemptId) == 0) { -unrollMemoryMap.remove(taskAttemptId) - } memoryManager.releaseUnrollMemory(memoryToRelease) } +if (unrollMemoryMap(taskAttemptId) == 0) { + unrollMemoryMap.remove(taskAttemptId) +} } } } @@ -530,11 +530,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val memoryToRelease = math.min(memory, pendingUnrollMemoryMap(taskAttemptId)) if (memoryToRelease > 0) { pendingUnrollMemoryMap(taskAttemptId) -= memoryToRelease - if (pendingUnrollMemoryMap(taskAttemptId) == 0) { -pendingUnrollMemoryMap.remove(taskAttemptId) - } memoryManager.releaseUnrollMemory(memoryToRelease) } +if (pendingUnrollMemoryMap(taskAttemptId) == 0) { + pendingUnrollMemoryMap.remove(taskAttemptId) +} } } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's value can be read thread-safely
Repository: spark Updated Branches: refs/heads/master ff6e4cbdc -> e33bfaed3 [SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's value can be read thread-safely ## What changes were proposed in this pull request? Make CollectionAccumulator and SetAccumulator's value can be read thread-safely to fix the ConcurrentModificationException reported in [JIRA](https://issues.apache.org/jira/browse/SPARK-17463). ## How was this patch tested? Existing tests. Author: Shixiong Zhu Closes #15063 from zsxwing/SPARK-17463. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e33bfaed Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e33bfaed Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e33bfaed Branch: refs/heads/master Commit: e33bfaed3b160fbc617c878067af17477a0044f5 Parents: ff6e4cb Author: Shixiong Zhu Authored: Wed Sep 14 13:33:51 2016 -0700 Committer: Josh Rosen Committed: Wed Sep 14 13:33:51 2016 -0700 -- .../org/apache/spark/executor/TaskMetrics.scala | 41 +--- .../org/apache/spark/util/AccumulatorV2.scala | 7 +++- .../org/apache/spark/util/JsonProtocol.scala| 11 +++--- .../apache/spark/util/JsonProtocolSuite.scala | 3 +- .../spark/sql/execution/debug/package.scala | 24 +++- 5 files changed, 54 insertions(+), 32 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e33bfaed/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala -- diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index dd149a9..52a3499 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,9 @@ package org.apache.spark.executor +import java.util.{ArrayList, Collections} + +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} import org.apache.spark._ @@ -99,7 +102,11 @@ class TaskMetrics private[spark] () extends Serializable { /** * Storage statuses of any blocks that have been updated as a result of this task. */ - def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.value + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = { +// This is called on driver. All accumulator updates have a fixed value. So it's safe to use +// `asScala` which accesses the internal values using `java.util.Iterator`. +_updatedBlockStatuses.value.asScala + } // Setters and increment-ers private[spark] def setExecutorDeserializeTime(v: Long): Unit = @@ -114,8 +121,10 @@ class TaskMetrics private[spark] () extends Serializable { private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) private[spark] def incUpdatedBlockStatuses(v: (BlockId, BlockStatus)): Unit = _updatedBlockStatuses.add(v) - private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = + private[spark] def setUpdatedBlockStatuses(v: java.util.List[(BlockId, BlockStatus)]): Unit = _updatedBlockStatuses.setValue(v) + private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = +_updatedBlockStatuses.setValue(v.asJava) /** * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted @@ -268,7 +277,7 @@ private[spark] object TaskMetrics extends Logging { val name = info.name.get val value = info.update.get if (name == UPDATED_BLOCK_STATUSES) { -tm.setUpdatedBlockStatuses(value.asInstanceOf[Seq[(BlockId, BlockStatus)]]) +tm.setUpdatedBlockStatuses(value.asInstanceOf[java.util.List[(BlockId, BlockStatus)]]) } else { tm.nameToAccums.get(name).foreach( _.asInstanceOf[LongAccumulator].setValue(value.asInstanceOf[Long]) @@ -299,8 +308,8 @@ private[spark] object TaskMetrics extends Logging { private[spark] class BlockStatusesAccumulator - extends AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { - private var _seq = ArrayBuffer.empty[(BlockId, BlockStatus)] + extends AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]] { + private val _seq = Collections.synchronizedList(new ArrayList[(BlockId, BlockStatus)]()) override def isZero(): Boolean = _seq.isEmpty @@ -308,25 +317,27 @@ private[spark] class BlockStatusesAccumulator override def copy(): BlockStatusesAccumulator = { val newAcc = new BlockStatusesAccumulator -newAcc._seq = _seq.clone() +newAcc._seq.addAll(_seq) newAcc } override def reset(): Unit = _seq.clear() -