[spark] branch branch-2.4 updated: [SPARK-32003][CORE][2.4] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost

2020-08-04 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 39d31dc  [SPARK-32003][CORE][2.4] When external shuffle service is 
used, unregister outputs for executor on fetch failure after executor is lost
39d31dc is described below

commit 39d31dcba362c1c34995d15495d6b3753f8355b5
Author: Wing Yew Poon 
AuthorDate: Tue Aug 4 14:36:59 2020 -0500

[SPARK-32003][CORE][2.4] When external shuffle service is used, unregister 
outputs for executor on fetch failure after executor is lost

### What changes were proposed in this pull request?

If an executor is lost, the `DAGScheduler` handles the executor loss by 
removing the executor but does not unregister its outputs if the external 
shuffle service is used. However, if the node on which the executor runs is 
lost, the shuffle service may not be able to serve the shuffle files.
In such a case, when fetches from the executor's outputs fail in the same 
stage, the `DAGScheduler` again removes the executor and by right, should 
unregister its outputs. It doesn't because the epoch used to track the executor 
failure has not increased.

We track the epoch for failed executors that result in lost file output 
separately, so we can unregister the outputs in this scenario. The idea to 
track a second epoch is due to Attila Zsolt Piros.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New unit test. This test fails without the change and passes with it.

Closes #29182 from wypoon/SPARK-32003-2.4.

Authored-by: Wing Yew Poon 
Signed-off-by: Imran Rashid 
---
 .../org/apache/spark/scheduler/DAGScheduler.scala  | 100 +---
 .../apache/spark/scheduler/DAGSchedulerSuite.scala | 101 +
 2 files changed, 151 insertions(+), 50 deletions(-)

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 d0d12d8..18baa0b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -171,13 +171,34 @@ private[spark] class DAGScheduler(
*/
   private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]]
 
-  // For tracking failed nodes, we use the MapOutputTracker's epoch number, 
which is sent with
-  // every task. When we detect a node failing, we note the current epoch 
number and failed
-  // executor, increment it for new tasks, and use this to ignore stray 
ShuffleMapTask results.
-  //
-  // TODO: Garbage collect information about failure epochs when we know there 
are no more
-  //   stray messages to detect.
-  private val failedEpoch = new HashMap[String, Long]
+  /**
+   * Tracks the latest epoch of a fully processed error related to the given 
executor. (We use
+   * the MapOutputTracker's epoch number, which is sent with every task.)
+   *
+   * When an executor fails, it can affect the results of many tasks, and we 
have to deal with
+   * all of them consistently. We don't simply ignore all future results from 
that executor,
+   * as the failures may have been transient; but we also don't want to 
"overreact" to follow-
+   * on errors we receive. Furthermore, we might receive notification of a 
task success, after
+   * we find out the executor has actually failed; we'll assume those 
successes are, in fact,
+   * simply delayed notifications and the results have been lost, if the tasks 
started in the
+   * same or an earlier epoch. In particular, we use this to control when we 
tell the
+   * BlockManagerMaster that the BlockManager has been lost.
+   */
+  private val executorFailureEpoch = new HashMap[String, Long]
+
+  /**
+   * Tracks the latest epoch of a fully processed error where shuffle files 
have been lost from
+   * the given executor.
+   *
+   * This is closely related to executorFailureEpoch. They only differ for the 
executor when
+   * there is an external shuffle service serving shuffle files and we haven't 
been notified that
+   * the entire worker has been lost. In that case, when an executor is lost, 
we do not update
+   * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only 
the executor
+   * fails, we do not unregister the shuffle data as it can still be served; 
but if there is
+   * a failure in the shuffle service (resulting in fetch failure), we 
unregister the shuffle
+   * data only once, even if we get many fetch failures.
+   */
+  private val shuffleFileLostEpoch = new HashMap[String, Long]
 
   private [scheduler] val outputCommitCoordinator = env.outputCommitCoordinator
 
@@ -1389,7 +1410,8 @@ private[spark] class DAGScheduler(
 

[spark] branch branch-3.0 updated: [SPARK-32003][CORE][3.0] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost

2020-08-04 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 fd445cb  [SPARK-32003][CORE][3.0] When external shuffle service is 
used, unregister outputs for executor on fetch failure after executor is lost
fd445cb is described below

commit fd445cb46ce940fbc68f85d2db14367d9f58739a
Author: Wing Yew Poon 
AuthorDate: Tue Aug 4 11:33:56 2020 -0500

[SPARK-32003][CORE][3.0] When external shuffle service is used, unregister 
outputs for executor on fetch failure after executor is lost

### What changes were proposed in this pull request?

If an executor is lost, the `DAGScheduler` handles the executor loss by 
removing the executor but does not unregister its outputs if the external 
shuffle service is used. However, if the node on which the executor runs is 
lost, the shuffle service may not be able to serve the shuffle files.
In such a case, when fetches from the executor's outputs fail in the same 
stage, the `DAGScheduler` again removes the executor and by right, should 
unregister its outputs. It doesn't because the epoch used to track the executor 
failure has not increased.

We track the epoch for failed executors that result in lost file output 
separately, so we can unregister the outputs in this scenario. The idea to 
track a second epoch is due to Attila Zsolt Piros.

### Why are the changes needed?

Without the changes, the loss of a node could require two stage attempts to 
recover instead of one.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New unit test. This test fails without the change and passes with it.

Closes #29193 from wypoon/SPARK-32003-3.0.

Authored-by: Wing Yew Poon 
Signed-off-by: Imran Rashid 
---
 .../org/apache/spark/scheduler/DAGScheduler.scala  | 100 +---
 .../apache/spark/scheduler/DAGSchedulerSuite.scala | 101 +
 2 files changed, 151 insertions(+), 50 deletions(-)

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 ec6eff3..51445bf 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -168,13 +168,34 @@ private[spark] class DAGScheduler(
*/
   private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]]
 
-  // For tracking failed nodes, we use the MapOutputTracker's epoch number, 
which is sent with
-  // every task. When we detect a node failing, we note the current epoch 
number and failed
-  // executor, increment it for new tasks, and use this to ignore stray 
ShuffleMapTask results.
-  //
-  // TODO: Garbage collect information about failure epochs when we know there 
are no more
-  //   stray messages to detect.
-  private val failedEpoch = new HashMap[String, Long]
+  /**
+   * Tracks the latest epoch of a fully processed error related to the given 
executor. (We use
+   * the MapOutputTracker's epoch number, which is sent with every task.)
+   *
+   * When an executor fails, it can affect the results of many tasks, and we 
have to deal with
+   * all of them consistently. We don't simply ignore all future results from 
that executor,
+   * as the failures may have been transient; but we also don't want to 
"overreact" to follow-
+   * on errors we receive. Furthermore, we might receive notification of a 
task success, after
+   * we find out the executor has actually failed; we'll assume those 
successes are, in fact,
+   * simply delayed notifications and the results have been lost, if the tasks 
started in the
+   * same or an earlier epoch. In particular, we use this to control when we 
tell the
+   * BlockManagerMaster that the BlockManager has been lost.
+   */
+  private val executorFailureEpoch = new HashMap[String, Long]
+
+  /**
+   * Tracks the latest epoch of a fully processed error where shuffle files 
have been lost from
+   * the given executor.
+   *
+   * This is closely related to executorFailureEpoch. They only differ for the 
executor when
+   * there is an external shuffle service serving shuffle files and we haven't 
been notified that
+   * the entire worker has been lost. In that case, when an executor is lost, 
we do not update
+   * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only 
the executor
+   * fails, we do not unregister the shuffle data as it can still be served; 
but if there is
+   * a failure in the shuffle service (resulting in fetch failure), we 
unregister the shuffle
+   * data only once, even if we get many fetch failures.
+   */
+  private val shuffleFileLostEpoch = new HashMap[String, Long]
 

[spark] branch master updated: [SPARK-32003][CORE] When external shuffle service is used, unregister outputs for executor on fetch failure after executor is lost

2020-07-22 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 e8c06af  [SPARK-32003][CORE] When external shuffle service is used, 
unregister outputs for executor on fetch failure after executor is lost
e8c06af is described below

commit e8c06af7d1ab19b6478ac542cf876259cac5e13a
Author: Wing Yew Poon 
AuthorDate: Wed Jul 22 09:53:16 2020 -0500

[SPARK-32003][CORE] When external shuffle service is used, unregister 
outputs for executor on fetch failure after executor is lost

### What changes were proposed in this pull request?

If an executor is lost, the `DAGScheduler` handles the executor loss by 
removing the executor but does not unregister its outputs if the external 
shuffle service is used. However, if the node on which the executor runs is 
lost, the shuffle service may not be able to serve the shuffle files.
In such a case, when fetches from the executor's outputs fail in the same 
stage, the `DAGScheduler` again removes the executor and by right, should 
unregister its outputs. It doesn't because the epoch used to track the executor 
failure has not increased.

We track the epoch for failed executors that result in lost file output 
separately, so we can unregister the outputs in this scenario. The idea to 
track a second epoch is due to Attila Zsolt Piros.

### Why are the changes needed?

Without the changes, the loss of a node could require two stage attempts to 
recover instead of one.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New unit test. This test fails without the change and passes with it.

Closes #28848 from wypoon/SPARK-32003.

Authored-by: Wing Yew Poon 
Signed-off-by: Imran Rashid 
---
 .../org/apache/spark/scheduler/DAGScheduler.scala  | 100 ++---
 .../apache/spark/scheduler/DAGSchedulerSuite.scala |  98 
 2 files changed, 148 insertions(+), 50 deletions(-)

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 cb024d0..73c95d1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -170,13 +170,34 @@ private[spark] class DAGScheduler(
*/
   private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]]
 
-  // For tracking failed nodes, we use the MapOutputTracker's epoch number, 
which is sent with
-  // every task. When we detect a node failing, we note the current epoch 
number and failed
-  // executor, increment it for new tasks, and use this to ignore stray 
ShuffleMapTask results.
-  //
-  // TODO: Garbage collect information about failure epochs when we know there 
are no more
-  //   stray messages to detect.
-  private val failedEpoch = new HashMap[String, Long]
+  /**
+   * Tracks the latest epoch of a fully processed error related to the given 
executor. (We use
+   * the MapOutputTracker's epoch number, which is sent with every task.)
+   *
+   * When an executor fails, it can affect the results of many tasks, and we 
have to deal with
+   * all of them consistently. We don't simply ignore all future results from 
that executor,
+   * as the failures may have been transient; but we also don't want to 
"overreact" to follow-
+   * on errors we receive. Furthermore, we might receive notification of a 
task success, after
+   * we find out the executor has actually failed; we'll assume those 
successes are, in fact,
+   * simply delayed notifications and the results have been lost, if the tasks 
started in the
+   * same or an earlier epoch. In particular, we use this to control when we 
tell the
+   * BlockManagerMaster that the BlockManager has been lost.
+   */
+  private val executorFailureEpoch = new HashMap[String, Long]
+
+  /**
+   * Tracks the latest epoch of a fully processed error where shuffle files 
have been lost from
+   * the given executor.
+   *
+   * This is closely related to executorFailureEpoch. They only differ for the 
executor when
+   * there is an external shuffle service serving shuffle files and we haven't 
been notified that
+   * the entire worker has been lost. In that case, when an executor is lost, 
we do not update
+   * the shuffleFileLostEpoch; we wait for a fetch failure. This way, if only 
the executor
+   * fails, we do not unregister the shuffle data as it can still be served; 
but if there is
+   * a failure in the shuffle service (resulting in fetch failure), we 
unregister the shuffle
+   * data only once, even if we get many fetch failures.
+   */
+  private val shuffleFileLostEpoch = new HashMap[String, Long]
 
   private [sche

[spark] branch master updated: [SPARK-27324][DOC][CORE] Document configurations related to executor metrics and modify a configuration

2020-01-31 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 387ce89  [SPARK-27324][DOC][CORE] Document configurations related to 
executor metrics and modify a configuration
387ce89 is described below

commit 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd
Author: Wing Yew Poon 
AuthorDate: Fri Jan 31 14:28:02 2020 -0600

[SPARK-27324][DOC][CORE] Document configurations related to executor 
metrics and modify a configuration

### What changes were proposed in this pull request?

Add a section to the Configuration page to document configurations for 
executor metrics.
At the same time, rename 
spark.eventLog.logStageExecutorProcessTreeMetrics.enabled to 
spark.executor.processTreeMetrics.enabled and make it independent of 
spark.eventLog.logStageExecutorMetrics.enabled.

### Why are the changes needed?

Executor metrics are new in Spark 3.0. They lack documentation.
Memory metrics as a whole are always collected, but the ones obtained from 
the process tree have to be optionally enabled. Making this depend on a single 
configuration makes for more intuitive behavior. Given this, the configuration 
property is renamed to better reflect its meaning.

### Does this PR introduce any user-facing change?

Yes, only in that the configurations are all new to 3.0.

### How was this patch tested?

Not necessary.

Closes #27329 from wypoon/SPARK-27324.

Authored-by: Wing Yew Poon 
Signed-off-by: Imran Rashid 
---
 .../spark/executor/ExecutorMetricsSource.scala |  3 +-
 .../spark/executor/ProcfsMetricsGetter.scala   |  8 ++---
 .../org/apache/spark/internal/config/package.scala | 17 +++---
 .../spark/deploy/history/HistoryServerSuite.scala  |  2 +-
 docs/configuration.md  | 37 ++
 docs/monitoring.md | 20 ++--
 6 files changed, 65 insertions(+), 22 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala 
b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala
index b052e43..14645f7 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala
@@ -32,8 +32,7 @@ import org.apache.spark.metrics.source.Source
  * spark.executor.metrics.pollingInterval=.
  * (2) Procfs metrics are gathered all in one-go and only conditionally:
  * if the /proc filesystem exists
- * and spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true
- * and spark.eventLog.logStageExecutorMetrics.enabled=true.
+ * and spark.executor.processTreeMetrics.enabled=true.
  */
 private[spark] class ExecutorMetricsSource extends Source {
 
diff --git 
a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala 
b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala
index 0d5dcfb4..80ef757 100644
--- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala
@@ -58,11 +58,9 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = 
"/proc/") extends L
   logWarning("Exception checking for procfs dir", ioe)
   false
   }
-  val shouldLogStageExecutorMetrics =
-SparkEnv.get.conf.get(config.EVENT_LOG_STAGE_EXECUTOR_METRICS)
-  val shouldLogStageExecutorProcessTreeMetrics =
-SparkEnv.get.conf.get(config.EVENT_LOG_PROCESS_TREE_METRICS)
-  procDirExists.get && shouldLogStageExecutorProcessTreeMetrics && 
shouldLogStageExecutorMetrics
+  val shouldPollProcessTreeMetrics =
+SparkEnv.get.conf.get(config.EXECUTOR_PROCESS_TREE_METRICS_ENABLED)
+  procDirExists.get && shouldPollProcessTreeMetrics
 }
   }
 
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 40b05cf..e68368f 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
@@ -148,11 +148,8 @@ package object config {
 
   private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS =
 ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled")
-  .booleanConf
-  .createWithDefault(false)
-
-  private[spark] val EVENT_LOG_PROCESS_TREE_METRICS =
-ConfigBuilder("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled")
+  .doc("Whether to write per-stage peaks of executor metrics (for each 
executor) " +
+"to the event log.")
   .booleanConf
   .createWit

[spark] branch master updated: [SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system

2019-12-09 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 729f43f  [SPARK-27189][CORE] Add Executor metrics and memory usage 
instrumentation to the metrics system
729f43f is described below

commit 729f43f499f3dd2718c0b28d73f2ca29cc811eac
Author: Luca Canali 
AuthorDate: Mon Dec 9 08:55:30 2019 -0600

[SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation 
to the metrics system

## What changes were proposed in this pull request?

This PR proposes to add instrumentation of memory usage via the Spark 
Dropwizard/Codahale metrics system. Memory usage metrics are available via the 
Executor metrics, recently implemented as detailed in 
https://issues.apache.org/jira/browse/SPARK-23206.
Additional notes: This takes advantage of the metrics poller introduced in 
#23767.

## Why are the changes needed?
Executor metrics bring have many useful insights on memory usage, in 
particular on the usage of storage memory and executor memory. This is useful 
for troubleshooting. Having the information in the metrics systems allows to 
add those metrics to Spark performance dashboards and study memory usage as a 
function of time, as in the example graph 
https://issues.apache.org/jira/secure/attachment/12962810/Example_dashboard_Spark_Memory_Metrics.PNG

## Does this PR introduce any user-facing change?
Adds `ExecutorMetrics` source to publish executor metrics via the 
Dropwizard metrics system. Details of the available metrics in 
docs/monitoring.md
Adds configuration parameter `spark.metrics.executormetrics.source.enabled`

## How was this patch tested?

Tested on YARN cluster and with an existing setup for a Spark dashboard 
based on InfluxDB and Grafana.

Closes #24132 from LucaCanali/memoryMetricsSource.

Authored-by: Luca Canali 
Signed-off-by: Imran Rashid 
---
 .../main/scala/org/apache/spark/SparkContext.scala | 16 +-
 .../scala/org/apache/spark/executor/Executor.scala | 11 +++-
 .../spark/executor/ExecutorMetricsPoller.scala |  4 +-
 .../spark/executor/ExecutorMetricsSource.scala | 65 ++
 .../org/apache/spark/internal/config/package.scala |  6 ++
 .../spark/metrics/source/SourceConfigSuite.scala   | 30 +-
 docs/monitoring.md | 41 ++
 7 files changed, 167 insertions(+), 6 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 0694501..96ca12b 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -42,7 +42,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
 import org.apache.spark.deploy.StandaloneResourceUtils._
-import org.apache.spark.executor.ExecutorMetrics
+import org.apache.spark.executor.{ExecutorMetrics, ExecutorMetricsSource}
 import org.apache.spark.input.{FixedLengthBinaryInputFormat, 
PortableDataStream, StreamInputFormat, WholeTextFileInputFormat}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
@@ -551,9 +551,16 @@ class SparkContext(config: SparkConf) extends Logging {
 _dagScheduler = new DAGScheduler(this)
 _heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet)
 
+val _executorMetricsSource =
+  if (_conf.get(METRICS_EXECUTORMETRICS_SOURCE_ENABLED)) {
+Some(new ExecutorMetricsSource)
+  } else {
+None
+  }
+
 // create and start the heartbeater for collecting memory metrics
 _heartbeater = new Heartbeater(
-  () => SparkContext.this.reportHeartBeat(),
+  () => SparkContext.this.reportHeartBeat(_executorMetricsSource),
   "driver-heartbeater",
   conf.get(EXECUTOR_HEARTBEAT_INTERVAL))
 _heartbeater.start()
@@ -622,6 +629,7 @@ class SparkContext(config: SparkConf) extends Logging {
 _env.metricsSystem.registerSource(_dagScheduler.metricsSource)
 _env.metricsSystem.registerSource(new 
BlockManagerSource(_env.blockManager))
 _env.metricsSystem.registerSource(new JVMCPUSource())
+_executorMetricsSource.foreach(_.register(_env.metricsSystem))
 _executorAllocationManager.foreach { e =>
   _env.metricsSystem.registerSource(e.executorAllocationManagerSource)
 }
@@ -2473,8 +2481,10 @@ class SparkContext(config: SparkConf) extends Logging {
   }
 
   /** Reports heartbeat metrics for the driver. */
-  private def reportHeartBeat(): Unit = {
+  private def reportHeartBeat(executorMetricsSource: 
Option[ExecutorMetricsSource]): Unit = {
 val currentMetrics = ExecutorMetrics.getCurrentMetric

[spark] branch master updated: [SPARK-29398][CORE] Support dedicated thread pools for RPC endpoints

2019-10-17 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 2f0a38c  [SPARK-29398][CORE] Support dedicated thread pools for RPC 
endpoints
2f0a38c is described below

commit 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931
Author: Marcelo Vanzin 
AuthorDate: Thu Oct 17 13:14:32 2019 -0500

[SPARK-29398][CORE] Support dedicated thread pools for RPC endpoints

The current RPC backend in Spark supports single- and multi-threaded
message delivery to endpoints, but they all share the same underlying
thread pool. So an RPC endpoint that blocks a dispatcher thread can
negatively affect other endpoints.

This can be more pronounced with configurations that limit the number
of RPC dispatch threads based on configuration and / or running
environment. And exposing the RPC layer to other code (for example
with something like SPARK-29396) could make it easy to affect normal
Spark operation with a badly written RPC handler.

This change adds a new RPC endpoint type that tells the RPC env to
create dedicated dispatch threads, so that those effects are minimised.
Other endpoints will still need CPU to process their messages, but
they won't be able to actively block the dispatch thread of these
isolated endpoints.

As part of the change, I've changed the most important Spark endpoints
(the driver, executor and block manager endpoints) to be isolated from
others. This means a couple of extra threads are created on the driver
and executor for these endpoints.

Tested with existing unit tests, which hammer the RPC system extensively,
and also by running applications on a cluster (with a prototype of
SPARK-29396).

Closes #26059 from vanzin/SPARK-29398.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 
---
 .../executor/CoarseGrainedExecutorBackend.scala|   2 +-
 .../scala/org/apache/spark/rpc/RpcEndpoint.scala   |  16 ++
 .../org/apache/spark/rpc/netty/Dispatcher.scala| 130 --
 .../scala/org/apache/spark/rpc/netty/Inbox.scala   |   6 +-
 .../org/apache/spark/rpc/netty/MessageLoop.scala   | 194 +
 .../cluster/CoarseGrainedSchedulerBackend.scala|   2 +-
 .../spark/storage/BlockManagerMasterEndpoint.scala |   4 +-
 .../spark/storage/BlockManagerSlaveEndpoint.scala  |   4 +-
 .../scala/org/apache/spark/rpc/RpcEnvSuite.scala   |  35 +++-
 .../org/apache/spark/rpc/netty/InboxSuite.scala|  23 +--
 10 files changed, 296 insertions(+), 120 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index fbf2dc7..b4bca1e 100644
--- 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -51,7 +51,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 userClassPath: Seq[URL],
 env: SparkEnv,
 resourcesFileOpt: Option[String])
-  extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging {
+  extends IsolatedRpcEndpoint with ExecutorBackend with Logging {
 
   private implicit val formats = DefaultFormats
 
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala 
b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala
index 97eed54..4728759 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala
@@ -146,3 +146,19 @@ private[spark] trait RpcEndpoint {
  * [[ThreadSafeRpcEndpoint]] for different messages.
  */
 private[spark] trait ThreadSafeRpcEndpoint extends RpcEndpoint
+
+/**
+ * An endpoint that uses a dedicated thread pool for delivering messages.
+ */
+private[spark] trait IsolatedRpcEndpoint extends RpcEndpoint {
+
+  /**
+   * How many threads to use for delivering messages. By default, use a single 
thread.
+   *
+   * Note that requesting more than one thread means that the endpoint should 
be able to handle
+   * messages arriving from many threads at once, and all the things that 
entails (including
+   * messages being delivered to the endpoint out of order).
+   */
+  def threadCount(): Int = 1
+
+}
diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala 
b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala
index 2f923d7..27c943d 100644
--- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala
@@ -17,20 +17,16 @@
 
 package org.apache.spark.rpc.netty
 
-import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, 
LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit}
+import

[spark] branch master updated: [SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver Components API

2019-10-15 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 2e28622  [SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver 
Components API
2e28622 is described below

commit 2e28622d8aeb9ce2460e803bb7d994196bcc0253
Author: Yifei Huang 
AuthorDate: Tue Oct 15 12:26:49 2019 -0500

[SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver Components API

### What changes were proposed in this pull request?

This is the next step of the Spark-25299 work of proposing a new Shuffle 
storage API. This patch includes the components of the plugin that hook into 
the driver, including driver shuffle initialization, application cleanup, and 
shuffle cleanup.

### How was this patch tested?
Existing unit tests, plus an additional test for testing the interactions 
between the driver and executor initialization.

Closes #25823 from yifeih/yh/upstream/driver-lifecycle.

Lead-authored-by: Yifei Huang 
Co-authored-by: mccheah 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/shuffle/api/ShuffleDataIO.java|  6 ++
 .../spark/shuffle/api/ShuffleDriverComponents.java | 64 +++
 .../shuffle/api/ShuffleExecutorComponents.java | 12 ++-
 .../shuffle/sort/io/LocalDiskShuffleDataIO.java|  8 +-
 java => LocalDiskShuffleDriverComponents.java} | 35 +---
 .../io/LocalDiskShuffleExecutorComponents.java |  7 +-
 .../scala/org/apache/spark/ContextCleaner.scala|  8 +-
 .../main/scala/org/apache/spark/Dependency.scala   |  1 +
 .../main/scala/org/apache/spark/SparkContext.scala | 17 +++-
 .../apache/spark/shuffle/ShuffleDataIOUtils.scala  | 42 ++
 .../spark/shuffle/sort/SortShuffleManager.scala| 15 ++--
 .../apache/spark/InternalAccumulatorSuite.scala|  3 +-
 .../shuffle/ShuffleDriverComponentsSuite.scala | 94 ++
 13 files changed, 281 insertions(+), 31 deletions(-)

diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java 
b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java
index e9e50ec..e4554bd 100644
--- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java
+++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java
@@ -46,4 +46,10 @@ public interface ShuffleDataIO {
* are only invoked on the executors.
*/
   ShuffleExecutorComponents executor();
+
+  /**
+   * Called once on driver process to bootstrap the shuffle metadata modules 
that
+   * are maintained by the driver.
+   */
+  ShuffleDriverComponents driver();
 }
diff --git 
a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java 
b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java
new file mode 100644
index 000..b4cec17
--- /dev/null
+++ 
b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java
@@ -0,0 +1,64 @@
+/*
+ * 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.shuffle.api;
+
+import java.util.Map;
+
+import org.apache.spark.annotation.Private;
+
+/**
+ * :: Private ::
+ * An interface for building shuffle support modules for the Driver.
+ */
+@Private
+public interface ShuffleDriverComponents {
+
+  /**
+   * Called once in the driver to bootstrap this module that is specific to 
this application.
+   * This method is called before submitting executor requests to the cluster 
manager.
+   *
+   * This method should prepare the module with its shuffle components i.e. 
registering against
+   * an external file servers or shuffle services, or creating tables in a 
shuffle
+   * storage data database.
+   *
+   * @return additional SparkConf settings necessary for initializing the 
executor components.
+   * This would include configurations that cannot be statically set on the 
application, like
+   * the host:port of external services for shuffle storage.
+   */
+  Map initializeApplication();
+
+  /**
+   * Called once at the end of the Spark application to clean up any existing 
shuffle state.
+   */
+  

[spark] branch master updated: [SPARK-27468][CORE] Track correct storage level of RDDs and partitions

2019-10-07 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 d2f21b0  [SPARK-27468][CORE] Track correct storage level of RDDs and 
partitions
d2f21b0 is described below

commit d2f21b019909e66bf49ad764b851b4a65c2438f8
Author: Marcelo Vanzin 
AuthorDate: Mon Oct 7 16:07:00 2019 -0500

[SPARK-27468][CORE] Track correct storage level of RDDs and partitions

Previously, the RDD level would change depending on the status reported
by executors for the block they were storing, and individual blocks would
reflect that. That is wrong because different blocks may be stored 
differently
in different executors.

So now the RDD tracks the user-provided storage level, while the individual
partitions reflect the current storage level of that particular block,
including the current number of replicas.

Closes #25779 from vanzin/SPARK-27468.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/status/AppStatusListener.scala| 20 +++
 .../scala/org/apache/spark/status/LiveEntity.scala | 37 +---
 .../spark/status/AppStatusListenerSuite.scala  | 40 +++---
 .../org/apache/spark/status/LiveEntitySuite.scala  |  5 +--
 4 files changed, 68 insertions(+), 34 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala 
b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index c85b3ca..7da0a9d 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -234,8 +234,8 @@ private[spark] class AppStatusListener(
 (partition.memoryUsed / partition.executors.length) * -1)
   rdd.diskUsed = addDeltaToValue(rdd.diskUsed,
 (partition.diskUsed / partition.executors.length) * -1)
-  partition.update(partition.executors
-.filter(!_.equals(event.executorId)), rdd.storageLevel,
+  partition.update(
+partition.executors.filter(!_.equals(event.executorId)),
 addDeltaToValue(partition.memoryUsed,
   (partition.memoryUsed / partition.executors.length) * -1),
 addDeltaToValue(partition.diskUsed,
@@ -495,7 +495,7 @@ private[spark] class AppStatusListener(
 
 event.stageInfo.rddInfos.foreach { info =>
   if (info.storageLevel.isValid) {
-liveUpdate(liveRDDs.getOrElseUpdate(info.id, new LiveRDD(info)), now)
+liveUpdate(liveRDDs.getOrElseUpdate(info.id, new LiveRDD(info, 
info.storageLevel)), now)
   }
 }
 
@@ -916,12 +916,6 @@ private[spark] class AppStatusListener(
 val diskDelta = event.blockUpdatedInfo.diskSize * (if 
(storageLevel.useDisk) 1 else -1)
 val memoryDelta = event.blockUpdatedInfo.memSize * (if 
(storageLevel.useMemory) 1 else -1)
 
-val updatedStorageLevel = if (storageLevel.isValid) {
-  Some(storageLevel.description)
-} else {
-  None
-}
-
 // We need information about the executor to update some memory accounting 
values in the
 // RDD info, so read that beforehand.
 val maybeExec = liveExecutors.get(executorId)
@@ -936,13 +930,9 @@ private[spark] class AppStatusListener(
 // Update the block entry in the RDD info, keeping track of the deltas 
above so that we
 // can update the executor information too.
 liveRDDs.get(block.rddId).foreach { rdd =>
-  if (updatedStorageLevel.isDefined) {
-rdd.setStorageLevel(updatedStorageLevel.get)
-  }
-
   val partition = rdd.partition(block.name)
 
-  val executors = if (updatedStorageLevel.isDefined) {
+  val executors = if (storageLevel.isValid) {
 val current = partition.executors
 if (current.contains(executorId)) {
   current
@@ -957,7 +947,7 @@ private[spark] class AppStatusListener(
 
   // Only update the partition if it's still stored in some executor, 
otherwise get rid of it.
   if (executors.nonEmpty) {
-partition.update(executors, rdd.storageLevel,
+partition.update(executors,
   addDeltaToValue(partition.memoryUsed, memoryDelta),
   addDeltaToValue(partition.diskUsed, diskDelta))
   } else {
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala 
b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index aa4a21c..00c991b 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -30,7 +30,7 @@ import org.apache.spark.executor.{ExecutorMetrics, 
TaskMetrics}
 import org.apache.spark.resource.ResourceInformation
 import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, Ta

[spark] branch master updated: [SPARK-29189][SQL] Add an option to ignore block locations when listing file

2019-10-07 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 64fe82b  [SPARK-29189][SQL] Add an option to ignore block locations 
when listing file
64fe82b is described below

commit 64fe82b519bdc854fcbef40e906ac1fb181534c2
Author: gwang3 
AuthorDate: Mon Oct 7 14:52:55 2019 -0500

[SPARK-29189][SQL] Add an option to ignore block locations when listing file

### What changes were proposed in this pull request?
In our PROD env, we have a pure Spark cluster, I think this is also pretty 
common, where computation is separated from storage layer. In such deploy mode, 
data locality is never reachable.
And there are some configurations in Spark scheduler to reduce waiting time 
for data locality(e.g. "spark.locality.wait"). While, problem is that, in 
listing file phase, the location informations of all the files, with all the 
blocks inside each file, are all fetched from the distributed file system. 
Actually, in a PROD environment, a table can be so huge that even fetching all 
these location informations need take tens of seconds.
To improve such scenario, Spark need provide an option, where data locality 
can be totally ignored, all we need in the listing file phase are the files 
locations, without any block location informations.

### Why are the changes needed?
And we made a benchmark in our PROD env, after ignore the block locations, 
we got a pretty huge improvement.

Table Size | Total File Number | Total Block Number | List File 
Duration(With Block Location) | List File Duration(Without Block Location)
-- | -- | -- | -- | --
22.6T | 3 | 12 | 16.841s | 1.730s
28.8 T | 42001 | 148964 | 10.099s | 2.858s
3.4 T | 2 | 2 | 5.833s | 4.881s

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Via ut.

Closes #25869 from wangshisan/SPARK-29189.

Authored-by: gwang3 
Signed-off-by: Imran Rashid 
---
 .../org/apache/spark/sql/internal/SQLConf.scala| 13 ++
 .../execution/datasources/InMemoryFileIndex.scala  | 11 ++--
 .../sql/execution/datasources/FileIndexSuite.scala | 29 ++
 3 files changed, 51 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 9b2e314..3d28b5e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -832,6 +832,17 @@ object SQLConf {
   .intConf
   .createWithDefault(1)
 
+  val IGNORE_DATA_LOCALITY =
+buildConf("spark.sql.sources.ignore.datalocality")
+  .doc("If true, Spark will not fetch the block locations for each file on 
" +
+"listing files. This speeds up file listing, but the scheduler cannot 
" +
+"schedule tasks to take advantage of data locality. It can be 
particularly " +
+"useful if data is read from a remote cluster so the scheduler could 
never " +
+"take advantage of locality anyway.")
+  .internal()
+  .booleanConf
+  .createWithDefault(false)
+
   // Whether to automatically resolve ambiguity in join conditions for 
self-joins.
   // See SPARK-6231.
   val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY =
@@ -2494,6 +2505,8 @@ class SQLConf extends Serializable with Logging {
 
   def defaultV2Catalog: Option[String] = getConf(DEFAULT_V2_CATALOG)
 
+  def ignoreDataLocality: Boolean = getConf(SQLConf.IGNORE_DATA_LOCALITY)
+
   /** ** SQLConf functionality methods  */
 
   /** Set Spark SQL configuration properties. */
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
index cf7a130..ed860f6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
@@ -171,6 +171,7 @@ object InMemoryFileIndex extends Logging {
   areRootPaths: Boolean): Seq[(Path, Seq[FileStatus])] = {
 
 val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles
+val ignoreLocality = sparkSession.sessionState.conf.ignoreDataLocality
 
 // Short-circuits parallel listing when serial listing is likely to be 
faster.
 if (paths.size <= 
sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) {
@@ -181,6 +182,7 @@ object InMemoryFileIndex extends Logging {
   filter,

[spark] branch master updated: [SPARK-29072][CORE] Put back usage of TimeTrackingOutputStream for UnsafeShuffleWriter and SortShuffleWriter

2019-09-16 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 67751e2  [SPARK-29072][CORE] Put back usage of 
TimeTrackingOutputStream for UnsafeShuffleWriter and SortShuffleWriter
67751e2 is described below

commit 67751e26940a16ab6f9950ae66a46b7cb901c102
Author: mcheah 
AuthorDate: Mon Sep 16 09:08:25 2019 -0500

[SPARK-29072][CORE] Put back usage of TimeTrackingOutputStream for 
UnsafeShuffleWriter and SortShuffleWriter

### What changes were proposed in this pull request?
The previous refactors of the shuffle writers using the shuffle writer 
plugin resulted in shuffle write metric updates - particularly write times - 
being lost in particular situations. This patch restores the lost metric 
updates.

### Why are the changes needed?
This fixes a regression. I'm pretty sure that without this, the Spark UI 
will lose shuffle write time information.

### Does this PR introduce any user-facing change?
No change from Spark 2.4. Without this, there would be a user-facing bug in 
Spark 3.0.

### How was this patch tested?
Existing unit tests.

Closes #25780 from mccheah/fix-write-metrics.

Authored-by: mcheah 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/shuffle/sort/UnsafeShuffleWriter.java|  2 ++
 .../spark/shuffle/ShufflePartitionPairsWriter.scala   | 15 ---
 2 files changed, 14 insertions(+), 3 deletions(-)

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 f59bddc..4d11abd 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
@@ -57,6 +57,7 @@ import org.apache.spark.shuffle.api.ShufflePartitionWriter;
 import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter;
 import org.apache.spark.shuffle.api.WritableByteChannelWrapper;
 import org.apache.spark.storage.BlockManager;
+import org.apache.spark.storage.TimeTrackingOutputStream;
 import org.apache.spark.unsafe.Platform;
 import org.apache.spark.util.Utils;
 
@@ -382,6 +383,7 @@ public class UnsafeShuffleWriter extends 
ShuffleWriter {
 ShufflePartitionWriter writer = 
mapWriter.getPartitionWriter(partition);
 OutputStream partitionOutput = writer.openStream();
 try {
+  partitionOutput = new TimeTrackingOutputStream(writeMetrics, 
partitionOutput);
   partitionOutput = 
blockManager.serializerManager().wrapForEncryption(partitionOutput);
   if (compressionCodec != null) {
 partitionOutput = 
compressionCodec.compressedOutputStream(partitionOutput);
diff --git 
a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala
 
b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala
index a988c5e..e0affb8 100644
--- 
a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala
+++ 
b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala
@@ -21,7 +21,7 @@ import java.io.{Closeable, IOException, OutputStream}
 
 import org.apache.spark.serializer.{SerializationStream, SerializerInstance, 
SerializerManager}
 import org.apache.spark.shuffle.api.ShufflePartitionWriter
-import org.apache.spark.storage.BlockId
+import org.apache.spark.storage.{BlockId, TimeTrackingOutputStream}
 import org.apache.spark.util.Utils
 import org.apache.spark.util.collection.PairsWriter
 
@@ -39,6 +39,7 @@ private[spark] class ShufflePartitionPairsWriter(
 
   private var isClosed = false
   private var partitionStream: OutputStream = _
+  private var timeTrackingStream: OutputStream = _
   private var wrappedStream: OutputStream = _
   private var objOut: SerializationStream = _
   private var numRecordsWritten = 0
@@ -59,7 +60,8 @@ private[spark] class ShufflePartitionPairsWriter(
   private def open(): Unit = {
 try {
   partitionStream = partitionWriter.openStream
-  wrappedStream = serializerManager.wrapStream(blockId, partitionStream)
+  timeTrackingStream = new TimeTrackingOutputStream(writeMetrics, 
partitionStream)
+  wrappedStream = serializerManager.wrapStream(blockId, timeTrackingStream)
   objOut = serializerInstance.serializeStream(wrappedStream)
 } catch {
   case e: Exception =>
@@ -78,6 +80,7 @@ private[spark] class ShufflePartitionPairsWriter(
   // Setting these to null will prevent the underlying streams from 
being closed twice
   // just in case any stream's close() implementation is not 
idempotent.
   wrappedStream = null
+  timeTrackingStream = null
   partitionStream = null
 } {
   // Normally clos

[spark] branch master updated: [SPARK-28770][CORE][TEST] Fix ReplayListenerSuite tests that sometimes fail

2019-09-05 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 151b954  [SPARK-28770][CORE][TEST] Fix ReplayListenerSuite tests that 
sometimes fail
151b954 is described below

commit 151b954e52c66f78a72530ab69f38100101f6cb7
Author: Wing Yew Poon 
AuthorDate: Thu Sep 5 15:55:22 2019 -0500

[SPARK-28770][CORE][TEST] Fix ReplayListenerSuite tests that sometimes fail

### What changes were proposed in this pull request?

`ReplayListenerSuite` depends on a listener class to listen for replayed 
events. This class was implemented by extending `EventLoggingListener`. 
`EventLoggingListener` does not log executor metrics update events, but uses 
them to update internal state; on a stage completion event, it then logs stage 
executor metrics events using this internal state. As executor metrics update 
events do not get written to the event log, they do not get replayed. The 
internal state of the replay listene [...]

We reimplement the replay listener to simply buffer each and every event it 
receives. This makes it a simpler yet better tool for verifying the events that 
get sent through the ReplayListenerBus.

### Why are the changes needed?

As explained above. Tests sometimes fail due to events being received by 
the `EventLoggingListener` that do not get logged (and thus do not get 
replayed) but influence other events that get logged.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing unit tests.

Closes #25673 from wypoon/SPARK-28770.

Authored-by: Wing Yew Poon 
Signed-off-by: Imran Rashid 
---
 .../spark/scheduler/ReplayListenerSuite.scala  | 36 ++
 1 file changed, 16 insertions(+), 20 deletions(-)

diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
index e796137..d65b5cb 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
@@ -21,12 +21,14 @@ import java.io._
 import java.net.URI
 import java.util.concurrent.atomic.AtomicInteger
 
+import scala.collection.mutable.ArrayBuffer
+
 import org.apache.hadoop.fs.Path
 import org.json4s.JsonAST.JValue
 import org.json4s.jackson.JsonMethods._
 import org.scalatest.BeforeAndAfter
 
-import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, 
SparkFunSuite}
+import org.apache.spark._
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec}
 import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils}
@@ -62,7 +64,7 @@ class ReplayListenerSuite extends SparkFunSuite with 
BeforeAndAfter with LocalSp
 
 val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath)
 val logData = fileSystem.open(logFilePath)
-val eventMonster = new EventMonster(conf)
+val eventMonster = new EventBufferingListener
 try {
   val replayer = new ReplayListenerBus()
   replayer.addListener(eventMonster)
@@ -108,7 +110,7 @@ class ReplayListenerSuite extends SparkFunSuite with 
BeforeAndAfter with LocalSp
 val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath)
 val replayer = new ReplayListenerBus()
 
-val eventMonster = new EventMonster(conf)
+val eventMonster = new EventBufferingListener
 replayer.addListener(eventMonster)
 
 // Verify the replay returns the events given the input maybe truncated.
@@ -145,7 +147,7 @@ class ReplayListenerSuite extends SparkFunSuite with 
BeforeAndAfter with LocalSp
 
 val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath)
 val logData = fileSystem.open(logFilePath)
-val eventMonster = new EventMonster(conf)
+val eventMonster = new EventBufferingListener
 try {
   val replayer = new ReplayListenerBus()
   replayer.addListener(eventMonster)
@@ -207,7 +209,7 @@ class ReplayListenerSuite extends SparkFunSuite with 
BeforeAndAfter with LocalSp
 
 // Replay events
 val logData = EventLoggingListener.openEventLog(eventLog.getPath(), 
fileSystem)
-val eventMonster = new EventMonster(conf)
+val eventMonster = new EventBufferingListener
 try {
   val replayer = new ReplayListenerBus()
   replayer.addListener(eventMonster)
@@ -219,11 +221,11 @@ class ReplayListenerSuite extends SparkFunSuite with 
BeforeAndAfter with LocalSp
 // Verify the same events are replayed in the same order
 assert(sc.eventLogger.isDefined)
 val originalEvents = sc.eventLogger.get.loggedEvents
+  .map(JsonProtocol.sparkEventFromJson(_))
 val replayedEvents = eventMonster.loggedEvents
+  .map

[spark-website] branch asf-site updated: CVE-2019-10099

2019-08-06 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 6e85b26  CVE-2019-10099
6e85b26 is described below

commit 6e85b26065809c56a9d317980388032f62a381e8
Author: Imran Rashid 
AuthorDate: Tue Aug 6 10:14:36 2019 -0500

CVE-2019-10099

ran jekyll build / serve locally.

Also updated a random formatting issue on the release process page -- 
(fixed just by running jekyll build).

Author: Imran Rashid 

Closes #209 from squito/CVE-2019-10099.
---
 security.md   | 24 
 site/release-process.html | 12 +++-
 site/security.html| 28 
 3 files changed, 59 insertions(+), 5 deletions(-)

diff --git a/security.md b/security.md
index 340622b..7e062b8 100644
--- a/security.md
+++ b/security.md
@@ -18,6 +18,30 @@ non-public list that will reach the Apache Security team, as 
well as the Spark P
 
 Known Security Issues
 
+CVE-2019-10099: Apache Spark unencrypted data on local 
disk
+
+Severity: Important
+
+Vendor: The Apache Software Foundation
+
+Versions affected:
+- All Spark 1.x, Spark 2.0.x, Spark 2.1.x, and 2.2.x versions
+- Spark 2.3.0 to 2.3.2
+
+Description:
+
+Prior to Spark 2.3.3, in certain situations Spark would write user data to 
local disk unencrypted, even if `spark.io.encryption.enabled=true`.  This 
includes cached blocks that are fetched to disk (controlled by 
`spark.maxRemoteBlockSizeFetchToMem`); in SparkR, using parallelize; in 
Pyspark, using broadcast and parallelize; and use of python udfs.
+
+
+Mitigation:
+
+- 1.x, 2.0.x, 2.1.x, 2.2.x, 2.3.x  users should upgrade to 2.3.3 or newer, 
including 2.4.x
+
+Credit:
+
+- This issue was reported by Thomas Graves of NVIDIA.
+
+
 CVE-2018-11760: Apache Spark local privilege 
escalation vulnerability
 
 Severity: Important
diff --git a/site/release-process.html b/site/release-process.html
index 2c5bb41..2e8e82e 100644
--- a/site/release-process.html
+++ b/site/release-process.html
@@ -299,11 +299,13 @@ changes or in the release news on the website later.
 Also check that all build and test passes are green from the RISELab 
Jenkins: https://amplab.cs.berkeley.edu/jenkins/ particularly look for Spark 
Packaging, QA Compile, QA Test.
 Note that not all permutations are run on PR therefore it is important to 
check Jenkins runs.
 
-To cut a release candidate, there are 4 steps:
-1. Create a git tag for the release candidate.
-1. Package the release binaries  sources, and upload them to the Apache 
staging SVN repo.
-1. Create the release docs, and upload them to the Apache staging SVN repo.
-1. Publish a snapshot to the Apache staging Maven repo.
+To cut a release candidate, there are 4 steps:
+
+  Create a git tag for the release candidate.
+  Package the release binaries  sources, and upload them to the 
Apache staging SVN repo.
+  Create the release docs, and upload them to the Apache staging SVN 
repo.
+  Publish a snapshot to the Apache staging Maven repo.
+
 
 The process of cutting a release candidate has been automated via the 
dev/create-release/do-release-docker.sh script.
 Run this script, type information it requires, and wait until it finishes. You 
can also do a single step via the -s option.
diff --git a/site/security.html b/site/security.html
index 9f5b5b5..4c78b81 100644
--- a/site/security.html
+++ b/site/security.html
@@ -211,6 +211,34 @@ non-public list that will reach the Apache Security team, 
as well as the Spark P
 
 Known Security Issues
 
+CVE-2019-10099: Apache Spark unencrypted data on local 
disk
+
+Severity: Important
+
+Vendor: The Apache Software Foundation
+
+Versions affected:
+
+  All Spark 1.x, Spark 2.0.x, Spark 2.1.x, and 2.2.x versions
+  Spark 2.3.0 to 2.3.2
+
+
+Description:
+
+Prior to Spark 2.3.3, in certain situations Spark would write user data to 
local disk unencrypted, even if spark.io.encryption.enabled=true.  
This includes cached blocks that are fetched to disk (controlled by 
spark.maxRemoteBlockSizeFetchToMem); in SparkR, using parallelize; 
in Pyspark, using broadcast and parallelize; and use of python udfs.
+
+Mitigation:
+
+
+  1.x, 2.0.x, 2.1.x, 2.2.x, 2.3.x  users should upgrade to 2.3.3 or newer, 
including 2.4.x
+
+
+Credit:
+
+
+  This issue was reported by Thomas Graves of NVIDIA.
+
+
 CVE-2018-11760: Apache Spark local privilege 
escalation vulnerability
 
 Severity: Important


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-26755][SCHEDULER] : Optimize Spark Scheduler to dequeue speculative tasks…

2019-07-30 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 70910e6  [SPARK-26755][SCHEDULER] : Optimize Spark Scheduler to 
dequeue speculative tasks…
70910e6 is described below

commit 70910e6ad00f0de4075217d5305d87a477ff1dc4
Author: pgandhi 
AuthorDate: Tue Jul 30 09:54:51 2019 -0500

[SPARK-26755][SCHEDULER] : Optimize Spark Scheduler to dequeue speculative 
tasks…

… more efficiently

This PR improves the performance of scheduling speculative tasks to be O(1) 
instead of O(numSpeculativeTasks), using the same approach used for scheduling 
regular tasks. The performance of this method is particularly important because 
a lock is held on the TaskSchedulerImpl which is a bottleneck for all 
scheduling operations. We ran a Join query on a large dataset with speculation 
enabled and out of 10 tasks for the ShuffleMapStage, the maximum number of 
speculatable tasks that wa [...]

In particular, this works by storing a separate stack of tasks by executor, 
node, and rack locality preferences. Then when trying to schedule a speculative 
task, rather than scanning all speculative tasks to find ones which match the 
given executor (or node, or rack) preference, we can jump to a quick check of 
tasks matching the resource offer. This technique was already used for regular 
tasks -- this change refactors the code to allow sharing with regular and 
speculative task execution.

## What changes were proposed in this pull request?

Have split the main queue "speculatableTasks" into 5 separate queues based 
on locality preference similar to how normal tasks are enqueued. Thus, the 
"dequeueSpeculativeTask" method will avoid performing locality checks for each 
task at runtime and simply return the preferable task to be executed.

## How was this patch tested?
We ran a spark job that performed a join on a 10 TB dataset to test the 
code change.
Original Code:
https://user-images.githubusercontent.com/8190/51873321-572df280-2322-11e9-9149-0aae08d5edc6.png;>

Optimized Code:
https://user-images.githubusercontent.com/8190/51873343-6745d200-2322-11e9-947b-2cfd0f06bcab.png;>

As you can see, the run time of the ShuffleMapStage came down from 40 min 
to 6 min approximately, thus, reducing the overall running time of the spark 
job by a significant amount.

Another example for the same job:

Original Code:
https://user-images.githubusercontent.com/8190/51873355-70cf3a00-2322-11e9-9c3a-af035449a306.png;>

Optimized Code:
https://user-images.githubusercontent.com/8190/51873367-7dec2900-2322-11e9-8d07-1b1b49285f71.png;>

Closes #23677 from pgandhi999/SPARK-26755.

Lead-authored-by: pgandhi 
Co-authored-by: pgandhi 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSetManager.scala| 292 -
 .../scheduler/OutputCommitCoordinatorSuite.scala   |  12 +-
 .../spark/scheduler/TaskSetManagerSuite.scala  | 122 -
 3 files changed, 242 insertions(+), 184 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index e7645fc..79a1afc 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -131,37 +131,17 @@ private[spark] class TaskSetManager(
   // same time for a barrier stage.
   private[scheduler] def isBarrier = taskSet.tasks.nonEmpty && 
taskSet.tasks(0).isBarrier
 
-  // Set of pending tasks for each executor. These collections are actually
-  // treated as stacks, in which new tasks are added to the end of the
-  // ArrayBuffer and removed from the end. This makes it faster to detect
-  // tasks that repeatedly fail because whenever a task failed, it is put
-  // back at the head of the stack. These collections may contain duplicates
-  // for two reasons:
-  // (1): Tasks are only removed lazily; when a task is launched, it remains
-  // in all the pending lists except the one that it was launched from.
-  // (2): Tasks may be re-added to these lists multiple times as a result
-  // of failures.
-  // Duplicates are handled in dequeueTaskFromList, which ensures that a
-  // task hasn't already started running before launching it.
-  private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
-
-  // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
-  // but at host level.
-  private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]]
-
-  // Set of pending tasks for each rack -- similar to the above.
-  private val pendingTasksForRa

[spark] branch master updated: [SPARK-28005][YARN] Remove unnecessary log from SparkRackResolver

2019-06-26 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 8313015  [SPARK-28005][YARN] Remove unnecessary log from 
SparkRackResolver
8313015 is described below

commit 8313015e8dd11aafc5b887773e467892d15134de
Author: Gabor Somogyi 
AuthorDate: Wed Jun 26 09:50:54 2019 -0500

[SPARK-28005][YARN] Remove unnecessary log from SparkRackResolver

## What changes were proposed in this pull request?

SparkRackResolver generates an INFO message every time is called with 0 
arguments.
In this PR I've deleted it because it's too verbose.

## How was this patch tested?

Existing unit tests + spark-shell.

Closes #24935 from gaborgsomogyi/SPARK-28005.

Authored-by: Gabor Somogyi 
Signed-off-by: Imran Rashid 
---
 .../main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala| 3 +++
 1 file changed, 3 insertions(+)

diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
index cab3272..51be932 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
@@ -67,6 +67,9 @@ private[spark] class SparkRackResolver(conf: Configuration) 
extends Logging {
   }
 
   private def coreResolve(hostNames: Seq[String]): Seq[Node] = {
+if (hostNames.isEmpty) {
+  return Seq.empty
+}
 val nodes = new ArrayBuffer[Node]
 // dnsToSwitchMapping is thread-safe
 val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-27630][CORE] Properly handle task end events from completed stages

2019-06-25 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 38263f6  [SPARK-27630][CORE] Properly handle task end events from 
completed stages
38263f6 is described below

commit 38263f6d153944b6f2f0248d9284861fc82532d6
Author: sychen 
AuthorDate: Tue Jun 25 14:30:13 2019 -0500

[SPARK-27630][CORE] Properly handle task end events from completed stages

## What changes were proposed in this pull request?
Track tasks separately for each stage attempt (instead of tracking by 
stage), and do NOT reset the numRunningTasks to 0 on StageCompleted.

In the case of stage retry, the `taskEnd` event from the zombie stage 
sometimes makes the number of `totalRunningTasks` negative, which will causes 
the job to get stuck.
Similar problem also exists with `stageIdToTaskIndices` & 
`stageIdToSpeculativeTaskIndices`.
If it is a failed `taskEnd` event of the zombie stage, this will cause 
`stageIdToTaskIndices` or `stageIdToSpeculativeTaskIndices` to remove the task 
index of the active stage, and the number of `totalPendingTasks` will increase 
unexpectedly.
## How was this patch tested?
unit test properly handle task end events from completed stages

Closes #24497 from cxzl25/fix_stuck_job_follow_up.

Authored-by: sychen 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/ExecutorAllocationManager.scala   | 113 -
 .../org/apache/spark/scheduler/DAGScheduler.scala  |   2 +-
 .../org/apache/spark/scheduler/SparkListener.scala |   5 +-
 .../spark/ExecutorAllocationManagerSuite.scala |  33 +-
 project/MimaExcludes.scala |   6 ++
 5 files changed, 104 insertions(+), 55 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index bb95fea..bceb26c 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -491,6 +491,10 @@ private[spark] class ExecutorAllocationManager(
 numExecutorsToAdd = 1
   }
 
+  private case class StageAttempt(stageId: Int, stageAttemptId: Int) {
+override def toString: String = s"Stage $stageId (Attempt $stageAttemptId)"
+  }
+
   /**
* A listener that notifies the given allocation manager of when to add and 
remove executors.
*
@@ -499,29 +503,32 @@ private[spark] class ExecutorAllocationManager(
*/
   private[spark] class ExecutorAllocationListener extends SparkListener {
 
-private val stageIdToNumTasks = new mutable.HashMap[Int, Int]
-// Number of running tasks per stage including speculative tasks.
+private val stageAttemptToNumTasks = new mutable.HashMap[StageAttempt, Int]
+// Number of running tasks per stageAttempt including speculative tasks.
 // Should be 0 when no stages are active.
-private val stageIdToNumRunningTask = new mutable.HashMap[Int, Int]
-private val stageIdToTaskIndices = new mutable.HashMap[Int, 
mutable.HashSet[Int]]
-// Number of speculative tasks to be scheduled in each stage
-private val stageIdToNumSpeculativeTasks = new mutable.HashMap[Int, Int]
-// The speculative tasks started in each stage
-private val stageIdToSpeculativeTaskIndices = new mutable.HashMap[Int, 
mutable.HashSet[Int]]
-
-// stageId to tuple (the number of task with locality preferences, a map 
where each pair is a
-// node and the number of tasks that would like to be scheduled on that 
node) map,
-// maintain the executor placement hints for each stage Id used by 
resource framework to better
-// place the executors.
-private val stageIdToExecutorPlacementHints = new mutable.HashMap[Int, 
(Int, Map[String, Int])]
+private val stageAttemptToNumRunningTask = new 
mutable.HashMap[StageAttempt, Int]
+private val stageAttemptToTaskIndices = new mutable.HashMap[StageAttempt, 
mutable.HashSet[Int]]
+// Number of speculative tasks to be scheduled in each stageAttempt
+private val stageAttemptToNumSpeculativeTasks = new 
mutable.HashMap[StageAttempt, Int]
+// The speculative tasks started in each stageAttempt
+private val stageAttemptToSpeculativeTaskIndices =
+  new mutable.HashMap[StageAttempt, mutable.HashSet[Int]]
+
+// stageAttempt to tuple (the number of task with locality preferences, a 
map where each pair
+// is a node and the number of tasks that would like to be scheduled on 
that node) map,
+// maintain the executor placement hints for each stageAttempt used by 
resource framework
+// to better place the executors.
+private val stageAttemptToExecutorPlacementHints =
+  new mutable.HashMap[StageAttempt, (Int, Map[String, Int])]
 
   

[spark] branch master updated: [SPARK-20286][CORE] Improve logic for timing out executors in dynamic allocation.

2019-06-05 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 b312033  [SPARK-20286][CORE] Improve logic for timing out executors in 
dynamic allocation.
b312033 is described below

commit b312033bd33cd6cbf1f166ccaa7a5df4e3421078
Author: Marcelo Vanzin 
AuthorDate: Wed Jun 5 08:09:44 2019 -0500

[SPARK-20286][CORE] Improve logic for timing out executors in dynamic 
allocation.

This change refactors the portions of the ExecutorAllocationManager class 
that
track executor state into a new class, to achieve a few goals:

- make the code easier to understand
- better separate concerns (task backlog vs. executor state)
- less synchronization between event and allocation threads
- less coupling between the allocation code and executor state tracking

The executor tracking code was moved to a new class (ExecutorMonitor) that
encapsulates all the logic of tracking what happens to executors and when
they can be timed out. The logic to actually remove the executors remains
in the EAM, since it still requires information that is not tracked by the
new executor monitor code.

In the executor monitor itself, of interest, specifically, is a change in
how cached blocks are tracked; instead of polling the block manager, the
monitor now uses events to track which executors have cached blocks, and
is able to detect also unpersist events and adjust the time when the 
executor
should be removed accordingly. (That's the bug mentioned in the PR title.)

Because of the refactoring, a few tests in the old EAM test suite were 
removed,
since they're now covered by the newly added test suite. The EAM suite was
also changed a little bit to not instantiate a SparkContext every time. This
allowed some cleanup, and the tests also run faster.

Tested with new and updated unit tests, and with multiple TPC-DS workloads
running with dynamic allocation on; also some manual tests for the caching
behavior.

Closes #24704 from vanzin/SPARK-20286.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/ExecutorAllocationClient.scala|   9 +-
 .../apache/spark/ExecutorAllocationManager.scala   | 239 +--
 .../main/scala/org/apache/spark/SparkContext.scala |   3 +-
 .../org/apache/spark/internal/config/package.scala |   8 +-
 .../cluster/CoarseGrainedSchedulerBackend.scala|   4 +
 .../spark/scheduler/dynalloc/ExecutorMonitor.scala | 283 
 .../apache/spark/storage/BlockManagerMaster.scala  |   9 -
 .../spark/storage/BlockManagerMasterEndpoint.scala |  31 -
 .../spark/storage/BlockManagerMessages.scala   |   3 -
 ...g.apache.spark.scheduler.ExternalClusterManager |   1 -
 .../spark/ExecutorAllocationManagerSuite.scala | 708 ++---
 .../deploy/ExternalShuffleServiceDbSuite.scala |   9 +-
 .../scheduler/dynalloc/ExecutorMonitorSuite.scala  | 289 +
 .../spark/storage/BlockManagerInfoSuite.scala  |  14 -
 .../scheduler/ExecutorAllocationManagerSuite.scala |   2 +-
 15 files changed, 822 insertions(+), 790 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index 63d87b4..cb965cb 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -23,11 +23,18 @@ package org.apache.spark
  */
 private[spark] trait ExecutorAllocationClient {
 
-
   /** Get the list of currently active executors */
   private[spark] def getExecutorIds(): Seq[String]
 
   /**
+   * Whether an executor is active. An executor is active when it can be used 
to execute tasks
+   * for jobs submitted by the application.
+   *
+   * @return whether the executor with the given ID is currently active.
+   */
+  def isExecutorActive(id: String): Boolean
+
+  /**
* Update the cluster manager on our scheduling needs. Three bits of 
information are included
* to help it make decisions.
* @param numExecutors The total number of executors we'd like to have. The 
cluster manager
diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index 1782027..63df7cc 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -30,7 +30,7 @@ import org.apache.spark.internal.config._
 import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL
 import org.apache.spark.metrics.source.Source
 import org.apache.spark.scheduler._
-import org.apache.spark.storage.BlockManagerMaster

[spark] branch master updated: [SPARK-27590][CORE] do not consider skipped tasks when scheduling speculative tasks

2019-05-07 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 d124ce9  [SPARK-27590][CORE] do not consider skipped tasks when 
scheduling speculative tasks
d124ce9 is described below

commit d124ce9c7ea537c59a776c05977c9f918d38febc
Author: Wenchen Fan 
AuthorDate: Tue May 7 12:02:08 2019 -0500

[SPARK-27590][CORE] do not consider skipped tasks when scheduling 
speculative tasks

## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24375

When `TaskSetManager` skips a task because its corresponding partition is 
already completed by other `TaskSetManager`s, we should not consider the 
duration of the task that is finished by other `TaskSetManager`s to schedule 
the speculative tasks of this `TaskSetManager`.

## How was this patch tested?

updated test case

Closes #24485 from cloud-fan/minor.

Authored-by: Wenchen Fan 
Signed-off-by: Imran Rashid 
---
 .../org/apache/spark/scheduler/DAGScheduler.scala  |  3 +--
 .../apache/spark/scheduler/TaskResultGetter.scala  |  5 ++---
 .../org/apache/spark/scheduler/TaskScheduler.scala |  2 +-
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 12 ---
 .../apache/spark/scheduler/TaskSetManager.scala| 24 +++---
 .../apache/spark/scheduler/DAGSchedulerSuite.scala |  6 ++
 .../scheduler/ExternalClusterManagerSuite.scala|  3 +--
 .../spark/scheduler/TaskSetManagerSuite.scala  |  4 ++--
 8 files changed, 25 insertions(+), 34 deletions(-)

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 b817eb6..1d4972e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1394,8 +1394,7 @@ private[spark] class DAGScheduler(
 // finished. Here we notify the task scheduler to skip running tasks 
for the same partition,
 // to save resource.
 if (task.stageAttemptId < stage.latestInfo.attemptNumber()) {
-  taskScheduler.notifyPartitionCompletion(
-stageId, task.partitionId, event.taskInfo.duration)
+  taskScheduler.notifyPartitionCompletion(stageId, task.partitionId)
 }
 
 task match {
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 09c4d9b..9b7f901 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -158,10 +158,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, 
scheduler: TaskSchedul
   // This method calls `TaskSchedulerImpl.handlePartitionCompleted` 
asynchronously. We do not want
   // DAGScheduler to call `TaskSchedulerImpl.handlePartitionCompleted` 
directly, as it's
   // synchronized and may hurt the throughput of the scheduler.
-  def enqueuePartitionCompletionNotification(
-  stageId: Int, partitionId: Int, taskDuration: Long): Unit = {
+  def enqueuePartitionCompletionNotification(stageId: Int, partitionId: Int): 
Unit = {
 getTaskResultExecutor.execute(() => Utils.logUncaughtExceptions {
-  scheduler.handlePartitionCompleted(stageId, partitionId, taskDuration)
+  scheduler.handlePartitionCompleted(stageId, partitionId)
 })
   }
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
index 1862e16..bfdbf02 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -70,7 +70,7 @@ private[spark] trait TaskScheduler {
 
   // Notify the corresponding `TaskSetManager`s of the stage, that a partition 
has already completed
   // and they can skip running tasks for it.
-  def notifyPartitionCompletion(stageId: Int, partitionId: Int, taskDuration: 
Long)
+  def notifyPartitionCompletion(stageId: Int, partitionId: Int)
 
   // Set the DAG scheduler for upcalls. This is guaranteed to be set before 
submitTasks is called.
   def setDAGScheduler(dagScheduler: DAGScheduler): Unit
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 7e820c3..532eb32 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -301,9 +301,8 @@ private[spark] class TaskSchedulerImpl(
 }
   }
 
-  override def notifyPartitionComp

[spark] branch master updated: [SPARK-13704][CORE][YARN] Reduce rack resolution time

2019-04-08 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 52838e7  [SPARK-13704][CORE][YARN] Reduce rack resolution time
52838e7 is described below

commit 52838e74afdd58a7c09707284b4e0232dc01ef26
Author: LantaoJin 
AuthorDate: Mon Apr 8 10:47:06 2019 -0500

[SPARK-13704][CORE][YARN] Reduce rack resolution time

## What changes were proposed in this pull request?

When you submit a stage on a large cluster, rack resolving takes a long 
time when initializing TaskSetManager because a script is invoked to resolve 
the rack of each host, one by one.
Based on current implementation, it takes 30~40 seconds to resolve the 
racks in our 5000 nodes' cluster. After applied the patch, it decreased to less 
than 15 seconds.

YARN-9332 has added an interface to handle multiple hosts in one invocation 
to save time. But before upgrading to the newest Hadoop, we could construct the 
same tool in Spark to resolve this issue.

## How was this patch tested?

UT and manually testing on a 5000 node cluster.

Closes #24245 from squito/SPARK-13704_update.

Lead-authored-by: LantaoJin 
Co-authored-by: Imran Rashid 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 30 ++--
 .../apache/spark/scheduler/TaskSetManager.scala| 33 ++--
 .../spark/scheduler/TaskSetManagerSuite.scala  | 76 +--
 ...calityPreferredContainerPlacementStrategy.scala |  5 +-
 .../spark/deploy/yarn/SparkRackResolver.scala  | 88 --
 .../apache/spark/deploy/yarn/YarnAllocator.scala   |  2 +-
 .../apache/spark/deploy/yarn/YarnRMClient.scala|  2 +-
 .../spark/scheduler/cluster/YarnScheduler.scala| 20 ++---
 .../spark/deploy/yarn/YarnAllocatorSuite.scala |  5 +-
 9 files changed, 219 insertions(+), 42 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index bffa1ff..e401c39 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -158,6 +158,8 @@ private[spark] class TaskSchedulerImpl(
 
   private[scheduler] var barrierCoordinator: RpcEndpoint = null
 
+  protected val defaultRackValue: Option[String] = None
+
   private def maybeInitBarrierCoordinator(): Unit = {
 if (barrierCoordinator == null) {
   barrierCoordinator = new BarrierCoordinator(barrierSyncTimeout, 
sc.listenerBus,
@@ -394,9 +396,10 @@ private[spark] class TaskSchedulerImpl(
 executorIdToRunningTaskIds(o.executorId) = HashSet[Long]()
 newExecAvail = true
   }
-  for (rack <- getRackForHost(o.host)) {
-hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += o.host
-  }
+}
+val hosts = offers.map(_.host).toSet.toSeq
+for ((host, Some(rack)) <- hosts.zip(getRacksForHosts(hosts))) {
+  hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += host
 }
 
 // Before making any offers, remove any nodes from the blacklist whose 
blacklist has expired. Do
@@ -830,8 +833,25 @@ private[spark] class TaskSchedulerImpl(
 blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(Set.empty)
   }
 
-  // By default, rack is unknown
-  def getRackForHost(value: String): Option[String] = None
+  /**
+   * Get the rack for one host.
+   *
+   * Note that [[getRacksForHosts]] should be preferred when possible as that 
can be much
+   * more efficient.
+   */
+  def getRackForHost(host: String): Option[String] = {
+getRacksForHosts(Seq(host)).head
+  }
+
+  /**
+   * Get racks for multiple hosts.
+   *
+   * The returned Sequence will be the same length as the hosts argument and 
can be zipped
+   * together with the hosts argument.
+   */
+  def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = {
+hosts.map(_ => defaultRackValue)
+  }
 
   private def waitBackendReady(): Unit = {
 if (backend.isReady) {
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 3977c0b..1444220 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -186,8 +186,24 @@ private[spark] class TaskSetManager(
 
   // Add all our tasks to the pending lists. We do this in reverse order
   // of task index so that tasks with low indices get launched first.
-  for (i <- (0 until numTasks).reverse) {
-addPendingTask(i)
+  addPendingTasks()
+
+  private def addPendingTasks(): Unit = {
+val (_, duration) = Utils.timeTakenMs {
+  for (i <- (0

[spark] branch master updated: [SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe ser/dser issue

2019-04-03 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 69dd44a  [SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix 
Kryo unsafe ser/dser issue
69dd44a is described below

commit 69dd44af19ed9a2d7873350e8610c9faeb151a7b
Author: LantaoJin 
AuthorDate: Wed Apr 3 20:09:50 2019 -0500

[SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe 
ser/dser issue

## What changes were proposed in this pull request?

HighlyCompressedMapStatus uses RoaringBitmap to record the empty blocks. 
But RoaringBitmap couldn't be ser/deser with unsafe KryoSerializer.

It's a bug of RoaringBitmap-0.5.11 and fixed in latest version.

This is an update of #24157

## How was this patch tested?

Add a UT

Closes #24264 from LantaoJin/SPARK-27216.

Lead-authored-by: LantaoJin 
Co-authored-by: Lantao Jin 
Signed-off-by: Imran Rashid 
---
 .../spark/serializer/KryoSerializerSuite.scala | 34 ++
 dev/deps/spark-deps-hadoop-2.7 |  3 +-
 dev/deps/spark-deps-hadoop-3.2 |  3 +-
 pom.xml|  2 +-
 4 files changed, 14 insertions(+), 28 deletions(-)

diff --git 
a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala 
b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index 16eec7e..2442670 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.serializer
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, 
FileOutputStream}
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
 import java.nio.ByteBuffer
 import java.util.concurrent.Executors
 
@@ -370,30 +370,6 @@ class KryoSerializerSuite extends SparkFunSuite with 
SharedSparkContext {
 assert(thrown.getCause.isInstanceOf[KryoException])
   }
 
-  test("SPARK-1: deserialize RoaringBitmap throw Buffer underflow 
exception") {
-withTempDir { dir =>
-  val tmpfile = dir.toString + "/RoaringBitmap"
-  val outStream = new FileOutputStream(tmpfile)
-  val output = new KryoOutput(outStream)
-  val bitmap = new RoaringBitmap
-  bitmap.add(1)
-  bitmap.add(3)
-  bitmap.add(5)
-  // Ignore Kryo because it doesn't use writeObject
-  bitmap.serialize(new KryoOutputObjectOutputBridge(null, output))
-  output.flush()
-  output.close()
-
-  val inStream = new FileInputStream(tmpfile)
-  val input = new KryoInput(inStream)
-  val ret = new RoaringBitmap
-  // Ignore Kryo because it doesn't use readObject
-  ret.deserialize(new KryoInputObjectInputBridge(null, input))
-  input.close()
-  assert(ret == bitmap)
-}
-  }
-
   test("KryoOutputObjectOutputBridge.writeObject and 
KryoInputObjectInputBridge.readObject") {
 val kryo = new KryoSerializer(conf).newKryo()
 
@@ -518,6 +494,14 @@ class KryoSerializerSuite extends SparkFunSuite with 
SharedSparkContext {
   assert(ThreadUtils.awaitResult(f, 10.seconds))
 }
   }
+
+  test("SPARK-27216: test RoaringBitmap ser/dser with Kryo") {
+val expected = new RoaringBitmap()
+expected.add(1787)
+val ser = new KryoSerializer(conf).newInstance()
+val actual: RoaringBitmap = ser.deserialize(ser.serialize(expected))
+assert(actual === expected)
+  }
 }
 
 class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with 
SharedSparkContext {
diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7
index 277274a..de0cb14 100644
--- a/dev/deps/spark-deps-hadoop-2.7
+++ b/dev/deps/spark-deps-hadoop-2.7
@@ -1,5 +1,5 @@
 JavaEWAH-0.3.2.jar
-RoaringBitmap-0.5.11.jar
+RoaringBitmap-0.7.45.jar
 ST4-4.0.4.jar
 activation-1.1.1.jar
 aircompressor-0.10.jar
@@ -178,6 +178,7 @@ scala-parser-combinators_2.12-1.1.0.jar
 scala-reflect-2.12.8.jar
 scala-xml_2.12-1.0.6.jar
 shapeless_2.12-2.3.2.jar
+shims-0.7.45.jar
 slf4j-api-1.7.16.jar
 slf4j-log4j12-1.7.16.jar
 snakeyaml-1.23.jar
diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2
index 5d40c24..326d085 100644
--- a/dev/deps/spark-deps-hadoop-3.2
+++ b/dev/deps/spark-deps-hadoop-3.2
@@ -1,6 +1,6 @@
 HikariCP-java7-2.4.12.jar
 JavaEWAH-0.3.2.jar
-RoaringBitmap-0.5.11.jar
+RoaringBitmap-0.7.45.jar
 ST4-4.0.4.jar
 accessors-smart-1.2.jar
 activation-1.1.1.jar
@@ -198,6 +198,7 @@ scala-parser-combinators_2.12-1.1.0.jar
 scala-reflect-2.12.8.jar
 scala-xml_2.12-1.0.6.jar
 shapeless_2.12-2.3.2.jar
+shims-0.7.45.jar
 slf4j-api-1.7.16.jar
 slf4j-log4j12-1.7.16.jar
 snakeyaml-1.23.jar
diff --git a/pom.

[spark] branch branch-2.3 updated: [SPARK-27112][CORE] : Create a resource ordering between threads to r…

2019-03-19 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new 7bb2b42  [SPARK-27112][CORE] : Create a resource ordering between 
threads to r…
7bb2b42 is described below

commit 7bb2b42de9a52e75107469f028c1e707d6a174b7
Author: pgandhi 
AuthorDate: Tue Mar 19 16:22:40 2019 -0500

[SPARK-27112][CORE] : Create a resource ordering between threads to r…

…esolve the deadlocks encountered when trying to kill executors either due 
to dynamic allocation or blacklisting

Closes #24072 from pgandhi999/SPARK-27112-2.

Authored-by: pgandhi 
Signed-off-by: Imran Rashid 

## What changes were proposed in this pull request?

There are two deadlocks as a result of the interplay between three 
different threads:

**task-result-getter thread**

**spark-dynamic-executor-allocation thread**

**dispatcher-event-loop thread(makeOffers())**

The fix ensures ordering synchronization constraint by acquiring lock on 
`TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in 
`makeOffers()` as well as killExecutors() method. This ensures resource 
ordering between the threads and thus, fixes the deadlocks.

## How was this patch tested?

Manual Tests

Closes #24134 from pgandhi999/branch-2.4-SPARK-27112.

Authored-by: pgandhi 
Signed-off-by: Imran Rashid 
(cherry picked from commit 95e73b328ac883be2ced9099f20c8878e498e297)
Signed-off-by: Imran Rashid 
---
 .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala   | 13 ++---
 1 file changed, 10 insertions(+), 3 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 8848add..ba45e28 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -238,7 +238,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 // Make fake resource offers on all executors
 private def makeOffers() {
   // Make sure no executor is killed while some task is launching on it
-  val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized {
+  val taskDescs = withLock {
 // Filter out executors under killing
 val activeExecutors = executorDataMap.filterKeys(executorIsAlive)
 val workOffers = activeExecutors.map {
@@ -263,7 +263,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 // Make fake resource offers on just one executor
 private def makeOffers(executorId: String) {
   // Make sure no executor is killed while some task is launching on it
-  val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized {
+  val taskDescs = withLock {
 // Filter out executors under killing
 if (executorIsAlive(executorId)) {
   val executorData = executorDataMap(executorId)
@@ -600,7 +600,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   force: Boolean): Seq[String] = {
 logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}")
 
-val response = synchronized {
+val response = withLock {
   val (knownExecutors, unknownExecutors) = 
executorIds.partition(executorDataMap.contains)
   unknownExecutors.foreach { id =>
 logWarning(s"Executor to kill $id does not exist!")
@@ -678,6 +678,13 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   }
 
   protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None }
+
+  // SPARK-27112: We need to ensure that there is ordering of lock acquisition
+  // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in 
order to fix
+  // the deadlock issue exposed in SPARK-27112
+  private def withLock[T](fn: => T): T = scheduler.synchronized {
+CoarseGrainedSchedulerBackend.this.synchronized { fn }
+  }
 }
 
 private[spark] object CoarseGrainedSchedulerBackend {


-
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-27112][CORE] : Create a resource ordering between threads to r…

2019-03-19 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 95e73b3  [SPARK-27112][CORE] : Create a resource ordering between 
threads to r…
95e73b3 is described below

commit 95e73b328ac883be2ced9099f20c8878e498e297
Author: pgandhi 
AuthorDate: Tue Mar 19 16:22:40 2019 -0500

[SPARK-27112][CORE] : Create a resource ordering between threads to r…

…esolve the deadlocks encountered when trying to kill executors either due 
to dynamic allocation or blacklisting

Closes #24072 from pgandhi999/SPARK-27112-2.

Authored-by: pgandhi 
Signed-off-by: Imran Rashid 

## What changes were proposed in this pull request?

There are two deadlocks as a result of the interplay between three 
different threads:

**task-result-getter thread**

**spark-dynamic-executor-allocation thread**

**dispatcher-event-loop thread(makeOffers())**

The fix ensures ordering synchronization constraint by acquiring lock on 
`TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in 
`makeOffers()` as well as killExecutors() method. This ensures resource 
ordering between the threads and thus, fixes the deadlocks.

## How was this patch tested?

Manual Tests

Closes #24134 from pgandhi999/branch-2.4-SPARK-27112.

Authored-by: pgandhi 
Signed-off-by: Imran Rashid 
---
 .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala   | 13 ++---
 1 file changed, 10 insertions(+), 3 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index de7c0d8..aa4e638 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -237,7 +237,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 // Make fake resource offers on all executors
 private def makeOffers() {
   // Make sure no executor is killed while some task is launching on it
-  val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized {
+  val taskDescs = withLock {
 // Filter out executors under killing
 val activeExecutors = executorDataMap.filterKeys(executorIsAlive)
 val workOffers = activeExecutors.map {
@@ -263,7 +263,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 // Make fake resource offers on just one executor
 private def makeOffers(executorId: String) {
   // Make sure no executor is killed while some task is launching on it
-  val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized {
+  val taskDescs = withLock {
 // Filter out executors under killing
 if (executorIsAlive(executorId)) {
   val executorData = executorDataMap(executorId)
@@ -607,7 +607,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   force: Boolean): Seq[String] = {
 logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}")
 
-val response = synchronized {
+val response = withLock {
   val (knownExecutors, unknownExecutors) = 
executorIds.partition(executorDataMap.contains)
   unknownExecutors.foreach { id =>
 logWarning(s"Executor to kill $id does not exist!")
@@ -685,6 +685,13 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   }
 
   protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None }
+
+  // SPARK-27112: We need to ensure that there is ordering of lock acquisition
+  // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in 
order to fix
+  // the deadlock issue exposed in SPARK-27112
+  private def withLock[T](fn: => T): T = scheduler.synchronized {
+CoarseGrainedSchedulerBackend.this.synchronized { fn }
+  }
 }
 
 private[spark] object CoarseGrainedSchedulerBackend {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-26288][CORE] add initRegisteredExecutorsDB

2019-03-19 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 8b0aa59  [SPARK-26288][CORE] add initRegisteredExecutorsDB
8b0aa59 is described below

commit 8b0aa59218c209d39cbba5959302d8668b885cf6
Author: weixiuli 
AuthorDate: Tue Mar 19 16:16:43 2019 -0500

[SPARK-26288][CORE] add initRegisteredExecutorsDB

## What changes were proposed in this pull request?

As we all know that spark on Yarn uses DB 
https://github.com/apache/spark/pull/7943 to record RegisteredExecutors 
information which can be reloaded and used again when the 
ExternalShuffleService is restarted .

The RegisteredExecutors information can't be recorded both in the mode of 
spark's standalone and spark on k8s , which will cause the RegisteredExecutors 
information to be lost ,when the ExternalShuffleService is restarted.

To solve the problem above, a method is proposed and is committed .

## How was this patch tested?
new  unit tests

Closes #23393 from weixiuli/SPARK-26288.

Authored-by: weixiuli 
Signed-off-by: Imran Rashid 
---
 .../shuffle/ExternalShuffleBlockHandler.java   |   5 +
 core/pom.xml   |   7 ++
 .../spark/deploy/ExternalShuffleService.scala  |  25 +++-
 .../org/apache/spark/deploy/worker/Worker.scala|   9 ++
 .../org/apache/spark/internal/config/package.scala |   7 ++
 .../deploy/ExternalShuffleServiceDbSuite.scala | 140 +
 .../apache/spark/deploy/worker/WorkerSuite.scala   |  51 +++-
 docs/spark-standalone.md   |  11 ++
 8 files changed, 253 insertions(+), 2 deletions(-)

diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
index b25e48a..70dcc8b 100644
--- 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
+++ 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java
@@ -66,6 +66,11 @@ public class ExternalShuffleBlockHandler extends RpcHandler {
   new ExternalShuffleBlockResolver(conf, registeredExecutorFile));
   }
 
+  @VisibleForTesting
+  public ExternalShuffleBlockResolver getBlockResolver() {
+return blockManager;
+  }
+
   /** Enables mocking out the StreamManager and BlockManager. */
   @VisibleForTesting
   public ExternalShuffleBlockHandler(
diff --git a/core/pom.xml b/core/pom.xml
index b9f78b2..45bda44 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -372,6 +372,13 @@
   tests
   test
 
+
+  org.apache.spark
+  spark-network-shuffle_${scala.binary.version}
+  ${project.version}
+  tests
+  test
+
 
 

[spark] branch master updated: [SPARK-27112][CORE] : Create a resource ordering between threads to resolve the deadlocks encountered …

2019-03-18 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 7043aee  [SPARK-27112][CORE] : Create a resource ordering between 
threads to resolve the deadlocks encountered …
7043aee is described below

commit 7043aee1ba95e92e1cbd0ebafcc5b09b69ee3082
Author: pgandhi 
AuthorDate: Mon Mar 18 10:33:51 2019 -0500

[SPARK-27112][CORE] : Create a resource ordering between threads to resolve 
the deadlocks encountered …

…when trying to kill executors either due to dynamic allocation or 
blacklisting

## What changes were proposed in this pull request?

There are two deadlocks as a result of the interplay between three 
different threads:

**task-result-getter thread**

**spark-dynamic-executor-allocation thread**

**dispatcher-event-loop thread(makeOffers())**

The fix ensures ordering synchronization constraint by acquiring lock on 
`TaskSchedulerImpl` before acquiring lock on `CoarseGrainedSchedulerBackend` in 
`makeOffers()` as well as killExecutors() method. This ensures resource 
ordering between the threads and thus, fixes the deadlocks.

## How was this patch tested?

Manual Tests

Closes #24072 from pgandhi999/SPARK-27112-2.

Authored-by: pgandhi 
Signed-off-by: Imran Rashid 
---
 .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala   | 13 ++---
 1 file changed, 10 insertions(+), 3 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index dc0f21c..808ef08 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -258,7 +258,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 // Make fake resource offers on all executors
 private def makeOffers() {
   // Make sure no executor is killed while some task is launching on it
-  val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized {
+  val taskDescs = withLock {
 // Filter out executors under killing
 val activeExecutors = executorDataMap.filterKeys(executorIsAlive)
 val workOffers = activeExecutors.map {
@@ -284,7 +284,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 // Make fake resource offers on just one executor
 private def makeOffers(executorId: String) {
   // Make sure no executor is killed while some task is launching on it
-  val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized {
+  val taskDescs = withLock {
 // Filter out executors under killing
 if (executorIsAlive(executorId)) {
   val executorData = executorDataMap(executorId)
@@ -631,7 +631,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   force: Boolean): Seq[String] = {
 logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}")
 
-val response = synchronized {
+val response = withLock {
   val (knownExecutors, unknownExecutors) = 
executorIds.partition(executorDataMap.contains)
   unknownExecutors.foreach { id =>
 logWarning(s"Executor to kill $id does not exist!")
@@ -730,6 +730,13 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 
   protected def currentDelegationTokens: Array[Byte] = delegationTokens.get()
 
+  // SPARK-27112: We need to ensure that there is ordering of lock acquisition
+  // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in 
order to fix
+  // the deadlock issue exposed in SPARK-27112
+  private def withLock[T](fn: => T): T = scheduler.synchronized {
+CoarseGrainedSchedulerBackend.this.synchronized { fn }
+  }
+
 }
 
 private[spark] object CoarseGrainedSchedulerBackend {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-26089][CORE] Handle corruption in large shuffle blocks

2019-03-12 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 688b0c0  [SPARK-26089][CORE] Handle corruption in large shuffle blocks
688b0c0 is described below

commit 688b0c01fac0db80f6473181673a89f1ce1be65b
Author: ankurgupta 
AuthorDate: Tue Mar 12 14:27:44 2019 -0500

[SPARK-26089][CORE] Handle corruption in large shuffle blocks

## What changes were proposed in this pull request?

SPARK-4105 added corruption detection in shuffle blocks but that was 
limited to blocks which are
smaller than maxBytesInFlight/3. This commit adds upon that by adding 
corruption check for large
blocks. There are two changes/improvements that are made in this commit:

1. Large blocks are checked upto maxBytesInFlight/3 size in a similar way 
as smaller blocks, so if a
large block is corrupt in the starting, that block will be re-fetched and 
if that also fails,
FetchFailureException will be thrown.
2. If large blocks are corrupt after size maxBytesInFlight/3, then any 
IOException thrown while
reading the stream will be converted to FetchFailureException.  This is 
slightly more aggressive
than was originally intended but since the consumer of the stream may have 
already read some records and processed them, we can't just re-fetch the block, 
we need to fail the whole task. Additionally, we also thought about maybe 
adding a new type of TaskEndReason, which would re-try the task couple of times 
before failing the previous stage, but given the complexity involved in that 
solution we decided to not proceed in that direction.

Thanks to squito for direction and support.

## How was this patch tested?

Changed the junit test for big blocks to check for corruption.

Closes #23453 from ankuriitg/ankurgupta/SPARK-26089.

Authored-by: ankurgupta 
Signed-off-by: Imran Rashid 
---
 .../org/apache/spark/internal/config/package.scala |   9 ++
 .../spark/shuffle/BlockStoreShuffleReader.scala|   1 +
 .../storage/ShuffleBlockFetcherIterator.scala  |  94 +
 .../main/scala/org/apache/spark/util/Utils.scala   |  45 +++
 .../storage/ShuffleBlockFetcherIteratorSuite.scala | 148 ++---
 .../scala/org/apache/spark/util/UtilsSuite.scala   |  54 +++-
 6 files changed, 306 insertions(+), 45 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 d6a359d..850d684 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
@@ -928,6 +928,15 @@ package object config {
   .booleanConf
   .createWithDefault(true)
 
+  private[spark] val SHUFFLE_DETECT_CORRUPT_MEMORY =
+ConfigBuilder("spark.shuffle.detectCorrupt.useExtraMemory")
+  .doc("If enabled, part of a compressed/encrypted stream will be 
de-compressed/de-crypted " +
+"by using extra memory to detect early corruption. Any IOException 
thrown will cause " +
+"the task to be retried once and if it fails again with same 
exception, then " +
+"FetchFailedException will be thrown to retry previous stage")
+  .booleanConf
+  .createWithDefault(false)
+
   private[spark] val SHUFFLE_SYNC =
 ConfigBuilder("spark.shuffle.sync")
   .doc("Whether to force outstanding writes to disk.")
diff --git 
a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala 
b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
index c5eefc7..c784371 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
@@ -55,6 +55,7 @@ private[spark] class BlockStoreShuffleReader[K, C](
   SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS),
   SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM),
   SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT),
+  SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT_MEMORY),
   readMetrics).toCompletionIterator
 
 val serializerInstance = dep.serializer.newInstance()
diff --git 
a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
 
b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
index c75b209..c89d5cc 100644
--- 
a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ 
b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.storage
 
-import java.io.{InputStream, IOException}
+

[spark] branch branch-2.3 updated: [SPARK-23433][SPARK-25250][CORE][BRANCH-2.3] Later created TaskSet should learn about the finished partitions

2019-03-07 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new a1ca566  [SPARK-23433][SPARK-25250][CORE][BRANCH-2.3] Later created 
TaskSet should learn about the finished partitions
a1ca566 is described below

commit a1ca5663725c278b6e3785042348819a25496fe4
Author: wuyi 
AuthorDate: Thu Mar 7 12:30:37 2019 -0600

[SPARK-23433][SPARK-25250][CORE][BRANCH-2.3] Later created TaskSet should 
learn about the finished partitions

## What changes were proposed in this pull request?

This is an optional solution for #22806 .

#21131 firstly implement that a previous successful completed task from 
zombie TaskSetManager could also succeed the active TaskSetManager, which based 
on an assumption that an active TaskSetManager always exists for that stage 
when this happen. But that's not always true as an active TaskSetManager may 
haven't been created when a previous task succeed, and this is the reason why 
#22806 hit the issue.

This pr extends #21131 's behavior by adding stageIdToFinishedPartitions 
into TaskSchedulerImpl, which recording the finished partition whenever a 
task(from zombie or active) succeed. Thus, a later created active 
TaskSetManager could also learn about the finished partition by looking into 
stageIdToFinishedPartitions and won't launch any duplicate tasks.

## How was this patch tested?

Add.

Closes #24007 from Ngone51/dev-23433-25250-branch-2.3.

Authored-by: wuyi 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 36 --
 .../apache/spark/scheduler/TaskSetManager.scala| 19 +++---
 .../spark/scheduler/TaskSchedulerImplSuite.scala   | 44 --
 3 files changed, 79 insertions(+), 20 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index edf79aa..ef3ce87 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -23,7 +23,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
 import java.util.concurrent.atomic.AtomicLong
 
 import scala.collection.Set
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.collection.mutable.{ArrayBuffer, BitSet, HashMap, HashSet}
 import scala.util.Random
 
 import org.apache.spark._
@@ -93,6 +93,9 @@ private[spark] class TaskSchedulerImpl(
   private[scheduler] val taskIdToTaskSetManager = new ConcurrentHashMap[Long, 
TaskSetManager]
   val taskIdToExecutorId = new HashMap[Long, String]
 
+  // Protected by `this`
+  private[scheduler] val stageIdToFinishedPartitions = new HashMap[Int, BitSet]
+
   @volatile private var hasReceivedTask = false
   @volatile private var hasLaunchedTask = false
   private val starvationTimer = new Timer(true)
@@ -223,7 +226,20 @@ private[spark] class TaskSchedulerImpl(
   private[scheduler] def createTaskSetManager(
   taskSet: TaskSet,
   maxTaskFailures: Int): TaskSetManager = {
-new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt)
+// only create a BitSet once for a certain stage since we only remove
+// that stage when an active TaskSetManager succeed.
+stageIdToFinishedPartitions.getOrElseUpdate(taskSet.stageId, new BitSet)
+val tsm = new TaskSetManager(this, taskSet, maxTaskFailures, 
blacklistTrackerOpt)
+// TaskSet got submitted by DAGScheduler may have some already completed
+// tasks since DAGScheduler does not always know all the tasks that have
+// been completed by other tasksets when completing a stage, so we mark
+// those tasks as finished here to avoid launching duplicate tasks, while
+// holding the TaskSchedulerImpl lock.
+// See SPARK-25250 and `markPartitionCompletedInAllTaskSets()`
+stageIdToFinishedPartitions.get(taskSet.stageId).foreach {
+  finishedPartitions => 
finishedPartitions.foreach(tsm.markPartitionCompleted(_, None))
+}
+tsm
   }
 
   override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = 
synchronized {
@@ -696,19 +712,31 @@ private[spark] class TaskSchedulerImpl(
   }
 
   /**
-   * Marks the task has completed in all TaskSetManagers for the given stage.
+   * Marks the task has completed in all TaskSetManagers(active / zombie) for 
the given stage.
*
* After stage failure and retry, there may be multiple TaskSetManagers for 
the stage.
* If an earlier attempt of a stage completes a task, we should ensure that 
the later attempts
* do not also submit those same tasks.  That also means that a task 
completion from an earlier
* attempt can lead to the entire stage gett

[spark] branch branch-2.4 updated: [SPARK-27065][CORE] avoid more than one active task set managers for a stage

2019-03-06 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 7df5aa6  [SPARK-27065][CORE] avoid more than one active task set 
managers for a stage
7df5aa6 is described below

commit 7df5aa6b8a81478e25532064e1177c37027ec423
Author: Wenchen Fan 
AuthorDate: Wed Mar 6 12:00:33 2019 -0600

[SPARK-27065][CORE] avoid more than one active task set managers for a stage

## What changes were proposed in this pull request?

This is another attempt to fix the more-than-one-active-task-set-managers 
bug.

https://github.com/apache/spark/pull/17208 is the first attempt. It marks 
the TSM as zombie before sending a task completion event to DAGScheduler. This 
is necessary, because when the DAGScheduler gets the task completion event, and 
it's for the last partition, then the stage is finished. However, if it's a 
shuffle stage and it has missing map outputs, DAGScheduler will resubmit it(see 
the 
[code](https://github.com/apache/spark/blob/v2.4.0/core/src/main/scala/org/apache/spark/schedule
 [...]

This fix has a hole: Let's say a stage has 10 partitions and 2 task set 
managers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 
10 and it completes. TSM2 finishes tasks for partitions 1-9, and thinks he is 
still active because he hasn't finished partition 10 yet. However, DAGScheduler 
gets task completion events for all the 10 partitions and thinks the stage is 
finished. Then the same problem occurs: DAGScheduler may resubmit the stage and 
cause more than one ac [...]

https://github.com/apache/spark/pull/21131 fixed this hole by notifying all 
the task set managers when a task finishes. For the above case, TSM2 will know 
that partition 10 is already completed, so he can mark himself as zombie after 
partitions 1-9 are completed.

However, #21131 still has a hole: TSM2 may be created after the task from 
TSM1 is completed. Then TSM2 can't get notified about the task completion, and 
leads to the more than one active TSM error.

#22806 and #23871 are created to fix this hole. However the fix is 
complicated and there are still ongoing discussions.

This PR proposes a simple fix, which can be easy to backport: mark all 
existing task set managers as zombie when trying to create a new task set 
manager.

After this PR, #21131 is still necessary, to avoid launching unnecessary 
tasks and fix [SPARK-25250](https://issues.apache.org/jira/browse/SPARK-25250 
). #22806 and #23871 are its followups to fix the hole.

## How was this patch tested?

existing tests.

Closes #23927 from cloud-fan/scheduler.

Authored-by: Wenchen Fan 
Signed-off-by: Imran Rashid 
(cherry picked from commit cb20fbc43e7f54af1ed30b9eb6d76ca50b4eb750)
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 20 -
 .../spark/scheduler/TaskSchedulerImplSuite.scala   | 33 ++
 2 files changed, 35 insertions(+), 18 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 4d45fd2..382a417 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -207,14 +207,20 @@ private[spark] class TaskSchedulerImpl(
   val stage = taskSet.stageId
   val stageTaskSets =
 taskSetsByStageIdAndAttempt.getOrElseUpdate(stage, new HashMap[Int, 
TaskSetManager])
-  stageTaskSets(taskSet.stageAttemptId) = manager
-  val conflictingTaskSet = stageTaskSets.exists { case (_, ts) =>
-ts.taskSet != taskSet && !ts.isZombie
-  }
-  if (conflictingTaskSet) {
-throw new IllegalStateException(s"more than one active taskSet for 
stage $stage:" +
-  s" ${stageTaskSets.toSeq.map{_._2.taskSet.id}.mkString(",")}")
+
+  // Mark all the existing TaskSetManagers of this stage as zombie, as we 
are adding a new one.
+  // This is necessary to handle a corner case. Let's say a stage has 10 
partitions and has 2
+  // TaskSetManagers: TSM1(zombie) and TSM2(active). TSM1 has a running 
task for partition 10
+  // and it completes. TSM2 finishes tasks for partition 1-9, and thinks 
he is still active
+  // because partition 10 is not completed yet. However, DAGScheduler gets 
task completion
+  // events for all the 10 partitions and thinks the stage is finished. If 
it's a shuffle stage
+  // and somehow it has missing map outputs, then DAGScheduler will 
resubmit it and create a
+  // TSM3 for it. As a stage can't have more than one active task s

[spark] branch branch-2.3 updated: [SPARK-27065][CORE] avoid more than one active task set managers for a stage

2019-03-06 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
 new 877b8db  [SPARK-27065][CORE] avoid more than one active task set 
managers for a stage
877b8db is described below

commit 877b8db25b70ffb0793a619342e7b8edda712b31
Author: Wenchen Fan 
AuthorDate: Wed Mar 6 12:00:33 2019 -0600

[SPARK-27065][CORE] avoid more than one active task set managers for a stage

## What changes were proposed in this pull request?

This is another attempt to fix the more-than-one-active-task-set-managers 
bug.

https://github.com/apache/spark/pull/17208 is the first attempt. It marks 
the TSM as zombie before sending a task completion event to DAGScheduler. This 
is necessary, because when the DAGScheduler gets the task completion event, and 
it's for the last partition, then the stage is finished. However, if it's a 
shuffle stage and it has missing map outputs, DAGScheduler will resubmit it(see 
the 
[code](https://github.com/apache/spark/blob/v2.4.0/core/src/main/scala/org/apache/spark/schedule
 [...]

This fix has a hole: Let's say a stage has 10 partitions and 2 task set 
managers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 
10 and it completes. TSM2 finishes tasks for partitions 1-9, and thinks he is 
still active because he hasn't finished partition 10 yet. However, DAGScheduler 
gets task completion events for all the 10 partitions and thinks the stage is 
finished. Then the same problem occurs: DAGScheduler may resubmit the stage and 
cause more than one ac [...]

https://github.com/apache/spark/pull/21131 fixed this hole by notifying all 
the task set managers when a task finishes. For the above case, TSM2 will know 
that partition 10 is already completed, so he can mark himself as zombie after 
partitions 1-9 are completed.

However, #21131 still has a hole: TSM2 may be created after the task from 
TSM1 is completed. Then TSM2 can't get notified about the task completion, and 
leads to the more than one active TSM error.

#22806 and #23871 are created to fix this hole. However the fix is 
complicated and there are still ongoing discussions.

This PR proposes a simple fix, which can be easy to backport: mark all 
existing task set managers as zombie when trying to create a new task set 
manager.

After this PR, #21131 is still necessary, to avoid launching unnecessary 
tasks and fix [SPARK-25250](https://issues.apache.org/jira/browse/SPARK-25250 
). #22806 and #23871 are its followups to fix the hole.

## How was this patch tested?

existing tests.

Closes #23927 from cloud-fan/scheduler.

Authored-by: Wenchen Fan 
Signed-off-by: Imran Rashid 
(cherry picked from commit cb20fbc43e7f54af1ed30b9eb6d76ca50b4eb750)
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 20 -
 .../spark/scheduler/TaskSchedulerImplSuite.scala   | 33 ++
 2 files changed, 35 insertions(+), 18 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 4edc6b2..edf79aa 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -185,14 +185,20 @@ private[spark] class TaskSchedulerImpl(
   val stage = taskSet.stageId
   val stageTaskSets =
 taskSetsByStageIdAndAttempt.getOrElseUpdate(stage, new HashMap[Int, 
TaskSetManager])
-  stageTaskSets(taskSet.stageAttemptId) = manager
-  val conflictingTaskSet = stageTaskSets.exists { case (_, ts) =>
-ts.taskSet != taskSet && !ts.isZombie
-  }
-  if (conflictingTaskSet) {
-throw new IllegalStateException(s"more than one active taskSet for 
stage $stage:" +
-  s" ${stageTaskSets.toSeq.map{_._2.taskSet.id}.mkString(",")}")
+
+  // Mark all the existing TaskSetManagers of this stage as zombie, as we 
are adding a new one.
+  // This is necessary to handle a corner case. Let's say a stage has 10 
partitions and has 2
+  // TaskSetManagers: TSM1(zombie) and TSM2(active). TSM1 has a running 
task for partition 10
+  // and it completes. TSM2 finishes tasks for partition 1-9, and thinks 
he is still active
+  // because partition 10 is not completed yet. However, DAGScheduler gets 
task completion
+  // events for all the 10 partitions and thinks the stage is finished. If 
it's a shuffle stage
+  // and somehow it has missing map outputs, then DAGScheduler will 
resubmit it and create a
+  // TSM3 for it. As a stage can't have more than one active task s

[spark] branch master updated: [SPARK-27065][CORE] avoid more than one active task set managers for a stage

2019-03-06 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 cb20fbc  [SPARK-27065][CORE] avoid more than one active task set 
managers for a stage
cb20fbc is described below

commit cb20fbc43e7f54af1ed30b9eb6d76ca50b4eb750
Author: Wenchen Fan 
AuthorDate: Wed Mar 6 12:00:33 2019 -0600

[SPARK-27065][CORE] avoid more than one active task set managers for a stage

## What changes were proposed in this pull request?

This is another attempt to fix the more-than-one-active-task-set-managers 
bug.

https://github.com/apache/spark/pull/17208 is the first attempt. It marks 
the TSM as zombie before sending a task completion event to DAGScheduler. This 
is necessary, because when the DAGScheduler gets the task completion event, and 
it's for the last partition, then the stage is finished. However, if it's a 
shuffle stage and it has missing map outputs, DAGScheduler will resubmit it(see 
the 
[code](https://github.com/apache/spark/blob/v2.4.0/core/src/main/scala/org/apache/spark/schedule
 [...]

This fix has a hole: Let's say a stage has 10 partitions and 2 task set 
managers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 
10 and it completes. TSM2 finishes tasks for partitions 1-9, and thinks he is 
still active because he hasn't finished partition 10 yet. However, DAGScheduler 
gets task completion events for all the 10 partitions and thinks the stage is 
finished. Then the same problem occurs: DAGScheduler may resubmit the stage and 
cause more than one ac [...]

https://github.com/apache/spark/pull/21131 fixed this hole by notifying all 
the task set managers when a task finishes. For the above case, TSM2 will know 
that partition 10 is already completed, so he can mark himself as zombie after 
partitions 1-9 are completed.

However, #21131 still has a hole: TSM2 may be created after the task from 
TSM1 is completed. Then TSM2 can't get notified about the task completion, and 
leads to the more than one active TSM error.

#22806 and #23871 are created to fix this hole. However the fix is 
complicated and there are still ongoing discussions.

This PR proposes a simple fix, which can be easy to backport: mark all 
existing task set managers as zombie when trying to create a new task set 
manager.

After this PR, #21131 is still necessary, to avoid launching unnecessary 
tasks and fix [SPARK-25250](https://issues.apache.org/jira/browse/SPARK-25250 
). #22806 and #23871 are its followups to fix the hole.

## How was this patch tested?

existing tests.

Closes #23927 from cloud-fan/scheduler.

Authored-by: Wenchen Fan 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 20 -
 .../spark/scheduler/TaskSchedulerImplSuite.scala   | 33 ++
 2 files changed, 35 insertions(+), 18 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index c23b201..db0fbfe 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -215,14 +215,20 @@ private[spark] class TaskSchedulerImpl(
   val stage = taskSet.stageId
   val stageTaskSets =
 taskSetsByStageIdAndAttempt.getOrElseUpdate(stage, new HashMap[Int, 
TaskSetManager])
-  stageTaskSets(taskSet.stageAttemptId) = manager
-  val conflictingTaskSet = stageTaskSets.exists { case (_, ts) =>
-ts.taskSet != taskSet && !ts.isZombie
-  }
-  if (conflictingTaskSet) {
-throw new IllegalStateException(s"more than one active taskSet for 
stage $stage:" +
-  s" ${stageTaskSets.toSeq.map{_._2.taskSet.id}.mkString(",")}")
+
+  // Mark all the existing TaskSetManagers of this stage as zombie, as we 
are adding a new one.
+  // This is necessary to handle a corner case. Let's say a stage has 10 
partitions and has 2
+  // TaskSetManagers: TSM1(zombie) and TSM2(active). TSM1 has a running 
task for partition 10
+  // and it completes. TSM2 finishes tasks for partition 1-9, and thinks 
he is still active
+  // because partition 10 is not completed yet. However, DAGScheduler gets 
task completion
+  // events for all the 10 partitions and thinks the stage is finished. If 
it's a shuffle stage
+  // and somehow it has missing map outputs, then DAGScheduler will 
resubmit it and create a
+  // TSM3 for it. As a stage can't have more than one active task set 
managers, we must mark
+  // TSM2 as zombie (it actually is).
+  stageTaskSets.foreach { ca

[spark] branch branch-2.4 updated: [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions

2019-03-06 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 db86ccb  [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should 
learn about the finished partitions
db86ccb is described below

commit db86ccb11821231d85b727fb889dec1d58b39e4d
Author: wuyi 
AuthorDate: Wed Mar 6 11:53:07 2019 -0600

[SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about 
the finished partitions

## What changes were proposed in this pull request?

This is an optional solution for #22806 .

#21131 firstly implement that a previous successful completed task from 
zombie TaskSetManager could also succeed the active TaskSetManager, which based 
on an assumption that an active TaskSetManager always exists for that stage 
when this happen.  But that's not always true as an active TaskSetManager may 
haven't been created when a previous task succeed, and this is the reason why 
#22806 hit the issue.

This pr extends #21131 's behavior by adding `stageIdToFinishedPartitions` 
into TaskSchedulerImpl, which recording the finished partition whenever a 
task(from zombie or active) succeed. Thus, a later created active 
TaskSetManager could also learn about the finished partition by looking into 
`stageIdToFinishedPartitions ` and won't launch any duplicate tasks.

## How was this patch tested?

Add.

Closes #23871 from Ngone51/dev-23433-25250.

Lead-authored-by: wuyi 
Co-authored-by: Ngone51 
Signed-off-by: Imran Rashid 
(cherry picked from commit e5c61436a5720f13eb6d530ebf80635522bd64c6)
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 36 --
 .../apache/spark/scheduler/TaskSetManager.scala| 19 +++---
 .../spark/scheduler/TaskSchedulerImplSuite.scala   | 44 --
 3 files changed, 79 insertions(+), 20 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 24d77f8..4d45fd2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -23,7 +23,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
 import java.util.concurrent.atomic.AtomicLong
 
 import scala.collection.Set
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.collection.mutable.{ArrayBuffer, BitSet, HashMap, HashSet}
 import scala.util.Random
 
 import org.apache.spark._
@@ -94,6 +94,9 @@ private[spark] class TaskSchedulerImpl(
   private[scheduler] val taskIdToTaskSetManager = new ConcurrentHashMap[Long, 
TaskSetManager]
   val taskIdToExecutorId = new HashMap[Long, String]
 
+  // Protected by `this`
+  private[scheduler] val stageIdToFinishedPartitions = new HashMap[Int, BitSet]
+
   @volatile private var hasReceivedTask = false
   @volatile private var hasLaunchedTask = false
   private val starvationTimer = new Timer(true)
@@ -236,7 +239,20 @@ private[spark] class TaskSchedulerImpl(
   private[scheduler] def createTaskSetManager(
   taskSet: TaskSet,
   maxTaskFailures: Int): TaskSetManager = {
-new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt)
+// only create a BitSet once for a certain stage since we only remove
+// that stage when an active TaskSetManager succeed.
+stageIdToFinishedPartitions.getOrElseUpdate(taskSet.stageId, new BitSet)
+val tsm = new TaskSetManager(this, taskSet, maxTaskFailures, 
blacklistTrackerOpt)
+// TaskSet got submitted by DAGScheduler may have some already completed
+// tasks since DAGScheduler does not always know all the tasks that have
+// been completed by other tasksets when completing a stage, so we mark
+// those tasks as finished here to avoid launching duplicate tasks, while
+// holding the TaskSchedulerImpl lock.
+// See SPARK-25250 and `markPartitionCompletedInAllTaskSets()`
+stageIdToFinishedPartitions.get(taskSet.stageId).foreach {
+  finishedPartitions => 
finishedPartitions.foreach(tsm.markPartitionCompleted(_, None))
+}
+tsm
   }
 
   override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = 
synchronized {
@@ -833,19 +849,31 @@ private[spark] class TaskSchedulerImpl(
   }
 
   /**
-   * Marks the task has completed in all TaskSetManagers for the given stage.
+   * Marks the task has completed in all TaskSetManagers(active / zombie) for 
the given stage.
*
* After stage failure and retry, there may be multiple TaskSetManagers for 
the stage.
* If an earlier attempt of a stage completes a task, we should ensure that 
the later attempts
* do not also submit those same ta

[spark] branch master updated: [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions

2019-03-06 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 e5c6143  [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should 
learn about the finished partitions
e5c6143 is described below

commit e5c61436a5720f13eb6d530ebf80635522bd64c6
Author: wuyi 
AuthorDate: Wed Mar 6 11:53:07 2019 -0600

[SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about 
the finished partitions

## What changes were proposed in this pull request?

This is an optional solution for #22806 .

#21131 firstly implement that a previous successful completed task from 
zombie TaskSetManager could also succeed the active TaskSetManager, which based 
on an assumption that an active TaskSetManager always exists for that stage 
when this happen.  But that's not always true as an active TaskSetManager may 
haven't been created when a previous task succeed, and this is the reason why 
#22806 hit the issue.

This pr extends #21131 's behavior by adding `stageIdToFinishedPartitions` 
into TaskSchedulerImpl, which recording the finished partition whenever a 
task(from zombie or active) succeed. Thus, a later created active 
TaskSetManager could also learn about the finished partition by looking into 
`stageIdToFinishedPartitions ` and won't launch any duplicate tasks.

## How was this patch tested?

Add.

Closes #23871 from Ngone51/dev-23433-25250.

Lead-authored-by: wuyi 
Co-authored-by: Ngone51 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/scheduler/TaskSchedulerImpl.scala | 36 --
 .../apache/spark/scheduler/TaskSetManager.scala| 19 +++---
 .../spark/scheduler/TaskSchedulerImplSuite.scala   | 44 --
 3 files changed, 79 insertions(+), 20 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 3f23bfe..c23b201 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -23,7 +23,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
 import java.util.concurrent.atomic.AtomicLong
 
 import scala.collection.Set
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.collection.mutable.{ArrayBuffer, BitSet, HashMap, HashSet}
 import scala.util.Random
 
 import org.apache.spark._
@@ -102,6 +102,9 @@ private[spark] class TaskSchedulerImpl(
   // Protected by `this`
   val taskIdToExecutorId = new HashMap[Long, String]
 
+  // Protected by `this`
+  private[scheduler] val stageIdToFinishedPartitions = new HashMap[Int, BitSet]
+
   @volatile private var hasReceivedTask = false
   @volatile private var hasLaunchedTask = false
   private val starvationTimer = new Timer(true)
@@ -244,7 +247,20 @@ private[spark] class TaskSchedulerImpl(
   private[scheduler] def createTaskSetManager(
   taskSet: TaskSet,
   maxTaskFailures: Int): TaskSetManager = {
-new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt)
+// only create a BitSet once for a certain stage since we only remove
+// that stage when an active TaskSetManager succeed.
+stageIdToFinishedPartitions.getOrElseUpdate(taskSet.stageId, new BitSet)
+val tsm = new TaskSetManager(this, taskSet, maxTaskFailures, 
blacklistTrackerOpt)
+// TaskSet got submitted by DAGScheduler may have some already completed
+// tasks since DAGScheduler does not always know all the tasks that have
+// been completed by other tasksets when completing a stage, so we mark
+// those tasks as finished here to avoid launching duplicate tasks, while
+// holding the TaskSchedulerImpl lock.
+// See SPARK-25250 and `markPartitionCompletedInAllTaskSets()`
+stageIdToFinishedPartitions.get(taskSet.stageId).foreach {
+  finishedPartitions => 
finishedPartitions.foreach(tsm.markPartitionCompleted(_, None))
+}
+tsm
   }
 
   override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = 
synchronized {
@@ -847,19 +863,31 @@ private[spark] class TaskSchedulerImpl(
   }
 
   /**
-   * Marks the task has completed in all TaskSetManagers for the given stage.
+   * Marks the task has completed in all TaskSetManagers(active / zombie) for 
the given stage.
*
* After stage failure and retry, there may be multiple TaskSetManagers for 
the stage.
* If an earlier attempt of a stage completes a task, we should ensure that 
the later attempts
* do not also submit those same tasks.  That also means that a task 
completion from an earlier
* attempt can lead to the entire stage getting marked as successful.
+   * And there is also the possibil

[spark] branch master updated: [SPARK-25865][CORE] Add GC information to ExecutorMetrics

2019-03-04 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 e5c502c  [SPARK-25865][CORE] Add GC information to ExecutorMetrics
e5c502c is described below

commit e5c502c596563dce8eb58f86e42c1aea2c51ed17
Author: LantaoJin 
AuthorDate: Mon Mar 4 14:26:02 2019 -0600

[SPARK-25865][CORE] Add GC information to ExecutorMetrics

## What changes were proposed in this pull request?

Only memory usage without GC information could not help us to determinate 
the proper settings of memory. We need the GC metrics about frequency of major 
& minor GC. For example, two cases, their configured memory for executor are 
all 10GB and their usages are all near 10GB. So should we increase or decrease 
the configured memory for them? This metrics may be helpful. We can increase 
configured memory for the first one if it has very frequency major GC and 
decrease the second one if on [...]
GC metrics are only useful in entire lifetime of executors instead of 
separated stages.

## How was this patch tested?

Adding UT.

Closes #22874 from LantaoJin/SPARK-25865.

Authored-by: LantaoJin 
Signed-off-by: Imran Rashid 
---
 .../org/apache/spark/internal/config/package.scala |  19 ++
 .../apache/spark/metrics/ExecutorMetricType.scala  |  63 +-
 .../application_list_json_expectation.json |  18 ++
 .../completed_app_list_json_expectation.json   |  18 ++
 ...arbage_collection_metrics_json_expectation.json | 122 
 ...ist_with_executor_metrics_json_expectation.json |  30 ++-
 ...utor_process_tree_metrics_json_expectation.json |  12 +-
 .../limit_app_list_json_expectation.json   |  33 ++--
 .../minDate_app_list_json_expectation.json |  18 ++
 .../minEndDate_app_list_json_expectation.json  |  18 ++
 .../spark-events/application_1536831636016_59384_1 | 214 +
 .../spark/deploy/history/HistoryServerSuite.scala  |   2 +
 .../scheduler/EventLoggingListenerSuite.scala  |  36 ++--
 .../org/apache/spark/util/JsonProtocolSuite.scala  |  18 +-
 dev/.rat-excludes  |   1 +
 15 files changed, 576 insertions(+), 46 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 3ef6cba..d6a359d 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
@@ -20,6 +20,7 @@ package org.apache.spark.internal
 import java.util.concurrent.TimeUnit
 
 import org.apache.spark.launcher.SparkLauncher
+import org.apache.spark.metrics.GarbageCollectionMetrics
 import org.apache.spark.network.util.ByteUnit
 import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode}
 import org.apache.spark.storage.{DefaultTopologyMapper, 
RandomBlockReplicationPolicy}
@@ -114,6 +115,24 @@ package object config {
   .booleanConf
   .createWithDefault(false)
 
+  private[spark] val EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS =
+ConfigBuilder("spark.eventLog.gcMetrics.youngGenerationGarbageCollectors")
+  .doc("Names of supported young generation garbage collector. A name 
usually is " +
+" the return of GarbageCollectorMXBean.getName. The built-in young 
generation garbage " +
+s"collectors are 
${GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS}")
+  .stringConf
+  .toSequence
+  
.createWithDefault(GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS)
+
+  private[spark] val EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS =
+ConfigBuilder("spark.eventLog.gcMetrics.oldGenerationGarbageCollectors")
+  .doc("Names of supported old generation garbage collector. A name 
usually is " +
+"the return of GarbageCollectorMXBean.getName. The built-in old 
generation garbage " +
+s"collectors are 
${GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS}")
+  .stringConf
+  .toSequence
+  
.createWithDefault(GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS)
+
   private[spark] val EVENT_LOG_OVERWRITE =
 
ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false)
 
diff --git 
a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala 
b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
index 704b36d..6d8e9a8 100644
--- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
@@ -19,9 +19,12 @@ package org.apache.spark.metrics
 import 

[spark] branch master updated: [SPARK-26688][YARN] Provide configuration of initially blacklisted YARN nodes

2019-03-04 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 caceaec  [SPARK-26688][YARN] Provide configuration of initially 
blacklisted YARN nodes
caceaec is described below

commit caceaec93203edaea1d521b88e82ef67094cdea9
Author: “attilapiros” 
AuthorDate: Mon Mar 4 14:14:20 2019 -0600

[SPARK-26688][YARN] Provide configuration of initially blacklisted YARN 
nodes

## What changes were proposed in this pull request?

Introducing new config for initially blacklisted YARN nodes.

## How was this patch tested?

With existing and a new unit test.

Closes #23616 from attilapiros/SPARK-26688.

Lead-authored-by: “attilapiros” 
Co-authored-by: Attila Zsolt Piros 
<2017933+attilapi...@users.noreply.github.com>
Signed-off-by: Imran Rashid 
---
 docs/running-on-yarn.md|  7 +
 .../yarn/YarnAllocatorBlacklistTracker.scala   |  4 ++-
 .../org/apache/spark/deploy/yarn/config.scala  |  6 
 .../yarn/YarnAllocatorBlacklistTrackerSuite.scala  | 33 --
 4 files changed, 41 insertions(+), 9 deletions(-)

diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 6ee4b3d..261f7e3 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -463,6 +463,13 @@ To use a custom metrics.properties for the application 
master and executors, upd
   
 
 
+  spark.yarn.exclude.nodes
+  (none)
+  
+  Comma-separated list of YARN node names which are excluded from resource 
allocation.
+  
+
+
   spark.yarn.metrics.namespace
   (none)
   
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala
index 268976b..fa8c961 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala
@@ -56,6 +56,8 @@ private[spark] class YarnAllocatorBlacklistTracker(
 
   private val maxFailuresPerHost = sparkConf.get(MAX_FAILED_EXEC_PER_NODE)
 
+  private val excludeNodes = sparkConf.get(YARN_EXCLUDE_NODES).toSet
+
   private val allocatorBlacklist = new HashMap[String, Long]()
 
   private var currentBlacklistedYarnNodes = Set.empty[String]
@@ -105,7 +107,7 @@ private[spark] class YarnAllocatorBlacklistTracker(
 
   private def refreshBlacklistedNodes(): Unit = {
 removeExpiredYarnBlacklistedNodes()
-val allBlacklistedNodes = schedulerBlacklist ++ allocatorBlacklist.keySet
+val allBlacklistedNodes = excludeNodes ++ schedulerBlacklist ++ 
allocatorBlacklist.keySet
 synchronizeBlacklistedNodeWithYarn(allBlacklistedNodes)
   }
 
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
index 5f8739b..4c187b2 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -310,6 +310,12 @@ package object config {
   .booleanConf
   .createWithDefault(false)
 
+  /* Initially blacklisted YARN nodes. */
+  private[spark] val YARN_EXCLUDE_NODES = 
ConfigBuilder("spark.yarn.exclude.nodes")
+  .stringConf
+  .toSequence
+  .createWithDefault(Nil)
+
   private[yarn] val YARN_EXECUTOR_RESOURCE_TYPES_PREFIX = 
"spark.yarn.executor.resource."
   private[yarn] val YARN_DRIVER_RESOURCE_TYPES_PREFIX = 
"spark.yarn.driver.resource."
   private[yarn] val YARN_AM_RESOURCE_TYPES_PREFIX = "spark.yarn.am.resource."
diff --git 
a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala
 
b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala
index 259d758..c07a4ac 100644
--- 
a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala
+++ 
b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala
@@ -25,7 +25,7 @@ import org.mockito.Mockito._
 import org.scalatest.{BeforeAndAfterEach, Matchers}
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
-import 
org.apache.spark.deploy.yarn.config.YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED
+import org.apache.spark.deploy.yarn.config.{YARN_EXCLUDE_NODES, 
YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED}
 import org.apache.spark.internal.config.{BLACKLIST_TIMEOUT_CONF, 
MAX_FAILED_EXEC_PER_NODE}
 import org.apache.spark.util.ManualClock
 
@@ -35,27 +35,31 @@ class YarnAllocat

[spark] branch master updated: [SPARK-24522][UI] Create filter to apply HTTP security checks consistently.

2019-01-08 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 2783e4c  [SPARK-24522][UI] Create filter to apply HTTP security checks 
consistently.
2783e4c is described below

commit 2783e4c45f55f4fc87748d1c4a454bfdf3024156
Author: Marcelo Vanzin 
AuthorDate: Tue Jan 8 11:25:33 2019 -0600

[SPARK-24522][UI] Create filter to apply HTTP security checks consistently.

Currently there is code scattered in a bunch of places to do different
things related to HTTP security, such as access control, setting
security-related headers, and filtering out bad content. This makes it
really easy to miss these things when writing new UI code.

This change creates a new filter that does all of those things, and
makes sure that all servlet handlers that are attached to the UI get
the new filter and any user-defined filters consistently. The extent
of the actual features should be the same as before.

The new filter is added at the end of the filter chain, because 
authentication
is done by custom filters and thus needs to happen first. This means that
custom filters see unfiltered HTTP requests - which is actually the current
behavior anyway.

As a side-effect of some of the code refactoring, handlers added after
the initial set also get wrapped with a GzipHandler, which didn't happen
before.

Tested with added unit tests and in a history server with SPNEGO auth
configured.

Closes #23302 from vanzin/SPARK-24522.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 
---
 .../apache/spark/deploy/history/HistoryPage.scala  |   5 +-
 .../spark/deploy/history/HistoryServer.scala   |   8 +-
 .../spark/deploy/master/ui/ApplicationPage.scala   |   3 +-
 .../apache/spark/deploy/master/ui/MasterPage.scala |   6 +-
 .../apache/spark/deploy/worker/ui/LogPage.scala|  28 ++--
 .../spark/deploy/worker/ui/WorkerWebUI.scala   |   1 -
 .../apache/spark/metrics/sink/MetricsServlet.scala |   2 +-
 .../spark/status/api/v1/SecurityFilter.scala   |  36 -
 .../org/apache/spark/ui/HttpSecurityFilter.scala   | 116 +++
 .../scala/org/apache/spark/ui/JettyUtils.scala | 154 +---
 .../main/scala/org/apache/spark/ui/UIUtils.scala   |  21 ---
 .../src/main/scala/org/apache/spark/ui/WebUI.scala |  15 +-
 .../spark/ui/exec/ExecutorThreadDumpPage.scala |   4 +-
 .../org/apache/spark/ui/jobs/AllJobsPage.scala |  16 +--
 .../scala/org/apache/spark/ui/jobs/JobPage.scala   |   3 +-
 .../scala/org/apache/spark/ui/jobs/JobsTab.scala   |   4 +-
 .../scala/org/apache/spark/ui/jobs/PoolPage.scala  |   3 +-
 .../scala/org/apache/spark/ui/jobs/StagePage.scala |  19 ++-
 .../org/apache/spark/ui/jobs/StageTable.scala  |  15 +-
 .../scala/org/apache/spark/ui/jobs/StagesTab.scala |   4 +-
 .../org/apache/spark/ui/storage/RDDPage.scala  |  11 +-
 .../apache/spark/ui/HttpSecurityFilterSuite.scala  | 157 +
 .../test/scala/org/apache/spark/ui/UISuite.scala   | 147 +--
 .../scala/org/apache/spark/ui/UIUtilsSuite.scala   |  39 -
 .../apache/spark/deploy/mesos/ui/DriverPage.scala  |   3 +-
 .../scheduler/cluster/YarnSchedulerBackend.scala   |  35 -
 .../cluster/YarnSchedulerBackendSuite.scala|  59 +++-
 .../spark/sql/execution/ui/AllExecutionsPage.scala |  19 +--
 .../spark/sql/execution/ui/ExecutionPage.scala |   3 +-
 .../thriftserver/ui/ThriftServerSessionPage.scala  |   3 +-
 .../org/apache/spark/streaming/ui/BatchPage.scala  |   8 +-
 31 files changed, 609 insertions(+), 338 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala 
b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
index 00ca4ef..7a8ab7f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
@@ -27,9 +27,8 @@ import org.apache.spark.ui.{UIUtils, WebUIPage}
 private[history] class HistoryPage(parent: HistoryServer) extends 
WebUIPage("") {
 
   def render(request: HttpServletRequest): Seq[Node] = {
-// stripXSS is called first to remove suspicious characters used in XSS 
attacks
-val requestedIncomplete =
-  
Option(UIUtils.stripXSS(request.getParameter("showIncomplete"))).getOrElse("false").toBoolean
+val requestedIncomplete = Option(request.getParameter("showIncomplete"))
+  .getOrElse("false").toBoolean
 
 val displayApplications = parent.getApplicationList()
   .exists(isApplicationCompleted(_) != requestedIncomplete)
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala 
b/core/src/main/scala/org/apa

[spark] branch master updated: [SPARK-25689][YARN] Make driver, not AM, manage delegation tokens.

2019-01-07 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 669e8a1  [SPARK-25689][YARN] Make driver, not AM, manage delegation 
tokens.
669e8a1 is described below

commit 669e8a155987995a1a5d49a96b88c05f39e41723
Author: Marcelo Vanzin 
AuthorDate: Mon Jan 7 14:40:08 2019 -0600

[SPARK-25689][YARN] Make driver, not AM, manage delegation tokens.

This change modifies the behavior of the delegation token code when running
on YARN, so that the driver controls the renewal, in both client and cluster
mode. For that, a few different things were changed:

* The AM code only runs code that needs DTs when DTs are available.

In a way, this restores the AM behavior to what it was pre-SPARK-23361, but
keeping the fix added in that bug. Basically, all the AM code is run in a
"UGI.doAs()" block; but code that needs to talk to HDFS (basically the
distributed cache handling code) was delayed to the point where the driver
is up and running, and thus when valid delegation tokens are available.

* SparkSubmit / ApplicationMaster now handle user login, not the token 
manager.

The previous AM code was relying on the token manager to keep the user
logged in when keytabs are used. This required some odd APIs in the token
manager and the AM so that the right UGI was exposed and used in the right
places.

After this change, the logged in user is handled separately from the token
manager, so the API was cleaned up, and, as explained above, the whole AM
runs under the logged in user, which also helps with simplifying some more 
code.

* Distributed cache configs are sent separately to the AM.

Because of the delayed initialization of the cached resources in the AM, it
became easier to write the cache config to a separate properties file 
instead
of bundling it with the rest of the Spark config. This also avoids having
to modify the SparkConf to hide things from the UI.

* Finally, the AM doesn't manage the token manager anymore.

The above changes allow the token manager to be completely handled by the
driver's scheduler backend code also in YARN mode (whether client or 
cluster),
making it similar to other RMs. To maintain the fix added in SPARK-23361 
also
in client mode, the AM now sends an extra message to the driver on 
initialization
to fetch delegation tokens; and although it might not really be needed, the
driver also keeps the running AM updated when new tokens are created.

Tested in a kerberized cluster with the same tests used to validate 
SPARK-23361,
in both client and cluster mode. Also tested with a non-kerberized cluster.

Closes #23338 from vanzin/SPARK-25689.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 
---
 .../security/HadoopDelegationTokenManager.scala| 110 ++---
 .../security/HiveDelegationTokenProvider.scala |  16 ++-
 .../cluster/CoarseGrainedClusterMessage.scala  |   3 +
 .../cluster/CoarseGrainedSchedulerBackend.scala|  40 --
 .../HadoopDelegationTokenManagerSuite.scala|   8 +-
 .../features/KerberosConfDriverFeatureStep.scala   |   2 +-
 .../k8s/KubernetesClusterSchedulerBackend.scala|   7 +-
 .../mesos/MesosCoarseGrainedSchedulerBackend.scala |   7 +-
 .../spark/deploy/yarn/ApplicationMaster.scala  | 135 ++---
 .../deploy/yarn/ApplicationMasterArguments.scala   |   5 +
 .../org/apache/spark/deploy/yarn/Client.scala  | 100 ---
 .../apache/spark/deploy/yarn/YarnRMClient.scala|   8 +-
 .../org/apache/spark/deploy/yarn/config.scala  |  10 --
 .../YARNHadoopDelegationTokenManager.scala |   7 +-
 .../cluster/YarnClientSchedulerBackend.scala   |   6 +
 .../scheduler/cluster/YarnSchedulerBackend.scala   |  17 ++-
 .../YARNHadoopDelegationTokenManagerSuite.scala|   2 +-
 17 files changed, 246 insertions(+), 237 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
index f7e3dde..d97857a 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
@@ -21,7 +21,6 @@ import java.io.File
 import java.net.URI
 import java.security.PrivilegedExceptionAction
 import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
-import java.util.concurrent.atomic.AtomicReference
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileSystem
@@ -39,32 +38,24 @@ import org.apache.spark.util.ThreadUtils
 /**
 

[spark] branch master updated: [SPARK-24958][CORE] Add memory from procfs to executor metrics.

2018-12-10 Thread irashid
This is an automated email from the ASF dual-hosted git repository.

irashid 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 90c77ea  [SPARK-24958][CORE] Add memory from procfs to executor 
metrics.
90c77ea is described below

commit 90c77ea3132d0b7a12c316bd42fb8d0f59bee253
Author: Reza Safi 
AuthorDate: Mon Dec 10 11:14:11 2018 -0600

[SPARK-24958][CORE] Add memory from procfs to executor metrics.

This adds the entire memory used by spark’s executor (as measured by 
procfs) to the executor metrics.  The memory usage is collected from the entire 
process tree under the executor.  The metrics are subdivided into memory used 
by java, by python, and by other processes, to aid users in diagnosing the 
source of high memory usage.
The additional metrics are sent to the driver in heartbeats, using the 
mechanism introduced by SPARK-23429.  This also slightly extends that approach 
to allow one ExecutorMetricType to collect multiple metrics.

Added unit tests and also tested on a live cluster.

Closes #22612 from rezasafi/ptreememory2.

Authored-by: Reza Safi 
Signed-off-by: Imran Rashid 
---
 .../main/scala/org/apache/spark/Heartbeater.scala  |  11 +-
 .../apache/spark/executor/ExecutorMetrics.scala|  23 +--
 .../spark/executor/ProcfsMetricsGetter.scala   | 228 +
 .../org/apache/spark/internal/config/package.scala |   5 +
 .../apache/spark/metrics/ExecutorMetricType.scala  |  74 ++-
 .../scala/org/apache/spark/status/api/v1/api.scala |   6 +-
 .../scala/org/apache/spark/util/JsonProtocol.scala |  16 +-
 .../application_list_json_expectation.json |  15 ++
 .../completed_app_list_json_expectation.json   |  15 ++
 ...ist_with_executor_metrics_json_expectation.json |  40 +++-
 ...utor_process_tree_metrics_json_expectation.json |  98 +
 .../limit_app_list_json_expectation.json   |  30 +--
 .../minDate_app_list_json_expectation.json |  15 ++
 .../minEndDate_app_list_json_expectation.json  |  15 ++
 core/src/test/resources/ProcfsMetrics/22763/stat   |   1 +
 core/src/test/resources/ProcfsMetrics/26109/stat   |   1 +
 .../spark-events/application_1538416563558_0014| 190 +
 .../spark/deploy/history/HistoryServerSuite.scala  |   3 +
 .../spark/executor/ProcfsMetricsGetterSuite.scala  |  41 
 .../scheduler/EventLoggingListenerSuite.scala  |  85 +---
 .../spark/status/AppStatusListenerSuite.scala  |  74 ---
 .../org/apache/spark/util/JsonProtocolSuite.scala  |  46 +++--
 dev/.rat-excludes  |   2 +
 23 files changed, 901 insertions(+), 133 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala 
b/core/src/main/scala/org/apache/spark/Heartbeater.scala
index 84091ee..1012755 100644
--- a/core/src/main/scala/org/apache/spark/Heartbeater.scala
+++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala
@@ -61,10 +61,17 @@ private[spark] class Heartbeater(
 
   /**
* Get the current executor level metrics. These are returned as an array, 
with the index
-   * determined by ExecutorMetricType.values
+   * determined by ExecutorMetricType.metricToOffset
*/
   def getCurrentMetrics(): ExecutorMetrics = {
-val metrics = 
ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray
+
+val metrics = new Array[Long](ExecutorMetricType.numMetrics)
+var offset = 0
+ExecutorMetricType.metricGetters.foreach { metric =>
+  val newMetrics = metric.getMetricValues(memoryManager)
+  Array.copy(newMetrics, 0, metrics, offset, newMetrics.size)
+  offset += newMetrics.length
+}
 new ExecutorMetrics(metrics)
   }
 }
diff --git 
a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala 
b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
index 1befd27..f19ac81 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
@@ -27,17 +27,15 @@ import org.apache.spark.metrics.ExecutorMetricType
  */
 @DeveloperApi
 class ExecutorMetrics private[spark] extends Serializable {
-
-  // Metrics are indexed by ExecutorMetricType.values
-  private val metrics = new Array[Long](ExecutorMetricType.values.length)
-
+  // Metrics are indexed by ExecutorMetricType.metricToOffset
+  private val metrics = new Array[Long](ExecutorMetricType.numMetrics)
   // the first element is initialized to -1, indicating that the values for 
the array
   // haven't been set yet.
   metrics(0) = -1
 
-  /** Returns the value for the specified metricType. */
-  def getMetricValue(metricType: ExecutorMetricType): Long = {
-metrics(ExecutorMetricType.metricIdxMap(metricType))
+  /** Returns the value for the specified metric. */
+  def getMetricValue(metricName: Str

spark git commit: [SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a ChunkedByteBuffer

2018-11-29 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master cb368f2c2 -> 59741887e


[SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to 
a ChunkedByteBuffer

## What changes were proposed in this pull request?

In `BlockManager`, `getRemoteValues` gets a `ChunkedByteBuffer` (by calling 
`getRemoteBytes`) and creates an `InputStream` from it. `getRemoteBytes`, in 
turn, gets a `ManagedBuffer` and converts it to a `ChunkedByteBuffer`.
Instead, expose a `getRemoteManagedBuffer` method so `getRemoteValues` can just 
get this `ManagedBuffer` and use its `InputStream`.
When reading a remote cache block from disk, this reduces heap memory usage 
significantly.
Retain `getRemoteBytes` for other callers.

## How was this patch tested?

Imran Rashid wrote an application 
(https://github.com/squito/spark_2gb_test/blob/master/src/main/scala/com/cloudera/sparktest/LargeBlocks.scala),
 that among other things, tests reading remote cache blocks. I ran this 
application, using 2500MB blocks, to test reading a cache block on disk. 
Without this change, with `--executor-memory 5g`, the test fails with 
`java.lang.OutOfMemoryError: Java heap space`. With the change, the test passes 
with `--executor-memory 2g`.
I also ran the unit tests in core. In particular, `DistributedSuite` has a set 
of tests that exercise the `getRemoteValues` code path. `BlockManagerSuite` has 
several tests that call `getRemoteBytes`; I left these unchanged, so 
`getRemoteBytes` still gets exercised.

Closes #23058 from wypoon/SPARK-25905.

Authored-by: Wing Yew Poon 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: 59741887e272be92ebd6e61783f99f7d8fc05456
Parents: cb368f2
Author: Wing Yew Poon 
Authored: Thu Nov 29 14:56:34 2018 -0600
Committer: Imran Rashid 
Committed: Thu Nov 29 14:56:34 2018 -0600

--
 .../org/apache/spark/storage/BlockManager.scala | 43 
 .../spark/util/io/ChunkedByteBuffer.scala   |  2 -
 .../org/apache/spark/DistributedSuite.scala |  2 +-
 3 files changed, 28 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/59741887/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 1b61729..1dfbc6e 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -692,9 +692,9 @@ private[spark] class BlockManager(
*/
   private def getRemoteValues[T: ClassTag](blockId: BlockId): 
Option[BlockResult] = {
 val ct = implicitly[ClassTag[T]]
-getRemoteBytes(blockId).map { data =>
+getRemoteManagedBuffer(blockId).map { data =>
   val values =
-serializerManager.dataDeserializeStream(blockId, 
data.toInputStream(dispose = true))(ct)
+serializerManager.dataDeserializeStream(blockId, 
data.createInputStream())(ct)
   new BlockResult(values, DataReadMethod.Network, data.size)
 }
   }
@@ -717,13 +717,9 @@ private[spark] class BlockManager(
   }
 
   /**
-   * Get block from remote block managers as serialized bytes.
+   * Get block from remote block managers as a ManagedBuffer.
*/
-  def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = {
-// TODO SPARK-25905 if we change this method to return the ManagedBuffer, 
then getRemoteValues
-// could just use the inputStream on the temp file, rather than reading 
the file into memory.
-// Until then, replication can cause the process to use too much memory 
and get killed
-// even though we've read the data to disk.
+  private def getRemoteManagedBuffer(blockId: BlockId): Option[ManagedBuffer] 
= {
 logDebug(s"Getting remote block $blockId")
 require(blockId != null, "BlockId is null")
 var runningFailureCount = 0
@@ -788,14 +784,13 @@ private[spark] class BlockManager(
   }
 
   if (data != null) {
-// SPARK-24307 undocumented "escape-hatch" in case there are any 
issues in converting to
-// ChunkedByteBuffer, to go back to old code-path.  Can be removed 
post Spark 2.4 if
-// new path is stable.
-if (remoteReadNioBufferConversion) {
-  return Some(new ChunkedByteBuffer(data.nioByteBuffer()))
-} else {
-  return Some(ChunkedByteBuffer.fromManagedBuffer(data))
-}
+// If the ManagedBuffer is a BlockManagerManagedBuffer, the disposal 
of the
+ 

spark git commit: [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize

2018-11-20 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master c34c42234 -> ab61ddb34


[SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP 
requestHeaderSize

## What changes were proposed in this pull request?

Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP 
requestHeaderSize.
This way long authorization field does not lead to HTTP 413.

## How was this patch tested?

Manually with curl (which version must be at least 7.55).

With the original default value (8k limit):

```bash
# Starting history server with default requestHeaderSize
$ ./sbin/start-history-server.sh
starting org.apache.spark.deploy.history.HistoryServer, logging to 
/Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out

# Creating huge header
$ echo -n "X-Custom-Header: " > cookie
$ printf 'A%.0s' {1..9500} >> cookie

# HTTP GET with huge header fails with 431
$ curl  -H cookie http://458apiros-MBP.lan:18080/
Bad Message 431reason: Request Header Fields Too Large

# The log contains the error
$ tail -1 
/Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out
18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192
```

After:

```bash
# Creating the history properties file with the increased requestHeaderSize
$ echo spark.ui.requestHeaderSize=1 > history.properties

# Starting Spark History Server with the settings
$ ./sbin/start-history-server.sh --properties-file history.properties
starting org.apache.spark.deploy.history.HistoryServer, logging to 
/Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out

# HTTP GET with huge header gives back HTML5 (I have added here only just a 
part of the response)
$ curl  -H cookie http://458apiros-MBP.lan:18080/

  ...
 
History Server
  
  
...
```

Closes #23090 from attilapiros/JettyHeaderSize.

Authored-by: “attilapiros” 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: ab61ddb34d58ab5701191c8fd3a24a62f6ebf37b
Parents: c34c422
Author: “attilapiros” 
Authored: Tue Nov 20 08:56:22 2018 -0600
Committer: Imran Rashid 
Committed: Tue Nov 20 08:56:22 2018 -0600

--
 .../scala/org/apache/spark/internal/config/package.scala | 6 ++
 core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 --
 docs/configuration.md| 8 
 3 files changed, 18 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ab61ddb3/core/src/main/scala/org/apache/spark/internal/config/package.scala
--
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 ab2b872..9cc48f6 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
@@ -570,6 +570,12 @@ package object config {
   .stringConf
   .createOptional
 
+  private[spark] val UI_REQUEST_HEADER_SIZE =
+ConfigBuilder("spark.ui.requestHeaderSize")
+  .doc("Value for HTTP request header size in bytes.")
+  .bytesConf(ByteUnit.BYTE)
+  .createWithDefaultString("8k")
+
   private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners")
 .doc("Class names of listeners to add to SparkContext during 
initialization.")
 .stringConf

http://git-wip-us.apache.org/repos/asf/spark/blob/ab61ddb3/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 52a9551..316af9b 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -356,13 +356,15 @@ private[spark] object JettyUtils extends Logging {
 
 (connector, connector.getLocalPort())
   }
+  val httpConfig = new HttpConfiguration()
+  httpConfig.setRequestHeaderSize(conf.get(UI_REQUEST_HEADER_SIZE).toInt)
 
   // If SSL is configured, create the secure connector first.
   val securePort = sslOptions.createJettySslContextFactory().map { factory 
=>
 val securePort = sslOptions.port.getOrElse(if (port > 0) 
Utils.userPort(port, 400) else 0)
 

spark git commit: [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize

2018-11-20 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 096e0d8f0 -> c28a27a25


[SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP 
requestHeaderSize

## What changes were proposed in this pull request?

Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP 
requestHeaderSize.
This way long authorization field does not lead to HTTP 413.

## How was this patch tested?

Manually with curl (which version must be at least 7.55).

With the original default value (8k limit):

```bash
# Starting history server with default requestHeaderSize
$ ./sbin/start-history-server.sh
starting org.apache.spark.deploy.history.HistoryServer, logging to 
/Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out

# Creating huge header
$ echo -n "X-Custom-Header: " > cookie
$ printf 'A%.0s' {1..9500} >> cookie

# HTTP GET with huge header fails with 431
$ curl  -H cookie http://458apiros-MBP.lan:18080/
Bad Message 431reason: Request Header Fields Too Large

# The log contains the error
$ tail -1 
/Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out
18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192
```

After:

```bash
# Creating the history properties file with the increased requestHeaderSize
$ echo spark.ui.requestHeaderSize=1 > history.properties

# Starting Spark History Server with the settings
$ ./sbin/start-history-server.sh --properties-file history.properties
starting org.apache.spark.deploy.history.HistoryServer, logging to 
/Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out

# HTTP GET with huge header gives back HTML5 (I have added here only just a 
part of the response)
$ curl  -H cookie http://458apiros-MBP.lan:18080/

  ...
 
History Server
  
  
...
```

Closes #23090 from attilapiros/JettyHeaderSize.

Authored-by: “attilapiros” 
Signed-off-by: Imran Rashid 
(cherry picked from commit ab61ddb34d58ab5701191c8fd3a24a62f6ebf37b)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.4
Commit: c28a27a2546ebbe0c001662126625638fcbb1100
Parents: 096e0d8
Author: “attilapiros” 
Authored: Tue Nov 20 08:56:22 2018 -0600
Committer: Imran Rashid 
Committed: Tue Nov 20 08:56:39 2018 -0600

--
 .../scala/org/apache/spark/internal/config/package.scala | 6 ++
 core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 --
 docs/configuration.md| 8 
 3 files changed, 18 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c28a27a2/core/src/main/scala/org/apache/spark/internal/config/package.scala
--
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 bde0995..3b3c45f 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
@@ -528,6 +528,12 @@ package object config {
   .stringConf
   .createOptional
 
+  private[spark] val UI_REQUEST_HEADER_SIZE =
+ConfigBuilder("spark.ui.requestHeaderSize")
+  .doc("Value for HTTP request header size in bytes.")
+  .bytesConf(ByteUnit.BYTE)
+  .createWithDefaultString("8k")
+
   private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners")
 .doc("Class names of listeners to add to SparkContext during 
initialization.")
 .stringConf

http://git-wip-us.apache.org/repos/asf/spark/blob/c28a27a2/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 52a9551..316af9b 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -356,13 +356,15 @@ private[spark] object JettyUtils extends Logging {
 
 (connector, connector.getLocalPort())
   }
+  val httpConfig = new HttpConfiguration()
+  httpConfig.setRequestHeaderSize(conf.get(UI_REQUEST_HEADER_SIZE).toInt)
 
   // If SSL is configured, create the secure connector first.
   val securePort = sslOptions.createJettySslContextFactory().map { factory 
=>

spark git commit: [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue

2018-11-08 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 947462f5a -> 47a668c2f


[SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue

JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never 
try to allocate an array that big.  This commit changes some defaults & configs 
to gracefully fallover to something that doesn't require one large array in 
some cases; in other cases it simply improves an error message for cases which 
will still fail.

Closes #22818 from squito/SPARK-25827.

Authored-by: Imran Rashid 
Signed-off-by: Imran Rashid 
(cherry picked from commit 8fbc1830f962c446b915d0d8ff2b13c5c75d22fc)


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

Branch: refs/heads/branch-2.4
Commit: 47a668c2f03d77078259531ddaccf80b001a8b5c
Parents: 947462f
Author: Imran Rashid 
Authored: Wed Nov 7 13:18:52 2018 +0100
Committer: Imran Rashid 
Committed: Thu Nov 8 15:49:19 2018 +0100

--
 .../org/apache/spark/internal/config/package.scala | 17 ++---
 .../scala/org/apache/spark/storage/DiskStore.scala |  6 --
 .../apache/spark/storage/memory/MemoryStore.scala  |  7 ---
 .../apache/spark/util/io/ChunkedByteBuffer.scala   |  2 +-
 .../org/apache/spark/mllib/linalg/Matrices.scala   | 13 +++--
 .../org/apache/spark/sql/internal/SQLConf.scala|  6 +++---
 .../main/scala/org/apache/spark/sql/Dataset.scala  |  7 +++
 7 files changed, 32 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/47a668c2/core/src/main/scala/org/apache/spark/internal/config/package.scala
--
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 5836d27..e2162db 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
@@ -387,8 +387,9 @@ package object config {
   .internal()
   .doc("The chunk size in bytes during writing out the bytes of 
ChunkedByteBuffer.")
   .bytesConf(ByteUnit.BYTE)
-  .checkValue(_ <= Int.MaxValue, "The chunk size during writing out the 
bytes of" +
-" ChunkedByteBuffer should not larger than Int.MaxValue.")
+  .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH,
+"The chunk size during writing out the bytes of" +
+" ChunkedByteBuffer should not larger than Int.MaxValue - 15.")
   .createWithDefault(64 * 1024 * 1024)
 
   private[spark] val CHECKPOINT_COMPRESS =
@@ -459,8 +460,9 @@ package object config {
 "otherwise specified. These buffers reduce the number of disk seeks 
and system calls " +
 "made in creating intermediate shuffle files.")
   .bytesConf(ByteUnit.KiB)
-  .checkValue(v => v > 0 && v <= Int.MaxValue / 1024,
-s"The file buffer size must be greater than 0 and less than 
${Int.MaxValue / 1024}.")
+  .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH 
/ 1024,
+s"The file buffer size must be greater than 0 and less than" +
+  s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.")
   .createWithDefaultString("32k")
 
   private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE =
@@ -468,8 +470,9 @@ package object config {
   .doc("The file system for this buffer size after each partition " +
 "is written in unsafe shuffle writer. In KiB unless otherwise 
specified.")
   .bytesConf(ByteUnit.KiB)
-  .checkValue(v => v > 0 && v <= Int.MaxValue / 1024,
-s"The buffer size must be greater than 0 and less than ${Int.MaxValue 
/ 1024}.")
+  .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH 
/ 1024,
+s"The buffer size must be greater than 0 and less than" +
+  s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.")
   .createWithDefaultString("32k")
 
   private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE =
@@ -580,7 +583,7 @@ package object config {
   .internal()
   .doc("For testing only, controls the size of chunks when memory mapping 
a file")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Int.MaxValue)
+  .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH)
 
   private[spark] val BARRIER_SYNC_TIMEOUT =
 ConfigBuilder("spark.barrier.sync.timeout")

http://git-wip-us.apache.org/repos/asf/spark/blob/47a668c2/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
--
diff --git 

spark git commit: [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue

2018-11-07 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 9e9fa2f69 -> 8fbc1830f


[SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue

JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never 
try to allocate an array that big.  This commit changes some defaults & configs 
to gracefully fallover to something that doesn't require one large array in 
some cases; in other cases it simply improves an error message for cases which 
will still fail.

Closes #22818 from squito/SPARK-25827.

Authored-by: Imran Rashid 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: 8fbc1830f962c446b915d0d8ff2b13c5c75d22fc
Parents: 9e9fa2f
Author: Imran Rashid 
Authored: Wed Nov 7 13:18:52 2018 +0100
Committer: Imran Rashid 
Committed: Wed Nov 7 13:18:52 2018 +0100

--
 .../org/apache/spark/internal/config/package.scala | 17 ++---
 .../scala/org/apache/spark/storage/DiskStore.scala |  6 --
 .../apache/spark/storage/memory/MemoryStore.scala  |  7 ---
 .../apache/spark/util/io/ChunkedByteBuffer.scala   |  2 +-
 .../org/apache/spark/mllib/linalg/Matrices.scala   | 13 +++--
 .../org/apache/spark/sql/internal/SQLConf.scala|  6 +++---
 .../main/scala/org/apache/spark/sql/Dataset.scala  |  7 +++
 7 files changed, 32 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8fbc1830/core/src/main/scala/org/apache/spark/internal/config/package.scala
--
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 2b3ba3c..d346013 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
@@ -416,8 +416,9 @@ package object config {
   .internal()
   .doc("The chunk size in bytes during writing out the bytes of 
ChunkedByteBuffer.")
   .bytesConf(ByteUnit.BYTE)
-  .checkValue(_ <= Int.MaxValue, "The chunk size during writing out the 
bytes of" +
-" ChunkedByteBuffer should not larger than Int.MaxValue.")
+  .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH,
+"The chunk size during writing out the bytes of" +
+" ChunkedByteBuffer should not larger than Int.MaxValue - 15.")
   .createWithDefault(64 * 1024 * 1024)
 
   private[spark] val CHECKPOINT_COMPRESS =
@@ -488,8 +489,9 @@ package object config {
 "otherwise specified. These buffers reduce the number of disk seeks 
and system calls " +
 "made in creating intermediate shuffle files.")
   .bytesConf(ByteUnit.KiB)
-  .checkValue(v => v > 0 && v <= Int.MaxValue / 1024,
-s"The file buffer size must be greater than 0 and less than 
${Int.MaxValue / 1024}.")
+  .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH 
/ 1024,
+s"The file buffer size must be greater than 0 and less than" +
+  s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.")
   .createWithDefaultString("32k")
 
   private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE =
@@ -497,8 +499,9 @@ package object config {
   .doc("The file system for this buffer size after each partition " +
 "is written in unsafe shuffle writer. In KiB unless otherwise 
specified.")
   .bytesConf(ByteUnit.KiB)
-  .checkValue(v => v > 0 && v <= Int.MaxValue / 1024,
-s"The buffer size must be greater than 0 and less than ${Int.MaxValue 
/ 1024}.")
+  .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH 
/ 1024,
+s"The buffer size must be greater than 0 and less than" +
+  s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.")
   .createWithDefaultString("32k")
 
   private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE =
@@ -610,7 +613,7 @@ package object config {
   .internal()
   .doc("For testing only, controls the size of chunks when memory mapping 
a file")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Int.MaxValue)
+  .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH)
 
   private[spark] val BARRIER_SYNC_TIMEOUT =
 ConfigBuilder("spark.barrier.sync.timeout")

http://git-wip-us.apache.org/repos/asf/spark/blob/8fbc1830/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
--
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala 

spark git commit: [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager.

2018-10-31 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master af3b81607 -> 68dde3481


[SPARK-23781][CORE] Merge token renewer functionality into 
HadoopDelegationTokenManager.

This avoids having two classes to deal with tokens; now the above
class is a one-stop shop for dealing with delegation tokens. The
YARN backend extends that class instead of doing composition like
before, resulting in a bit less code there too.

The renewer functionality is basically the same code that used to
be in YARN's AMCredentialRenewer. That is also the reason why the
public API of HadoopDelegationTokenManager is a little bit odd;
the YARN AM has some odd requirements for how this all should be
initialized, and the weirdness is needed currently to support that.

Tested:
- YARN with stress app for DT renewal
- Mesos and K8S with basic kerberos tests (both tgt and keytab)

Closes #22624 from vanzin/SPARK-23781.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: 68dde3481ea458b0b8deeec2f99233c2d4c1e056
Parents: af3b816
Author: Marcelo Vanzin 
Authored: Wed Oct 31 13:00:10 2018 -0500
Committer: Imran Rashid 
Committed: Wed Oct 31 13:00:10 2018 -0500

--
 .../main/scala/org/apache/spark/SparkConf.scala |   4 +-
 .../apache/spark/deploy/SparkHadoopUtil.scala   |  14 -
 .../security/HadoopDelegationTokenManager.scala | 278 ++-
 .../HadoopFSDelegationTokenProvider.scala   |   5 +-
 .../apache/spark/internal/config/package.scala  |   4 +
 .../cluster/CoarseGrainedSchedulerBackend.scala |  40 ++-
 .../HadoopDelegationTokenManagerSuite.scala | 142 +++---
 .../spark/deploy/k8s/KubernetesConf.scala   |   3 +-
 .../hadooputils/HadoopKerberosLogin.scala   |  10 +-
 ...KubernetesHadoopDelegationTokenManager.scala |  35 +--
 .../MesosCoarseGrainedSchedulerBackend.scala|  19 +-
 .../MesosHadoopDelegationTokenManager.scala | 160 ---
 .../spark/deploy/yarn/ApplicationMaster.scala   |  24 +-
 .../org/apache/spark/deploy/yarn/Client.scala   |   2 +-
 .../org/apache/spark/deploy/yarn/config.scala   |   4 -
 .../yarn/security/AMCredentialRenewer.scala | 177 
 .../YARNHadoopDelegationTokenManager.scala  |  48 ++--
 .../cluster/YarnSchedulerBackend.scala  |   5 +-
 .../YARNHadoopDelegationTokenManagerSuite.scala |   5 +-
 19 files changed, 355 insertions(+), 624 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/68dde348/core/src/main/scala/org/apache/spark/SparkConf.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala 
b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 5166543..8537c53 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -731,7 +731,9 @@ private[spark] object SparkConf extends Logging {
 KEYTAB.key -> Seq(
   AlternateConfig("spark.yarn.keytab", "3.0")),
 PRINCIPAL.key -> Seq(
-  AlternateConfig("spark.yarn.principal", "3.0"))
+  AlternateConfig("spark.yarn.principal", "3.0")),
+KERBEROS_RELOGIN_PERIOD.key -> Seq(
+  AlternateConfig("spark.yarn.kerberos.relogin.period", "3.0"))
   )
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/68dde348/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
--
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 78a7cf6..5979151 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -414,20 +414,6 @@ object SparkHadoopUtil {
   def get: SparkHadoopUtil = instance
 
   /**
-   * Given an expiration date for the current set of credentials, calculate 
the time when new
-   * credentials should be created.
-   *
-   * @param expirationDate Drop-dead expiration date
-   * @param conf Spark configuration
-   * @return Timestamp when new credentials should be created.
-   */
-  private[spark] def nextCredentialRenewalTime(expirationDate: Long, conf: 
SparkConf): Long = {
-val ct = System.currentTimeMillis
-val ratio = conf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO)
-(ct + (ratio * (expirationDate - ct))).toLong
-  }
-
-  /**
* Returns a Configuration object with Spark configuration applied on top. 
Unlike
* the instance method, this will always return a Configuration instance, 
and 

spark git commit: [SPARK-23429][CORE][FOLLOWUP] MetricGetter should rename to ExecutorMetricType in comments

2018-10-30 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master ce40efa20 -> a129f0795


[SPARK-23429][CORE][FOLLOWUP] MetricGetter should rename to ExecutorMetricType 
in comments

## What changes were proposed in this pull request?

MetricGetter should rename to ExecutorMetricType in comments.

## How was this patch tested?

Just comments, no need to test.

Closes #22884 from LantaoJin/SPARK-23429_FOLLOWUP.

Authored-by: LantaoJin 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: a129f079557204e3694754a5f9184c7f178cdf2a
Parents: ce40efa
Author: LantaoJin 
Authored: Tue Oct 30 11:01:55 2018 -0500
Committer: Imran Rashid 
Committed: Tue Oct 30 11:01:55 2018 -0500

--
 core/src/main/scala/org/apache/spark/Heartbeater.scala | 2 +-
 .../src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala | 2 +-
 core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala  | 2 +-
 core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/core/src/main/scala/org/apache/spark/Heartbeater.scala
--
diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala 
b/core/src/main/scala/org/apache/spark/Heartbeater.scala
index 5ba1b9b..84091ee 100644
--- a/core/src/main/scala/org/apache/spark/Heartbeater.scala
+++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala
@@ -61,7 +61,7 @@ private[spark] class Heartbeater(
 
   /**
* Get the current executor level metrics. These are returned as an array, 
with the index
-   * determined by MetricGetter.values
+   * determined by ExecutorMetricType.values
*/
   def getCurrentMetrics(): ExecutorMetrics = {
 val metrics = 
ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray

http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala 
b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
index 2933f3b..1befd27 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
@@ -28,7 +28,7 @@ import org.apache.spark.metrics.ExecutorMetricType
 @DeveloperApi
 class ExecutorMetrics private[spark] extends Serializable {
 
-  // Metrics are indexed by MetricGetter.values
+  // Metrics are indexed by ExecutorMetricType.values
   private val metrics = new Array[Long](ExecutorMetricType.values.length)
 
   // the first element is initialized to -1, indicating that the values for 
the array

http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/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 f93d8a8..34b1160 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -265,7 +265,7 @@ private[spark] class DAGScheduler(
   // (taskId, stageId, stageAttemptId, accumUpdates)
   accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])],
   blockManagerId: BlockManagerId,
-  // executor metrics indexed by MetricGetter.values
+  // executor metrics indexed by ExecutorMetricType.values
   executorUpdates: ExecutorMetrics): Boolean = {
 listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates,
   Some(executorUpdates)))

http://git-wip-us.apache.org/repos/asf/spark/blob/a129f079/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
--
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala 
b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 293e836..e92b8a2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -175,7 +175,7 @@ case class SparkListenerExecutorMetricsUpdate(
  * @param execId executor id
  * @param stageId stage id
  * @param stageAttemptId stage attempt
- * @param executorMetrics executor level metrics, 

spark git commit: [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue

2018-10-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 9c0c6d4d5 -> 1001d2314


[SPARK-25704][CORE] Allocate a bit less than Int.MaxValue

JVMs don't you allocate arrays of length exactly Int.MaxValue, so leave
a little extra room.  This is necessary when reading blocks >2GB off
the network (for remote reads or for cache replication).

Unit tests via jenkins, ran a test with blocks over 2gb on a cluster

Closes #22705 from squito/SPARK-25704.

Authored-by: Imran Rashid 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.4
Commit: 1001d2314275c902da519725da266a23b537e33a
Parents: 9c0c6d4
Author: Imran Rashid 
Authored: Fri Oct 19 12:52:41 2018 -0500
Committer: Imran Rashid 
Committed: Fri Oct 19 12:54:08 2018 -0500

--
 .../org/apache/spark/storage/BlockManager.scala |  6 ++
 .../apache/spark/util/io/ChunkedByteBuffer.scala| 16 +---
 2 files changed, 11 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1001d231/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 0fe82ac..c01a453 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -133,8 +133,6 @@ private[spark] class BlockManager(
 
   private[spark] val externalShuffleServiceEnabled =
 conf.get(config.SHUFFLE_SERVICE_ENABLED)
-  private val chunkSize =
-conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", 
Int.MaxValue.toString).toInt
   private val remoteReadNioBufferConversion =
 conf.getBoolean("spark.network.remoteReadNioBufferConversion", false)
 
@@ -451,7 +449,7 @@ private[spark] class BlockManager(
 new EncryptedBlockData(tmpFile, blockSize, conf, 
key).toChunkedByteBuffer(allocator)
 
   case None =>
-ChunkedByteBuffer.fromFile(tmpFile, 
conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt)
+ChunkedByteBuffer.fromFile(tmpFile)
 }
 putBytes(blockId, buffer, level)(classTag)
 tmpFile.delete()
@@ -797,7 +795,7 @@ private[spark] class BlockManager(
 if (remoteReadNioBufferConversion) {
   return Some(new ChunkedByteBuffer(data.nioByteBuffer()))
 } else {
-  return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize))
+  return Some(ChunkedByteBuffer.fromManagedBuffer(data))
 }
   }
   logDebug(s"The value of block $blockId is null")

http://git-wip-us.apache.org/repos/asf/spark/blob/1001d231/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala 
b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
index 4aa8d45..9547cb4 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
@@ -30,6 +30,7 @@ import org.apache.spark.internal.config
 import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, 
ManagedBuffer}
 import org.apache.spark.network.util.{ByteArrayWritableChannel, 
LimitedInputStream}
 import org.apache.spark.storage.StorageUtils
+import org.apache.spark.unsafe.array.ByteArrayMethods
 import org.apache.spark.util.Utils
 
 /**
@@ -169,24 +170,25 @@ private[spark] class ChunkedByteBuffer(var chunks: 
Array[ByteBuffer]) {
 
 }
 
-object ChunkedByteBuffer {
+private[spark] object ChunkedByteBuffer {
+
+
   // TODO eliminate this method if we switch BlockManager to getting 
InputStreams
-  def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): 
ChunkedByteBuffer = {
+  def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = {
 data match {
   case f: FileSegmentManagedBuffer =>
-fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength)
+fromFile(f.getFile, f.getOffset, f.getLength)
   case other =>
 new ChunkedByteBuffer(other.nioByteBuffer())
 }
   }
 
-  def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = {
-fromFile(file, maxChunkSize, 0, file.length())
+  def fromFile(file: File): ChunkedByteBuffer = {
+fromFile(file, 0, file.length())
   }
 
   private def fromFile(
   file: File,
-  maxChunkSize: Int,
   offset: Long,
   length: Long): 

spark git commit: [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue

2018-10-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 130121711 -> 43717dee5


[SPARK-25704][CORE] Allocate a bit less than Int.MaxValue

JVMs don't you allocate arrays of length exactly Int.MaxValue, so leave
a little extra room.  This is necessary when reading blocks >2GB off
the network (for remote reads or for cache replication).

Unit tests via jenkins, ran a test with blocks over 2gb on a cluster

Closes #22705 from squito/SPARK-25704.

Authored-by: Imran Rashid 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: 43717dee570dc41d71f0b27b8939f6297a029a02
Parents: 1301217
Author: Imran Rashid 
Authored: Fri Oct 19 12:52:41 2018 -0500
Committer: Imran Rashid 
Committed: Fri Oct 19 12:52:41 2018 -0500

--
 .../org/apache/spark/storage/BlockManager.scala |  6 ++
 .../apache/spark/util/io/ChunkedByteBuffer.scala| 16 +---
 2 files changed, 11 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/43717dee/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 0fe82ac..c01a453 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -133,8 +133,6 @@ private[spark] class BlockManager(
 
   private[spark] val externalShuffleServiceEnabled =
 conf.get(config.SHUFFLE_SERVICE_ENABLED)
-  private val chunkSize =
-conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", 
Int.MaxValue.toString).toInt
   private val remoteReadNioBufferConversion =
 conf.getBoolean("spark.network.remoteReadNioBufferConversion", false)
 
@@ -451,7 +449,7 @@ private[spark] class BlockManager(
 new EncryptedBlockData(tmpFile, blockSize, conf, 
key).toChunkedByteBuffer(allocator)
 
   case None =>
-ChunkedByteBuffer.fromFile(tmpFile, 
conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt)
+ChunkedByteBuffer.fromFile(tmpFile)
 }
 putBytes(blockId, buffer, level)(classTag)
 tmpFile.delete()
@@ -797,7 +795,7 @@ private[spark] class BlockManager(
 if (remoteReadNioBufferConversion) {
   return Some(new ChunkedByteBuffer(data.nioByteBuffer()))
 } else {
-  return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize))
+  return Some(ChunkedByteBuffer.fromManagedBuffer(data))
 }
   }
   logDebug(s"The value of block $blockId is null")

http://git-wip-us.apache.org/repos/asf/spark/blob/43717dee/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala 
b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
index 4aa8d45..9547cb4 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
@@ -30,6 +30,7 @@ import org.apache.spark.internal.config
 import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, 
ManagedBuffer}
 import org.apache.spark.network.util.{ByteArrayWritableChannel, 
LimitedInputStream}
 import org.apache.spark.storage.StorageUtils
+import org.apache.spark.unsafe.array.ByteArrayMethods
 import org.apache.spark.util.Utils
 
 /**
@@ -169,24 +170,25 @@ private[spark] class ChunkedByteBuffer(var chunks: 
Array[ByteBuffer]) {
 
 }
 
-object ChunkedByteBuffer {
+private[spark] object ChunkedByteBuffer {
+
+
   // TODO eliminate this method if we switch BlockManager to getting 
InputStreams
-  def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): 
ChunkedByteBuffer = {
+  def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = {
 data match {
   case f: FileSegmentManagedBuffer =>
-fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength)
+fromFile(f.getFile, f.getOffset, f.getLength)
   case other =>
 new ChunkedByteBuffer(other.nioByteBuffer())
 }
   }
 
-  def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = {
-fromFile(file, maxChunkSize, 0, file.length())
+  def fromFile(file: File): ChunkedByteBuffer = {
+fromFile(file, 0, file.length())
   }
 
   private def fromFile(
   file: File,
-  maxChunkSize: Int,
   offset: Long,
   length: Long): 

spark git commit: [SPARK-20327][YARN] Follow up: fix resource request tests on Hadoop 3.

2018-10-17 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 24f5bbd77 -> 7d425b190


[SPARK-20327][YARN] Follow up: fix resource request tests on Hadoop 3.

The test fix is to allocate a `Resource` object only after the resource
types have been initialized. Otherwise the YARN classes get in a weird
state and throw a different exception than expected, because the resource
has a different view of the registered resources.

I also removed a test for a null resource since that seems unnecessary
and made the fix more complicated.

All the other changes are just cleanup; basically simplify the tests by
defining what is being tested and deriving the resource type registration
and the SparkConf from that data, instead of having redundant definitions
in the tests.

Ran tests with Hadoop 3 (and also without it).

Closes #22751 from vanzin/SPARK-20327.fix.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: 7d425b190a91f49193eeb58c398e497ff92c6169
Parents: 24f5bbd
Author: Marcelo Vanzin 
Authored: Wed Oct 17 10:40:47 2018 -0500
Committer: Imran Rashid 
Committed: Wed Oct 17 10:40:47 2018 -0500

--
 .../deploy/yarn/ResourceRequestHelper.scala |   4 +-
 .../apache/spark/deploy/yarn/ClientSuite.scala  |  70 +++
 .../yarn/ResourceRequestHelperSuite.scala   | 181 ++-
 .../deploy/yarn/ResourceRequestTestHelper.scala |  13 +-
 4 files changed, 90 insertions(+), 178 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7d425b19/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala
--
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala
index 9534f3a..012268e 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala
@@ -29,7 +29,7 @@ import org.apache.spark.{SparkConf, SparkException}
 import org.apache.spark.deploy.yarn.config._
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{CausedBy, Utils}
 
 /**
  * This helper class uses some of Hadoop 3 methods from the YARN API,
@@ -121,6 +121,8 @@ private object ResourceRequestHelper extends Logging {
 case _: MatchError =>
   throw new IllegalArgumentException(s"Resource request for '$name' 
('$rawAmount') " +
   s"does not match pattern $AMOUNT_AND_UNIT_REGEX.")
+case CausedBy(e: IllegalArgumentException) =>
+  throw new IllegalArgumentException(s"Invalid request for $name: 
${e.getMessage}")
 case e: InvocationTargetException if e.getCause != null => throw 
e.getCause
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/7d425b19/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
--
diff --git 
a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
 
b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
index 533cb2b..b3286e8 100644
--- 
a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
+++ 
b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
@@ -200,20 +200,6 @@ class ClientSuite extends SparkFunSuite with Matchers {
 appContext.getMaxAppAttempts should be (42)
   }
 
-  test("resource request (client mode)") {
-val sparkConf = new SparkConf().set("spark.submit.deployMode", "client")
-  .set(YARN_AM_RESOURCE_TYPES_PREFIX + "fpga", "2")
-  .set(YARN_AM_RESOURCE_TYPES_PREFIX + "gpu", "3")
-testResourceRequest(sparkConf, List("gpu", "fpga"), Seq(("fpga", 2), 
("gpu", 3)))
-  }
-
-  test("resource request (cluster mode)") {
-val sparkConf = new SparkConf().set("spark.submit.deployMode", "cluster")
-  .set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + "fpga", "4")
-  .set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + "gpu", "5")
-testResourceRequest(sparkConf, List("gpu", "fpga"), Seq(("fpga", 4), 
("gpu", 5)))
-  }
-
   test("spark.yarn.jars with multiple paths and globs") {
 val libs = Utils.createTempDir()
 val single = Utils.createTempDir()
@@ -372,6 +358,35 @@ class ClientSuite 

spark git commit: [SPARK-25535][CORE] Work around bad error handling in commons-crypto.

2018-10-09 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master deb9588b2 -> 3eee9e024


[SPARK-25535][CORE] Work around bad error handling in commons-crypto.

The commons-crypto library does some questionable error handling internally,
which can lead to JVM crashes if some call into native code fails and cleans
up state it should not.

While the library is not fixed, this change adds some workarounds in Spark code
so that when an error is detected in the commons-crypto side, Spark avoids
calling into the library further.

Tested with existing and added unit tests.

Closes #22557 from vanzin/SPARK-25535.

Authored-by: Marcelo Vanzin 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: 3eee9e02463e10570a29fad00823c953debd945e
Parents: deb9588
Author: Marcelo Vanzin 
Authored: Tue Oct 9 09:27:08 2018 -0500
Committer: Imran Rashid 
Committed: Tue Oct 9 09:27:08 2018 -0500

--
 .../apache/spark/network/crypto/AuthEngine.java |  95 -
 .../spark/network/crypto/TransportCipher.java   |  60 ++--
 .../spark/network/crypto/AuthEngineSuite.java   |  17 +++
 .../spark/security/CryptoStreamUtils.scala  | 137 +--
 .../spark/security/CryptoStreamUtilsSuite.scala |  37 -
 5 files changed, 295 insertions(+), 51 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3eee9e02/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
 
b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
index 056505e..64fdb32 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
@@ -159,15 +159,21 @@ class AuthEngine implements Closeable {
 // accurately report the errors when they happen.
 RuntimeException error = null;
 byte[] dummy = new byte[8];
-try {
-  doCipherOp(encryptor, dummy, true);
-} catch (Exception e) {
-  error = new RuntimeException(e);
+if (encryptor != null) {
+  try {
+doCipherOp(Cipher.ENCRYPT_MODE, dummy, true);
+  } catch (Exception e) {
+error = new RuntimeException(e);
+  }
+  encryptor = null;
 }
-try {
-  doCipherOp(decryptor, dummy, true);
-} catch (Exception e) {
-  error = new RuntimeException(e);
+if (decryptor != null) {
+  try {
+doCipherOp(Cipher.DECRYPT_MODE, dummy, true);
+  } catch (Exception e) {
+error = new RuntimeException(e);
+  }
+  decryptor = null;
 }
 random.close();
 
@@ -189,11 +195,11 @@ class AuthEngine implements Closeable {
   }
 
   private byte[] decrypt(byte[] in) throws GeneralSecurityException {
-return doCipherOp(decryptor, in, false);
+return doCipherOp(Cipher.DECRYPT_MODE, in, false);
   }
 
   private byte[] encrypt(byte[] in) throws GeneralSecurityException {
-return doCipherOp(encryptor, in, false);
+return doCipherOp(Cipher.ENCRYPT_MODE, in, false);
   }
 
   private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec 
key)
@@ -205,11 +211,13 @@ class AuthEngine implements Closeable {
 byte[] iv = new byte[conf.ivLength()];
 System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length));
 
-encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf);
-encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv));
+CryptoCipher _encryptor = CryptoCipherFactory.getCryptoCipher(cipher, 
cryptoConf);
+_encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv));
+this.encryptor = _encryptor;
 
-decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf);
-decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv));
+CryptoCipher _decryptor = CryptoCipherFactory.getCryptoCipher(cipher, 
cryptoConf);
+_decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv));
+this.decryptor = _decryptor;
   }
 
   /**
@@ -241,29 +249,52 @@ class AuthEngine implements Closeable {
 return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm());
   }
 
-  private byte[] doCipherOp(CryptoCipher cipher, byte[] in, boolean isFinal)
+  private byte[] doCipherOp(int mode, byte[] in, boolean isFinal)
 throws GeneralSecurityException {
 
-Preconditions.checkState(cipher != null);
+CryptoCipher cipher;
+switch (mode) {
+  case 

[3/3] spark git commit: [PYSPARK][SQL] Updates to RowQueue

2018-09-25 Thread irashid
[PYSPARK][SQL] Updates to RowQueue

Tested with updates to RowQueueSuite

(cherry picked from commit 6d742d1bd71aa3803dce91a830b37284cb18cf70)


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

Branch: refs/heads/branch-2.2
Commit: 4f10aff403ccc8287a816cb94ddf7f11e185907a
Parents: dd0e7cf
Author: Imran Rashid 
Authored: Thu Sep 6 12:11:47 2018 -0500
Committer: Imran Rashid 
Committed: Tue Sep 25 11:46:06 2018 -0500

--
 .../spark/sql/execution/python/RowQueue.scala   | 27 ++-
 .../sql/execution/python/RowQueueSuite.scala| 28 +++-
 2 files changed, 41 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4f10aff4/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
index cd1e77f..4d6820c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
@@ -21,9 +21,10 @@ import java.io._
 
 import com.google.common.io.Closeables
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkEnv, SparkException}
 import org.apache.spark.io.NioBufferedFileInputStream
 import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager}
+import org.apache.spark.serializer.SerializerManager
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.unsafe.Platform
 import org.apache.spark.unsafe.memory.MemoryBlock
@@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: 
MemoryBlock, numFields
  * A RowQueue that is backed by a file on disk. This queue will stop accepting 
new rows once any
  * reader has begun reading from the queue.
  */
-private[python] case class DiskRowQueue(file: File, fields: Int) extends 
RowQueue {
-  private var out = new DataOutputStream(
-new BufferedOutputStream(new FileOutputStream(file.toString)))
+private[python] case class DiskRowQueue(
+file: File,
+fields: Int,
+serMgr: SerializerManager) extends RowQueue {
+
+  private var out = new DataOutputStream(serMgr.wrapForEncryption(
+new BufferedOutputStream(new FileOutputStream(file.toString
   private var unreadBytes = 0L
 
   private var in: DataInputStream = _
@@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 if (out != null) {
   out.close()
   out = null
-  in = new DataInputStream(new NioBufferedFileInputStream(file))
+  in = new DataInputStream(serMgr.wrapForEncryption(
+new NioBufferedFileInputStream(file)))
 }
 
 if (unreadBytes > 0) {
@@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 private[python] case class HybridRowQueue(
 memManager: TaskMemoryManager,
 tempDir: File,
-numFields: Int)
+numFields: Int,
+serMgr: SerializerManager)
   extends MemoryConsumer(memManager) with RowQueue {
 
   // Each buffer should have at least one row
@@ -212,7 +219,7 @@ private[python] case class HybridRowQueue(
   }
 
   private def createDiskQueue(): RowQueue = {
-DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields)
+DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr)
   }
 
   private def createNewQueue(required: Long): RowQueue = {
@@ -279,3 +286,9 @@ private[python] case class HybridRowQueue(
 }
   }
 }
+
+private[python] object HybridRowQueue {
+  def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): 
HybridRowQueue = {
+HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/4f10aff4/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
index ffda33c..1ec9986 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
@@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python
 import java.io.File
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
-import 

[1/3] spark git commit: [SPARK-25253][PYSPARK] Refactor local connection & auth code

2018-09-25 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 bd12eb75d -> 4f10aff40


[SPARK-25253][PYSPARK] Refactor local connection & auth code

This eliminates some duplication in the code to connect to a server on 
localhost to talk directly to the jvm.  Also it gives consistent ipv6 and error 
handling.  Two other incidental changes, that shouldn't matter:
1) python barrier tasks perform authentication immediately (rather than waiting 
for the BARRIER_FUNCTION indicator)
2) for `rdd._load_from_socket`, the timeout is only increased after 
authentication.

Closes #22247 from squito/py_connection_refactor.

Authored-by: Imran Rashid 
Signed-off-by: hyukjinkwon 
(cherry picked from commit 38391c9aa8a88fcebb337934f30298a32d91596b)
(cherry picked from commit a2a54a5f49364a1825932c9f04eb0ff82dd7d465)


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

Branch: refs/heads/branch-2.2
Commit: fc1c4e7d24f7d0afb3b79d66aa9812e7dddc2f38
Parents: bd12eb7
Author: Imran Rashid 
Authored: Wed Aug 29 09:47:38 2018 +0800
Committer: Imran Rashid 
Committed: Tue Sep 25 11:45:59 2018 -0500

--
 python/pyspark/java_gateway.py | 32 +++-
 python/pyspark/rdd.py  | 24 ++--
 python/pyspark/worker.py   |  7 ++-
 3 files changed, 35 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fc1c4e7d/python/pyspark/java_gateway.py
--
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 7abf2c1..191dfce 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -133,7 +133,7 @@ def launch_gateway(conf=None):
 return gateway
 
 
-def do_server_auth(conn, auth_secret):
+def _do_server_auth(conn, auth_secret):
 """
 Performs the authentication protocol defined by the SocketAuthHelper class 
on the given
 file-like object 'conn'.
@@ -144,3 +144,33 @@ def do_server_auth(conn, auth_secret):
 if reply != "ok":
 conn.close()
 raise Exception("Unexpected reply from iterator server.")
+
+
+def local_connect_and_auth(port, auth_secret):
+"""
+Connect to local host, authenticate with it, and return a (sockfile,sock) 
for that connection.
+Handles IPV4 & IPV6, does some error handling.
+:param port
+:param auth_secret
+:return: a tuple with (sockfile, sock)
+"""
+sock = None
+errors = []
+# Support for both IPv4 and IPv6.
+# On most of IPv6-ready systems, IPv6 will take precedence.
+for res in socket.getaddrinfo("127.0.0.1", port, socket.AF_UNSPEC, 
socket.SOCK_STREAM):
+af, socktype, proto, _, sa = res
+try:
+sock = socket.socket(af, socktype, proto)
+sock.settimeout(15)
+sock.connect(sa)
+sockfile = sock.makefile("rwb", 65536)
+_do_server_auth(sockfile, auth_secret)
+return (sockfile, sock)
+except socket.error as e:
+emsg = _exception_message(e)
+errors.append("tried to connect to %s, but an error occured: %s" % 
(sa, emsg))
+sock.close()
+sock = None
+else:
+raise Exception("could not open socket: %s" % errors)

http://git-wip-us.apache.org/repos/asf/spark/blob/fc1c4e7d/python/pyspark/rdd.py
--
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 864cebb..7d84cbd 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -39,7 +39,7 @@ if sys.version > '3':
 else:
 from itertools import imap as map, ifilter as filter
 
-from pyspark.java_gateway import do_server_auth
+from pyspark.java_gateway import local_connect_and_auth
 from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
 BatchedSerializer, CloudPickleSerializer, PairDeserializer, \
 PickleSerializer, pack_long, AutoBatchedSerializer, write_with_length, \
@@ -122,30 +122,10 @@ def _parse_memory(s):
 
 
 def _load_from_socket(sock_info, serializer):
-port, auth_secret = sock_info
-sock = None
-# Support for both IPv4 and IPv6.
-# On most of IPv6-ready systems, IPv6 will take precedence.
-for res in socket.getaddrinfo("localhost", port, socket.AF_UNSPEC, 
socket.SOCK_STREAM):
-af, socktype, proto, canonname, sa = res
-sock = socket.socket(af, socktype, proto)
-try:
-sock.settimeout(15)
-sock.connect(sa)
-except socket.error:
-sock.close()
-sock = None
-continue
-break
-  

[2/3] spark git commit: [PYSPARK] Updates to pyspark broadcast

2018-09-25 Thread irashid
[PYSPARK] Updates to pyspark broadcast

(cherry picked from commit 09dd34cb1706f2477a89174d6a1a0f17ed5b0a65)


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

Branch: refs/heads/branch-2.2
Commit: dd0e7cf5287148618404593ca095dd900b6e993f
Parents: fc1c4e7
Author: Imran Rashid 
Authored: Mon Aug 13 21:35:34 2018 -0500
Committer: Imran Rashid 
Committed: Tue Sep 25 11:46:03 2018 -0500

--
 .../org/apache/spark/api/python/PythonRDD.scala | 349 ---
 .../spark/api/python/PythonRDDSuite.scala   |  23 +-
 dev/sparktestsupport/modules.py |   2 +
 python/pyspark/broadcast.py |  58 ++-
 python/pyspark/context.py   |  63 +++-
 python/pyspark/serializers.py   |  58 +++
 python/pyspark/test_broadcast.py| 126 +++
 python/pyspark/test_serializers.py  |  90 +
 python/pyspark/worker.py|  24 +-
 9 files changed, 705 insertions(+), 88 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dd0e7cf5/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 7b5a179..2f4e3bc 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, 
Map => JMap}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.concurrent.Promise
+import scala.concurrent.duration.Duration
 import scala.language.existentials
-import scala.util.control.NonFatal
+import scala.util.Try
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.compress.CompressionCodec
@@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, 
JavaSparkContext}
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.input.PortableDataStream
 import org.apache.spark.internal.Logging
+import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.security.SocketAuthHelper
 import org.apache.spark.util._
@@ -293,19 +296,51 @@ private[spark] class PythonRunner(
 val newBids = broadcastVars.map(_.id).toSet
 // number of different broadcasts
 val toRemove = oldBids.diff(newBids)
-val cnt = toRemove.size + newBids.diff(oldBids).size
+val addedBids = newBids.diff(oldBids)
+val cnt = toRemove.size + addedBids.size
+val needsDecryptionServer = env.serializerManager.encryptionEnabled && 
addedBids.nonEmpty
+dataOut.writeBoolean(needsDecryptionServer)
 dataOut.writeInt(cnt)
-for (bid <- toRemove) {
-  // remove the broadcast from worker
-  dataOut.writeLong(- bid - 1)  // bid >= 0
-  oldBids.remove(bid)
+def sendBidsToRemove(): Unit = {
+  for (bid <- toRemove) {
+// remove the broadcast from worker
+dataOut.writeLong(-bid - 1) // bid >= 0
+oldBids.remove(bid)
+  }
 }
-for (broadcast <- broadcastVars) {
-  if (!oldBids.contains(broadcast.id)) {
+if (needsDecryptionServer) {
+  // if there is encryption, we setup a server which reads the 
encrypted files, and sends
+// the decrypted data to python
+  val idsAndFiles = broadcastVars.flatMap { broadcast =>
+  if (oldBids.contains(broadcast.id)) {
+  None
+} else {
+  Some((broadcast.id, broadcast.value.path))
+}
+}
+  val server = new EncryptedPythonBroadcastServer(env, idsAndFiles)
+  dataOut.writeInt(server.port)
+  logTrace(s"broadcast decryption server setup on ${server.port}")
+  PythonRDD.writeUTF(server.secret, dataOut)
+  sendBidsToRemove()
+  idsAndFiles.foreach { case (id, _) =>
 // send new broadcast
-dataOut.writeLong(broadcast.id)
-PythonRDD.writeUTF(broadcast.value.path, dataOut)
-oldBids.add(broadcast.id)
+dataOut.writeLong(id)
+oldBids.add(id)
+  }
+  dataOut.flush()
+  logTrace("waiting for python to read decrypted broadcast data from 
server")
+  server.waitTillBroadcastDataSent()
+  logTrace("done sending decrypted data to python")
+} else {
+ 

spark git commit: [SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite

2018-09-18 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 123f0041d -> a6f37b074


[SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite

PythonForeachWriterSuite was failing because RowQueue now needs to have a 
handle on a SparkEnv with a SerializerManager, so added a mock env with a 
serializer manager.

Also fixed a typo in the `finally` that was hiding the real exception.

Tested PythonForeachWriterSuite locally, full tests via jenkins.

Closes #22452 from squito/SPARK-25456.

Authored-by: Imran Rashid 
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/master
Commit: a6f37b0742d87d5c8ee3e134999d665e5719e822
Parents: 123f004
Author: Imran Rashid 
Authored: Tue Sep 18 16:33:37 2018 -0500
Committer: Imran Rashid 
Committed: Tue Sep 18 16:33:37 2018 -0500

--
 .../execution/python/PythonForeachWriterSuite.scala   | 14 +++---
 1 file changed, 11 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a6f37b07/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
index 07e6034..d02014c 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
@@ -19,17 +19,20 @@ package org.apache.spark.sql.execution.python
 
 import scala.collection.mutable.ArrayBuffer
 
+import org.mockito.Mockito.when
 import org.scalatest.concurrent.Eventually
+import org.scalatest.mockito.MockitoSugar
 import org.scalatest.time.SpanSugar._
 
 import org.apache.spark._
 import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager}
+import org.apache.spark.serializer.{JavaSerializer, SerializerManager}
 import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, 
UnsafeProjection}
 import 
org.apache.spark.sql.execution.python.PythonForeachWriter.UnsafeRowBuffer
 import org.apache.spark.sql.types.{DataType, IntegerType}
 import org.apache.spark.util.Utils
 
-class PythonForeachWriterSuite extends SparkFunSuite with Eventually {
+class PythonForeachWriterSuite extends SparkFunSuite with Eventually with 
MockitoSugar {
 
   testWithBuffer("UnsafeRowBuffer: iterator blocks when no data is available") 
{ b =>
 b.assertIteratorBlocked()
@@ -75,7 +78,7 @@ class PythonForeachWriterSuite extends SparkFunSuite with 
Eventually {
 tester = new BufferTester(memBytes, sleepPerRowReadMs)
 f(tester)
   } finally {
-if (tester == null) tester.close()
+if (tester != null) tester.close()
   }
 }
   }
@@ -83,7 +86,12 @@ class PythonForeachWriterSuite extends SparkFunSuite with 
Eventually {
 
   class BufferTester(memBytes: Long, sleepPerRowReadMs: Int) {
 private val buffer = {
-  val mem = new TestMemoryManager(new SparkConf())
+  val mockEnv = mock[SparkEnv]
+  val conf = new SparkConf()
+  val serializerManager = new SerializerManager(new JavaSerializer(conf), 
conf, None)
+  when(mockEnv.serializerManager).thenReturn(serializerManager)
+  SparkEnv.set(mockEnv)
+  val mem = new TestMemoryManager(conf)
   mem.limit(memBytes)
   val taskM = new TaskMemoryManager(mem, 0)
   new UnsafeRowBuffer(taskM, Utils.createTempDir(), 1)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite

2018-09-18 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 67f2cb6e0 -> 76514a015


[SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite

PythonForeachWriterSuite was failing because RowQueue now needs to have a 
handle on a SparkEnv with a SerializerManager, so added a mock env with a 
serializer manager.

Also fixed a typo in the `finally` that was hiding the real exception.

Tested PythonForeachWriterSuite locally, full tests via jenkins.

Closes #22452 from squito/SPARK-25456.

Authored-by: Imran Rashid 
Signed-off-by: Imran Rashid 
(cherry picked from commit a6f37b0742d87d5c8ee3e134999d665e5719e822)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.4
Commit: 76514a015168de8d8b54b3abf6b835050eefd8c2
Parents: 67f2cb6
Author: Imran Rashid 
Authored: Tue Sep 18 16:33:37 2018 -0500
Committer: Imran Rashid 
Committed: Tue Sep 18 16:33:49 2018 -0500

--
 .../execution/python/PythonForeachWriterSuite.scala   | 14 +++---
 1 file changed, 11 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/76514a01/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
index 07e6034..d02014c 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala
@@ -19,17 +19,20 @@ package org.apache.spark.sql.execution.python
 
 import scala.collection.mutable.ArrayBuffer
 
+import org.mockito.Mockito.when
 import org.scalatest.concurrent.Eventually
+import org.scalatest.mockito.MockitoSugar
 import org.scalatest.time.SpanSugar._
 
 import org.apache.spark._
 import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager}
+import org.apache.spark.serializer.{JavaSerializer, SerializerManager}
 import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, 
UnsafeProjection}
 import 
org.apache.spark.sql.execution.python.PythonForeachWriter.UnsafeRowBuffer
 import org.apache.spark.sql.types.{DataType, IntegerType}
 import org.apache.spark.util.Utils
 
-class PythonForeachWriterSuite extends SparkFunSuite with Eventually {
+class PythonForeachWriterSuite extends SparkFunSuite with Eventually with 
MockitoSugar {
 
   testWithBuffer("UnsafeRowBuffer: iterator blocks when no data is available") 
{ b =>
 b.assertIteratorBlocked()
@@ -75,7 +78,7 @@ class PythonForeachWriterSuite extends SparkFunSuite with 
Eventually {
 tester = new BufferTester(memBytes, sleepPerRowReadMs)
 f(tester)
   } finally {
-if (tester == null) tester.close()
+if (tester != null) tester.close()
   }
 }
   }
@@ -83,7 +86,12 @@ class PythonForeachWriterSuite extends SparkFunSuite with 
Eventually {
 
   class BufferTester(memBytes: Long, sleepPerRowReadMs: Int) {
 private val buffer = {
-  val mem = new TestMemoryManager(new SparkConf())
+  val mockEnv = mock[SparkEnv]
+  val conf = new SparkConf()
+  val serializerManager = new SerializerManager(new JavaSerializer(conf), 
conf, None)
+  when(mockEnv.serializerManager).thenReturn(serializerManager)
+  SparkEnv.set(mockEnv)
+  val mem = new TestMemoryManager(conf)
   mem.limit(memBytes)
   val taskM = new TaskMemoryManager(mem, 0)
   new UnsafeRowBuffer(taskM, Utils.createTempDir(), 1)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[1/3] spark git commit: [PYSPARK] Updates to pyspark broadcast

2018-09-17 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.4 08f7b145c -> 7beb34175


[PYSPARK] Updates to pyspark broadcast

(cherry picked from commit 58419b92673c46911c25bc6c6b13397f880c6424)


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

Branch: refs/heads/branch-2.4
Commit: 963af1301131f63a955288edcd6bc76956bb6447
Parents: 08f7b14
Author: Imran Rashid 
Authored: Mon Aug 13 21:35:34 2018 -0500
Committer: Imran Rashid 
Committed: Mon Sep 17 15:28:31 2018 -0500

--
 .../org/apache/spark/api/python/PythonRDD.scala | 299 ---
 .../apache/spark/api/python/PythonRunner.scala  |  52 +++-
 .../spark/api/python/PythonRDDSuite.scala   |  23 +-
 dev/sparktestsupport/modules.py |   2 +
 python/pyspark/broadcast.py |  58 +++-
 python/pyspark/context.py   |  64 ++--
 python/pyspark/serializers.py   |  51 
 python/pyspark/sql/session.py   |  12 +-
 python/pyspark/sql/tests.py |  45 ++-
 python/pyspark/test_broadcast.py| 126 
 python/pyspark/test_serializers.py  |  90 ++
 python/pyspark/tests.py |   9 +-
 python/pyspark/worker.py|  22 +-
 .../spark/sql/api/python/PythonSQLUtils.scala   |  47 ++-
 .../sql/execution/arrow/ArrowConverters.scala   |   9 +-
 15 files changed, 789 insertions(+), 120 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/963af130/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index e639a84..8b5a7a9 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, 
Map => JMap}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.concurrent.Promise
+import scala.concurrent.duration.Duration
 import scala.language.existentials
-import scala.util.control.NonFatal
+import scala.util.Try
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.compress.CompressionCodec
@@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, 
JavaSparkContext}
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.input.PortableDataStream
 import org.apache.spark.internal.Logging
+import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.security.SocketAuthHelper
 import org.apache.spark.util._
@@ -169,27 +172,34 @@ private[spark] object PythonRDD extends Logging {
 
   def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: 
Int):
   JavaRDD[Array[Byte]] = {
-val file = new DataInputStream(new FileInputStream(filename))
+readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism)
+  }
+
+  def readRDDFromInputStream(
+  sc: SparkContext,
+  in: InputStream,
+  parallelism: Int): JavaRDD[Array[Byte]] = {
+val din = new DataInputStream(in)
 try {
   val objs = new mutable.ArrayBuffer[Array[Byte]]
   try {
 while (true) {
-  val length = file.readInt()
+  val length = din.readInt()
   val obj = new Array[Byte](length)
-  file.readFully(obj)
+  din.readFully(obj)
   objs += obj
 }
   } catch {
 case eof: EOFException => // No-op
   }
-  JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
+  JavaRDD.fromRDD(sc.parallelize(objs, parallelism))
 } finally {
-  file.close()
+  din.close()
 }
   }
 
-  def readBroadcastFromFile(sc: JavaSparkContext, path: String): 
Broadcast[PythonBroadcast] = {
-sc.broadcast(new PythonBroadcast(path))
+  def setupBroadcast(path: String): PythonBroadcast = {
+new PythonBroadcast(path)
   }
 
   def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) {
@@ -419,34 +429,15 @@ private[spark] object PythonRDD extends Logging {
*/
   private[spark] def serveToStream(
   threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = {
-val serverSocket = new ServerSocket(0, 1, 
InetAddress.getByName("localhost"))
-// Close the socket if no connection in 15 seconds
-serverSocket.setSoTimeout(15000)
-
-new Thread(threadName) {
-  setDaemon(true)
-  override def run() {
-

[3/3] spark git commit: [CORE] Updates to remote cache reads

2018-09-17 Thread irashid
[CORE] Updates to remote cache reads

Covered by tests in DistributedSuite

(cherry picked from commit a97001d21757ae214c86371141bd78a376200f66)


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

Branch: refs/heads/branch-2.4
Commit: 7beb3417591b0f9c436d9175f3343ee79795d536
Parents: 80e317b
Author: Imran Rashid 
Authored: Wed Aug 22 16:38:28 2018 -0500
Committer: Imran Rashid 
Committed: Mon Sep 17 15:28:40 2018 -0500

--
 .../spark/network/buffer/ManagedBuffer.java |  5 +-
 .../spark/network/shuffle/DownloadFile.java | 47 ++
 .../network/shuffle/DownloadFileManager.java| 36 
 .../shuffle/DownloadFileWritableChannel.java| 30 +++
 .../network/shuffle/ExternalShuffleClient.java  |  4 +-
 .../network/shuffle/OneForOneBlockFetcher.java  | 28 +++---
 .../spark/network/shuffle/ShuffleClient.java|  4 +-
 .../network/shuffle/SimpleDownloadFile.java | 91 
 .../spark/network/shuffle/TempFileManager.java  | 36 
 .../spark/network/BlockTransferService.scala|  6 +-
 .../netty/NettyBlockTransferService.scala   |  4 +-
 .../org/apache/spark/storage/BlockManager.scala | 78 ++---
 .../org/apache/spark/storage/DiskStore.scala| 16 
 .../storage/ShuffleBlockFetcherIterator.scala   | 21 +++--
 .../spark/storage/BlockManagerSuite.scala   |  8 +-
 .../ShuffleBlockFetcherIteratorSuite.scala  |  6 +-
 16 files changed, 328 insertions(+), 92 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7beb3417/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
index 1861f8d..2d573f5 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
@@ -36,7 +36,10 @@ import java.nio.ByteBuffer;
  */
 public abstract class ManagedBuffer {
 
-  /** Number of bytes of the data. */
+  /**
+   * Number of bytes of the data. If this buffer will decrypt for all of the 
views into the data,
+   * this is the size of the decrypted data.
+   */
   public abstract long size();
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/7beb3417/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
--
diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
new file mode 100644
index 000..633622b
--- /dev/null
+++ 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
@@ -0,0 +1,47 @@
+/*
+ * 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.network.shuffle;
+
+import java.io.IOException;
+
+/**
+ * A handle on the file used when fetching remote data to disk.  Used to 
ensure the lifecycle of
+ * writing the data, reading it back, and then cleaning it up is followed.  
Specific implementations
+ * may also handle encryption.  The data can be read only via 
DownloadFileWritableChannel,
+ * which ensures data is not read until after the writer is closed.
+ */
+public interface DownloadFile {
+  /**
+   * Delete the file.
+   *
+   * @return  true if and only if the file or directory is
+   *  successfully deleted; false otherwise
+   */
+  boolean delete();
+
+  /**
+   * A channel for writing data to the file.  This special channel allows 
access to the data for
+   * reading, after the channel is closed, via {@link 

[2/3] spark git commit: [PYSPARK][SQL] Updates to RowQueue

2018-09-17 Thread irashid
[PYSPARK][SQL] Updates to RowQueue

Tested with updates to RowQueueSuite

(cherry picked from commit 8f5a5a9e5b9f273443b2721f80c99dc7397ef4c0)


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

Branch: refs/heads/branch-2.4
Commit: 80e317b3a5cafb95a5b5a00d1d8b3b8c595985e2
Parents: 963af13
Author: Imran Rashid 
Authored: Thu Sep 6 12:11:47 2018 -0500
Committer: Imran Rashid 
Committed: Mon Sep 17 15:28:36 2018 -0500

--
 .../spark/sql/execution/python/RowQueue.scala   | 27 ++-
 .../sql/execution/python/RowQueueSuite.scala| 28 +++-
 2 files changed, 41 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/80e317b3/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
index e2fa6e7..d2820ff 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
@@ -21,9 +21,10 @@ import java.io._
 
 import com.google.common.io.Closeables
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkEnv, SparkException}
 import org.apache.spark.io.NioBufferedFileInputStream
 import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager}
+import org.apache.spark.serializer.SerializerManager
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.unsafe.Platform
 import org.apache.spark.unsafe.memory.MemoryBlock
@@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: 
MemoryBlock, numFields
  * A RowQueue that is backed by a file on disk. This queue will stop accepting 
new rows once any
  * reader has begun reading from the queue.
  */
-private[python] case class DiskRowQueue(file: File, fields: Int) extends 
RowQueue {
-  private var out = new DataOutputStream(
-new BufferedOutputStream(new FileOutputStream(file.toString)))
+private[python] case class DiskRowQueue(
+file: File,
+fields: Int,
+serMgr: SerializerManager) extends RowQueue {
+
+  private var out = new DataOutputStream(serMgr.wrapForEncryption(
+new BufferedOutputStream(new FileOutputStream(file.toString
   private var unreadBytes = 0L
 
   private var in: DataInputStream = _
@@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 if (out != null) {
   out.close()
   out = null
-  in = new DataInputStream(new NioBufferedFileInputStream(file))
+  in = new DataInputStream(serMgr.wrapForEncryption(
+new NioBufferedFileInputStream(file)))
 }
 
 if (unreadBytes > 0) {
@@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 private[python] case class HybridRowQueue(
 memManager: TaskMemoryManager,
 tempDir: File,
-numFields: Int)
+numFields: Int,
+serMgr: SerializerManager)
   extends MemoryConsumer(memManager) with RowQueue {
 
   // Each buffer should have at least one row
@@ -212,7 +219,7 @@ private[python] case class HybridRowQueue(
   }
 
   private def createDiskQueue(): RowQueue = {
-DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields)
+DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr)
   }
 
   private def createNewQueue(required: Long): RowQueue = {
@@ -279,3 +286,9 @@ private[python] case class HybridRowQueue(
 }
   }
 }
+
+private[python] object HybridRowQueue {
+  def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): 
HybridRowQueue = {
+HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/80e317b3/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
index ffda33c..1ec9986 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
@@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python
 import java.io.File
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
-import 

[1/3] spark git commit: [CORE] Updates to remote cache reads

2018-09-17 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 553af22f2 -> a97001d21


[CORE] Updates to remote cache reads

Covered by tests in DistributedSuite


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

Branch: refs/heads/master
Commit: a97001d21757ae214c86371141bd78a376200f66
Parents: 8f5a5a9
Author: Imran Rashid 
Authored: Wed Aug 22 16:38:28 2018 -0500
Committer: Imran Rashid 
Committed: Mon Sep 17 14:06:09 2018 -0500

--
 .../spark/network/buffer/ManagedBuffer.java |  5 +-
 .../spark/network/shuffle/DownloadFile.java | 47 ++
 .../network/shuffle/DownloadFileManager.java| 36 
 .../shuffle/DownloadFileWritableChannel.java| 30 +++
 .../network/shuffle/ExternalShuffleClient.java  |  4 +-
 .../network/shuffle/OneForOneBlockFetcher.java  | 28 +++---
 .../spark/network/shuffle/ShuffleClient.java|  4 +-
 .../network/shuffle/SimpleDownloadFile.java | 91 
 .../spark/network/shuffle/TempFileManager.java  | 36 
 .../spark/network/BlockTransferService.scala|  6 +-
 .../netty/NettyBlockTransferService.scala   |  4 +-
 .../org/apache/spark/storage/BlockManager.scala | 78 ++---
 .../org/apache/spark/storage/DiskStore.scala| 16 
 .../storage/ShuffleBlockFetcherIterator.scala   | 21 +++--
 .../spark/storage/BlockManagerSuite.scala   |  8 +-
 .../ShuffleBlockFetcherIteratorSuite.scala  |  6 +-
 16 files changed, 328 insertions(+), 92 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a97001d2/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
index 1861f8d..2d573f5 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
@@ -36,7 +36,10 @@ import java.nio.ByteBuffer;
  */
 public abstract class ManagedBuffer {
 
-  /** Number of bytes of the data. */
+  /**
+   * Number of bytes of the data. If this buffer will decrypt for all of the 
views into the data,
+   * this is the size of the decrypted data.
+   */
   public abstract long size();
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/a97001d2/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
--
diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
new file mode 100644
index 000..633622b
--- /dev/null
+++ 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
@@ -0,0 +1,47 @@
+/*
+ * 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.network.shuffle;
+
+import java.io.IOException;
+
+/**
+ * A handle on the file used when fetching remote data to disk.  Used to 
ensure the lifecycle of
+ * writing the data, reading it back, and then cleaning it up is followed.  
Specific implementations
+ * may also handle encryption.  The data can be read only via 
DownloadFileWritableChannel,
+ * which ensures data is not read until after the writer is closed.
+ */
+public interface DownloadFile {
+  /**
+   * Delete the file.
+   *
+   * @return  true if and only if the file or directory is
+   *  successfully deleted; false otherwise
+   */
+  boolean delete();
+
+  /**
+   * A channel for writing data to the file.  This special channel allows 
access to the data for
+   * reading, after the channel is closed, via 

[3/3] spark git commit: [PYSPARK] Updates to pyspark broadcast

2018-09-17 Thread irashid
[PYSPARK] Updates to pyspark broadcast


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

Branch: refs/heads/master
Commit: 58419b92673c46911c25bc6c6b13397f880c6424
Parents: 553af22
Author: Imran Rashid 
Authored: Mon Aug 13 21:35:34 2018 -0500
Committer: Imran Rashid 
Committed: Mon Sep 17 14:06:09 2018 -0500

--
 .../org/apache/spark/api/python/PythonRDD.scala | 299 ---
 .../apache/spark/api/python/PythonRunner.scala  |  52 +++-
 .../spark/api/python/PythonRDDSuite.scala   |  23 +-
 dev/sparktestsupport/modules.py |   2 +
 python/pyspark/broadcast.py |  58 +++-
 python/pyspark/context.py   |  64 ++--
 python/pyspark/serializers.py   |  51 
 python/pyspark/sql/session.py   |  12 +-
 python/pyspark/sql/tests.py |  45 ++-
 python/pyspark/test_broadcast.py| 126 
 python/pyspark/test_serializers.py  |  90 ++
 python/pyspark/tests.py |   9 +-
 python/pyspark/worker.py|  22 +-
 .../spark/sql/api/python/PythonSQLUtils.scala   |  47 ++-
 .../sql/execution/arrow/ArrowConverters.scala   |   9 +-
 15 files changed, 789 insertions(+), 120 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/58419b92/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index e639a84..8b5a7a9 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, 
Map => JMap}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.concurrent.Promise
+import scala.concurrent.duration.Duration
 import scala.language.existentials
-import scala.util.control.NonFatal
+import scala.util.Try
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.compress.CompressionCodec
@@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, 
JavaSparkContext}
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.input.PortableDataStream
 import org.apache.spark.internal.Logging
+import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.security.SocketAuthHelper
 import org.apache.spark.util._
@@ -169,27 +172,34 @@ private[spark] object PythonRDD extends Logging {
 
   def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: 
Int):
   JavaRDD[Array[Byte]] = {
-val file = new DataInputStream(new FileInputStream(filename))
+readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism)
+  }
+
+  def readRDDFromInputStream(
+  sc: SparkContext,
+  in: InputStream,
+  parallelism: Int): JavaRDD[Array[Byte]] = {
+val din = new DataInputStream(in)
 try {
   val objs = new mutable.ArrayBuffer[Array[Byte]]
   try {
 while (true) {
-  val length = file.readInt()
+  val length = din.readInt()
   val obj = new Array[Byte](length)
-  file.readFully(obj)
+  din.readFully(obj)
   objs += obj
 }
   } catch {
 case eof: EOFException => // No-op
   }
-  JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
+  JavaRDD.fromRDD(sc.parallelize(objs, parallelism))
 } finally {
-  file.close()
+  din.close()
 }
   }
 
-  def readBroadcastFromFile(sc: JavaSparkContext, path: String): 
Broadcast[PythonBroadcast] = {
-sc.broadcast(new PythonBroadcast(path))
+  def setupBroadcast(path: String): PythonBroadcast = {
+new PythonBroadcast(path)
   }
 
   def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) {
@@ -419,34 +429,15 @@ private[spark] object PythonRDD extends Logging {
*/
   private[spark] def serveToStream(
   threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = {
-val serverSocket = new ServerSocket(0, 1, 
InetAddress.getByName("localhost"))
-// Close the socket if no connection in 15 seconds
-serverSocket.setSoTimeout(15000)
-
-new Thread(threadName) {
-  setDaemon(true)
-  override def run() {
-try {
-  val sock = serverSocket.accept()
-  authHelper.authClient(sock)
-
-  val out = new BufferedOutputStream(sock.getOutputStream)
- 

[2/3] spark git commit: [PYSPARK][SQL] Updates to RowQueue

2018-09-17 Thread irashid
[PYSPARK][SQL] Updates to RowQueue

Tested with updates to RowQueueSuite


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

Branch: refs/heads/master
Commit: 8f5a5a9e5b9f273443b2721f80c99dc7397ef4c0
Parents: 58419b9
Author: Imran Rashid 
Authored: Thu Sep 6 12:11:47 2018 -0500
Committer: Imran Rashid 
Committed: Mon Sep 17 14:06:09 2018 -0500

--
 .../spark/sql/execution/python/RowQueue.scala   | 27 ++-
 .../sql/execution/python/RowQueueSuite.scala| 28 +++-
 2 files changed, 41 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8f5a5a9e/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
index e2fa6e7..d2820ff 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
@@ -21,9 +21,10 @@ import java.io._
 
 import com.google.common.io.Closeables
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkEnv, SparkException}
 import org.apache.spark.io.NioBufferedFileInputStream
 import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager}
+import org.apache.spark.serializer.SerializerManager
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.unsafe.Platform
 import org.apache.spark.unsafe.memory.MemoryBlock
@@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: 
MemoryBlock, numFields
  * A RowQueue that is backed by a file on disk. This queue will stop accepting 
new rows once any
  * reader has begun reading from the queue.
  */
-private[python] case class DiskRowQueue(file: File, fields: Int) extends 
RowQueue {
-  private var out = new DataOutputStream(
-new BufferedOutputStream(new FileOutputStream(file.toString)))
+private[python] case class DiskRowQueue(
+file: File,
+fields: Int,
+serMgr: SerializerManager) extends RowQueue {
+
+  private var out = new DataOutputStream(serMgr.wrapForEncryption(
+new BufferedOutputStream(new FileOutputStream(file.toString
   private var unreadBytes = 0L
 
   private var in: DataInputStream = _
@@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 if (out != null) {
   out.close()
   out = null
-  in = new DataInputStream(new NioBufferedFileInputStream(file))
+  in = new DataInputStream(serMgr.wrapForEncryption(
+new NioBufferedFileInputStream(file)))
 }
 
 if (unreadBytes > 0) {
@@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 private[python] case class HybridRowQueue(
 memManager: TaskMemoryManager,
 tempDir: File,
-numFields: Int)
+numFields: Int,
+serMgr: SerializerManager)
   extends MemoryConsumer(memManager) with RowQueue {
 
   // Each buffer should have at least one row
@@ -212,7 +219,7 @@ private[python] case class HybridRowQueue(
   }
 
   private def createDiskQueue(): RowQueue = {
-DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields)
+DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr)
   }
 
   private def createNewQueue(required: Long): RowQueue = {
@@ -279,3 +286,9 @@ private[python] case class HybridRowQueue(
 }
   }
 }
+
+private[python] object HybridRowQueue {
+  def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): 
HybridRowQueue = {
+HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/8f5a5a9e/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
index ffda33c..1ec9986 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
@@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python
 import java.io.File
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, 
TestMemoryManager}

spark git commit: [HOTFIX] fix lint-java

2018-09-13 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 575fea120 -> f3bbb7ceb


[HOTFIX] fix lint-java


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

Branch: refs/heads/branch-2.3
Commit: f3bbb7ceb9ae8038c3612f1fe5b8b44f0652711a
Parents: 575fea1
Author: Imran Rashid 
Authored: Thu Sep 13 09:47:45 2018 -0500
Committer: Imran Rashid 
Committed: Thu Sep 13 09:47:45 2018 -0500

--
 .../java/org/apache/spark/network/shuffle/DownloadFile.java| 6 +++---
 .../spark/network/shuffle/DownloadFileWritableChannel.java | 3 +--
 2 files changed, 4 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f3bbb7ce/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
--
diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
index 76c2d02..633622b 100644
--- 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
+++ 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
@@ -32,16 +32,16 @@ public interface DownloadFile {
* @return  true if and only if the file or directory is
*  successfully deleted; false otherwise
*/
-  public boolean delete();
+  boolean delete();
 
   /**
* A channel for writing data to the file.  This special channel allows 
access to the data for
* reading, after the channel is closed, via {@link 
DownloadFileWritableChannel#closeAndRead()}.
*/
-  public DownloadFileWritableChannel openForWriting() throws IOException;
+  DownloadFileWritableChannel openForWriting() throws IOException;
 
   /**
* The path of the file, intended only for debug purposes.
*/
-  public String path();
+  String path();
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f3bbb7ce/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java
--
diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java
index acf194c..dbbbac4 100644
--- 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java
+++ 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java
@@ -19,7 +19,6 @@ package org.apache.spark.network.shuffle;
 
 import org.apache.spark.network.buffer.ManagedBuffer;
 
-import java.io.OutputStream;
 import java.nio.channels.WritableByteChannel;
 
 /**
@@ -27,5 +26,5 @@ import java.nio.channels.WritableByteChannel;
  * after the writer has been closed.  Used with DownloadFile and 
DownloadFileManager.
  */
 public interface DownloadFileWritableChannel extends WritableByteChannel {
-  public ManagedBuffer closeAndRead();
+  ManagedBuffer closeAndRead();
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[1/4] spark git commit: [CORE] Updates to remote cache reads

2018-09-13 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 9ac9f36c4 -> 575fea120


[CORE] Updates to remote cache reads

Covered by tests in DistributedSuite


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

Branch: refs/heads/branch-2.3
Commit: 575fea120e25249716e3f680396580c5f9e26b5b
Parents: 6d742d1
Author: Imran Rashid 
Authored: Wed Aug 22 16:38:28 2018 -0500
Committer: Imran Rashid 
Committed: Thu Sep 13 09:19:56 2018 -0500

--
 .../spark/network/buffer/ManagedBuffer.java |  5 +-
 .../spark/network/shuffle/DownloadFile.java | 47 ++
 .../network/shuffle/DownloadFileManager.java| 36 
 .../shuffle/DownloadFileWritableChannel.java| 31 +++
 .../network/shuffle/ExternalShuffleClient.java  |  4 +-
 .../network/shuffle/OneForOneBlockFetcher.java  | 28 +++---
 .../spark/network/shuffle/ShuffleClient.java|  4 +-
 .../network/shuffle/SimpleDownloadFile.java | 91 
 .../spark/network/shuffle/TempFileManager.java  | 36 
 .../spark/network/BlockTransferService.scala|  6 +-
 .../netty/NettyBlockTransferService.scala   |  4 +-
 .../org/apache/spark/storage/BlockManager.scala | 78 ++---
 .../org/apache/spark/storage/DiskStore.scala| 16 
 .../storage/ShuffleBlockFetcherIterator.scala   | 21 +++--
 .../spark/storage/BlockManagerSuite.scala   |  8 +-
 .../ShuffleBlockFetcherIteratorSuite.scala  |  6 +-
 16 files changed, 329 insertions(+), 92 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/575fea12/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
index 1861f8d..2d573f5 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java
@@ -36,7 +36,10 @@ import java.nio.ByteBuffer;
  */
 public abstract class ManagedBuffer {
 
-  /** Number of bytes of the data. */
+  /**
+   * Number of bytes of the data. If this buffer will decrypt for all of the 
views into the data,
+   * this is the size of the decrypted data.
+   */
   public abstract long size();
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/575fea12/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
--
diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
new file mode 100644
index 000..76c2d02
--- /dev/null
+++ 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java
@@ -0,0 +1,47 @@
+/*
+ * 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.network.shuffle;
+
+import java.io.IOException;
+
+/**
+ * A handle on the file used when fetching remote data to disk.  Used to 
ensure the lifecycle of
+ * writing the data, reading it back, and then cleaning it up is followed.  
Specific implementations
+ * may also handle encryption.  The data can be read only via 
DownloadFileWritableChannel,
+ * which ensures data is not read until after the writer is closed.
+ */
+public interface DownloadFile {
+  /**
+   * Delete the file.
+   *
+   * @return  true if and only if the file or directory is
+   *  successfully deleted; false otherwise
+   */
+  public boolean delete();
+
+  /**
+   * A channel for writing data to the file.  This special channel allows 
access to the data for
+   * reading, after the channel is 

[3/4] spark git commit: [PYSPARK][SQL] Updates to RowQueue

2018-09-13 Thread irashid
[PYSPARK][SQL] Updates to RowQueue

Tested with updates to RowQueueSuite


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

Branch: refs/heads/branch-2.3
Commit: 6d742d1bd71aa3803dce91a830b37284cb18cf70
Parents: 09dd34c
Author: Imran Rashid 
Authored: Thu Sep 6 12:11:47 2018 -0500
Committer: Imran Rashid 
Committed: Thu Sep 13 09:19:56 2018 -0500

--
 .../spark/sql/execution/python/RowQueue.scala   | 27 ++-
 .../sql/execution/python/RowQueueSuite.scala| 28 +++-
 2 files changed, 41 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6d742d1b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
index e2fa6e7..d2820ff 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala
@@ -21,9 +21,10 @@ import java.io._
 
 import com.google.common.io.Closeables
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkEnv, SparkException}
 import org.apache.spark.io.NioBufferedFileInputStream
 import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager}
+import org.apache.spark.serializer.SerializerManager
 import org.apache.spark.sql.catalyst.expressions.UnsafeRow
 import org.apache.spark.unsafe.Platform
 import org.apache.spark.unsafe.memory.MemoryBlock
@@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: 
MemoryBlock, numFields
  * A RowQueue that is backed by a file on disk. This queue will stop accepting 
new rows once any
  * reader has begun reading from the queue.
  */
-private[python] case class DiskRowQueue(file: File, fields: Int) extends 
RowQueue {
-  private var out = new DataOutputStream(
-new BufferedOutputStream(new FileOutputStream(file.toString)))
+private[python] case class DiskRowQueue(
+file: File,
+fields: Int,
+serMgr: SerializerManager) extends RowQueue {
+
+  private var out = new DataOutputStream(serMgr.wrapForEncryption(
+new BufferedOutputStream(new FileOutputStream(file.toString
   private var unreadBytes = 0L
 
   private var in: DataInputStream = _
@@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 if (out != null) {
   out.close()
   out = null
-  in = new DataInputStream(new NioBufferedFileInputStream(file))
+  in = new DataInputStream(serMgr.wrapForEncryption(
+new NioBufferedFileInputStream(file)))
 }
 
 if (unreadBytes > 0) {
@@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: 
Int) extends RowQueu
 private[python] case class HybridRowQueue(
 memManager: TaskMemoryManager,
 tempDir: File,
-numFields: Int)
+numFields: Int,
+serMgr: SerializerManager)
   extends MemoryConsumer(memManager) with RowQueue {
 
   // Each buffer should have at least one row
@@ -212,7 +219,7 @@ private[python] case class HybridRowQueue(
   }
 
   private def createDiskQueue(): RowQueue = {
-DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields)
+DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr)
   }
 
   private def createNewQueue(required: Long): RowQueue = {
@@ -279,3 +286,9 @@ private[python] case class HybridRowQueue(
 }
   }
 }
+
+private[python] object HybridRowQueue {
+  def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): 
HybridRowQueue = {
+HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/6d742d1b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
--
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
index ffda33c..1ec9986 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala
@@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python
 import java.io.File
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, 
TestMemoryManager}

[2/4] spark git commit: [PYSPARK] Updates to pyspark broadcast

2018-09-13 Thread irashid
[PYSPARK] Updates to pyspark broadcast


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

Branch: refs/heads/branch-2.3
Commit: 09dd34cb1706f2477a89174d6a1a0f17ed5b0a65
Parents: a2a54a5
Author: Imran Rashid 
Authored: Mon Aug 13 21:35:34 2018 -0500
Committer: Imran Rashid 
Committed: Thu Sep 13 09:19:56 2018 -0500

--
 .../org/apache/spark/api/python/PythonRDD.scala | 297 ---
 .../apache/spark/api/python/PythonRunner.scala  |  52 +++-
 .../spark/api/python/PythonRDDSuite.scala   |  23 +-
 dev/sparktestsupport/modules.py |   2 +
 python/pyspark/broadcast.py |  58 +++-
 python/pyspark/context.py   |  49 ++-
 python/pyspark/serializers.py   |  58 
 python/pyspark/test_broadcast.py| 126 
 python/pyspark/test_serializers.py  |  90 ++
 python/pyspark/worker.py|  24 +-
 10 files changed, 695 insertions(+), 84 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/09dd34cb/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 8bc0ff7..5e6bd96 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, 
Map => JMap}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.concurrent.Promise
+import scala.concurrent.duration.Duration
 import scala.language.existentials
-import scala.util.control.NonFatal
+import scala.util.Try
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.compress.CompressionCodec
@@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, 
JavaSparkContext}
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.input.PortableDataStream
 import org.apache.spark.internal.Logging
+import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.security.SocketAuthHelper
 import org.apache.spark.util._
@@ -168,27 +171,34 @@ private[spark] object PythonRDD extends Logging {
 
   def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: 
Int):
   JavaRDD[Array[Byte]] = {
-val file = new DataInputStream(new FileInputStream(filename))
+readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism)
+  }
+
+  def readRDDFromInputStream(
+  sc: SparkContext,
+  in: InputStream,
+  parallelism: Int): JavaRDD[Array[Byte]] = {
+val din = new DataInputStream(in)
 try {
   val objs = new mutable.ArrayBuffer[Array[Byte]]
   try {
 while (true) {
-  val length = file.readInt()
+  val length = din.readInt()
   val obj = new Array[Byte](length)
-  file.readFully(obj)
+  din.readFully(obj)
   objs += obj
 }
   } catch {
 case eof: EOFException => // No-op
   }
-  JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
+  JavaRDD.fromRDD(sc.parallelize(objs, parallelism))
 } finally {
-  file.close()
+  din.close()
 }
   }
 
-  def readBroadcastFromFile(sc: JavaSparkContext, path: String): 
Broadcast[PythonBroadcast] = {
-sc.broadcast(new PythonBroadcast(path))
+  def setupBroadcast(path: String): PythonBroadcast = {
+new PythonBroadcast(path)
   }
 
   def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) {
@@ -398,34 +408,15 @@ private[spark] object PythonRDD extends Logging {
* data collected from this job, and the secret for authentication.
*/
   def serveIterator(items: Iterator[_], threadName: String): Array[Any] = {
-val serverSocket = new ServerSocket(0, 1, 
InetAddress.getByName("localhost"))
-// Close the socket if no connection in 15 seconds
-serverSocket.setSoTimeout(15000)
-
-new Thread(threadName) {
-  setDaemon(true)
-  override def run() {
-try {
-  val sock = serverSocket.accept()
-  authHelper.authClient(sock)
-
-  val out = new DataOutputStream(new 
BufferedOutputStream(sock.getOutputStream))
-  Utils.tryWithSafeFinally {
-writeIteratorToStream(items, out)
-  } {
-out.close()
-sock.close()
-  }
-} catch {
-  case NonFatal(e) =>
-   

[4/4] spark git commit: [SPARK-25253][PYSPARK] Refactor local connection & auth code

2018-09-13 Thread irashid
[SPARK-25253][PYSPARK] Refactor local connection & auth code

This eliminates some duplication in the code to connect to a server on 
localhost to talk directly to the jvm.  Also it gives consistent ipv6 and error 
handling.  Two other incidental changes, that shouldn't matter:
1) python barrier tasks perform authentication immediately (rather than waiting 
for the BARRIER_FUNCTION indicator)
2) for `rdd._load_from_socket`, the timeout is only increased after 
authentication.

Closes #22247 from squito/py_connection_refactor.

Authored-by: Imran Rashid 
Signed-off-by: hyukjinkwon 
(cherry picked from commit 38391c9aa8a88fcebb337934f30298a32d91596b)


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

Branch: refs/heads/branch-2.3
Commit: a2a54a5f49364a1825932c9f04eb0ff82dd7d465
Parents: 9ac9f36
Author: Imran Rashid 
Authored: Wed Aug 29 09:47:38 2018 +0800
Committer: Imran Rashid 
Committed: Thu Sep 13 09:19:56 2018 -0500

--
 python/pyspark/java_gateway.py | 32 +++-
 python/pyspark/rdd.py  | 24 ++--
 python/pyspark/worker.py   |  7 ++-
 3 files changed, 35 insertions(+), 28 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a2a54a5f/python/pyspark/java_gateway.py
--
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 0afbe9d..eed866d 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -134,7 +134,7 @@ def launch_gateway(conf=None):
 return gateway
 
 
-def do_server_auth(conn, auth_secret):
+def _do_server_auth(conn, auth_secret):
 """
 Performs the authentication protocol defined by the SocketAuthHelper class 
on the given
 file-like object 'conn'.
@@ -145,3 +145,33 @@ def do_server_auth(conn, auth_secret):
 if reply != "ok":
 conn.close()
 raise Exception("Unexpected reply from iterator server.")
+
+
+def local_connect_and_auth(port, auth_secret):
+"""
+Connect to local host, authenticate with it, and return a (sockfile,sock) 
for that connection.
+Handles IPV4 & IPV6, does some error handling.
+:param port
+:param auth_secret
+:return: a tuple with (sockfile, sock)
+"""
+sock = None
+errors = []
+# Support for both IPv4 and IPv6.
+# On most of IPv6-ready systems, IPv6 will take precedence.
+for res in socket.getaddrinfo("127.0.0.1", port, socket.AF_UNSPEC, 
socket.SOCK_STREAM):
+af, socktype, proto, _, sa = res
+try:
+sock = socket.socket(af, socktype, proto)
+sock.settimeout(15)
+sock.connect(sa)
+sockfile = sock.makefile("rwb", 65536)
+_do_server_auth(sockfile, auth_secret)
+return (sockfile, sock)
+except socket.error as e:
+emsg = _exception_message(e)
+errors.append("tried to connect to %s, but an error occured: %s" % 
(sa, emsg))
+sock.close()
+sock = None
+else:
+raise Exception("could not open socket: %s" % errors)

http://git-wip-us.apache.org/repos/asf/spark/blob/a2a54a5f/python/pyspark/rdd.py
--
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index eadb5ab..3f6a8e6 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -39,7 +39,7 @@ if sys.version > '3':
 else:
 from itertools import imap as map, ifilter as filter
 
-from pyspark.java_gateway import do_server_auth
+from pyspark.java_gateway import local_connect_and_auth
 from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
 BatchedSerializer, CloudPickleSerializer, PairDeserializer, \
 PickleSerializer, pack_long, AutoBatchedSerializer, write_with_length, \
@@ -139,30 +139,10 @@ def _parse_memory(s):
 
 
 def _load_from_socket(sock_info, serializer):
-port, auth_secret = sock_info
-sock = None
-# Support for both IPv4 and IPv6.
-# On most of IPv6-ready systems, IPv6 will take precedence.
-for res in socket.getaddrinfo("localhost", port, socket.AF_UNSPEC, 
socket.SOCK_STREAM):
-af, socktype, proto, canonname, sa = res
-sock = socket.socket(af, socktype, proto)
-try:
-sock.settimeout(15)
-sock.connect(sa)
-except socket.error:
-sock.close()
-sock = None
-continue
-break
-if not sock:
-raise Exception("could not open socket")
+(sockfile, sock) = local_connect_and_auth(*sock_info)
 # The RDD 

spark git commit: [PYSPARK] Updates to Accumulators

2018-08-03 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.1 a3eb07db3 -> b2e0f68f6


[PYSPARK] Updates to Accumulators

(cherry picked from commit 15fc2372269159ea2556b028d4eb8860c4108650)


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

Branch: refs/heads/branch-2.1
Commit: b2e0f68f615cbe2cf74f9813ece76c311fe8e911
Parents: a3eb07d
Author: LucaCanali 
Authored: Wed Jul 18 23:19:02 2018 +0200
Committer: Imran Rashid 
Committed: Fri Aug 3 16:30:40 2018 -0500

--
 .../org/apache/spark/api/python/PythonRDD.scala | 12 +++--
 python/pyspark/accumulators.py  | 53 +++-
 python/pyspark/context.py   |  5 +-
 3 files changed, 53 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b2e0f68f/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index b1190b9..de548e8 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -886,8 +886,9 @@ class BytesToString extends 
org.apache.spark.api.java.function.Function[Array[By
  */
 private[spark] class PythonAccumulatorV2(
 @transient private val serverHost: String,
-private val serverPort: Int)
-  extends CollectionAccumulator[Array[Byte]] {
+private val serverPort: Int,
+private val secretToken: String)
+  extends CollectionAccumulator[Array[Byte]] with Logging{
 
   Utils.checkHost(serverHost, "Expected hostname")
 
@@ -902,12 +903,17 @@ private[spark] class PythonAccumulatorV2(
   private def openSocket(): Socket = synchronized {
 if (socket == null || socket.isClosed) {
   socket = new Socket(serverHost, serverPort)
+  logInfo(s"Connected to AccumulatorServer at host: $serverHost port: 
$serverPort")
+  // send the secret just for the initial authentication when opening a 
new connection
+  
socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8))
 }
 socket
   }
 
   // Need to override so the types match with PythonFunction
-  override def copyAndReset(): PythonAccumulatorV2 = new 
PythonAccumulatorV2(serverHost, serverPort)
+  override def copyAndReset(): PythonAccumulatorV2 = {
+new PythonAccumulatorV2(serverHost, serverPort, secretToken)
+  }
 
   override def merge(other: AccumulatorV2[Array[Byte], JList[Array[Byte]]]): 
Unit = synchronized {
 val otherPythonAccumulator = other.asInstanceOf[PythonAccumulatorV2]

http://git-wip-us.apache.org/repos/asf/spark/blob/b2e0f68f/python/pyspark/accumulators.py
--
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index 6ef8cf5..bc0be07 100644
--- a/python/pyspark/accumulators.py
+++ b/python/pyspark/accumulators.py
@@ -228,20 +228,49 @@ class 
_UpdateRequestHandler(SocketServer.StreamRequestHandler):
 
 def handle(self):
 from pyspark.accumulators import _accumulatorRegistry
-while not self.server.server_shutdown:
-# Poll every 1 second for new data -- don't block in case of 
shutdown.
-r, _, _ = select.select([self.rfile], [], [], 1)
-if self.rfile in r:
-num_updates = read_int(self.rfile)
-for _ in range(num_updates):
-(aid, update) = pickleSer._read_with_length(self.rfile)
-_accumulatorRegistry[aid] += update
-# Write a byte in acknowledgement
-self.wfile.write(struct.pack("!b", 1))
+auth_token = self.server.auth_token
+
+def poll(func):
+while not self.server.server_shutdown:
+# Poll every 1 second for new data -- don't block in case of 
shutdown.
+r, _, _ = select.select([self.rfile], [], [], 1)
+if self.rfile in r:
+if func():
+break
+
+def accum_updates():
+num_updates = read_int(self.rfile)
+for _ in range(num_updates):
+(aid, update) = pickleSer._read_with_length(self.rfile)
+_accumulatorRegistry[aid] += update
+# Write a byte in acknowledgement
+self.wfile.write(struct.pack("!b", 1))
+return False
+
+def authenticate_and_accum_updates():
+received_token = self.rfile.read(len(auth_token))
+if 

spark git commit: [PYSPARK] Updates to Accumulators

2018-08-03 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 22ce8051f -> a5624c7ae


[PYSPARK] Updates to Accumulators

(cherry picked from commit 15fc2372269159ea2556b028d4eb8860c4108650)


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

Branch: refs/heads/branch-2.2
Commit: a5624c7ae29d6d49117dd78642879bf978212d30
Parents: 22ce805
Author: LucaCanali 
Authored: Wed Jul 18 23:19:02 2018 +0200
Committer: Imran Rashid 
Committed: Fri Aug 3 14:49:38 2018 -0500

--
 .../org/apache/spark/api/python/PythonRDD.scala | 12 +++--
 python/pyspark/accumulators.py  | 53 +++-
 python/pyspark/context.py   |  5 +-
 3 files changed, 53 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a5624c7a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 0662792..7b5a179 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -900,8 +900,9 @@ class BytesToString extends 
org.apache.spark.api.java.function.Function[Array[By
  */
 private[spark] class PythonAccumulatorV2(
 @transient private val serverHost: String,
-private val serverPort: Int)
-  extends CollectionAccumulator[Array[Byte]] {
+private val serverPort: Int,
+private val secretToken: String)
+  extends CollectionAccumulator[Array[Byte]] with Logging{
 
   Utils.checkHost(serverHost, "Expected hostname")
 
@@ -916,12 +917,17 @@ private[spark] class PythonAccumulatorV2(
   private def openSocket(): Socket = synchronized {
 if (socket == null || socket.isClosed) {
   socket = new Socket(serverHost, serverPort)
+  logInfo(s"Connected to AccumulatorServer at host: $serverHost port: 
$serverPort")
+  // send the secret just for the initial authentication when opening a 
new connection
+  
socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8))
 }
 socket
   }
 
   // Need to override so the types match with PythonFunction
-  override def copyAndReset(): PythonAccumulatorV2 = new 
PythonAccumulatorV2(serverHost, serverPort)
+  override def copyAndReset(): PythonAccumulatorV2 = {
+new PythonAccumulatorV2(serverHost, serverPort, secretToken)
+  }
 
   override def merge(other: AccumulatorV2[Array[Byte], JList[Array[Byte]]]): 
Unit = synchronized {
 val otherPythonAccumulator = other.asInstanceOf[PythonAccumulatorV2]

http://git-wip-us.apache.org/repos/asf/spark/blob/a5624c7a/python/pyspark/accumulators.py
--
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index 6ef8cf5..bc0be07 100644
--- a/python/pyspark/accumulators.py
+++ b/python/pyspark/accumulators.py
@@ -228,20 +228,49 @@ class 
_UpdateRequestHandler(SocketServer.StreamRequestHandler):
 
 def handle(self):
 from pyspark.accumulators import _accumulatorRegistry
-while not self.server.server_shutdown:
-# Poll every 1 second for new data -- don't block in case of 
shutdown.
-r, _, _ = select.select([self.rfile], [], [], 1)
-if self.rfile in r:
-num_updates = read_int(self.rfile)
-for _ in range(num_updates):
-(aid, update) = pickleSer._read_with_length(self.rfile)
-_accumulatorRegistry[aid] += update
-# Write a byte in acknowledgement
-self.wfile.write(struct.pack("!b", 1))
+auth_token = self.server.auth_token
+
+def poll(func):
+while not self.server.server_shutdown:
+# Poll every 1 second for new data -- don't block in case of 
shutdown.
+r, _, _ = select.select([self.rfile], [], [], 1)
+if self.rfile in r:
+if func():
+break
+
+def accum_updates():
+num_updates = read_int(self.rfile)
+for _ in range(num_updates):
+(aid, update) = pickleSer._read_with_length(self.rfile)
+_accumulatorRegistry[aid] += update
+# Write a byte in acknowledgement
+self.wfile.write(struct.pack("!b", 1))
+return False
+
+def authenticate_and_accum_updates():
+received_token = self.rfile.read(len(auth_token))
+if 

spark git commit: [PYSPARK] Updates to Accumulators

2018-08-02 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 5b187a85a -> 8080c937d


[PYSPARK] Updates to Accumulators

(cherry picked from commit 15fc2372269159ea2556b028d4eb8860c4108650)


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

Branch: refs/heads/branch-2.3
Commit: 8080c937d3752aee2fd36f0045a057f7130f6fe4
Parents: 5b187a8
Author: LucaCanali 
Authored: Wed Jul 18 23:19:02 2018 +0200
Committer: Imran Rashid 
Committed: Thu Aug 2 21:05:03 2018 -0500

--
 .../org/apache/spark/api/python/PythonRDD.scala | 12 +++--
 python/pyspark/accumulators.py  | 53 +++-
 python/pyspark/context.py   |  5 +-
 3 files changed, 53 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8080c937/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala 
b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index a1ee2f7..8bc0ff7 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -586,8 +586,9 @@ class BytesToString extends 
org.apache.spark.api.java.function.Function[Array[By
  */
 private[spark] class PythonAccumulatorV2(
 @transient private val serverHost: String,
-private val serverPort: Int)
-  extends CollectionAccumulator[Array[Byte]] {
+private val serverPort: Int,
+private val secretToken: String)
+  extends CollectionAccumulator[Array[Byte]] with Logging{
 
   Utils.checkHost(serverHost)
 
@@ -602,12 +603,17 @@ private[spark] class PythonAccumulatorV2(
   private def openSocket(): Socket = synchronized {
 if (socket == null || socket.isClosed) {
   socket = new Socket(serverHost, serverPort)
+  logInfo(s"Connected to AccumulatorServer at host: $serverHost port: 
$serverPort")
+  // send the secret just for the initial authentication when opening a 
new connection
+  
socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8))
 }
 socket
   }
 
   // Need to override so the types match with PythonFunction
-  override def copyAndReset(): PythonAccumulatorV2 = new 
PythonAccumulatorV2(serverHost, serverPort)
+  override def copyAndReset(): PythonAccumulatorV2 = {
+new PythonAccumulatorV2(serverHost, serverPort, secretToken)
+  }
 
   override def merge(other: AccumulatorV2[Array[Byte], JList[Array[Byte]]]): 
Unit = synchronized {
 val otherPythonAccumulator = other.asInstanceOf[PythonAccumulatorV2]

http://git-wip-us.apache.org/repos/asf/spark/blob/8080c937/python/pyspark/accumulators.py
--
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index 6ef8cf5..bc0be07 100644
--- a/python/pyspark/accumulators.py
+++ b/python/pyspark/accumulators.py
@@ -228,20 +228,49 @@ class 
_UpdateRequestHandler(SocketServer.StreamRequestHandler):
 
 def handle(self):
 from pyspark.accumulators import _accumulatorRegistry
-while not self.server.server_shutdown:
-# Poll every 1 second for new data -- don't block in case of 
shutdown.
-r, _, _ = select.select([self.rfile], [], [], 1)
-if self.rfile in r:
-num_updates = read_int(self.rfile)
-for _ in range(num_updates):
-(aid, update) = pickleSer._read_with_length(self.rfile)
-_accumulatorRegistry[aid] += update
-# Write a byte in acknowledgement
-self.wfile.write(struct.pack("!b", 1))
+auth_token = self.server.auth_token
+
+def poll(func):
+while not self.server.server_shutdown:
+# Poll every 1 second for new data -- don't block in case of 
shutdown.
+r, _, _ = select.select([self.rfile], [], [], 1)
+if self.rfile in r:
+if func():
+break
+
+def accum_updates():
+num_updates = read_int(self.rfile)
+for _ in range(num_updates):
+(aid, update) = pickleSer._read_with_length(self.rfile)
+_accumulatorRegistry[aid] += update
+# Write a byte in acknowledgement
+self.wfile.write(struct.pack("!b", 1))
+return False
+
+def authenticate_and_accum_updates():
+received_token = self.rfile.read(len(auth_token))
+if isinstance(received_token, bytes):
+  

spark git commit: [SPARK-24801][CORE] Avoid memory waste by empty byte[] arrays in SaslEncryption$EncryptedMessage

2018-07-26 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master fa09d9192 -> 094aa5971


[SPARK-24801][CORE] Avoid memory waste by empty byte[] arrays in 
SaslEncryption$EncryptedMessage

## What changes were proposed in this pull request?

Initialize SaslEncryption$EncryptedMessage.byteChannel lazily,
so that empty, not yet used instances of ByteArrayWritableChannel
referenced by this field don't use up memory.

I analyzed a heap dump from Yarn Node Manager where this code is used, and 
found that there are over 40,000 of the above objects in memory, each with a 
big empty byte[] array. The reason they are all there is because of Netty 
queued up a large number of messages in memory  before transferTo() is called. 
There is a small number of netty ChannelOutboundBuffer objects, and then 
collectively , via linked lists starting from their flushedEntry data fields, 
they end up referencing over 40K ChannelOutboundBuffer$Entry objects, which 
ultimately reference EncryptedMessage objects.

## How was this patch tested?

Ran all the tests locally.

Author: Misha Dmitriev 

Closes #21811 from countmdm/misha/spark-24801.


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

Branch: refs/heads/master
Commit: 094aa597155dfcbf41a2490c9e462415e3824901
Parents: fa09d91
Author: Misha Dmitriev 
Authored: Thu Jul 26 22:15:12 2018 -0500
Committer: Imran Rashid 
Committed: Thu Jul 26 22:15:12 2018 -0500

--
 .../org/apache/spark/network/sasl/SaslEncryption.java | 10 +++---
 1 file changed, 7 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/094aa597/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
--
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
 
b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
index 3ac9081..d3b2a33 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
@@ -135,13 +135,14 @@ class SaslEncryption {
 private final boolean isByteBuf;
 private final ByteBuf buf;
 private final FileRegion region;
+private final int maxOutboundBlockSize;
 
 /**
  * A channel used to buffer input data for encryption. The channel has an 
upper size bound
  * so that if the input is larger than the allowed buffer, it will be 
broken into multiple
- * chunks.
+ * chunks. Made non-final to enable lazy initialization, which saves 
memory.
  */
-private final ByteArrayWritableChannel byteChannel;
+private ByteArrayWritableChannel byteChannel;
 
 private ByteBuf currentHeader;
 private ByteBuffer currentChunk;
@@ -157,7 +158,7 @@ class SaslEncryption {
   this.isByteBuf = msg instanceof ByteBuf;
   this.buf = isByteBuf ? (ByteBuf) msg : null;
   this.region = isByteBuf ? null : (FileRegion) msg;
-  this.byteChannel = new ByteArrayWritableChannel(maxOutboundBlockSize);
+  this.maxOutboundBlockSize = maxOutboundBlockSize;
 }
 
 /**
@@ -292,6 +293,9 @@ class SaslEncryption {
 }
 
 private void nextChunk() throws IOException {
+  if (byteChannel == null) {
+byteChannel = new ByteArrayWritableChannel(maxOutboundBlockSize);
+  }
   byteChannel.reset();
   if (isByteBuf) {
 int copied = byteChannel.write(buf.nioBuffer());


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-16630][YARN] Blacklist a node if executors won't launch on it

2018-06-21 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master c0cad596b -> b56e9c613


[SPARK-16630][YARN] Blacklist a node if executors won't launch on it

## What changes were proposed in this pull request?

This change extends YARN resource allocation handling with blacklisting 
functionality.
This handles cases when node is messed up or misconfigured such that a 
container won't launch on it. Before this change backlisting only focused on 
task execution but this change introduces YarnAllocatorBlacklistTracker which 
tracks allocation failures per host (when enabled via 
"spark.yarn.blacklist.executor.launch.blacklisting.enabled").

## How was this patch tested?

### With unit tests

Including a new suite: YarnAllocatorBlacklistTrackerSuite.

 Manually

It was tested on a cluster by deleting the Spark jars on one of the node.

 Behaviour before these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf 
spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6"
```

Log is:
```
18/04/12 06:49:36 INFO yarn.ApplicationMaster: Final app status: FAILED, 
exitCode: 11, (reason: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Unregistering ApplicationMaster 
with FAILED (diag message: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO impl.AMRMClientImpl: Waiting for application to be 
successfully unregistered.
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Deleting staging directory 
hdfs://apiros-1.gce.test.com:8020/user/systest/.sparkStaging/application_1523459048274_0016
18/04/12 06:49:39 INFO util.ShutdownHookManager: Shutdown hook called
```

 Behaviour after these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf 
spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6" --conf 
"spark.yarn.blacklist.executor.launch.blacklisting.enabled=true"
```

And the log is:
```
18/04/13 05:37:43 INFO yarn.YarnAllocator: Will request 1 executor 
container(s), each with 1 core(s) and 4505 MB memory (including 409 MB of 
overhead)
18/04/13 05:37:43 INFO yarn.YarnAllocator: Submitted 1 unlocalized container 
requests.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Launching container 
container_1523459048274_0025_01_08 on host apiros-4.gce.test.com for 
executor with ID 6
18/04/13 05:37:43 INFO yarn.YarnAllocator: Received 1 containers from YARN, 
launching executors on 1 of them.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Completed container 
container_1523459048274_0025_01_07 on host: apiros-4.gce.test.com (state: 
COMPLETE, exit status: 1)
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as 
YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN 
application master's blacklist: List(apiros-4.gce.test.com)
18/04/13 05:37:43 WARN yarn.YarnAllocator: Container marked as failed: 
container_1523459048274_0025_01_07 on host: apiros-4.gce.test.com. Exit 
status: 1. Diagnostics: Exception from container-launch.
Container id: container_1523459048274_0025_01_07
Exit code: 1
Stack trace: ExitCodeException exitCode=1:
at org.apache.hadoop.util.Shell.runCommand(Shell.java:604)
at org.apache.hadoop.util.Shell.run(Shell.java:507)
at 
org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:789)
at 
org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)
at 
org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)
at 
org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
```

Where the most important part is:

```
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as 
YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN 
application master's blacklist: List(apiros-4.gce.test.com)
```

And execution was continued (no shutdown called).

### Testing the backlisting of the whole cluster

Starting Spark with YARN blacklisting enabled then removing a the Spark core 
jar one by one from all the cluster nodes. Then executing a simple spark job 
which fails checking the yarn log the expected exit status is contained:

```
18/06/15 01:07:10 INFO yarn.ApplicationMaster: Final app status: FAILED, 
exitCode: 11, (reason: Due to executor failures all available nodes are 
blacklisted)
18/06/15 01:07:13 INFO 

spark git commit: [SPARK-24416] Fix configuration specification for killBlacklisted executors

2018-06-12 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 2824f1436 -> 3af1d3e6d


[SPARK-24416] Fix configuration specification for killBlacklisted executors

## What changes were proposed in this pull request?

spark.blacklist.killBlacklistedExecutors is defined as

(Experimental) If set to "true", allow Spark to automatically kill, and attempt 
to re-create, executors when they are blacklisted. Note that, when an entire 
node is added to the blacklist, all of the executors on that node will be 
killed.

I presume the killing of blacklisted executors only happens after the stage 
completes successfully and all tasks have completed or on fetch failures 
(updateBlacklistForFetchFailure/updateBlacklistForSuccessfulTaskSet). It is 
confusing because the definition states that the executor will be attempted to 
be recreated as soon as it is blacklisted. This is not true while the stage is 
in progress and an executor is blacklisted, it will not attempt to cleanup 
until the stage finishes.

Author: Sanket Chintapalli 

Closes #21475 from redsanket/SPARK-24416.


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

Branch: refs/heads/master
Commit: 3af1d3e6d95719e15a997877d5ecd3bb40c08b9c
Parents: 2824f14
Author: Sanket Chintapalli 
Authored: Tue Jun 12 13:55:08 2018 -0500
Committer: Imran Rashid 
Committed: Tue Jun 12 13:55:08 2018 -0500

--
 docs/configuration.md | 7 ---
 1 file changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3af1d3e6/docs/configuration.md
--
diff --git a/docs/configuration.md b/docs/configuration.md
index 5588c37..6aa7878 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1656,9 +1656,10 @@ Apart from these, the following properties are also 
available, and may be useful
   spark.blacklist.killBlacklistedExecutors
   false
   
-(Experimental) If set to "true", allow Spark to automatically kill, and 
attempt to re-create,
-executors when they are blacklisted.  Note that, when an entire node is 
added to the blacklist,
-all of the executors on that node will be killed.
+(Experimental) If set to "true", allow Spark to automatically kill the 
executors 
+when they are blacklisted on fetch failure or blacklisted for the entire 
application, 
+as controlled by spark.blacklist.application.*. Note that, when an entire 
node is added 
+to the blacklist, all of the executors on that node will be killed.
   
 
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-23433][CORE] Late zombie task completions update all tasksets

2018-05-03 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 4f1ae3af9 -> 154bbc959


[SPARK-23433][CORE] Late zombie task completions update all tasksets

Fetch failure lead to multiple tasksets which are active for a given
stage.  While there is only one "active" version of the taskset, the
earlier attempts can still have running tasks, which can complete
successfully.  So a task completion needs to update every taskset
so that it knows the partition is completed.  That way the final active
taskset does not try to submit another task for the same partition,
and so that it knows when it is completed and when it should be
marked as a "zombie".

Added a regression test.

Author: Imran Rashid 

Closes #21131 from squito/SPARK-23433.

(cherry picked from commit 94641fe6cc68e5977dd8663b8f232a287a783acb)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.2
Commit: 154bbc95989b34b2007022b6082ed17864bbaa32
Parents: 4f1ae3a
Author: Imran Rashid 
Authored: Thu May 3 10:59:18 2018 -0500
Committer: Imran Rashid 
Committed: Thu May 3 10:59:45 2018 -0500

--
 .../spark/scheduler/TaskSchedulerImpl.scala |  14 +++
 .../apache/spark/scheduler/TaskSetManager.scala |  20 +++-
 .../scheduler/TaskSchedulerImplSuite.scala  | 104 +++
 3 files changed, 137 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/154bbc95/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 1b6bc91..df6407b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -693,6 +693,20 @@ private[spark] class TaskSchedulerImpl private[scheduler](
 }
   }
 
+  /**
+   * Marks the task has completed in all TaskSetManagers for the given stage.
+   *
+   * After stage failure and retry, there may be multiple TaskSetManagers for 
the stage.
+   * If an earlier attempt of a stage completes a task, we should ensure that 
the later attempts
+   * do not also submit those same tasks.  That also means that a task 
completion from an  earlier
+   * attempt can lead to the entire stage getting marked as successful.
+   */
+  private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, 
partitionId: Int) = {
+taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm 
=>
+  tsm.markPartitionCompleted(partitionId)
+}
+  }
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/154bbc95/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 2f4e46c..d9515fb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -74,6 +74,8 @@ private[spark] class TaskSetManager(
   val ser = env.closureSerializer.newInstance()
 
   val tasks = taskSet.tasks
+  private[scheduler] val partitionToIndex = tasks.zipWithIndex
+.map { case (t, idx) => t.partitionId -> idx }.toMap
   val numTasks = tasks.length
   val copiesRunning = new Array[Int](numTasks)
 
@@ -149,7 +151,7 @@ private[spark] class TaskSetManager(
   private[scheduler] val speculatableTasks = new HashSet[Int]
 
   // Task index, start and finish time for each task attempt (indexed by task 
ID)
-  private val taskInfos = new HashMap[Long, TaskInfo]
+  private[scheduler] val taskInfos = new HashMap[Long, TaskInfo]
 
   // Use a MedianHeap to record durations of successful tasks so we know when 
to launch
   // speculative tasks. This is only used when speculation is enabled, to 
avoid the overhead
@@ -744,6 +746,9 @@ private[spark] class TaskSetManager(
   logInfo("Ignoring task-finished event for " + info.id + " in stage " + 
taskSet.id +
 " because task " + index + " has already completed successfully")
 }
+// There may be multiple tasksets for this stage -- we let all of them 
know that the partition
+// was completed.  This may result in some of the tasksets getting 
completed.
+

spark git commit: [SPARK-23433][CORE] Late zombie task completions update all tasksets

2018-05-03 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 61e7bc0c1 -> 8509284e1


[SPARK-23433][CORE] Late zombie task completions update all tasksets

Fetch failure lead to multiple tasksets which are active for a given
stage.  While there is only one "active" version of the taskset, the
earlier attempts can still have running tasks, which can complete
successfully.  So a task completion needs to update every taskset
so that it knows the partition is completed.  That way the final active
taskset does not try to submit another task for the same partition,
and so that it knows when it is completed and when it should be
marked as a "zombie".

Added a regression test.

Author: Imran Rashid 

Closes #21131 from squito/SPARK-23433.

(cherry picked from commit 94641fe6cc68e5977dd8663b8f232a287a783acb)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.3
Commit: 8509284e1ec048d5afa87d41071c0429924e45c9
Parents: 61e7bc0
Author: Imran Rashid 
Authored: Thu May 3 10:59:18 2018 -0500
Committer: Imran Rashid 
Committed: Thu May 3 10:59:30 2018 -0500

--
 .../spark/scheduler/TaskSchedulerImpl.scala |  14 +++
 .../apache/spark/scheduler/TaskSetManager.scala |  20 +++-
 .../scheduler/TaskSchedulerImplSuite.scala  | 104 +++
 3 files changed, 137 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8509284e/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 0c11806..8e97b3d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -689,6 +689,20 @@ private[spark] class TaskSchedulerImpl(
 }
   }
 
+  /**
+   * Marks the task has completed in all TaskSetManagers for the given stage.
+   *
+   * After stage failure and retry, there may be multiple TaskSetManagers for 
the stage.
+   * If an earlier attempt of a stage completes a task, we should ensure that 
the later attempts
+   * do not also submit those same tasks.  That also means that a task 
completion from an  earlier
+   * attempt can lead to the entire stage getting marked as successful.
+   */
+  private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, 
partitionId: Int) = {
+taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm 
=>
+  tsm.markPartitionCompleted(partitionId)
+}
+  }
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/8509284e/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index c3ed11b..b52e376 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -74,6 +74,8 @@ private[spark] class TaskSetManager(
   val ser = env.closureSerializer.newInstance()
 
   val tasks = taskSet.tasks
+  private[scheduler] val partitionToIndex = tasks.zipWithIndex
+.map { case (t, idx) => t.partitionId -> idx }.toMap
   val numTasks = tasks.length
   val copiesRunning = new Array[Int](numTasks)
 
@@ -154,7 +156,7 @@ private[spark] class TaskSetManager(
   private[scheduler] val speculatableTasks = new HashSet[Int]
 
   // Task index, start and finish time for each task attempt (indexed by task 
ID)
-  private val taskInfos = new HashMap[Long, TaskInfo]
+  private[scheduler] val taskInfos = new HashMap[Long, TaskInfo]
 
   // Use a MedianHeap to record durations of successful tasks so we know when 
to launch
   // speculative tasks. This is only used when speculation is enabled, to 
avoid the overhead
@@ -755,6 +757,9 @@ private[spark] class TaskSetManager(
   logInfo("Ignoring task-finished event for " + info.id + " in stage " + 
taskSet.id +
 " because task " + index + " has already completed successfully")
 }
+// There may be multiple tasksets for this stage -- we let all of them 
know that the partition
+// was completed.  This may result in some of the tasksets getting 
completed.
+sched.markPartitionCompletedInAllTaskSets(stageId, 

spark git commit: [SPARK-23433][CORE] Late zombie task completions update all tasksets

2018-05-03 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 96a50016b -> 94641fe6c


[SPARK-23433][CORE] Late zombie task completions update all tasksets

Fetch failure lead to multiple tasksets which are active for a given
stage.  While there is only one "active" version of the taskset, the
earlier attempts can still have running tasks, which can complete
successfully.  So a task completion needs to update every taskset
so that it knows the partition is completed.  That way the final active
taskset does not try to submit another task for the same partition,
and so that it knows when it is completed and when it should be
marked as a "zombie".

Added a regression test.

Author: Imran Rashid 

Closes #21131 from squito/SPARK-23433.


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

Branch: refs/heads/master
Commit: 94641fe6cc68e5977dd8663b8f232a287a783acb
Parents: 96a5001
Author: Imran Rashid 
Authored: Thu May 3 10:59:18 2018 -0500
Committer: Imran Rashid 
Committed: Thu May 3 10:59:18 2018 -0500

--
 .../spark/scheduler/TaskSchedulerImpl.scala |  14 +++
 .../apache/spark/scheduler/TaskSetManager.scala |  20 +++-
 .../scheduler/TaskSchedulerImplSuite.scala  | 104 +++
 3 files changed, 137 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/94641fe6/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 0c11806..8e97b3d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -689,6 +689,20 @@ private[spark] class TaskSchedulerImpl(
 }
   }
 
+  /**
+   * Marks the task has completed in all TaskSetManagers for the given stage.
+   *
+   * After stage failure and retry, there may be multiple TaskSetManagers for 
the stage.
+   * If an earlier attempt of a stage completes a task, we should ensure that 
the later attempts
+   * do not also submit those same tasks.  That also means that a task 
completion from an  earlier
+   * attempt can lead to the entire stage getting marked as successful.
+   */
+  private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, 
partitionId: Int) = {
+taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm 
=>
+  tsm.markPartitionCompleted(partitionId)
+}
+  }
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/94641fe6/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 8a96a76..195fc80 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -73,6 +73,8 @@ private[spark] class TaskSetManager(
   val ser = env.closureSerializer.newInstance()
 
   val tasks = taskSet.tasks
+  private[scheduler] val partitionToIndex = tasks.zipWithIndex
+.map { case (t, idx) => t.partitionId -> idx }.toMap
   val numTasks = tasks.length
   val copiesRunning = new Array[Int](numTasks)
 
@@ -153,7 +155,7 @@ private[spark] class TaskSetManager(
   private[scheduler] val speculatableTasks = new HashSet[Int]
 
   // Task index, start and finish time for each task attempt (indexed by task 
ID)
-  private val taskInfos = new HashMap[Long, TaskInfo]
+  private[scheduler] val taskInfos = new HashMap[Long, TaskInfo]
 
   // Use a MedianHeap to record durations of successful tasks so we know when 
to launch
   // speculative tasks. This is only used when speculation is enabled, to 
avoid the overhead
@@ -754,6 +756,9 @@ private[spark] class TaskSetManager(
   logInfo("Ignoring task-finished event for " + info.id + " in stage " + 
taskSet.id +
 " because task " + index + " has already completed successfully")
 }
+// There may be multiple tasksets for this stage -- we let all of them 
know that the partition
+// was completed.  This may result in some of the tasksets getting 
completed.
+sched.markPartitionCompletedInAllTaskSets(stageId, 
tasks(index).partitionId)
 // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which 
holds the
 // 

spark git commit: [SPARK-23888][CORE] correct the comment of hasAttemptOnHost()

2018-04-23 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master e82cb6834 -> c8f3ac69d


[SPARK-23888][CORE] correct the comment of hasAttemptOnHost()

TaskSetManager.hasAttemptOnHost had a misleading comment.  The comment
said that it only checked for running tasks, but really it checked for
any tasks that might have run in the past as well.  This updates to line
up with the implementation.

Author: wuyi 

Closes #20998 from Ngone51/SPARK-23888.


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

Branch: refs/heads/master
Commit: c8f3ac69d176bd10b8de1c147b6903a247943d51
Parents: e82cb68
Author: wuyi 
Authored: Mon Apr 23 15:35:45 2018 -0500
Committer: Imran Rashid 
Committed: Mon Apr 23 15:35:50 2018 -0500

--
 .../src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c8f3ac69/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index d958658..8a96a76 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -287,7 +287,7 @@ private[spark] class TaskSetManager(
 None
   }
 
-  /** Check whether a task is currently running an attempt on a given host */
+  /** Check whether a task once ran an attempt on a given host */
   private def hasAttemptOnHost(taskIndex: Int, host: String): Boolean = {
 taskAttempts(taskIndex).exists(_.host == host)
   }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure

2018-04-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 32bec6ca3 -> 7fb11176f


[SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure

## What changes were proposed in this pull request?

There‘s a miswrite in BlacklistTracker's updateBlacklistForFetchFailure:
```
val blacklistedExecsOnNode =
nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
blacklistedExecsOnNode += exec
```
where first **exec** should be **host**.
## How was this patch tested?

adjust existed test.

Author: wuyi 

Closes #21104 from Ngone51/SPARK-24021.

(cherry picked from commit 0deaa5251326a32a3d2d2b8851193ca926303972)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.3
Commit: 7fb11176f285b4de47e61511c09acbbb79e5c44c
Parents: 32bec6c
Author: wuyi 
Authored: Thu Apr 19 09:00:33 2018 -0500
Committer: Imran Rashid 
Committed: Thu Apr 19 09:00:46 2018 -0500

--
 .../scala/org/apache/spark/scheduler/BlacklistTracker.scala | 2 +-
 .../org/apache/spark/scheduler/BlacklistTrackerSuite.scala  | 5 +
 2 files changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7fb11176/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala 
b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
index 952598f..30cf75d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
@@ -210,7 +210,7 @@ private[scheduler] class BlacklistTracker (
 updateNextExpiryTime()
 killBlacklistedExecutor(exec)
 
-val blacklistedExecsOnNode = 
nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
+val blacklistedExecsOnNode = 
nodeToBlacklistedExecs.getOrElseUpdate(host, HashSet[String]())
 blacklistedExecsOnNode += exec
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/7fb11176/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
index 88a57c5..00867ef 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
@@ -574,6 +574,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with 
BeforeAndAfterEach with M
 verify(allocationClientMock, never).killExecutors(any(), any(), any(), 
any())
 verify(allocationClientMock, never).killExecutorsOnHost(any())
 
+assert(blacklist.nodeToBlacklistedExecs.contains("hostA"))
+assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1"))
+
 // Enable auto-kill. Blacklist an executor and make sure killExecutors is 
called.
 conf.set(config.BLACKLIST_KILL_ENABLED, true)
 blacklist = new BlacklistTracker(listenerBusMock, conf, 
Some(allocationClientMock), clock)
@@ -589,6 +592,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with 
BeforeAndAfterEach with M
   1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
 assert(blacklist.nextExpiryTime === 1000 + 
blacklist.BLACKLIST_TIMEOUT_MILLIS)
 assert(blacklist.nodeIdToBlacklistExpiryTime.isEmpty)
+assert(blacklist.nodeToBlacklistedExecs.contains("hostA"))
+assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1"))
 
 // Enable external shuffle service to see if all the executors on this 
node will be killed.
 conf.set(config.SHUFFLE_SERVICE_ENABLED, true)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure

2018-04-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master d96c3e33c -> 0deaa5251


[SPARK-24021][CORE] fix bug in BlacklistTracker's updateBlacklistForFetchFailure

## What changes were proposed in this pull request?

There‘s a miswrite in BlacklistTracker's updateBlacklistForFetchFailure:
```
val blacklistedExecsOnNode =
nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
blacklistedExecsOnNode += exec
```
where first **exec** should be **host**.
## How was this patch tested?

adjust existed test.

Author: wuyi 

Closes #21104 from Ngone51/SPARK-24021.


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

Branch: refs/heads/master
Commit: 0deaa5251326a32a3d2d2b8851193ca926303972
Parents: d96c3e3
Author: wuyi 
Authored: Thu Apr 19 09:00:33 2018 -0500
Committer: Imran Rashid 
Committed: Thu Apr 19 09:00:33 2018 -0500

--
 .../scala/org/apache/spark/scheduler/BlacklistTracker.scala | 2 +-
 .../org/apache/spark/scheduler/BlacklistTrackerSuite.scala  | 5 +
 2 files changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0deaa525/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala 
b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
index 952598f..30cf75d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
@@ -210,7 +210,7 @@ private[scheduler] class BlacklistTracker (
 updateNextExpiryTime()
 killBlacklistedExecutor(exec)
 
-val blacklistedExecsOnNode = 
nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
+val blacklistedExecsOnNode = 
nodeToBlacklistedExecs.getOrElseUpdate(host, HashSet[String]())
 blacklistedExecsOnNode += exec
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0deaa525/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
index 06d7afa..96c8404 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
@@ -574,6 +574,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with 
BeforeAndAfterEach with M
 verify(allocationClientMock, never).killExecutors(any(), any(), any(), 
any())
 verify(allocationClientMock, never).killExecutorsOnHost(any())
 
+assert(blacklist.nodeToBlacklistedExecs.contains("hostA"))
+assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1"))
+
 // Enable auto-kill. Blacklist an executor and make sure killExecutors is 
called.
 conf.set(config.BLACKLIST_KILL_ENABLED, true)
 blacklist = new BlacklistTracker(listenerBusMock, conf, 
Some(allocationClientMock), clock)
@@ -589,6 +592,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with 
BeforeAndAfterEach with M
   1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS)
 assert(blacklist.nextExpiryTime === 1000 + 
blacklist.BLACKLIST_TIMEOUT_MILLIS)
 assert(blacklist.nodeIdToBlacklistExpiryTime.isEmpty)
+assert(blacklist.nodeToBlacklistedExecs.contains("hostA"))
+assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1"))
 
 // Enable external shuffle service to see if all the executors on this 
node will be killed.
 conf.set(config.SHUFFLE_SERVICE_ENABLED, true)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-23948] Trigger mapstage's job listener in submitMissingTasks

2018-04-17 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 564019b92 -> 6b99d5bc3


[SPARK-23948] Trigger mapstage's job listener in submitMissingTasks

## What changes were proposed in this pull request?

SparkContext submitted a map stage from `submitMapStage` to `DAGScheduler`,
`markMapStageJobAsFinished` is called only in 
(https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L933
 and 
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1314);

But think about below scenario:
1. stage0 and stage1 are all `ShuffleMapStage` and stage1 depends on stage0;
2. We submit stage1 by `submitMapStage`;
3. When stage 1 running, `FetchFailed` happened, stage0 and stage1 got 
resubmitted as stage0_1 and stage1_1;
4. When stage0_1 running, speculated tasks in old stage1 come as succeeded, but 
stage1 is not inside `runningStages`. So even though all splits(including the 
speculated tasks) in stage1 succeeded, job listener in stage1 will not be 
called;
5. stage0_1 finished, stage1_1 starts running. When `submitMissingTasks`, there 
is no missing tasks. But in current code, job listener is not triggered.

We should call the job listener for map stage in `5`.

## How was this patch tested?

Not added yet.

Author: jinxing 

(cherry picked from commit 3990daaf3b6ca2c5a9f7790030096262efb12cb2)

Author: jinxing 

Closes #21085 from squito/cp.


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

Branch: refs/heads/branch-2.3
Commit: 6b99d5bc3f3898a0aff30468a623a3f64bb20b62
Parents: 564019b
Author: jinxing 
Authored: Tue Apr 17 15:53:29 2018 -0500
Committer: Imran Rashid 
Committed: Tue Apr 17 15:53:29 2018 -0500

--
 .../apache/spark/scheduler/DAGScheduler.scala   | 33 +++--
 .../spark/scheduler/DAGSchedulerSuite.scala | 52 
 2 files changed, 70 insertions(+), 15 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6b99d5bc/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 8c46a84..78b6b34 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1092,17 +1092,16 @@ class DAGScheduler(
   // the stage as completed here in case there are no tasks to run
   markStageAsFinished(stage, None)
 
-  val debugString = stage match {
+  stage match {
 case stage: ShuffleMapStage =>
-  s"Stage ${stage} is actually done; " +
-s"(available: ${stage.isAvailable}," +
-s"available outputs: ${stage.numAvailableOutputs}," +
-s"partitions: ${stage.numPartitions})"
+  logDebug(s"Stage ${stage} is actually done; " +
+  s"(available: ${stage.isAvailable}," +
+  s"available outputs: ${stage.numAvailableOutputs}," +
+  s"partitions: ${stage.numPartitions})")
+  markMapStageJobsAsFinished(stage)
 case stage : ResultStage =>
-  s"Stage ${stage} is actually done; (partitions: 
${stage.numPartitions})"
+  logDebug(s"Stage ${stage} is actually done; (partitions: 
${stage.numPartitions})")
   }
-  logDebug(debugString)
-
   submitWaitingChildStages(stage)
 }
   }
@@ -1307,13 +1306,7 @@ class DAGScheduler(
   shuffleStage.findMissingPartitions().mkString(", "))
 submitStage(shuffleStage)
   } else {
-// Mark any map-stage jobs waiting on this stage as finished
-if (shuffleStage.mapStageJobs.nonEmpty) {
-  val stats = 
mapOutputTracker.getStatistics(shuffleStage.shuffleDep)
-  for (job <- shuffleStage.mapStageJobs) {
-markMapStageJobAsFinished(job, stats)
-  }
-}
+markMapStageJobsAsFinished(shuffleStage)
 submitWaitingChildStages(shuffleStage)
   }
 }
@@ -1433,6 +1426,16 @@ class DAGScheduler(
 }
   }
 
+  private[scheduler] def markMapStageJobsAsFinished(shuffleStage: 
ShuffleMapStage): Unit = {
+// Mark any map-stage jobs waiting on this stage as finished
+if (shuffleStage.isAvailable && shuffleStage.mapStageJobs.nonEmpty) {
+  val stats = 

spark git commit: [SPARK-23948] Trigger mapstage's job listener in submitMissingTasks

2018-04-17 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master ed4101d29 -> 3990daaf3


[SPARK-23948] Trigger mapstage's job listener in submitMissingTasks

## What changes were proposed in this pull request?

SparkContext submitted a map stage from `submitMapStage` to `DAGScheduler`,
`markMapStageJobAsFinished` is called only in 
(https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L933
 and 
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1314);

But think about below scenario:
1. stage0 and stage1 are all `ShuffleMapStage` and stage1 depends on stage0;
2. We submit stage1 by `submitMapStage`;
3. When stage 1 running, `FetchFailed` happened, stage0 and stage1 got 
resubmitted as stage0_1 and stage1_1;
4. When stage0_1 running, speculated tasks in old stage1 come as succeeded, but 
stage1 is not inside `runningStages`. So even though all splits(including the 
speculated tasks) in stage1 succeeded, job listener in stage1 will not be 
called;
5. stage0_1 finished, stage1_1 starts running. When `submitMissingTasks`, there 
is no missing tasks. But in current code, job listener is not triggered.

We should call the job listener for map stage in `5`.

## How was this patch tested?

Not added yet.

Author: jinxing 

Closes #21019 from jinxing64/SPARK-23948.


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

Branch: refs/heads/master
Commit: 3990daaf3b6ca2c5a9f7790030096262efb12cb2
Parents: ed4101d
Author: jinxing 
Authored: Tue Apr 17 08:55:01 2018 -0500
Committer: Imran Rashid 
Committed: Tue Apr 17 08:55:01 2018 -0500

--
 .../apache/spark/scheduler/DAGScheduler.scala   | 33 +++--
 .../spark/scheduler/DAGSchedulerSuite.scala | 52 
 2 files changed, 70 insertions(+), 15 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3990daaf/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 8c46a84..78b6b34 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1092,17 +1092,16 @@ class DAGScheduler(
   // the stage as completed here in case there are no tasks to run
   markStageAsFinished(stage, None)
 
-  val debugString = stage match {
+  stage match {
 case stage: ShuffleMapStage =>
-  s"Stage ${stage} is actually done; " +
-s"(available: ${stage.isAvailable}," +
-s"available outputs: ${stage.numAvailableOutputs}," +
-s"partitions: ${stage.numPartitions})"
+  logDebug(s"Stage ${stage} is actually done; " +
+  s"(available: ${stage.isAvailable}," +
+  s"available outputs: ${stage.numAvailableOutputs}," +
+  s"partitions: ${stage.numPartitions})")
+  markMapStageJobsAsFinished(stage)
 case stage : ResultStage =>
-  s"Stage ${stage} is actually done; (partitions: 
${stage.numPartitions})"
+  logDebug(s"Stage ${stage} is actually done; (partitions: 
${stage.numPartitions})")
   }
-  logDebug(debugString)
-
   submitWaitingChildStages(stage)
 }
   }
@@ -1307,13 +1306,7 @@ class DAGScheduler(
   shuffleStage.findMissingPartitions().mkString(", "))
 submitStage(shuffleStage)
   } else {
-// Mark any map-stage jobs waiting on this stage as finished
-if (shuffleStage.mapStageJobs.nonEmpty) {
-  val stats = 
mapOutputTracker.getStatistics(shuffleStage.shuffleDep)
-  for (job <- shuffleStage.mapStageJobs) {
-markMapStageJobAsFinished(job, stats)
-  }
-}
+markMapStageJobsAsFinished(shuffleStage)
 submitWaitingChildStages(shuffleStage)
   }
 }
@@ -1433,6 +1426,16 @@ class DAGScheduler(
 }
   }
 
+  private[scheduler] def markMapStageJobsAsFinished(shuffleStage: 
ShuffleMapStage): Unit = {
+// Mark any map-stage jobs waiting on this stage as finished
+if (shuffleStage.isAvailable && shuffleStage.mapStageJobs.nonEmpty) {
+  val stats = mapOutputTracker.getStatistics(shuffleStage.shuffleDep)
+  for (job <- shuffleStage.mapStageJobs) {
+

spark git commit: [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher.

2018-04-11 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 653fe0241 -> 3cb82047f


[SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher.

The current in-process launcher implementation just calls the SparkSubmit
object, which, in case of errors, will more often than not exit the JVM.
This is not desirable since this launcher is meant to be used inside other
applications, and that would kill the application.

The change turns SparkSubmit into a class, and abstracts aways some of
the functionality used to print error messages and abort the submission
process. The default implementation uses the logging system for messages,
and throws exceptions for errors. As part of that I also moved some code
that doesn't really belong in SparkSubmit to a better location.

The command line invocation of spark-submit now uses a special implementation
of the SparkSubmit class that overrides those behaviors to do what is expected
from the command line version (print to the terminal, exit the JVM, etc).

A lot of the changes are to replace calls to methods such as "printErrorAndExit"
with the new API.

As part of adding tests for this, I had to fix some small things in the
launcher option parser so that things like "--version" can work when
used in the launcher library.

There is still code that prints directly to the terminal, like all the
Ivy-related code in SparkSubmitUtils, and other areas where some re-factoring
would help, like the CommandLineUtils class, but I chose to leave those
alone to keep this change more focused.

Aside from existing and added unit tests, I ran command line tools with
a bunch of different arguments to make sure messages and errors behave
like before.

Author: Marcelo Vanzin 

Closes #20925 from vanzin/SPARK-22941.


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

Branch: refs/heads/master
Commit: 3cb82047f2f51af553df09b9323796af507d36f8
Parents: 653fe02
Author: Marcelo Vanzin 
Authored: Wed Apr 11 10:13:44 2018 -0500
Committer: Imran Rashid 
Committed: Wed Apr 11 10:13:44 2018 -0500

--
 .../apache/spark/deploy/DependencyUtils.scala   |  30 +-
 .../org/apache/spark/deploy/SparkSubmit.scala   | 318 ++-
 .../spark/deploy/SparkSubmitArguments.scala |  90 +++---
 .../spark/deploy/worker/DriverWrapper.scala |   4 +-
 .../apache/spark/util/CommandLineUtils.scala|  18 +-
 .../spark/launcher/SparkLauncherSuite.java  |  37 ++-
 .../apache/spark/deploy/SparkSubmitSuite.scala  |  69 ++--
 .../deploy/rest/StandaloneRestSubmitSuite.scala |   2 +-
 .../apache/spark/launcher/AbstractLauncher.java |   6 +-
 .../spark/launcher/InProcessLauncher.java   |  14 +-
 .../launcher/SparkSubmitCommandBuilder.java |  82 +++--
 project/MimaExcludes.scala  |   7 +-
 .../deploy/mesos/MesosClusterDispatcher.scala   |  10 +-
 .../mesos/MesosClusterDispatcherArguments.scala |   6 +-
 14 files changed, 401 insertions(+), 292 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3cb82047/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala
--
diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala 
b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala
index fac834a..178bdcf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala
@@ -25,9 +25,10 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
 
 import org.apache.spark.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.internal.Logging
 import org.apache.spark.util.{MutableURLClassLoader, Utils}
 
-private[deploy] object DependencyUtils {
+private[deploy] object DependencyUtils extends Logging {
 
   def resolveMavenDependencies(
   packagesExclusions: String,
@@ -75,7 +76,7 @@ private[deploy] object DependencyUtils {
   def addJarsToClassPath(jars: String, loader: MutableURLClassLoader): Unit = {
 if (jars != null) {
   for (jar <- jars.split(",")) {
-SparkSubmit.addJarToClasspath(jar, loader)
+addJarToClasspath(jar, loader)
   }
 }
   }
@@ -151,6 +152,31 @@ private[deploy] object DependencyUtils {
 }.mkString(",")
   }
 
+  def addJarToClasspath(localJar: String, loader: MutableURLClassLoader): Unit 
= {
+val uri = Utils.resolveURI(localJar)
+uri.getScheme match {
+  case "file" | "local" =>
+val file = new File(uri.getPath)
+if 

spark git commit: [SPARK-23053][CORE][BRANCH-2.1] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

2018-02-20 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.1 1df8020e1 -> 24fe6eb0f


[SPARK-23053][CORE][BRANCH-2.1] taskBinarySerialization and task partitions 
calculate in DagScheduler.submitMissingTasks should keep the same RDD 
checkpoint status

## What changes were proposed in this pull request?
This PR backports [#20244](https://github.com/apache/spark/pull/20244)

When we run concurrent jobs using the same rdd which is marked to do 
checkpoint. If one job has finished running the job, and start the process of 
RDD.doCheckpoint, while another job is submitted, then submitStage and 
submitMissingTasks will be called. In 
[submitMissingTasks](https://github.com/apache/spark/blob/branch-2.1/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L932),
 will serialize taskBinaryBytes and calculate task partitions which are both 
affected by the status of checkpoint, if the former is calculated before 
doCheckpoint finished, while the latter is calculated after doCheckpoint 
finished, when run task, rdd.compute will be called, for some rdds with 
particular partition type such as 
[UnionRDD](https://github.com/apache/spark/blob/branch-2.1/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala)
 who will do partition type cast, will get a ClassCastException because the 
part params is actually a CheckpointRDDPartition.
This error occurs  because rdd.doCheckpoint occurs in the same thread that 
called sc.runJob, while the task serialization occurs in the DAGSchedulers 
event loop.

## How was this patch tested?
the exist tests.

Author: huangtengfei 

Closes #20635 from ivoson/branch-2.1-23053.


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

Branch: refs/heads/branch-2.1
Commit: 24fe6eb0f6e2fe31d1c93fc65bc294ebb94e2dcf
Parents: 1df8020
Author: huangtengfei 
Authored: Tue Feb 20 21:01:45 2018 -0600
Committer: Imran Rashid 
Committed: Tue Feb 20 21:01:45 2018 -0600

--
 .../apache/spark/scheduler/DAGScheduler.scala   | 27 +---
 1 file changed, 18 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/24fe6eb0/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 01a95c0..9d46d69 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -38,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.partial.{ApproximateActionListener, 
ApproximateEvaluator, PartialResult}
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{RDD, RDDCheckpointData}
 import org.apache.spark.rpc.RpcTimeout
 import org.apache.spark.storage._
 import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -982,15 +982,24 @@ class DAGScheduler(
 // might modify state of objects referenced in their closures. This is 
necessary in Hadoop
 // where the JobConf/Configuration object is not thread-safe.
 var taskBinary: Broadcast[Array[Byte]] = null
+var partitions: Array[Partition] = null
 try {
   // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep).
   // For ResultTask, serialize and broadcast (rdd, func).
-  val taskBinaryBytes: Array[Byte] = stage match {
-case stage: ShuffleMapStage =>
-  JavaUtils.bufferToArray(
-closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
-case stage: ResultStage =>
-  JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, 
stage.func): AnyRef))
+  var taskBinaryBytes: Array[Byte] = null
+  // taskBinaryBytes and partitions are both effected by the checkpoint 
status. We need
+  // this synchronization in case another concurrent job is checkpointing 
this RDD, so we get a
+  // consistent view of both variables.
+  RDDCheckpointData.synchronized {
+taskBinaryBytes = stage match {
+  case stage: ShuffleMapStage =>
+JavaUtils.bufferToArray(
+  closureSerializer.serialize((stage.rdd, stage.shuffleDep): 
AnyRef))
+  case stage: ResultStage =>
+JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, 
stage.func): AnyRef))
+}
+
+

spark git commit: [SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stag…

2018-02-15 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 0bd7765cd -> 75bb19a01


[SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stag…

…e page

## What changes were proposed in this pull request?

Fixing exception got at sorting tasks by Host / Executor ID:
```
java.lang.IllegalArgumentException: Invalid sort column: Host
at org.apache.spark.ui.jobs.ApiHelper$.indexName(StagePage.scala:1017)
at 
org.apache.spark.ui.jobs.TaskDataSource.sliceData(StagePage.scala:694)
at org.apache.spark.ui.PagedDataSource.pageData(PagedTable.scala:61)
at org.apache.spark.ui.PagedTable$class.table(PagedTable.scala:96)
at org.apache.spark.ui.jobs.TaskPagedTable.table(StagePage.scala:708)
at org.apache.spark.ui.jobs.StagePage.liftedTree1$1(StagePage.scala:293)
at org.apache.spark.ui.jobs.StagePage.render(StagePage.scala:282)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
at 
org.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
at 
org.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)
```

Moreover some refactoring to avoid similar problems by introducing constants 
for each header name and reusing them at the identification of the 
corresponding sorting index.

## How was this patch tested?

Manually:

![screen shot 2018-02-13 at 18 57 
10](https://user-images.githubusercontent.com/2017933/36166532-1cfdf3b8-10f3-11e8-8d32-5fcaad2af214.png)

(cherry picked from commit 1dc2c1d5e85c5f404f470aeb44c1f3c22786bdea)

Author: “attilapiros” 

Closes #20623 from squito/fix_backport.


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

Branch: refs/heads/branch-2.3
Commit: 75bb19a018f9260eab3ea0ba3ea46e84b87eabf2
Parents: 0bd7765
Author: “attilapiros” 
Authored: Thu Feb 15 14:03:41 2018 -0600
Committer: Imran Rashid 
Committed: Thu Feb 15 14:03:41 2018 -0600

--
 .../org/apache/spark/status/storeTypes.scala|   2 +
 .../org/apache/spark/ui/jobs/StagePage.scala| 121 ---
 .../org/apache/spark/ui/StagePageSuite.scala|  63 +-
 3 files changed, 139 insertions(+), 47 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/75bb19a0/core/src/main/scala/org/apache/spark/status/storeTypes.scala
--
diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala 
b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
index 412644d..646cf25 100644
--- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala
+++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
@@ -109,6 +109,7 @@ private[spark] object TaskIndexNames {
   final val DURATION = "dur"
   final val ERROR = "err"
   final val EXECUTOR = "exe"
+  final val HOST = "hst"
   final val EXEC_CPU_TIME = "ect"
   final val EXEC_RUN_TIME = "ert"
   final val GC_TIME = "gc"
@@ -165,6 +166,7 @@ private[spark] class TaskDataWrapper(
 val duration: Long,
 @KVIndexParam(value = TaskIndexNames.EXECUTOR, parent = 
TaskIndexNames.STAGE)
 val executorId: String,
+@KVIndexParam(value = TaskIndexNames.HOST, parent = TaskIndexNames.STAGE)
 val host: String,
 @KVIndexParam(value = TaskIndexNames.STATUS, parent = TaskIndexNames.STAGE)
 val status: String,

http://git-wip-us.apache.org/repos/asf/spark/blob/75bb19a0/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala 
b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 8af2537..a4710f6 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -742,37 +742,39 @@ private[ui] class TaskPagedTable(
   }
 
   def headers: Seq[Node] = {
+import ApiHelper._
+
 val taskHeadersAndCssClasses: Seq[(String, String)] =
   Seq(
-("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality 
Level", ""),
-("Executor ID", ""), ("Host", ""), ("Launch Time", ""), ("Duration", 
""),
-("Scheduler Delay", 

spark git commit: [SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stage page

2018-02-15 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master db45daab9 -> 1dc2c1d5e


[SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stage page

## What changes were proposed in this pull request?

Fixing exception got at sorting tasks by Host / Executor ID:
```
java.lang.IllegalArgumentException: Invalid sort column: Host
at org.apache.spark.ui.jobs.ApiHelper$.indexName(StagePage.scala:1017)
at 
org.apache.spark.ui.jobs.TaskDataSource.sliceData(StagePage.scala:694)
at org.apache.spark.ui.PagedDataSource.pageData(PagedTable.scala:61)
at org.apache.spark.ui.PagedTable$class.table(PagedTable.scala:96)
at org.apache.spark.ui.jobs.TaskPagedTable.table(StagePage.scala:708)
at org.apache.spark.ui.jobs.StagePage.liftedTree1$1(StagePage.scala:293)
at org.apache.spark.ui.jobs.StagePage.render(StagePage.scala:282)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
at 
org.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
at 
org.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)
```

Moreover some refactoring to avoid similar problems by introducing constants 
for each header name and reusing them at the identification of the 
corresponding sorting index.

## How was this patch tested?

Manually:

![screen shot 2018-02-13 at 18 57 
10](https://user-images.githubusercontent.com/2017933/36166532-1cfdf3b8-10f3-11e8-8d32-5fcaad2af214.png)

Author: “attilapiros” 

Closes #20601 from attilapiros/SPARK-23413.


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

Branch: refs/heads/master
Commit: 1dc2c1d5e85c5f404f470aeb44c1f3c22786bdea
Parents: db45daa
Author: “attilapiros” 
Authored: Thu Feb 15 13:51:24 2018 -0600
Committer: Imran Rashid 
Committed: Thu Feb 15 13:51:24 2018 -0600

--
 .../org/apache/spark/status/storeTypes.scala|   2 +
 .../org/apache/spark/ui/jobs/StagePage.scala| 121 ---
 .../org/apache/spark/ui/StagePageSuite.scala|  63 +-
 3 files changed, 139 insertions(+), 47 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1dc2c1d5/core/src/main/scala/org/apache/spark/status/storeTypes.scala
--
diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala 
b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
index 412644d..646cf25 100644
--- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala
+++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
@@ -109,6 +109,7 @@ private[spark] object TaskIndexNames {
   final val DURATION = "dur"
   final val ERROR = "err"
   final val EXECUTOR = "exe"
+  final val HOST = "hst"
   final val EXEC_CPU_TIME = "ect"
   final val EXEC_RUN_TIME = "ert"
   final val GC_TIME = "gc"
@@ -165,6 +166,7 @@ private[spark] class TaskDataWrapper(
 val duration: Long,
 @KVIndexParam(value = TaskIndexNames.EXECUTOR, parent = 
TaskIndexNames.STAGE)
 val executorId: String,
+@KVIndexParam(value = TaskIndexNames.HOST, parent = TaskIndexNames.STAGE)
 val host: String,
 @KVIndexParam(value = TaskIndexNames.STATUS, parent = TaskIndexNames.STAGE)
 val status: String,

http://git-wip-us.apache.org/repos/asf/spark/blob/1dc2c1d5/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala 
b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 5c2b0c3..a9265d4 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -750,37 +750,39 @@ private[ui] class TaskPagedTable(
   }
 
   def headers: Seq[Node] = {
+import ApiHelper._
+
 val taskHeadersAndCssClasses: Seq[(String, String)] =
   Seq(
-("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality 
Level", ""),
-("Executor ID", ""), ("Host", ""), ("Launch Time", ""), ("Duration", 
""),
-("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY),
-("Task Deserialization Time", 

spark git commit: [SPARK-23235][CORE] Add executor Threaddump to api

2018-02-13 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 2ee76c22b -> a5a4b8350


[SPARK-23235][CORE] Add executor Threaddump to api

## What changes were proposed in this pull request?

Extending api with the executor thread dump data.

For this new REST URL is introduced:
- GET 
http://localhost:4040/api/v1/applications/{applicationId}/executors/{executorId}/threads


Example response:

``` javascript
[ {
  "threadId" : 52,
  "threadName" : "context-cleaner-periodic-gc",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native 
Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : 
"Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1385411893})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 48,
  "threadName" : "dag-scheduler-event-loop",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native 
Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingDeque.takeFirst(LinkedBlockingDeque.java:492)\njava.util.concurrent.LinkedBlockingDeque.take(LinkedBlockingDeque.java:680)\norg.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:46)",
  "blockedByLock" : 
"Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1138053349})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 17,
  "threadName" : "dispatcher-event-loop-0",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native 
Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : 
"Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ 
"Lock(java.util.concurrent.ThreadPoolExecutor$Worker832743930})" ]
}, {
  "threadId" : 18,
  "threadName" : "dispatcher-event-loop-1",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native 
Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : 
"Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ 
"Lock(java.util.concurrent.ThreadPoolExecutor$Worker834153999})" ]
}, {
  "threadId" : 19,
  "threadName" : "dispatcher-event-loop-2",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native 
Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : 
"Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ 
"Lock(java.util.concurrent.ThreadPoolExecutor$Worker664836465})" ]
}, {
  "threadId" : 20,
  "threadName" : "dispatcher-event-loop-3",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native 

spark git commit: [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

2018-02-13 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.2 14b5dbfa9 -> 73263b215


[SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in 
DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

## What changes were proposed in this pull request?

When we run concurrent jobs using the same rdd which is marked to do 
checkpoint. If one job has finished running the job, and start the process of 
RDD.doCheckpoint, while another job is submitted, then submitStage and 
submitMissingTasks will be called. In 
[submitMissingTasks](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961),
 will serialize taskBinaryBytes and calculate task partitions which are both 
affected by the status of checkpoint, if the former is calculated before 
doCheckpoint finished, while the latter is calculated after doCheckpoint 
finished, when run task, rdd.compute will be called, for some rdds with 
particular partition type such as 
[UnionRDD](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala)
 who will do partition type cast, will get a ClassCastException because the 
part params is actually a CheckpointRDDPartition.
This error occurs  because rdd.doCheckpoint occurs in the same thread that 
called sc.runJob, while the task serialization occurs in the DAGSchedulers 
event loop.

## How was this patch tested?

the exist uts and also add a test case in DAGScheduerSuite to show the 
exception case.

Author: huangtengfei 

Closes #20244 from ivoson/branch-taskpart-mistype.

(cherry picked from commit 091a000d27f324de8c5c527880854ecfcf5de9a4)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.2
Commit: 73263b2150ededd8000b3fe4f276625d90de2507
Parents: 14b5dbf
Author: huangtengfei 
Authored: Tue Feb 13 09:59:21 2018 -0600
Committer: Imran Rashid 
Committed: Tue Feb 13 10:00:23 2018 -0600

--
 .../apache/spark/scheduler/DAGScheduler.scala   | 27 +---
 1 file changed, 18 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/73263b21/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 b009424..87e407f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -38,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.partial.{ApproximateActionListener, 
ApproximateEvaluator, PartialResult}
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{RDD, RDDCheckpointData}
 import org.apache.spark.rpc.RpcTimeout
 import org.apache.spark.storage._
 import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -992,15 +992,24 @@ class DAGScheduler(
 // might modify state of objects referenced in their closures. This is 
necessary in Hadoop
 // where the JobConf/Configuration object is not thread-safe.
 var taskBinary: Broadcast[Array[Byte]] = null
+var partitions: Array[Partition] = null
 try {
   // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep).
   // For ResultTask, serialize and broadcast (rdd, func).
-  val taskBinaryBytes: Array[Byte] = stage match {
-case stage: ShuffleMapStage =>
-  JavaUtils.bufferToArray(
-closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
-case stage: ResultStage =>
-  JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, 
stage.func): AnyRef))
+  var taskBinaryBytes: Array[Byte] = null
+  // taskBinaryBytes and partitions are both effected by the checkpoint 
status. We need
+  // this synchronization in case another concurrent job is checkpointing 
this RDD, so we get a
+  // consistent view of both variables.
+  RDDCheckpointData.synchronized {
+taskBinaryBytes = stage match {
+  case stage: ShuffleMapStage =>
+JavaUtils.bufferToArray(
+  closureSerializer.serialize((stage.rdd, stage.shuffleDep): 
AnyRef))
+  case stage: ResultStage =>
+ 

spark git commit: [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

2018-02-13 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 1c81c0c62 -> dbb1b399b


[SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in 
DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

## What changes were proposed in this pull request?

When we run concurrent jobs using the same rdd which is marked to do 
checkpoint. If one job has finished running the job, and start the process of 
RDD.doCheckpoint, while another job is submitted, then submitStage and 
submitMissingTasks will be called. In 
[submitMissingTasks](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961),
 will serialize taskBinaryBytes and calculate task partitions which are both 
affected by the status of checkpoint, if the former is calculated before 
doCheckpoint finished, while the latter is calculated after doCheckpoint 
finished, when run task, rdd.compute will be called, for some rdds with 
particular partition type such as 
[UnionRDD](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala)
 who will do partition type cast, will get a ClassCastException because the 
part params is actually a CheckpointRDDPartition.
This error occurs  because rdd.doCheckpoint occurs in the same thread that 
called sc.runJob, while the task serialization occurs in the DAGSchedulers 
event loop.

## How was this patch tested?

the exist uts and also add a test case in DAGScheduerSuite to show the 
exception case.

Author: huangtengfei 

Closes #20244 from ivoson/branch-taskpart-mistype.

(cherry picked from commit 091a000d27f324de8c5c527880854ecfcf5de9a4)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.3
Commit: dbb1b399b6cf8372a3659c472f380142146b1248
Parents: 1c81c0c
Author: huangtengfei 
Authored: Tue Feb 13 09:59:21 2018 -0600
Committer: Imran Rashid 
Committed: Tue Feb 13 10:00:05 2018 -0600

--
 .../apache/spark/scheduler/DAGScheduler.scala   | 27 +---
 1 file changed, 18 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dbb1b399/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 199937b..8c46a84 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.partial.{ApproximateActionListener, 
ApproximateEvaluator, PartialResult}
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{RDD, RDDCheckpointData}
 import org.apache.spark.rpc.RpcTimeout
 import org.apache.spark.storage._
 import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -1016,15 +1016,24 @@ class DAGScheduler(
 // might modify state of objects referenced in their closures. This is 
necessary in Hadoop
 // where the JobConf/Configuration object is not thread-safe.
 var taskBinary: Broadcast[Array[Byte]] = null
+var partitions: Array[Partition] = null
 try {
   // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep).
   // For ResultTask, serialize and broadcast (rdd, func).
-  val taskBinaryBytes: Array[Byte] = stage match {
-case stage: ShuffleMapStage =>
-  JavaUtils.bufferToArray(
-closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
-case stage: ResultStage =>
-  JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, 
stage.func): AnyRef))
+  var taskBinaryBytes: Array[Byte] = null
+  // taskBinaryBytes and partitions are both effected by the checkpoint 
status. We need
+  // this synchronization in case another concurrent job is checkpointing 
this RDD, so we get a
+  // consistent view of both variables.
+  RDDCheckpointData.synchronized {
+taskBinaryBytes = stage match {
+  case stage: ShuffleMapStage =>
+JavaUtils.bufferToArray(
+  closureSerializer.serialize((stage.rdd, stage.shuffleDep): 
AnyRef))
+  case stage: ResultStage =>
+

spark git commit: [SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

2018-02-13 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master d6e1958a2 -> 091a000d2


[SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in 
DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

## What changes were proposed in this pull request?

When we run concurrent jobs using the same rdd which is marked to do 
checkpoint. If one job has finished running the job, and start the process of 
RDD.doCheckpoint, while another job is submitted, then submitStage and 
submitMissingTasks will be called. In 
[submitMissingTasks](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961),
 will serialize taskBinaryBytes and calculate task partitions which are both 
affected by the status of checkpoint, if the former is calculated before 
doCheckpoint finished, while the latter is calculated after doCheckpoint 
finished, when run task, rdd.compute will be called, for some rdds with 
particular partition type such as 
[UnionRDD](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala)
 who will do partition type cast, will get a ClassCastException because the 
part params is actually a CheckpointRDDPartition.
This error occurs  because rdd.doCheckpoint occurs in the same thread that 
called sc.runJob, while the task serialization occurs in the DAGSchedulers 
event loop.

## How was this patch tested?

the exist uts and also add a test case in DAGScheduerSuite to show the 
exception case.

Author: huangtengfei 

Closes #20244 from ivoson/branch-taskpart-mistype.


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

Branch: refs/heads/master
Commit: 091a000d27f324de8c5c527880854ecfcf5de9a4
Parents: d6e1958
Author: huangtengfei 
Authored: Tue Feb 13 09:59:21 2018 -0600
Committer: Imran Rashid 
Committed: Tue Feb 13 09:59:21 2018 -0600

--
 .../apache/spark/scheduler/DAGScheduler.scala   | 27 +---
 1 file changed, 18 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/091a000d/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 199937b..8c46a84 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.partial.{ApproximateActionListener, 
ApproximateEvaluator, PartialResult}
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{RDD, RDDCheckpointData}
 import org.apache.spark.rpc.RpcTimeout
 import org.apache.spark.storage._
 import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -1016,15 +1016,24 @@ class DAGScheduler(
 // might modify state of objects referenced in their closures. This is 
necessary in Hadoop
 // where the JobConf/Configuration object is not thread-safe.
 var taskBinary: Broadcast[Array[Byte]] = null
+var partitions: Array[Partition] = null
 try {
   // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep).
   // For ResultTask, serialize and broadcast (rdd, func).
-  val taskBinaryBytes: Array[Byte] = stage match {
-case stage: ShuffleMapStage =>
-  JavaUtils.bufferToArray(
-closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef))
-case stage: ResultStage =>
-  JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, 
stage.func): AnyRef))
+  var taskBinaryBytes: Array[Byte] = null
+  // taskBinaryBytes and partitions are both effected by the checkpoint 
status. We need
+  // this synchronization in case another concurrent job is checkpointing 
this RDD, so we get a
+  // consistent view of both variables.
+  RDDCheckpointData.synchronized {
+taskBinaryBytes = stage match {
+  case stage: ShuffleMapStage =>
+JavaUtils.bufferToArray(
+  closureSerializer.serialize((stage.rdd, stage.shuffleDep): 
AnyRef))
+  case stage: ResultStage =>
+JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, 
stage.func): AnyRef))
+}
+
+partitions = 

spark git commit: [SPARK-23189][CORE][WEB UI] Reflect stage level blacklisting on executor tab

2018-02-13 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 116c581d2 -> d6e1958a2


[SPARK-23189][CORE][WEB UI] Reflect stage level blacklisting on executor tab

## What changes were proposed in this pull request?

The purpose of this PR to reflect the stage level blacklisting on the executor 
tab for the currently active stages.

After this change in the executor tab at the Status column one of the following 
label will be:

- "Blacklisted" when the executor is blacklisted application level (old flag)
- "Dead" when the executor is not Blacklisted and not Active
- "Blacklisted in Stages: [...]" when the executor is Active but the there are 
active blacklisted stages for the executor. Within the [] coma separated active 
stageIDs are listed.
- "Active" when the executor is Active and there is no active blacklisted 
stages for the executor

## How was this patch tested?

Both with unit tests and manually.

 Manual test

Spark was started as:

```bash
 bin/spark-shell --master "local-cluster[2,1,1024]" --conf 
"spark.blacklist.enabled=true" --conf 
"spark.blacklist.stage.maxFailedTasksPerExecutor=1" --conf 
"spark.blacklist.application.maxFailedTasksPerExecutor=10"
```

And the job was:
```scala
import org.apache.spark.SparkEnv

val pairs = sc.parallelize(1 to 1, 10).map { x =>
  if (SparkEnv.get.executorId.toInt == 0) throw new RuntimeException("Bad 
executor")
  else  {
Thread.sleep(10)
(x % 10, x)
  }
}

val all = pairs.cogroup(pairs)

all.collect()
```

UI screenshots about the running:

- One executor is blacklisted in the two stages:

![One executor is blacklisted in two 
stages](https://issues.apache.org/jira/secure/attachment/12908314/multiple_stages_1.png)

- One stage completes the other one is still running:

![One stage completes the other is still 
running](https://issues.apache.org/jira/secure/attachment/12908315/multiple_stages_2.png)

- Both stages are completed:

![Both stages are 
completed](https://issues.apache.org/jira/secure/attachment/12908316/multiple_stages_3.png)

### Unit tests

In AppStatusListenerSuite.scala both the node blacklisting for a stage and the 
executor blacklisting for stage are tested.

Author: “attilapiros” 

Closes #20408 from attilapiros/SPARK-23189.


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

Branch: refs/heads/master
Commit: d6e1958a2472898e60bd013902c2f35111596e40
Parents: 116c581
Author: “attilapiros” 
Authored: Tue Feb 13 09:54:52 2018 -0600
Committer: Imran Rashid 
Committed: Tue Feb 13 09:54:52 2018 -0600

--
 .../org/apache/spark/ui/static/executorspage.js | 21 ++---
 .../apache/spark/status/AppStatusListener.scala | 49 +++-
 .../org/apache/spark/status/LiveEntity.scala|  7 ++-
 .../org/apache/spark/status/api/v1/api.scala|  3 +-
 .../executor_list_json_expectation.json |  3 +-
 .../executor_memory_usage_expectation.json  | 15 --
 .../executor_node_blacklisting_expectation.json | 15 --
 ...blacklisting_unblacklisting_expectation.json | 15 --
 .../spark/status/AppStatusListenerSuite.scala   | 21 +
 9 files changed, 113 insertions(+), 36 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d6e1958a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
--
diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js 
b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
index d430d8c..6717af3 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
@@ -25,12 +25,18 @@ function getThreadDumpEnabled() {
 return threadDumpEnabled;
 }
 
-function formatStatus(status, type) {
+function formatStatus(status, type, row) {
+if (row.isBlacklisted) {
+return "Blacklisted";
+}
+
 if (status) {
-return "Active"
-} else {
-return "Dead"
+if (row.blacklistedInStages.length == 0) {
+return "Active"
+}
+return "Active (Blacklisted in Stages: [" + 
row.blacklistedInStages.join(", ") + "])";
 }
+return "Dead"
 }
 
 jQuery.extend(jQuery.fn.dataTableExt.oSort, {
@@ -415,9 +421,10 @@ $(document).ready(function () {
 }
 },
 {data: 'hostPort'},
-{data: 'isActive', render: function (data, type, row) {
-if (row.isBlacklisted) return 

spark git commit: [SPARK-23253][CORE][SHUFFLE] Only write shuffle temporary index file when there is not an existing one

2018-02-02 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master b9503fcbb -> dd52681bf


[SPARK-23253][CORE][SHUFFLE] Only write shuffle temporary index file when there 
is not an existing one

## What changes were proposed in this pull request?

Shuffle Index temporay file is used for atomic creating shuffle index file, it 
is not needed when the index file already exists after another attempts of same 
task had it done.

## How was this patch tested?

exitsting ut

cc squito

Author: Kent Yao 

Closes #20422 from yaooqinn/SPARK-23253.


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

Branch: refs/heads/master
Commit: dd52681bf542386711609cb037a55b3d264eddef
Parents: b9503fc
Author: Kent Yao 
Authored: Fri Feb 2 09:10:50 2018 -0600
Committer: Imran Rashid 
Committed: Fri Feb 2 09:10:50 2018 -0600

--
 .../shuffle/IndexShuffleBlockResolver.scala | 27 +
 .../sort/IndexShuffleBlockResolverSuite.scala   | 59 ++--
 2 files changed, 56 insertions(+), 30 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/dd52681b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala 
b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
index 266ee42..c5f3f6e 100644
--- 
a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
+++ 
b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
@@ -141,19 +141,6 @@ private[spark] class IndexShuffleBlockResolver(
 val indexFile = getIndexFile(shuffleId, mapId)
 val indexTmp = Utils.tempFileWith(indexFile)
 try {
-  val out = new DataOutputStream(new BufferedOutputStream(new 
FileOutputStream(indexTmp)))
-  Utils.tryWithSafeFinally {
-// We take in lengths of each block, need to convert it to offsets.
-var offset = 0L
-out.writeLong(offset)
-for (length <- lengths) {
-  offset += length
-  out.writeLong(offset)
-}
-  } {
-out.close()
-  }
-
   val dataFile = getDataFile(shuffleId, mapId)
   // There is only one IndexShuffleBlockResolver per executor, this 
synchronization make sure
   // the following check and rename are atomic.
@@ -166,10 +153,22 @@ private[spark] class IndexShuffleBlockResolver(
   if (dataTmp != null && dataTmp.exists()) {
 dataTmp.delete()
   }
-  indexTmp.delete()
 } else {
   // This is the first successful attempt in writing the map outputs 
for this task,
   // so override any existing index and data files with the ones we 
wrote.
+  val out = new DataOutputStream(new BufferedOutputStream(new 
FileOutputStream(indexTmp)))
+  Utils.tryWithSafeFinally {
+// We take in lengths of each block, need to convert it to offsets.
+var offset = 0L
+out.writeLong(offset)
+for (length <- lengths) {
+  offset += length
+  out.writeLong(offset)
+}
+  } {
+out.close()
+  }
+
   if (indexFile.exists()) {
 indexFile.delete()
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/dd52681b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
 
b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
index d21ce73..4ce379b 100644
--- 
a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.shuffle.sort
 
-import java.io.{File, FileInputStream, FileOutputStream}
+import java.io.{DataInputStream, File, FileInputStream, FileOutputStream}
 
 import org.mockito.{Mock, MockitoAnnotations}
 import org.mockito.Answers.RETURNS_SMART_NULLS
@@ -64,6 +64,9 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite 
with BeforeAndAfterEa
   }
 
   test("commit shuffle files multiple times") {
+val shuffleId = 1
+val mapId = 2
+val idxName = s"shuffle_${shuffleId}_${mapId}_0.index"
 val resolver = new IndexShuffleBlockResolver(conf, blockManager)
 val lengths = 

spark git commit: [SPARK-23209][core] Allow credential manager to work when Hive not available.

2018-01-29 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 438631031 -> 75131ee86


[SPARK-23209][core] Allow credential manager to work when Hive not available.

The JVM seems to be doing early binding of classes that the Hive provider
depends on, causing an error to be thrown before it was caught by the code
in the class.

The fix wraps the creation of the provider in a try..catch so that
the provider can be ignored when dependencies are missing.

Added a unit test (which fails without the fix), and also tested
that getting tokens still works in a real cluster.

Author: Marcelo Vanzin 

Closes #20399 from vanzin/SPARK-23209.

(cherry picked from commit b834446ec1338349f6d974afd96f677db3e8fd1a)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.3
Commit: 75131ee867bca17ca3aade5f832f1d49b7cfcff5
Parents: 4386310
Author: Marcelo Vanzin 
Authored: Mon Jan 29 16:09:14 2018 -0600
Committer: Imran Rashid 
Committed: Mon Jan 29 16:09:28 2018 -0600

--
 .../security/HadoopDelegationTokenManager.scala | 17 +-
 .../HadoopDelegationTokenManagerSuite.scala | 58 
 2 files changed, 72 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/75131ee8/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
index 116a686..5151df0 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
@@ -64,9 +64,9 @@ private[spark] class HadoopDelegationTokenManager(
   }
 
   private def getDelegationTokenProviders: Map[String, 
HadoopDelegationTokenProvider] = {
-val providers = List(new HadoopFSDelegationTokenProvider(fileSystems),
-  new HiveDelegationTokenProvider,
-  new HBaseDelegationTokenProvider)
+val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystems)) ++
+  safeCreateProvider(new HiveDelegationTokenProvider) ++
+  safeCreateProvider(new HBaseDelegationTokenProvider)
 
 // Filter out providers for which 
spark.security.credentials.{service}.enabled is false.
 providers
@@ -75,6 +75,17 @@ private[spark] class HadoopDelegationTokenManager(
   .toMap
   }
 
+  private def safeCreateProvider(
+  createFn: => HadoopDelegationTokenProvider): 
Option[HadoopDelegationTokenProvider] = {
+try {
+  Some(createFn)
+} catch {
+  case t: Throwable =>
+logDebug(s"Failed to load built in provider.", t)
+None
+}
+  }
+
   def isServiceEnabled(serviceName: String): Boolean = {
 val key = providerEnabledConfig.format(serviceName)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/75131ee8/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
index eeffc36..2849a10 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.deploy.security
 
+import org.apache.commons.io.IOUtils
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileSystem
 import org.apache.hadoop.security.Credentials
@@ -110,7 +111,64 @@ class HadoopDelegationTokenManagerSuite extends 
SparkFunSuite with Matchers {
 creds.getAllTokens.size should be (0)
   }
 
+  test("SPARK-23209: obtain tokens when Hive classes are not available") {
+// This test needs a custom class loader to hide Hive classes which are in 
the classpath.
+// Because the manager code loads the Hive provider directly instead of 
using reflection, we
+// need to drive the test through the custom class loader so a new copy 
that cannot find
+// Hive classes is loaded.
+val currentLoader = Thread.currentThread().getContextClassLoader()
+val noHive = new ClassLoader() {
+  override def loadClass(name: 

spark git commit: [SPARK-23209][core] Allow credential manager to work when Hive not available.

2018-01-29 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master e30b34f7b -> b834446ec


[SPARK-23209][core] Allow credential manager to work when Hive not available.

The JVM seems to be doing early binding of classes that the Hive provider
depends on, causing an error to be thrown before it was caught by the code
in the class.

The fix wraps the creation of the provider in a try..catch so that
the provider can be ignored when dependencies are missing.

Added a unit test (which fails without the fix), and also tested
that getting tokens still works in a real cluster.

Author: Marcelo Vanzin 

Closes #20399 from vanzin/SPARK-23209.


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

Branch: refs/heads/master
Commit: b834446ec1338349f6d974afd96f677db3e8fd1a
Parents: e30b34f
Author: Marcelo Vanzin 
Authored: Mon Jan 29 16:09:14 2018 -0600
Committer: Imran Rashid 
Committed: Mon Jan 29 16:09:14 2018 -0600

--
 .../security/HadoopDelegationTokenManager.scala | 17 +-
 .../HadoopDelegationTokenManagerSuite.scala | 58 
 2 files changed, 72 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b834446e/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
index 116a686..5151df0 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
@@ -64,9 +64,9 @@ private[spark] class HadoopDelegationTokenManager(
   }
 
   private def getDelegationTokenProviders: Map[String, 
HadoopDelegationTokenProvider] = {
-val providers = List(new HadoopFSDelegationTokenProvider(fileSystems),
-  new HiveDelegationTokenProvider,
-  new HBaseDelegationTokenProvider)
+val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystems)) ++
+  safeCreateProvider(new HiveDelegationTokenProvider) ++
+  safeCreateProvider(new HBaseDelegationTokenProvider)
 
 // Filter out providers for which 
spark.security.credentials.{service}.enabled is false.
 providers
@@ -75,6 +75,17 @@ private[spark] class HadoopDelegationTokenManager(
   .toMap
   }
 
+  private def safeCreateProvider(
+  createFn: => HadoopDelegationTokenProvider): 
Option[HadoopDelegationTokenProvider] = {
+try {
+  Some(createFn)
+} catch {
+  case t: Throwable =>
+logDebug(s"Failed to load built in provider.", t)
+None
+}
+  }
+
   def isServiceEnabled(serviceName: String): Boolean = {
 val key = providerEnabledConfig.format(serviceName)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/b834446e/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
index eeffc36..2849a10 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.deploy.security
 
+import org.apache.commons.io.IOUtils
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileSystem
 import org.apache.hadoop.security.Credentials
@@ -110,7 +111,64 @@ class HadoopDelegationTokenManagerSuite extends 
SparkFunSuite with Matchers {
 creds.getAllTokens.size should be (0)
   }
 
+  test("SPARK-23209: obtain tokens when Hive classes are not available") {
+// This test needs a custom class loader to hide Hive classes which are in 
the classpath.
+// Because the manager code loads the Hive provider directly instead of 
using reflection, we
+// need to drive the test through the custom class loader so a new copy 
that cannot find
+// Hive classes is loaded.
+val currentLoader = Thread.currentThread().getContextClassLoader()
+val noHive = new ClassLoader() {
+  override def loadClass(name: String, resolve: Boolean): Class[_] = {
+if (name.startsWith("org.apache.hive") || 

[2/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting

2018-01-24 Thread irashid
http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/spark-events/application_1516285256255_0012
--
diff --git 
a/core/src/test/resources/spark-events/application_1516285256255_0012 
b/core/src/test/resources/spark-events/application_1516285256255_0012
new file mode 100755
index 000..3e1736c
--- /dev/null
+++ b/core/src/test/resources/spark-events/application_1516285256255_0012
@@ -0,0 +1,71 @@
+{"Event":"SparkListenerLogStart","Spark Version":"2.3.0-SNAPSHOT"}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java 
Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre","Java 
Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 
2.11.8"},"Spark 
Properties":{"spark.blacklist.enabled":"true","spark.driver.host":"apiros-1.gce.test.com","spark.eventLog.enabled":"true","spark.driver.port":"33058","spark.repl.class.uri":"spark://apiros-1.gce.test.com:33058/classes","spark.jars":"","spark.repl.class.outputDir":"/tmp/spark-6781fb17-e07a-4b32-848b-9936c2e88b33/repl-c0fd7008-04be-471e-a173-6ad3e62d53d7","spark.app.name":"Spark
 
shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"1","spark.scheduler.mode":"FIFO","spark.executor.instances":"8","spark.ui.showConsoleProgress":"true","spark.blacklist.stage.maxFailedTasksPerExecutor":"1","spark.executor.id":"driver","spark.submit.deployMode":"client","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.e
 
xecutor.memory":"2G","spark.home":"/github/spark","spark.sql.catalogImplementation":"hive","spark.driver.appUIAddress":"http://apiros-1.gce.test.com:4040","spark.blacklist.application.maxFailedTasksPerExecutor":"10","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"apiros-1.gce.test.com","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://apiros-1.gce.test.com:8088/proxy/application_1516285256255_0012","spark.app.id":"application_1516285256255_0012"},"System
 
Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot
 64-Bit Tiered 
Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java
 Virtual Machine Specification","java.vendor":"Oracle 
Corporation","java.vm.specification.version":"1.8","user.home":"*(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","su
 
n.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/amd64","user.dir":"*(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed
 
mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK
 Runtime 
Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java
 Platform API 
Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.
 
8.0.161-0.b14.el7_4.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-0.b14.el7_4.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"*(redacted)","java.specification.vendor":"Oracle
 
Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-693.5.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle
 
Corporation","user.country":"*(redacted)","sun.jnu.encoding":"UTF-8","user.language":"*(redacted)","java.vendor.url":"*(redacted)","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.nam
 e":"Linux","java.vm.vendor":"Oracle 
Corporation","java.vendor.url.bug":"*(redacted)","user.name":"*(redacted)","java.vm.name":"OpenJDK
 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit 
--master yarn --deploy-mode client --conf 
spark.blacklist.stage.maxFailedTasksPerExecutor=1 --conf 
spark.blacklist.enabled=true --conf 

[4/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting

2018-01-24 Thread irashid
http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
--
diff --git 
a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
index 31093a6..03f886a 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
@@ -421,7 +421,8 @@
   "shuffleWrite" : 13180,
   "shuffleWriteRecords" : 0,
   "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0
+  "diskBytesSpilled" : 0,
+  "isBlacklistedForStage" : false
 }
   },
   "killedTasksSummary" : { }

http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
--
diff --git 
a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
index 601d706..947c899 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
@@ -421,7 +421,8 @@
   "shuffleWrite" : 13180,
   "shuffleWriteRecords" : 0,
   "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0
+  "diskBytesSpilled" : 0,
+  "isBlacklistedForStage" : false
 }
   },
   "killedTasksSummary" : { }

http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
--
diff --git 
a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
index 9cdcef0..963f010 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
@@ -465,7 +465,8 @@
   "shuffleWrite" : 0,
   "shuffleWriteRecords" : 0,
   "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0
+  "diskBytesSpilled" : 0,
+  "isBlacklistedForStage" : false
 }
   },
   "killedTasksSummary" : { }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[1/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting

2018-01-24 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master de36f65d3 -> 0ec95bb7d


http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala 
b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 7aa60f2..87f12f3 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -156,6 +156,8 @@ class HistoryServerSuite extends SparkFunSuite with 
BeforeAndAfter with Matchers
   "applications/local-1426533911241/1/stages/0/0/taskList",
 "stage task list from multi-attempt app json(2)" ->
   "applications/local-1426533911241/2/stages/0/0/taskList",
+"blacklisting for stage" -> 
"applications/app-20180109111548-/stages/0/0",
+"blacklisting node for stage" -> 
"applications/application_1516285256255_0012/stages/0/0",
 
 "rdd list storage json" -> "applications/local-1422981780767/storage/rdd",
 "executor node blacklisting" -> 
"applications/app-20161116163331-/executors",

http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
index cd1b7a9..afebcdd 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
@@ -92,7 +92,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with 
BeforeAndAfterEach with M
   }
 
   def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = {
-new TaskSetBlacklist(conf, stageId, clock)
+new TaskSetBlacklist(listenerBusMock, conf, stageId, stageAttemptId = 0, 
clock = clock)
   }
 
   test("executors can be blacklisted with only a few failures per stage") {

http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
index 18981d5..6e2709d 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
@@ -16,18 +16,32 @@
  */
 package org.apache.spark.scheduler
 
+import org.mockito.Matchers.isA
+import org.mockito.Mockito.{never, verify}
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mockito.MockitoSugar
+
 import org.apache.spark.{SparkConf, SparkFunSuite}
 import org.apache.spark.internal.config
-import org.apache.spark.util.{ManualClock, SystemClock}
+import org.apache.spark.util.ManualClock
+
+class TaskSetBlacklistSuite extends SparkFunSuite with BeforeAndAfterEach with 
MockitoSugar {
 
-class TaskSetBlacklistSuite extends SparkFunSuite {
+  private var listenerBusMock: LiveListenerBus = _
+
+  override def beforeEach(): Unit = {
+listenerBusMock = mock[LiveListenerBus]
+super.beforeEach()
+  }
 
   test("Blacklisting tasks, executors, and nodes") {
 val conf = new SparkConf().setAppName("test").setMaster("local")
   .set(config.BLACKLIST_ENABLED.key, "true")
 val clock = new ManualClock
+val attemptId = 0
+val taskSetBlacklist = new TaskSetBlacklist(
+  listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = 
clock)
 
-val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, clock = 
clock)
 clock.setTime(0)
 // We will mark task 0 & 1 failed on both executor 1 & 2.
 // We should blacklist all executors on that host, for all tasks for the 
stage.  Note the API
@@ -46,27 +60,53 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
   val shouldBeBlacklisted = (executor == "exec1" && index == 0)
   assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) 
=== shouldBeBlacklisted)
 }
+
 assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
+verify(listenerBusMock, never())
+  .post(isA(classOf[SparkListenerExecutorBlacklistedForStage]))
+
 assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
+verify(listenerBusMock, never())
+  .post(isA(classOf[SparkListenerNodeBlacklistedForStage]))
 
 // Mark task 1 failed on exec1 -- this pushes the executor into the 
blacklist
 taskSetBlacklist.updateBlacklistForFailedTask(
   "hostA", exec = "exec1", index = 1, 

[5/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting

2018-01-24 Thread irashid
  }}
-{executor.map(_.isBlacklisted).getOrElse(false)}
+{
+  if (executor.map(_.isBlacklisted).getOrElse(false)) {
+for application
+  } else if (v.isBlacklistedForStage) {
+for stage
+  } else {
+false
+  }
+}
   
 }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
--
diff --git 
a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
index f2c3ec5..4fecf84 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
@@ -1,4 +1,34 @@
 [ {
+  "id" : "application_1516285256255_0012",
+  "name" : "Spark shell",
+  "attempts" : [ {
+"startTime" : "2018-01-18T18:30:35.119GMT",
+"endTime" : "2018-01-18T18:38:27.938GMT",
+"lastUpdated" : "",
+"duration" : 472819,
+"sparkUser" : "attilapiros",
+"completed" : true,
+"appSparkVersion" : "2.3.0-SNAPSHOT",
+"lastUpdatedEpoch" : 0,
+"startTimeEpoch" : 1516300235119,
+"endTimeEpoch" : 1516300707938
+  } ]
+}, {
+  "id" : "app-20180109111548-",
+  "name" : "Spark shell",
+  "attempts" : [ {
+"startTime" : "2018-01-09T10:15:42.372GMT",
+"endTime" : "2018-01-09T10:24:37.606GMT",
+"lastUpdated" : "",
+"duration" : 535234,
+"sparkUser" : "attilapiros",
+"completed" : true,
+"appSparkVersion" : "2.3.0-SNAPSHOT",
+"lastUpdatedEpoch" : 0,
+"startTimeEpoch" : 1515492942372,
+"endTimeEpoch" : 1515493477606
+  } ]
+}, {
   "id" : "app-20161116163331-",
   "name" : "Spark shell",
   "attempts" : [ {
@@ -9,9 +39,9 @@
 "sparkUser" : "jose",
 "completed" : true,
 "appSparkVersion" : "2.1.0-SNAPSHOT",
-"endTimeEpoch" : 1479335620587,
+"lastUpdatedEpoch" : 0,
 "startTimeEpoch" : 1479335609916,
-    "lastUpdatedEpoch" : 0
+"endTimeEpoch" : 1479335620587
   } ]
 }, {
   "id" : "app-20161115172038-",
@@ -24,9 +54,9 @@
 "sparkUser" : "jose",
 "completed" : true,
 "appSparkVersion" : "2.1.0-SNAPSHOT",
-"endTimeEpoch" : 1479252138874,
+"lastUpdatedEpoch" : 0,
 "startTimeEpoch" : 1479252037079,
-"lastUpdatedEpoch" : 0
+"endTimeEpoch" : 1479252138874
   } ]
 }, {
   "id" : "local-1430917381534",
@@ -39,9 +69,9 @@
 "sparkUser" : "irashid",
 "completed" : true,
     "appSparkVersion" : "1.4.0-SNAPSHOT",
-"endTimeEpoch" : 1430917391398,
+"lastUpdatedEpoch" : 0,
 "startTimeEpoch" : 1430917380893,
-"lastUpdatedEpoch" : 0
+"endTimeEpoch" : 1430917391398
   } ]
 }, {
   "id" : "local-1430917381535",
@@ -55,9 +85,9 @@
 "sparkUser" : "irashid",
 "completed" : true,
 "appSparkVersion" : "1.4.0-SNAPSHOT",
-"endTimeEpoch" : 1430917380950,
+"lastUpdatedEpoch" : 0,
 "startTimeEpoch" : 1430917380893,
-"lastUpdatedEpoch" : 0
+"endTimeEpoch" : 1430917380950
   }, {
 "attemptId" : "1",
 "startTime" : "2015-05-06T13:03:00.880GMT",
@@ -67,9 +97,9 @@
 "sparkUser" : "irashid",
 "completed" : true,
 "appSparkVersion" : "1.4.0-SNAPSHOT",
-"endTimeEpoch" : 1430917380890,
+"lastUpdatedEpoch" : 0,
 "startTimeEpoch" : 1430917380880,
-"lastUpdatedEpoch" : 0
+"endTimeEpoch" : 1430917380890
   } ]
 }, {
   "id" : "local-1426533911241",
@@ -83,9 +113,9 @@
 "sparkUser" : "irashid",
 "completed" : true,
 "appSparkVersion" : "",
-"endTimeEpoch" : 1426633945177,
+"lastUpdatedEpoch" : 0,
 "startTimeEpoch" : 1426633910242,
-"lastUpdated

[3/5] spark git commit: [SPARK-22577][CORE] executor page blacklist status should update with TaskSet level blacklisting

2018-01-24 Thread irashid
http://git-wip-us.apache.org/repos/asf/spark/blob/0ec95bb7/core/src/test/resources/spark-events/app-20180109111548-
--
diff --git a/core/src/test/resources/spark-events/app-20180109111548- 
b/core/src/test/resources/spark-events/app-20180109111548-
new file mode 100755
index 000..50893d3
--- /dev/null
+++ b/core/src/test/resources/spark-events/app-20180109111548-
@@ -0,0 +1,59 @@
+{"Event":"SparkListenerLogStart","Spark Version":"2.3.0-SNAPSHOT"}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java 
Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre","Java
 Version":"1.8.0_152 (Oracle Corporation)","Scala Version":"version 
2.11.8"},"Spark 
Properties":{"spark.blacklist.enabled":"true","spark.driver.host":"172.30.65.138","spark.eventLog.enabled":"true","spark.driver.port":"64273","spark.repl.class.uri":"spark://172.30.65.138:64273/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/9g/gf583nd1765cvfgb_lsvwgp0gp/T/spark-811c1b49-eb66-4bfb-91ae-33b45efa269d/repl-c4438f51-ee23-41ed-8e04-71496e2f40f5","spark.app.name":"Spark
 
shell","spark.scheduler.mode":"FIFO","spark.ui.showConsoleProgress":"true","spark.blacklist.stage.maxFailedTasksPerExecutor":"1","spark.executor.id":"driver","spark.submit.deployMode":"client","spark.master":"local-cluster[2,1,1024]","spark.home":"*(redacted)","spark.sql.catalogImplementation":"in-memory","spark.blacklist.application.maxF
 ailedTasksPerExecutor":"10","spark.app.id":"app-20180109111548-"},"System 
Properties":{"java.io.tmpdir":"/var/folders/9g/gf583nd1765cvfgb_lsvwgp0gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot
 64-Bit Tiered 
Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java
 Virtual Machine Specification","java.vendor":"Oracle 
Corporation","java.vm.specification.version":"1.8","user.home":"*(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib","user.dir":"*(redacted)","java.library.path":"*(redacted)","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.152-b16","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/l
 ib/endorsed","java.runtime.version":"1.8.0_152-b16","java.vm.info":"mixed 
mode","java.ext.dirs":"*(redacted)","java.runtime.name":"Java(TM) SE 
Runtime 
Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java
 Platform API 
Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre/classes","file
 
.encoding":"UTF-8","user.timezone":"*(redacted)","java.specification.vendor":"Oracle
 
Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.12.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle
 
Corporation","user.country":"*(redacted)","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"*(redacted)","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"*(redacted)","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac
 OS X","java.vm.vendor":"Oracle 
Corporation","java.vendor.url.bug":"*(redacted)","user.name":"*(redacted)","java.vm.name":"Java
 HotSpot(TM) 64-Bit Server 
VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master 
local-cluster[2,1,1024] --conf spark.blacklist.stage.maxF
 ailedTasksPerExecutor=1 --conf spark.blacklist.enabled=true --conf 
spark.blacklist.application.maxFailedTasksPerExecutor=10 --conf 
spark.eventLog.enabled=true --class org.apache.spark.repl.Main --name Spark 
shell 
spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/jre","java.version":"1.8.0_152","sun.io.unicode.encoding":"UnicodeBig"},"Classpath
 

spark git commit: [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.

2018-01-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 4b79514c9 -> d0cb19873


[SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.

The code was sorting "0" as "less than" negative values, which is a little
wrong. Fix is simple, most of the changes are the added test and related
cleanup.

Author: Marcelo Vanzin 

Closes #20284 from vanzin/SPARK-23103.

(cherry picked from commit aa3a1276f9e23ffbb093d00743e63cd4369f9f57)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.3
Commit: d0cb19873bb325be7e31de62b0ba117dd6b92619
Parents: 4b79514
Author: Marcelo Vanzin 
Authored: Fri Jan 19 13:32:20 2018 -0600
Committer: Imran Rashid 
Committed: Fri Jan 19 13:32:38 2018 -0600

--
 .../spark/util/kvstore/LevelDBTypeInfo.java |  2 +-
 .../spark/util/kvstore/DBIteratorSuite.java |  7 +-
 .../apache/spark/util/kvstore/LevelDBSuite.java | 77 ++--
 .../spark/status/AppStatusListenerSuite.scala   |  8 +-
 4 files changed, 52 insertions(+), 42 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d0cb1987/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
--
diff --git 
a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
 
b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
index 232ee41..f4d3592 100644
--- 
a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
+++ 
b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
@@ -493,7 +493,7 @@ class LevelDBTypeInfo {
 byte[] key = new byte[bytes * 2 + 2];
 long longValue = ((Number) value).longValue();
 key[0] = prefix;
-key[1] = longValue > 0 ? POSITIVE_MARKER : NEGATIVE_MARKER;
+key[1] = longValue >= 0 ? POSITIVE_MARKER : NEGATIVE_MARKER;
 
 for (int i = 0; i < key.length - 2; i++) {
   int masked = (int) ((longValue >>> (4 * i)) & 0xF);

http://git-wip-us.apache.org/repos/asf/spark/blob/d0cb1987/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
--
diff --git 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
index 9a81f86..1e06243 100644
--- 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
+++ 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
@@ -73,7 +73,9 @@ public abstract class DBIteratorSuite {
   private static final BaseComparator NATURAL_ORDER = (t1, t2) -> 
t1.key.compareTo(t2.key);
   private static final BaseComparator REF_INDEX_ORDER = (t1, t2) -> 
t1.id.compareTo(t2.id);
   private static final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> 
t1.name.compareTo(t2.name);
-  private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num 
- t2.num;
+  private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> {
+return Integer.valueOf(t1.num).compareTo(t2.num);
+  };
   private static final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> 
t1.child.compareTo(t2.child);
 
   /**
@@ -112,7 +114,8 @@ public abstract class DBIteratorSuite {
   t.key = "key" + i;
   t.id = "id" + i;
   t.name = "name" + RND.nextInt(MAX_ENTRIES);
-  t.num = RND.nextInt(MAX_ENTRIES);
+  // Force one item to have an integer value of zero to test the fix for 
SPARK-23103.
+  t.num = (i != 0) ? (int) RND.nextLong() : 0;
   t.child = "child" + (i % MIN_ENTRIES);
   allEntries.add(t);
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d0cb1987/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
--
diff --git 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
index 2b07d24..b8123ac 100644
--- 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
+++ 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
@@ -21,6 +21,8 @@ import java.io.File;
 import java.util.Arrays;
 import java.util.List;
 import java.util.NoSuchElementException;
+import java.util.stream.Collectors;
+import 

spark git commit: [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.

2018-01-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master fed2139f0 -> aa3a1276f


[SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.

The code was sorting "0" as "less than" negative values, which is a little
wrong. Fix is simple, most of the changes are the added test and related
cleanup.

Author: Marcelo Vanzin 

Closes #20284 from vanzin/SPARK-23103.


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

Branch: refs/heads/master
Commit: aa3a1276f9e23ffbb093d00743e63cd4369f9f57
Parents: fed2139
Author: Marcelo Vanzin 
Authored: Fri Jan 19 13:32:20 2018 -0600
Committer: Imran Rashid 
Committed: Fri Jan 19 13:32:20 2018 -0600

--
 .../spark/util/kvstore/LevelDBTypeInfo.java |  2 +-
 .../spark/util/kvstore/DBIteratorSuite.java |  7 +-
 .../apache/spark/util/kvstore/LevelDBSuite.java | 77 ++--
 .../spark/status/AppStatusListenerSuite.scala   |  8 +-
 4 files changed, 52 insertions(+), 42 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/aa3a1276/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
--
diff --git 
a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
 
b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
index 232ee41..f4d3592 100644
--- 
a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
+++ 
b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java
@@ -493,7 +493,7 @@ class LevelDBTypeInfo {
 byte[] key = new byte[bytes * 2 + 2];
 long longValue = ((Number) value).longValue();
 key[0] = prefix;
-key[1] = longValue > 0 ? POSITIVE_MARKER : NEGATIVE_MARKER;
+key[1] = longValue >= 0 ? POSITIVE_MARKER : NEGATIVE_MARKER;
 
 for (int i = 0; i < key.length - 2; i++) {
   int masked = (int) ((longValue >>> (4 * i)) & 0xF);

http://git-wip-us.apache.org/repos/asf/spark/blob/aa3a1276/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
--
diff --git 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
index 9a81f86..1e06243 100644
--- 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
+++ 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java
@@ -73,7 +73,9 @@ public abstract class DBIteratorSuite {
   private static final BaseComparator NATURAL_ORDER = (t1, t2) -> 
t1.key.compareTo(t2.key);
   private static final BaseComparator REF_INDEX_ORDER = (t1, t2) -> 
t1.id.compareTo(t2.id);
   private static final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> 
t1.name.compareTo(t2.name);
-  private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num 
- t2.num;
+  private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> {
+return Integer.valueOf(t1.num).compareTo(t2.num);
+  };
   private static final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> 
t1.child.compareTo(t2.child);
 
   /**
@@ -112,7 +114,8 @@ public abstract class DBIteratorSuite {
   t.key = "key" + i;
   t.id = "id" + i;
   t.name = "name" + RND.nextInt(MAX_ENTRIES);
-  t.num = RND.nextInt(MAX_ENTRIES);
+  // Force one item to have an integer value of zero to test the fix for 
SPARK-23103.
+  t.num = (i != 0) ? (int) RND.nextLong() : 0;
   t.child = "child" + (i % MIN_ENTRIES);
   allEntries.add(t);
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/aa3a1276/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
--
diff --git 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
index 2b07d24..b8123ac 100644
--- 
a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
+++ 
b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java
@@ -21,6 +21,8 @@ import java.io.File;
 import java.util.Arrays;
 import java.util.List;
 import java.util.NoSuchElementException;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 
 import org.apache.commons.io.FileUtils;
 import org.iq80.leveldb.DBIterator;
@@ -74,11 +76,7 @@ public class 

spark git commit: [SPARK-20664][CORE] Delete stale application data from SHS.

2018-01-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 ffe45913d -> 4b79514c9


[SPARK-20664][CORE] Delete stale application data from SHS.

Detect the deletion of event log files from storage, and remove
data about the related application attempt in the SHS.

Also contains code to fix SPARK-21571 based on code by ericvandenbergfb.

Author: Marcelo Vanzin 

Closes #20138 from vanzin/SPARK-20664.

(cherry picked from commit fed2139f053fac4a9a6952ff0ab1cc2a5f657bd0)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.3
Commit: 4b79514c90ca76674d17fd80d125e9dbfb0e845e
Parents: ffe4591
Author: Marcelo Vanzin 
Authored: Fri Jan 19 13:26:37 2018 -0600
Committer: Imran Rashid 
Committed: Fri Jan 19 13:27:02 2018 -0600

--
 .../deploy/history/FsHistoryProvider.scala  | 297 ---
 .../deploy/history/FsHistoryProviderSuite.scala | 117 +++-
 .../deploy/history/HistoryServerSuite.scala |   4 +-
 3 files changed, 306 insertions(+), 112 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4b79514c/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 94c80eb..f9d0b5e 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -19,7 +19,7 @@ package org.apache.spark.deploy.history
 
 import java.io.{File, FileNotFoundException, IOException}
 import java.util.{Date, ServiceLoader, UUID}
-import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit}
+import java.util.concurrent.{ExecutorService, TimeUnit}
 import java.util.zip.{ZipEntry, ZipOutputStream}
 
 import scala.collection.JavaConverters._
@@ -29,7 +29,7 @@ import scala.xml.Node
 
 import com.fasterxml.jackson.annotation.JsonIgnore
 import com.google.common.io.ByteStreams
-import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
+import com.google.common.util.concurrent.MoreExecutors
 import org.apache.hadoop.fs.{FileStatus, Path}
 import org.apache.hadoop.fs.permission.FsAction
 import org.apache.hadoop.hdfs.DistributedFileSystem
@@ -116,8 +116,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
   // Used by check event thread and clean log thread.
   // Scheduled thread pool size must be one, otherwise it will have concurrent 
issues about fs
   // and applications between check task and clean task.
-  private val pool = Executors.newScheduledThreadPool(1, new 
ThreadFactoryBuilder()
-.setNameFormat("spark-history-task-%d").setDaemon(true).build())
+  private val pool = 
ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-history-task-%d")
 
   // The modification time of the newest log detected during the last scan.   
Currently only
   // used for logging msgs (logs are re-scanned based on file size, rather 
than modtime)
@@ -174,7 +173,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
* Fixed size thread pool to fetch and parse log files.
*/
   private val replayExecutor: ExecutorService = {
-if (!conf.contains("spark.testing")) {
+if (Utils.isTesting) {
   ThreadUtils.newDaemonFixedThreadPool(NUM_PROCESSING_THREADS, 
"log-replay-executor")
 } else {
   MoreExecutors.sameThreadExecutor()
@@ -275,7 +274,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
 try {
   Some(load(appId).toApplicationInfo())
 } catch {
-  case e: NoSuchElementException =>
+  case _: NoSuchElementException =>
 None
 }
   }
@@ -405,49 +404,70 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
 try {
   val newLastScanTime = getNewLastScanTime()
   logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime")
-  // scan for modified applications, replay and merge them
-  val logInfos = Option(fs.listStatus(new 
Path(logDir))).map(_.toSeq).getOrElse(Nil)
+
+  val updated = Option(fs.listStatus(new 
Path(logDir))).map(_.toSeq).getOrElse(Nil)
 .filter { entry =>
   !entry.isDirectory() &&
 // FsHistoryProvider generates a hidden file which can't be read.  
Accidentally
 // reading a garbage file is safe, but we would 

spark git commit: [SPARK-20664][CORE] Delete stale application data from SHS.

2018-01-19 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 07296a61c -> fed2139f0


[SPARK-20664][CORE] Delete stale application data from SHS.

Detect the deletion of event log files from storage, and remove
data about the related application attempt in the SHS.

Also contains code to fix SPARK-21571 based on code by ericvandenbergfb.

Author: Marcelo Vanzin 

Closes #20138 from vanzin/SPARK-20664.


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

Branch: refs/heads/master
Commit: fed2139f053fac4a9a6952ff0ab1cc2a5f657bd0
Parents: 07296a6
Author: Marcelo Vanzin 
Authored: Fri Jan 19 13:26:37 2018 -0600
Committer: Imran Rashid 
Committed: Fri Jan 19 13:26:37 2018 -0600

--
 .../deploy/history/FsHistoryProvider.scala  | 297 ---
 .../deploy/history/FsHistoryProviderSuite.scala | 117 +++-
 .../deploy/history/HistoryServerSuite.scala |   4 +-
 3 files changed, 306 insertions(+), 112 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/fed2139f/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 94c80eb..f9d0b5e 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -19,7 +19,7 @@ package org.apache.spark.deploy.history
 
 import java.io.{File, FileNotFoundException, IOException}
 import java.util.{Date, ServiceLoader, UUID}
-import java.util.concurrent.{Executors, ExecutorService, Future, TimeUnit}
+import java.util.concurrent.{ExecutorService, TimeUnit}
 import java.util.zip.{ZipEntry, ZipOutputStream}
 
 import scala.collection.JavaConverters._
@@ -29,7 +29,7 @@ import scala.xml.Node
 
 import com.fasterxml.jackson.annotation.JsonIgnore
 import com.google.common.io.ByteStreams
-import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
+import com.google.common.util.concurrent.MoreExecutors
 import org.apache.hadoop.fs.{FileStatus, Path}
 import org.apache.hadoop.fs.permission.FsAction
 import org.apache.hadoop.hdfs.DistributedFileSystem
@@ -116,8 +116,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
   // Used by check event thread and clean log thread.
   // Scheduled thread pool size must be one, otherwise it will have concurrent 
issues about fs
   // and applications between check task and clean task.
-  private val pool = Executors.newScheduledThreadPool(1, new 
ThreadFactoryBuilder()
-.setNameFormat("spark-history-task-%d").setDaemon(true).build())
+  private val pool = 
ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-history-task-%d")
 
   // The modification time of the newest log detected during the last scan.   
Currently only
   // used for logging msgs (logs are re-scanned based on file size, rather 
than modtime)
@@ -174,7 +173,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
* Fixed size thread pool to fetch and parse log files.
*/
   private val replayExecutor: ExecutorService = {
-if (!conf.contains("spark.testing")) {
+if (Utils.isTesting) {
   ThreadUtils.newDaemonFixedThreadPool(NUM_PROCESSING_THREADS, 
"log-replay-executor")
 } else {
   MoreExecutors.sameThreadExecutor()
@@ -275,7 +274,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
 try {
   Some(load(appId).toApplicationInfo())
 } catch {
-  case e: NoSuchElementException =>
+  case _: NoSuchElementException =>
 None
 }
   }
@@ -405,49 +404,70 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
 try {
   val newLastScanTime = getNewLastScanTime()
   logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime")
-  // scan for modified applications, replay and merge them
-  val logInfos = Option(fs.listStatus(new 
Path(logDir))).map(_.toSeq).getOrElse(Nil)
+
+  val updated = Option(fs.listStatus(new 
Path(logDir))).map(_.toSeq).getOrElse(Nil)
 .filter { entry =>
   !entry.isDirectory() &&
 // FsHistoryProvider generates a hidden file which can't be read.  
Accidentally
 // reading a garbage file is safe, but we would log an error which 
can be scary to
 // the end-user.
 !entry.getPath().getName().startsWith(".") &&
- 

spark git commit: [SPARK-22850][CORE] Ensure queued events are delivered to all event queues.

2018-01-04 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/branch-2.3 cd92913f3 -> bc4bef472


[SPARK-22850][CORE] Ensure queued events are delivered to all event queues.

The code in LiveListenerBus was queueing events before start in the
queues themselves; so in situations like the following:

   bus.post(someEvent)
   bus.addToEventLogQueue(listener)
   bus.start()

"someEvent" would not be delivered to "listener" if that was the first
listener in the queue, because the queue wouldn't exist when the
event was posted.

This change buffers the events before starting the bus in the bus itself,
so that they can be delivered to all registered queues when the bus is
started.

Also tweaked the unit tests to cover the behavior above.

Author: Marcelo Vanzin 

Closes #20039 from vanzin/SPARK-22850.

(cherry picked from commit d2cddc88eac32f26b18ec26bb59e85c6f09a8c88)
Signed-off-by: Imran Rashid 


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

Branch: refs/heads/branch-2.3
Commit: bc4bef472de0e99f74a80954d694c3d1744afe3a
Parents: cd92913
Author: Marcelo Vanzin 
Authored: Thu Jan 4 16:19:00 2018 -0600
Committer: Imran Rashid 
Committed: Thu Jan 4 16:19:22 2018 -0600

--
 .../spark/scheduler/LiveListenerBus.scala   | 45 +---
 .../spark/scheduler/SparkListenerSuite.scala| 21 +
 2 files changed, 52 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bc4bef47/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala 
b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
index 2312140..ba6387a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
@@ -62,6 +62,9 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
 
   private val queues = new CopyOnWriteArrayList[AsyncEventQueue]()
 
+  // Visible for testing.
+  @volatile private[scheduler] var queuedEvents = new 
mutable.ListBuffer[SparkListenerEvent]()
+
   /** Add a listener to queue shared by all non-internal listeners. */
   def addToSharedQueue(listener: SparkListenerInterface): Unit = {
 addToQueue(listener, SHARED_QUEUE)
@@ -125,13 +128,39 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
 
   /** Post an event to all queues. */
   def post(event: SparkListenerEvent): Unit = {
-if (!stopped.get()) {
-  metrics.numEventsPosted.inc()
-  val it = queues.iterator()
-  while (it.hasNext()) {
-it.next().post(event)
+if (stopped.get()) {
+  return
+}
+
+metrics.numEventsPosted.inc()
+
+// If the event buffer is null, it means the bus has been started and we 
can avoid
+// synchronization and post events directly to the queues. This should be 
the most
+// common case during the life of the bus.
+if (queuedEvents == null) {
+  postToQueues(event)
+  return
+}
+
+// Otherwise, need to synchronize to check whether the bus is started, to 
make sure the thread
+// calling start() picks up the new event.
+synchronized {
+  if (!started.get()) {
+queuedEvents += event
+return
   }
 }
+
+// If the bus was already started when the check above was made, just post 
directly to the
+// queues.
+postToQueues(event)
+  }
+
+  private def postToQueues(event: SparkListenerEvent): Unit = {
+val it = queues.iterator()
+while (it.hasNext()) {
+  it.next().post(event)
+}
   }
 
   /**
@@ -149,7 +178,11 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
 }
 
 this.sparkContext = sc
-queues.asScala.foreach(_.start(sc))
+queues.asScala.foreach { q =>
+  q.start(sc)
+  queuedEvents.foreach(q.post)
+}
+queuedEvents = null
 metricsSystem.registerSource(metrics)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/bc4bef47/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 1beb36a..da6ecb8 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -48,7 +48,7 @@ class 

spark git commit: [SPARK-22850][CORE] Ensure queued events are delivered to all event queues.

2018-01-04 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 93f92c0ed -> d2cddc88e


[SPARK-22850][CORE] Ensure queued events are delivered to all event queues.

The code in LiveListenerBus was queueing events before start in the
queues themselves; so in situations like the following:

   bus.post(someEvent)
   bus.addToEventLogQueue(listener)
   bus.start()

"someEvent" would not be delivered to "listener" if that was the first
listener in the queue, because the queue wouldn't exist when the
event was posted.

This change buffers the events before starting the bus in the bus itself,
so that they can be delivered to all registered queues when the bus is
started.

Also tweaked the unit tests to cover the behavior above.

Author: Marcelo Vanzin 

Closes #20039 from vanzin/SPARK-22850.


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

Branch: refs/heads/master
Commit: d2cddc88eac32f26b18ec26bb59e85c6f09a8c88
Parents: 93f92c0
Author: Marcelo Vanzin 
Authored: Thu Jan 4 16:19:00 2018 -0600
Committer: Imran Rashid 
Committed: Thu Jan 4 16:19:00 2018 -0600

--
 .../spark/scheduler/LiveListenerBus.scala   | 45 +---
 .../spark/scheduler/SparkListenerSuite.scala| 21 +
 2 files changed, 52 insertions(+), 14 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d2cddc88/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala 
b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
index 2312140..ba6387a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala
@@ -62,6 +62,9 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
 
   private val queues = new CopyOnWriteArrayList[AsyncEventQueue]()
 
+  // Visible for testing.
+  @volatile private[scheduler] var queuedEvents = new 
mutable.ListBuffer[SparkListenerEvent]()
+
   /** Add a listener to queue shared by all non-internal listeners. */
   def addToSharedQueue(listener: SparkListenerInterface): Unit = {
 addToQueue(listener, SHARED_QUEUE)
@@ -125,13 +128,39 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
 
   /** Post an event to all queues. */
   def post(event: SparkListenerEvent): Unit = {
-if (!stopped.get()) {
-  metrics.numEventsPosted.inc()
-  val it = queues.iterator()
-  while (it.hasNext()) {
-it.next().post(event)
+if (stopped.get()) {
+  return
+}
+
+metrics.numEventsPosted.inc()
+
+// If the event buffer is null, it means the bus has been started and we 
can avoid
+// synchronization and post events directly to the queues. This should be 
the most
+// common case during the life of the bus.
+if (queuedEvents == null) {
+  postToQueues(event)
+  return
+}
+
+// Otherwise, need to synchronize to check whether the bus is started, to 
make sure the thread
+// calling start() picks up the new event.
+synchronized {
+  if (!started.get()) {
+queuedEvents += event
+return
   }
 }
+
+// If the bus was already started when the check above was made, just post 
directly to the
+// queues.
+postToQueues(event)
+  }
+
+  private def postToQueues(event: SparkListenerEvent): Unit = {
+val it = queues.iterator()
+while (it.hasNext()) {
+  it.next().post(event)
+}
   }
 
   /**
@@ -149,7 +178,11 @@ private[spark] class LiveListenerBus(conf: SparkConf) {
 }
 
 this.sparkContext = sc
-queues.asScala.foreach(_.start(sc))
+queues.asScala.foreach { q =>
+  q.start(sc)
+  queuedEvents.foreach(q.post)
+}
+queuedEvents = null
 metricsSystem.registerSource(metrics)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d2cddc88/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 1beb36a..da6ecb8 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -48,7 +48,7 @@ class SparkListenerSuite extends SparkFunSuite with 
LocalSparkContext with Match
 

spark git commit: [SPARK-22921][PROJECT-INFRA] Bug fix in jira assigning

2017-12-29 Thread irashid
Repository: spark
Updated Branches:
  refs/heads/master 66a7d6b30 -> ccda75b0d


[SPARK-22921][PROJECT-INFRA] Bug fix in jira assigning

Small bug fix from last pr, ran a successful merge with this code.

Author: Imran Rashid 

Closes #20117 from squito/SPARK-22921.


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

Branch: refs/heads/master
Commit: ccda75b0d17637abaa7ab54b68e4db8e1a16f2ce
Parents: 66a7d6b
Author: Imran Rashid 
Authored: Fri Dec 29 17:07:01 2017 -0600
Committer: Imran Rashid 
Committed: Fri Dec 29 17:07:01 2017 -0600

--
 dev/merge_spark_pr.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ccda75b0/dev/merge_spark_pr.py
--
diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py
index cd5dd1e..57ca840 100755
--- a/dev/merge_spark_pr.py
+++ b/dev/merge_spark_pr.py
@@ -242,8 +242,8 @@ def resolve_jira_issue(merge_branches, comment, 
default_jira_id=""):
 cur_summary = issue.fields.summary
 cur_assignee = issue.fields.assignee
 if cur_assignee is None:
-cur_assignee = choose_jira_assignee(issue)
-# Check again, we might not have chose an assignee
+cur_assignee = choose_jira_assignee(issue, asf_jira)
+# Check again, we might not have chosen an assignee
 if cur_assignee is None:
 cur_assignee = "NOT ASSIGNED!!!"
 else:


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



  1   2   3   >