(spark) branch master updated: [SPARK-48544][SQL] Reduce memory pressure of empty TreeNode BitSets

2024-06-10 Thread joshrosen
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

2023-09-29 Thread joshrosen
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 JsonProtocol extends JsonUtils {

[spark] branch master updated: [SPARK-44818] Fix race for pending task kill issued before taskThread is initialized

2023-08-21 Thread joshrosen
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

2023-05-15 Thread joshrosen
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 at c

[spark] branch master updated: [SPARK-40261][CORE] Exclude DirectTaskResult metadata when calculating result size

2022-08-31 Thread joshrosen
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()

2022-08-29 Thread joshrosen
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 @@ private[spark

[spark] branch master updated: [SPARK-40211][CORE][SQL] Allow customize initial partitions number in take() behavior

2022-08-26 Thread joshrosen
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.
+  if (resu

[spark] branch master updated (50c163578cf -> 6cd9d88e237)

2022-08-18 Thread joshrosen
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

2022-08-10 Thread joshrosen
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

2022-06-30 Thread joshrosen
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/org/apache/

[spark] branch branch-3.2 updated: [SPARK-39422][SQL] Improve error message for 'SHOW CREATE TABLE' with unsupported serdes

2022-06-09 Thread joshrosen
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

2022-06-09 Thread joshrosen
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

2022-06-09 Thread joshrosen
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

2022-06-02 Thread joshrosen
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   | 2 +-
 hadoop-cloud/src/test/resources/log4j2.properties

[spark] branch master updated: [SPARK-39361] Don't use Log4J2's extended throwable conversion pattern in default logging configurations

2022-06-02 Thread joshrosen
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/test/resources/log4j2.properties | 4 ++--
 mllib/src/t

[spark] 01/02: [SPARK-32911][CORE] Free memory in UnsafeExternalSorter.SpillableIterator.spill() when all records have been read

2022-05-31 Thread joshrosen
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

2022-05-31 Thread joshrosen
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 {
-  synchr

[spark] branch branch-3.0 updated (9b268122f68 -> c653d287b15)

2022-05-31 Thread joshrosen
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

2022-05-31 Thread joshrosen
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 (MemoryBlock page

[spark] branch branch-3.2 updated: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator

2022-05-31 Thread joshrosen
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 (MemoryBlock page

[spark] branch branch-3.3 updated: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator

2022-05-31 Thread joshrosen
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 (MemoryBlock page

[spark] branch master updated: [SPARK-39283][CORE] Fix deadlock between TaskMemoryManager and UnsafeExternalSorter.SpillableIterator

2022-05-31 Thread joshrosen
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 (inMemSorterToFree !

[spark] branch branch-3.0 updated: [SPARK-37784][SQL] Correctly handle UDTs in CodeGenerator.addBufferedState()

2022-01-04 Thread joshrosen
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()

2022-01-04 Thread joshrosen
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()

2022-01-04 Thread joshrosen
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()

2022-01-04 Thread joshrosen
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

2021-11-19 Thread joshrosen
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 +111,7 @@ object Tr

[spark] 01/01: hacky wip towards python udf profiling

2021-11-18 Thread joshrosen
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)

2021-11-18 Thread joshrosen
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()

2021-10-18 Thread joshrosen
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 m

[spark] branch branch-3.0 updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler

2021-10-14 Thread joshrosen
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.

It's sometimes important to preserve exception wrapping behavior, but I

[spark] branch branch-3.1 updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler

2021-10-14 Thread joshrosen
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.

It's sometimes important to preserve exception wrapping behavior, but I

[spark] branch branch-3.2 updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler

2021-10-14 Thread joshrosen
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.

It's sometimes important to preserve exception wrapping behavior, but I

[spark] branch master updated: [SPARK-23626][CORE] Eagerly compute RDD.partitions on entire DAG when submitting job to DAGScheduler

2021-10-14 Thread joshrosen
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's sometimes important to preserve exception wrapping behavior, but I 
don't

[spark] branch branch-3.2 updated: [SPARK-36774][CORE][TESTS] Move SparkSubmitTestUtils to core module and use it in SparkSubmitSuite

2021-09-16 Thread joshrosen
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/HiveSparkSubmitSuite.scala  |  5 +++
 9 files changed, 32 insertions(+), 60 deletions(-)

diff --git 
a/sql/core/src/test/scala/or

[spark] branch master updated (f1f2ec3 -> 3ae6e67)

2021-09-16 Thread joshrosen
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)

2019-08-22 Thread joshrosen
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

2019-06-21 Thread joshrosen
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

2019-06-19 Thread joshrosen
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.ml.attribute.Att

[spark] branch branch-2.4 updated: [SPARK-26555][SQL][BRANCH-2.4] make ScalaReflection subtype checking thread safe

2019-06-19 Thread joshrosen
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

2019-06-19 Thread joshrosen
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 to non-JNI implement

[spark] branch master updated: [SPARK-27839][SQL] Change UTF8String.replace() to operate on UTF8 bytes

2019-06-19 Thread joshrosen
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/unsafe/src/main

[spark] branch master updated: [SPARK-27684][SQL] Avoid conversion overhead for primitive types

2019-05-30 Thread joshrosen
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

2019-05-14 Thread joshrosen
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

2018-01-10 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Wed Jan 10 00:45:47 2018 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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<Long, LinkedList<WeakReference>> 
bufferPoolsBySize =
-new HashMap<>();
+  private final Map<Long, LinkedList<WeakReference<long[]>>> 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<WeakReference> pool = 
bufferPoolsBySize.get(size);
+final LinkedList<WeakReference<long[]>> 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<long[]> 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)
+|| (

spark git commit: [SPARK-22997] Add additional defenses against use of freed MemoryBlocks

2018-01-10 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Wed Jan 10 00:45:47 2018 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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<Long, LinkedList<WeakReference>> 
bufferPoolsBySize =
-new HashMap<>();
+  private final Map<Long, LinkedList<WeakReference<long[]>>> 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<WeakReference> pool = 
bufferPoolsBySize.get(size);
+final LinkedList<WeakReference<long[]>> 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<long[]> 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(), n

spark git commit: [SPARK-21444] Be more defensive when removing broadcasts in MapOutputTracker

2017-07-17 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Mon Jul 17 20:40:32 2017 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2017-06-11 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Sun Jun 11 18:34:12 2017 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

spark git commit: HOTFIX: fix Scalastyle break introduced in 4d57981cfb18e7500cde6c03ae46c7c9b697d064

2017-05-30 Thread joshrosen
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

2017-03-27 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Mon Mar 27 10:23:28 2017 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2017-02-14 Thread joshrosen
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);
   } 

spark git commit: [SPARK-18952][BACKPORT] Regex strings not properly escaped in codegen for aggregations

2017-01-09 Thread joshrosen
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 

spark git commit: [SPARK-18952] Regex strings not properly escaped in codegen for aggregations

2017-01-09 Thread joshrosen
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)
   

spark git commit: [SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors

2016-12-20 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Mon Dec 19 18:43:59 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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.enab

spark git commit: [SPARK-18553][CORE][BRANCH-1.6] Fix leak of TaskSetManager following executor loss

2016-12-01 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Thu Dec 1 10:42:27 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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[Lon

spark git commit: [SPARK-18553][CORE] Fix leak of TaskSetManager following executor loss

2016-11-29 Thread joshrosen
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 <joshro...@databricks.com>

Closes #16045 from JoshRosen/fix-leak-following-total-executor-loss-master.

(cherry picked from commit 9a02f6821265ff67ba3f7b095cd1afaebd25a898)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Tue Nov 29 16:27:25 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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

spark git commit: [SPARK-18553][CORE] Fix leak of TaskSetManager following executor loss

2016-11-29 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Tue Nov 29 16:27:25 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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 ru

spark git commit: [SPARK-18553][CORE][BRANCH-2.0] Fix leak of TaskSetManager following executor loss

2016-11-28 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Mon Nov 28 13:17:24 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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 t

spark git commit: [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed

2016-11-16 Thread joshrosen
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
 

spark git commit: [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed

2016-11-16 Thread joshrosen
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


spark git commit: [SPARK-18418] Fix flags for make_binary_release for hadoop profile

2016-11-12 Thread joshrosen
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 <hol...@us.ibm.com>

Closes #15860 from holdenk/minor-fix-release-build-script.

(cherry picked from commit 1386fd28daf798bf152606f4da30a36223d75d18)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <hol...@us.ibm.com>
Authored: Sat Nov 12 14:50:37 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-11-12 Thread joshrosen
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 <hol...@us.ibm.com>

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 <hol...@us.ibm.com>
Authored: Sat Nov 12 14:50:37 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-11-07 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Mon Nov 7 16:14:19 2016 -0800
Committer: Josh Rosen <joshro...@databricks.com>
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(

spark git commit: [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness

2016-10-21 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Fri Oct 21 11:25:01 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-10-21 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Fri Oct 21 11:25:01 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-10-06 Thread joshrosen
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

2016-10-06 Thread joshrosen
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

2016-09-29 Thread joshrosen
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 <joshro...@databricks.com>

Closes #15289 from JoshRosen/SPARK-17712.

(cherry picked from commit 37eb9184f1e9f1c07142c66936671f4711ef407d)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Wed Sep 28 19:03:05 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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: com

spark git commit: [SPARK-16343][SQL] Improve the PushDownPredicate rule to pushdown predicates correctly in non-deterministic condition.

2016-09-29 Thread joshrosen
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)
-

spark git commit: [SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in MemoryStore

2016-09-27 Thread joshrosen
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

2016-09-27 Thread joshrosen
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

2016-09-27 Thread joshrosen
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 <joshro...@databricks.com>

Closes #15265 from JoshRosen/SPARK-17618-master.

(cherry picked from commit 2f84a686604b298537bfd4d087b41594d2aa7ec6)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Tue Sep 27 14:14:27 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-09-27 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Tue Sep 27 14:14:27 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-09-22 Thread joshrosen
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 

spark git commit: [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames

2016-09-22 Thread joshrosen
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 = {}
+
+  

spark git commit: [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job (branch-1.6 backport)

2016-09-22 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Thu Sep 22 11:05:35 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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
inde

spark git commit: [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published

2016-09-21 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Wed Sep 21 11:38:10 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-09-21 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Wed Sep 21 11:38:10 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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

2016-09-19 Thread joshrosen
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 <joshro...@databricks.com>

Closes #15156 from JoshRosen/SPARK-17160.

(cherry picked from commit e719b1c045ba185d242d21bbfcdee2c84dafc587)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Mon Sep 19 20:20:36 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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 c

spark git commit: [SPARK-17160] Properly escape field names in code-generated error messages

2016-09-19 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Mon Sep 19 20:20:36 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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 

spark git commit: [SPARK-17473][SQL] fixing docker integration tests error due to different versions of jars.

2016-09-19 Thread joshrosen
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.

2016-09-19 Thread joshrosen
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()

2016-09-17 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Sat Sep 17 11:46:15 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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 unr

spark git commit: [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes()

2016-09-17 Thread joshrosen
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 <joshro...@databricks.com>

Closes #15043 from JoshRosen/partially-serialized-block-values-iterator-bugfix.

(cherry picked from commit 8faa5217b44e8d52eab7eb2d53d0652abaaf43cd)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Sat Sep 17 11:46:15 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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, 
ChunkedByteBuffe

spark git commit: [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions

2016-09-15 Thread joshrosen
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 <joshro...@databricks.com>

Closes #15085 from JoshRosen/SPARK-17484.

(cherry picked from commit 1202075c95eabba0ffebc170077df798f271a139)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Thu Sep 15 11:54:17 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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) {
+

spark git commit: [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions

2016-09-15 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Thu Sep 15 11:54:17 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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 {

spark git commit: [SPARK-17547] Ensure temp shuffle data file is cleaned up after error

2016-09-15 Thread joshrosen
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 <joshro...@databricks.com>

Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer.

(cherry picked from commit 5b8f7377d54f83b93ef2bfc2a01ca65fae6d3032)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Thu Sep 15 11:22:58 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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<K, V> extends 
ShuffleWriter<K, V> {
 
 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<K, V> extends 
ShuffleWriter<K, V> {
 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.exi

spark git commit: [SPARK-17547] Ensure temp shuffle data file is cleaned up after error

2016-09-15 Thread joshrosen
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 <joshro...@databricks.com>

Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer.

(cherry picked from commit 5b8f7377d54f83b93ef2bfc2a01ca65fae6d3032)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


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 <joshro...@databricks.com>
Authored: Thu Sep 15 11:22:58 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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<K, V> extends 
ShuffleWriter<K, V> {
 
 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<K, V> extends 
ShuffleWriter<K, V> {
 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.exi

spark git commit: [SPARK-17547] Ensure temp shuffle data file is cleaned up after error

2016-09-15 Thread joshrosen
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 <joshro...@databricks.com>

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 <joshro...@databricks.com>
Authored: Thu Sep 15 11:22:58 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
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<K, V> extends 
ShuffleWriter<K, V> {
 
 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<K, V> extends 
ShuffleWriter<K, V> {
 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());
+  }
 }
-shuff

spark git commit: [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak

2016-09-14 Thread joshrosen
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

2016-09-14 Thread joshrosen
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

2016-09-14 Thread joshrosen
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

2016-09-14 Thread joshrosen
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()
+

spark git commit: [SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's value can be read thread-safely

2016-09-14 Thread joshrosen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 fab77dadf -> fffcec90b


[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.

(cherry picked from commit e33bfaed3b160fbc617c878067af17477a0044f5)
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/fffcec90
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fffcec90
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fffcec90

Branch: refs/heads/branch-2.0
Commit: fffcec90b65047c3031c2b96679401f8fbef6337
Parents: fab77da
Author: Shixiong Zhu 
Authored: Wed Sep 14 13:33:51 2016 -0700
Committer: Josh Rosen 
Committed: Wed Sep 14 13:34:27 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/fffcec90/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 

spark git commit: [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job

2016-09-12 Thread joshrosen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 37f45bf0d -> a3fc5762b


[SPARK-17485] Prevent failed remote reads of cached blocks from failing entire 
job

## 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 <joshro...@databricks.com>

Closes #15037 from JoshRosen/SPARK-17485.

(cherry picked from commit f9c580f11098d95f098936a0b90fa21d71021205)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a3fc5762
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a3fc5762
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a3fc5762

Branch: refs/heads/branch-2.0
Commit: a3fc5762b896e6531a66802dbfe583c98eccc42b
Parents: 37f45bf
Author: Josh Rosen <joshro...@databricks.com>
Authored: Mon Sep 12 15:43:57 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
Committed: Mon Sep 12 15:44:20 2016 -0700

--
 .../spark/storage/BlockFetchException.scala | 24 
 .../org/apache/spark/storage/BlockManager.scala |  5 ++--
 .../spark/storage/BlockManagerSuite.scala   | 10 +++-
 3 files changed, 6 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a3fc5762/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/a3fc5762/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 git commit: [SPARK-14818] Post-2.0 MiMa exclusion and build changes

2016-09-12 Thread joshrosen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 0a36e360c -> 37f45bf0d


[SPARK-14818] Post-2.0 MiMa exclusion and build changes

This patch makes a handful of post-Spark-2.0 MiMa exclusion and build updates. 
It should be merged to master and a subset of it should be picked into 
branch-2.0 in order to test Spark 2.0.1-SNAPSHOT.

- Remove the ` sketch`, `mllibLocal`, and `streamingKafka010` from the list of 
excluded subprojects so that MiMa checks them.
- Remove now-unnecessary special-case handling of the Kafka 0.8 artifact in 
`mimaSettings`.
- Move the exclusion added in SPARK-14743 from `v20excludes` to `v21excludes`, 
since that patch was only merged into master and not branch-2.0.
- Add exclusions for an API change introduced by SPARK-17096 / #14675.
- Add missing exclusions for the `o.a.spark.internal` and 
`o.a.spark.sql.internal` packages.

Author: Josh Rosen <joshro...@databricks.com>

Closes #15061 from JoshRosen/post-2.0-mima-changes.

(cherry picked from commit 7c51b99a428a965ff7d136e1cdda20305d260453)
Signed-off-by: Josh Rosen <joshro...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/37f45bf0
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/37f45bf0
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/37f45bf0

Branch: refs/heads/branch-2.0
Commit: 37f45bf0d95f463c38e5636690545472c0399222
Parents: 0a36e36
Author: Josh Rosen <joshro...@databricks.com>
Authored: Mon Sep 12 15:24:33 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
Committed: Mon Sep 12 15:34:49 2016 -0700

--
 project/MimaBuild.scala| 11 ++-
 project/MimaExcludes.scala |  7 +++
 project/SparkBuild.scala   |  2 +-
 3 files changed, 10 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/37f45bf0/project/MimaBuild.scala
--
diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala
index 2a989dd..77397ea 100644
--- a/project/MimaBuild.scala
+++ b/project/MimaBuild.scala
@@ -88,15 +88,8 @@ object MimaBuild {
 
   def mimaSettings(sparkHome: File, projectRef: ProjectRef) = {
 val organization = "org.apache.spark"
-val previousSparkVersion = "1.6.0"
-// This check can be removed post-2.0
-val project = if (previousSparkVersion == "1.6.0" &&
-  projectRef.project == "streaming-kafka-0-8"
-) {
-  "streaming-kafka"
-} else {
-  projectRef.project
-}
+val previousSparkVersion = "2.0.0"
+val project = projectRef.project
 val fullId = "spark-" + project + "_2.11"
 mimaDefaultSettings ++
 Seq(previousArtifact := Some(organization % fullId % previousSparkVersion),

http://git-wip-us.apache.org/repos/asf/spark/blob/37f45bf0/project/MimaExcludes.scala
--
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 4bd6156..c38a49a 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -42,12 +42,15 @@ object MimaExcludes {
 Seq(
   excludePackage("org.apache.spark.rpc"),
   excludePackage("org.spark-project.jetty"),
+  excludePackage("org.spark_project.jetty"),
+  excludePackage("org.apache.spark.internal"),
   excludePackage("org.apache.spark.unused"),
   excludePackage("org.apache.spark.unsafe"),
   excludePackage("org.apache.spark.memory"),
   excludePackage("org.apache.spark.util.collection.unsafe"),
   excludePackage("org.apache.spark.sql.catalyst"),
   excludePackage("org.apache.spark.sql.execution"),
+  excludePackage("org.apache.spark.sql.internal"),
   
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"),
   
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"),
   ProblemFilters.exclude[MissingMethodProblem](
@@ -777,6 +780,10 @@ object MimaExcludes {
   
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jdbc"),
   
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"),
   
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema")
+) ++ Seq(
+  // SPARK-17096: Improve exception string reported through the 
StreamingQueryListener
+  
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.Streamin

spark git commit: [SPARK-14818] Post-2.0 MiMa exclusion and build changes

2016-09-12 Thread joshrosen
Repository: spark
Updated Branches:
  refs/heads/master 3d40896f4 -> 7c51b99a4


[SPARK-14818] Post-2.0 MiMa exclusion and build changes

This patch makes a handful of post-Spark-2.0 MiMa exclusion and build updates. 
It should be merged to master and a subset of it should be picked into 
branch-2.0 in order to test Spark 2.0.1-SNAPSHOT.

- Remove the ` sketch`, `mllibLocal`, and `streamingKafka010` from the list of 
excluded subprojects so that MiMa checks them.
- Remove now-unnecessary special-case handling of the Kafka 0.8 artifact in 
`mimaSettings`.
- Move the exclusion added in SPARK-14743 from `v20excludes` to `v21excludes`, 
since that patch was only merged into master and not branch-2.0.
- Add exclusions for an API change introduced by SPARK-17096 / #14675.
- Add missing exclusions for the `o.a.spark.internal` and 
`o.a.spark.sql.internal` packages.

Author: Josh Rosen <joshro...@databricks.com>

Closes #15061 from JoshRosen/post-2.0-mima-changes.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7c51b99a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7c51b99a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7c51b99a

Branch: refs/heads/master
Commit: 7c51b99a428a965ff7d136e1cdda20305d260453
Parents: 3d40896
Author: Josh Rosen <joshro...@databricks.com>
Authored: Mon Sep 12 15:24:33 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
Committed: Mon Sep 12 15:24:33 2016 -0700

--
 project/MimaBuild.scala| 11 ++-
 project/MimaExcludes.scala | 12 +---
 project/SparkBuild.scala   |  2 +-
 3 files changed, 12 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7c51b99a/project/MimaBuild.scala
--
diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala
index 2a989dd..77397ea 100644
--- a/project/MimaBuild.scala
+++ b/project/MimaBuild.scala
@@ -88,15 +88,8 @@ object MimaBuild {
 
   def mimaSettings(sparkHome: File, projectRef: ProjectRef) = {
 val organization = "org.apache.spark"
-val previousSparkVersion = "1.6.0"
-// This check can be removed post-2.0
-val project = if (previousSparkVersion == "1.6.0" &&
-  projectRef.project == "streaming-kafka-0-8"
-) {
-  "streaming-kafka"
-} else {
-  projectRef.project
-}
+val previousSparkVersion = "2.0.0"
+val project = projectRef.project
 val fullId = "spark-" + project + "_2.11"
 mimaDefaultSettings ++
 Seq(previousArtifact := Some(organization % fullId % previousSparkVersion),

http://git-wip-us.apache.org/repos/asf/spark/blob/7c51b99a/project/MimaExcludes.scala
--
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 16f26e7..fbd78ae 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -37,6 +37,8 @@ object MimaExcludes {
   // Exclude rules for 2.1.x
   lazy val v21excludes = v20excludes ++ {
 Seq(
+  // [SPARK-14743] Improve delegation token handling in secure cluster
+  
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTimeFromNowToRenewal"),
   // [SPARK-16199][SQL] Add a method to list the referenced columns in 
data source Filter
   
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.Filter.references"),
   // [SPARK-16853][SQL] Fixes encoder error in DataSet typed select
@@ -51,12 +53,15 @@ object MimaExcludes {
 Seq(
   excludePackage("org.apache.spark.rpc"),
   excludePackage("org.spark-project.jetty"),
+  excludePackage("org.spark_project.jetty"),
+  excludePackage("org.apache.spark.internal"),
   excludePackage("org.apache.spark.unused"),
   excludePackage("org.apache.spark.unsafe"),
   excludePackage("org.apache.spark.memory"),
   excludePackage("org.apache.spark.util.collection.unsafe"),
   excludePackage("org.apache.spark.sql.catalyst"),
   excludePackage("org.apache.spark.sql.execution"),
+  excludePackage("org.apache.spark.sql.internal"),
   
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"),
   
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"),
   ProblemFilters.exclude[MissingMethodProblem](
@@ -787,9 +792,10 @@ object MimaExcludes {
   
ProblemFilters.exclude[IncompatibleResultTyp

spark git commit: [SPARK-17483] Refactoring in BlockManager status reporting and block removal

2016-09-12 Thread joshrosen
Repository: spark
Updated Branches:
  refs/heads/master 1742c3ab8 -> 3d40896f4


[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 <joshro...@databricks.com>

Closes #15036 from JoshRosen/cache-failure-race-conditions-refactorings-only.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3d40896f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3d40896f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3d40896f

Branch: refs/heads/master
Commit: 3d40896f410590c0be044b3fa7e5d32115fac05e
Parents: 1742c3a
Author: Josh Rosen <joshro...@databricks.com>
Authored: Mon Sep 12 13:09:33 2016 -0700
Committer: Josh Rosen <joshro...@databricks.com>
Committed: Mon Sep 12 13:09:33 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/3d40896f/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 0614646..9e63777 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -217,7 +217,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
   }
@@ -298,7 +298,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 =>
@@ -333,10 +333,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.
@@ -352,17 +351,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.diskSize
+master

spark git commit: [SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the whole RDD in memory

2016-09-12 Thread joshrosen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 30521522d -> 0a36e360c


[SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the 
whole RDD in memory

## What changes were proposed in this pull request?

   MemoryStore may throw OutOfMemoryError when trying to cache a super big RDD 
that cannot fit in memory.
   ```
   scala> sc.parallelize(1 to 10, 100).map(x => new 
Array[Long](1000)).cache().count()

   java.lang.OutOfMemoryError: Java heap space
at 
$line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:24)
at 
$line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:23)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at scala.collection.Iterator$JoinIterator.next(Iterator.scala:232)
at 
org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:683)
at 
org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43)
at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1684)
at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
at org.apache.spark.scheduler.Task.run(Task.scala:86)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
   ```

Spark MemoryStore uses SizeTrackingVector as a temporary unrolling buffer to 
store all input values that it has read so far before transferring the values 
to storage memory cache. The problem is that when the input RDD is too big for 
caching in memory, the temporary unrolling memory SizeTrackingVector is not 
garbage collected in time. As SizeTrackingVector can occupy all available 
storage memory, it may cause the executor JVM to run out of memory quickly.

More info can be found at https://issues.apache.org/jira/browse/SPARK-17503

## How was this patch tested?

Unit test and manual test.

### Before change

Heap memory consumption
https://cloud.githubusercontent.com/assets/2595532/18429524/60d73a26-7906-11e6-9768-6f286f5c58c8.png;>

Heap dump
https://cloud.githubusercontent.com/assets/2595532/18429577/cbc1ef20-7906-11e6-847b-b5903f450b3b.png;>

### After change

Heap memory consumption
https://cloud.githubusercontent.com/assets/2595532/18429503/4abe9342-7906-11e6-844a-b2f815072624.png;>

Author: Sean Zhong 

Closes #15056 from clockfly/memory_store_leak.

(cherry picked from commit 1742c3ab86d75ce3d352f7cddff65e62fb7c8dd4)
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/0a36e360
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0a36e360
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0a36e360

Branch: refs/heads/branch-2.0
Commit: 0a36e360cd4bb2c66687caf017fbeeece41a7ccd
Parents: 3052152
Author: Sean Zhong 
Authored: Mon Sep 12 11:30:06 2016 -0700
Committer: Josh Rosen 
Committed: Mon Sep 12 11:33:37 2016 -0700

--
 .../spark/storage/memory/MemoryStore.scala  | 40 -
 .../PartiallyUnrolledIteratorSuite.scala| 61 
 2 files changed, 87 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0a36e360/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 0349da0..5fd4e88 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
@@ -661,31 +661,43 @@ private[spark] class MemoryStore(
 private[storage] class PartiallyUnrolledIterator[T](
 memoryStore: MemoryStore,
 unrollMemory: Long,
-unrolled: Iterator[T],
+private[this] var unrolled: Iterator[T],
 rest: Iterator[T])
   extends Iterator[T] {
 
-  private[this] var unrolledIteratorIsConsumed: Boolean = false
-  private[this] var iter: Iterator[T] = {
-val completionIterator = CompletionIterator[T, Iterator[T]](unrolled, {
-  

spark git commit: [SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the whole RDD in memory

2016-09-12 Thread joshrosen
Repository: spark
Updated Branches:
  refs/heads/master 8087ecf8d -> 1742c3ab8


[SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the 
whole RDD in memory

## What changes were proposed in this pull request?

   MemoryStore may throw OutOfMemoryError when trying to cache a super big RDD 
that cannot fit in memory.
   ```
   scala> sc.parallelize(1 to 10, 100).map(x => new 
Array[Long](1000)).cache().count()

   java.lang.OutOfMemoryError: Java heap space
at 
$line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:24)
at 
$line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:23)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at scala.collection.Iterator$JoinIterator.next(Iterator.scala:232)
at 
org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:683)
at 
org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43)
at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1684)
at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
at org.apache.spark.scheduler.Task.run(Task.scala:86)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
   ```

Spark MemoryStore uses SizeTrackingVector as a temporary unrolling buffer to 
store all input values that it has read so far before transferring the values 
to storage memory cache. The problem is that when the input RDD is too big for 
caching in memory, the temporary unrolling memory SizeTrackingVector is not 
garbage collected in time. As SizeTrackingVector can occupy all available 
storage memory, it may cause the executor JVM to run out of memory quickly.

More info can be found at https://issues.apache.org/jira/browse/SPARK-17503

## How was this patch tested?

Unit test and manual test.

### Before change

Heap memory consumption
https://cloud.githubusercontent.com/assets/2595532/18429524/60d73a26-7906-11e6-9768-6f286f5c58c8.png;>

Heap dump
https://cloud.githubusercontent.com/assets/2595532/18429577/cbc1ef20-7906-11e6-847b-b5903f450b3b.png;>

### After change

Heap memory consumption
https://cloud.githubusercontent.com/assets/2595532/18429503/4abe9342-7906-11e6-844a-b2f815072624.png;>

Author: Sean Zhong 

Closes #15056 from clockfly/memory_store_leak.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1742c3ab
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1742c3ab
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1742c3ab

Branch: refs/heads/master
Commit: 1742c3ab86d75ce3d352f7cddff65e62fb7c8dd4
Parents: 8087ecf
Author: Sean Zhong 
Authored: Mon Sep 12 11:30:06 2016 -0700
Committer: Josh Rosen 
Committed: Mon Sep 12 11:30:06 2016 -0700

--
 .../spark/storage/memory/MemoryStore.scala  | 40 -
 .../PartiallyUnrolledIteratorSuite.scala| 61 
 2 files changed, 87 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1742c3ab/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 d220ab5..1a3bf2b 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
@@ -663,31 +663,43 @@ private[spark] class MemoryStore(
 private[storage] class PartiallyUnrolledIterator[T](
 memoryStore: MemoryStore,
 unrollMemory: Long,
-unrolled: Iterator[T],
+private[this] var unrolled: Iterator[T],
 rest: Iterator[T])
   extends Iterator[T] {
 
-  private[this] var unrolledIteratorIsConsumed: Boolean = false
-  private[this] var iter: Iterator[T] = {
-val completionIterator = CompletionIterator[T, Iterator[T]](unrolled, {
-  unrolledIteratorIsConsumed = true
-  memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 
unrollMemory)
-})
-completionIterator 

  1   2   3   4   5   6   7   8   9   10   >