Commit graph

6452 commits

Author SHA1 Message Date
Yinan Li 3f4060c340 [SPARK-22646][K8S] Spark on Kubernetes - basic submission client
This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31.

This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable.

The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default.

This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles.

* The patch contains unit tests which are passing.
* Manual testing: ./build/mvn -Pkubernetes clean package succeeded.
* It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
* There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI.
* Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926

Author: Yinan Li <liyinan926@gmail.com>

Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 15:15:05 -08:00
Sandor Murakozi e4639fa68f [SPARK-21672][CORE] Remove SHS-specific application / attempt data …
…structures

## What changes were proposed in this pull request?

In general, the SHS pages now use the public API types to represent applications. Some internal code paths still used its own view of what applications and attempts look like (`ApplicationHistoryInfo` and `ApplicationAttemptInfo`), declared in ApplicationHistoryProvider.scala.

This pull request removes these classes and updates the rest of the code to use `status.api.v1.ApplicationInfo` and `status.api.v1.ApplicationAttemptInfo` instead.

Furthermore `status.api.v1.ApplicationInfo` and `status.api.v1.ApplicationAttemptInfo` were changed to case class to
- facilitate copying instances
- equality checking in test code
- nicer toString()

To simplify the code a bit `v1.` prefixes were also removed from occurrences of v1.ApplicationInfo and v1.ApplicationAttemptInfo as there is no more ambiguity between classes in history and status.api.v1.

## How was this patch tested?

By running existing automated tests.

Author: Sandor Murakozi <smurakozi@gmail.com>

Closes #19920 from smurakozi/SPARK-21672.
2017-12-08 14:17:50 -08:00
Li Jin 26e66453de [SPARK-22655][PYSPARK] Throw exception rather than exit silently in PythonRunner when Spark session is stopped
## What changes were proposed in this pull request?

During Spark shutdown, if there are some active tasks, sometimes they will complete with incorrect results. The issue is in PythonRunner where it is returning partial result instead of throwing exception during Spark shutdown.

This patch makes it so that these tasks fail instead of complete with partial results.

## How was this patch tested?

Existing tests.

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19852 from icexelloss/python-runner-shutdown.
2017-12-08 20:44:21 +09:00
Juliusz Sompolski f28b1a4c41 [SPARK-22721] BytesToBytesMap peak memory not updated.
## What changes were proposed in this pull request?

Follow-up to earlier commit.
The peak memory of BytesToBytesMap is not updated in more places - spill() and destructiveIterator().

## How was this patch tested?

Manually.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19923 from juliuszsompolski/SPARK-22721cd.
2017-12-08 12:19:45 +01:00
kellyzly f41c0a93fd [SPARK-22660][BUILD] Use position() and limit() to fix ambiguity issue in scala-2.12
…a-2.12 and JDK9

## What changes were proposed in this pull request?
Some compile error after upgrading to scala-2.12
```javascript
spark_source/core/src/main/scala/org/apache/spark/executor/Executor.scala:455: ambiguous reference to overloaded definition, method limit in class ByteBuffer of type (x$1: Int)java.nio.ByteBuffer
method limit in class Buffer of type ()Int
match expected type ?
     val resultSize = serializedDirectResult.limit
error
```
The limit method was moved from ByteBuffer to the superclass Buffer and it can no longer be called without (). The same reason for position method.

```javascript
/home/zly/prj/oss/jdk9_HOS_SOURCE/spark_source/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala:427: ambiguous reference to overloaded definition, [error] both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit [error] and  method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit [error] match argument types (java.util.Map[String,String])
 [error]       props.putAll(outputSerdeProps.toMap.asJava)
 [error]             ^
 ```
This is because the key type is Object instead of String which is unsafe.

## How was this patch tested?

running tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: kellyzly <kellyzly@126.com>

Closes #19854 from kellyzly/SPARK-22660.
2017-12-07 10:04:04 -06:00
Brad Kaiser beb717f648 [SPARK-22618][CORE] Catch exception in removeRDD to stop jobs from dying
## What changes were proposed in this pull request?

I propose that BlockManagerMasterEndpoint.removeRdd() should catch and log any IOExceptions it receives. As it is now, the exception can bubble up to the main thread and kill user applications when called from RDD.unpersist(). I think this change is a better experience for the end user.

I chose to catch the exception in BlockManagerMasterEndpoint.removeRdd() instead of RDD.unpersist() because this way the RDD.unpersist() blocking option will still work correctly. Otherwise, blocking will get short circuited by the first error.

## How was this patch tested?

This patch was tested with a job that shows the job killing behavior mentioned above.

rxin, it looks like you originally wrote this method, I would appreciate it if you took a look. Thanks.

This contribution is my original work and is licensed under the project's open source license.

Author: Brad Kaiser <kaiserb@us.ibm.com>

Closes #19836 from brad-kaiser/catch-unpersist-exception.
2017-12-07 21:04:09 +08:00
Juliusz Sompolski d32337b1ef [SPARK-22721] BytesToBytesMap peak memory usage not accurate after reset()
## What changes were proposed in this pull request?

BytesToBytesMap doesn't update peak memory usage before shrinking back to initial capacity in reset(), so after a disk spill one never knows what was the size of hash table was before spilling.

## How was this patch tested?

Checked manually.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #19915 from juliuszsompolski/SPARK-22721.
2017-12-07 13:05:59 +01:00
Devaraj K 51066b437b [SPARK-14228][CORE][YARN] Lost executor of RPC disassociated, and occurs exception: Could not find CoarseGrainedScheduler or it has been stopped
## What changes were proposed in this pull request?
I see the two instances where the exception is occurring.

**Instance 1:**

```
17/11/10 15:49:32 ERROR util.Utils: Uncaught exception in thread driver-revive-thread
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:140)
        at org.apache.spark.rpc.netty.NettyRpcEnv.send(NettyRpcEnv.scala:187)
        at org.apache.spark.rpc.netty.NettyRpcEndpointRef.send(NettyRpcEnv.scala:521)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at scala.Option.foreach(Option.scala:257)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1.apply$mcV$sp(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1344)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1.run(CoarseGrainedSchedulerBackend.scala:124)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
```

In CoarseGrainedSchedulerBackend.scala, driver-revive-thread starts with DriverEndpoint.onStart() and keeps sending the ReviveOffers messages periodically till it gets shutdown as part DriverEndpoint.onStop(). There is no proper coordination between the driver-revive-thread(shutdown) and the RpcEndpoint unregister, RpcEndpoint unregister happens first and then driver-revive-thread shuts down as part of DriverEndpoint.onStop(), In-between driver-revive-thread may try to send the ReviveOffers message which is leading to the above exception.

To fix this issue, this PR moves the shutting down of driver-revive-thread to CoarseGrainedSchedulerBackend.stop() which executes before the DriverEndpoint unregister.

**Instance 2:**

```
17/11/10 16:31:38 ERROR cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: Error requesting driver to remove executor 1 for reason Executor for container container_1508535467865_0226_01_000002 exited because of a YARN event (e.g., pre-emption) and not because of an error in the running job.
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        at org.apache.spark.rpc.netty.Dispatcher.postLocalMessage(Dispatcher.scala:135)
        at org.apache.spark.rpc.netty.NettyRpcEnv.ask(NettyRpcEnv.scala:229)
        at org.apache.spark.rpc.netty.NettyRpcEndpointRef.ask(NettyRpcEnv.scala:516)
        at org.apache.spark.rpc.RpcEndpointRef.ask(RpcEndpointRef.scala:63)
        at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receive$1.applyOrElse(YarnSchedulerBackend.scala:269)
        at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117)
        at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205)
        at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101)
        at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:221)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
```

Here YarnDriverEndpoint tries to send remove executor messages after the Yarn scheduler backend service stop, which is leading to the above exception. To avoid the above exception,
1) We may add a condition(which checks whether service has stopped or not) before sending executor remove message
2) Add a warn log message in onFailure case when the service is already stopped

In this PR, chosen the 2) option which adds a log message in the case of onFailure without the exception stack trace since the option 1) would need to to go through for every remove executor message.

## How was this patch tested?
I verified it manually, I don't see these exceptions with the PR changes.

Author: Devaraj K <devaraj@apache.org>

Closes #19741 from devaraj-kavali/SPARK-14228.
2017-12-06 10:39:15 -08:00
Reynold Xin 4286cba7da [SPARK-22710] ConfigBuilder.fallbackConf should trigger onCreate function
## What changes were proposed in this pull request?
I was looking at the config code today and found that configs defined using ConfigBuilder.fallbackConf didn't trigger onCreate function. This patch fixes it.

This doesn't require backporting since we currently have no configs that use it.

## How was this patch tested?
Added a test case for all the config final creator functions in ConfigEntrySuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #19905 from rxin/SPARK-22710.
2017-12-06 10:11:25 -08:00
Carson Wang 03fdc92e42 [SPARK-22681] Accumulator should only be updated once for each task in result stage
## What changes were proposed in this pull request?
As the doc says "For accumulator updates performed inside actions only, Spark guarantees that each task’s update to the accumulator will only be applied once, i.e. restarted tasks will not update the value."
But currently the code doesn't guarantee this.

## How was this patch tested?
New added tests.

Author: Carson Wang <carson.wang@intel.com>

Closes #19877 from carsonwang/fixAccum.
2017-12-05 09:15:22 -08:00
Marcelo Vanzin e1dd03e42c [SPARK-22372][CORE, YARN] Make cluster submission use SparkApplication.
The main goal of this change is to allow multiple cluster-mode
submissions from the same JVM, without having them end up with
mixed configuration. That is done by extending the SparkApplication
trait, and doing so was reasonably trivial for standalone and
mesos modes.

For YARN mode, there was a complication. YARN used a "SPARK_YARN_MODE"
system property to control behavior indirectly in a whole bunch of
places, mainly in the SparkHadoopUtil / YarnSparkHadoopUtil classes.
Most of the changes here are removing that.

Since we removed support for Hadoop 1.x, some methods that lived in
YarnSparkHadoopUtil can now live in SparkHadoopUtil. The remaining
methods don't need to be part of the class, and can be called directly
from the YarnSparkHadoopUtil object, so now there's a single
implementation of SparkHadoopUtil.

There were two places in the code that relied on  SPARK_YARN_MODE to
make decisions about YARN-specific functionality, and now explicitly check
the master from the configuration for that instead:

* fetching the external shuffle service port, which can come from the YARN
  configuration.

* propagation of the authentication secret using Hadoop credentials. This also
  was cleaned up a little to not need so many methods in `SparkHadoopUtil`.

With those out of the way, actually changing the YARN client
to extend SparkApplication was easy.

Tested with existing unit tests, and also by running YARN apps
with auth and kerberos both on and off in a real cluster.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19631 from vanzin/SPARK-22372.
2017-12-04 11:05:03 -08:00
Reza Safi f81401e1cb [SPARK-22162] Executors and the driver should use consistent JobIDs in the RDD commit protocol
I have modified SparkHadoopWriter so that executors and the driver always use consistent JobIds during the hadoop commit. Before SPARK-18191, spark always used the rddId, it just incorrectly named the variable stageId. After SPARK-18191, it used the rddId as the jobId on the driver's side, and the stageId as the jobId on the executors' side. With this change executors and the driver will consistently uses rddId as the jobId. Also with this change, during the hadoop commit protocol spark uses  actual stageId to check whether a stage can be committed unlike before that  it was using executors' jobId to do this check.
In addition to the existing unit tests, a test has been added to check whether executors and the driver are using the same JobId. The test failed before this change and passed after applying this fix.

Author: Reza Safi <rezasafi@cloudera.com>

Closes #19848 from rezasafi/stagerddsimple.
2017-12-04 09:23:48 -08:00
Shixiong Zhu ee10ca7ec6 [SPARK-22638][SS] Use a separate queue for StreamingQueryListenerBus
## What changes were proposed in this pull request?

Use a separate Spark event queue for StreamingQueryListenerBus so that if there are many non-streaming events, streaming query listeners don't need to wait for other Spark listeners and can catch up.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19838 from zsxwing/SPARK-22638.
2017-12-01 13:02:03 -08:00
Thomas Graves dc365422bb [SPARK-22653] executorAddress registered in CoarseGrainedSchedulerBac…
https://issues.apache.org/jira/browse/SPARK-22653
executorRef.address can be null, pass the executorAddress which accounts for it being null a few lines above the fix.

Manually tested this patch. You can reproduce the issue by running a simple spark-shell in yarn client mode with dynamic allocation and request some executors up front. Let those executors idle timeout. Get a heap dump. Without this fix, you will see that addressToExecutorId still contains the ids, with the fix addressToExecutorId is properly cleaned up.

Author: Thomas Graves <tgraves@oath.com>

Closes #19850 from tgravescs/SPARK-22653.
2017-12-01 10:53:16 +08:00
Jakub Dubovsky ab6f60c4d6 [SPARK-22585][CORE] Path in addJar is not url encoded
## What changes were proposed in this pull request?

This updates a behavior of `addJar` method of `sparkContext` class. If path without any scheme is passed as input it is used literally without url encoding/decoding it.

## How was this patch tested?

A unit test is added for this.

Author: Jakub Dubovsky <jakub.dubovsky@seznam.cz>

Closes #19834 from james64/SPARK-22585-encode-add-jar.
2017-11-30 10:24:30 +09:00
Marcelo Vanzin 8ff474f6e5 [SPARK-20650][CORE] Remove JobProgressListener.
The only remaining use of this class was the SparkStatusTracker, which
was modified to use the new status store. The test code to wait for
executors was moved to TestUtils and now uses the SparkStatusTracker API.

Indirectly, ConsoleProgressBar also uses this data. Because it has
some lower latency requirements, a shortcut to efficiently get the
active stages from the active listener was added to the AppStateStore.

Now that all UI code goes through the status store to get its data,
the FsHistoryProvider can be cleaned up to only replay event logs
when needed - that is, when there is no pre-existing disk store for
the application.

As part of this change I also modified the streaming UI to read the needed
data from the store, which was missed in the previous patch that made
JobProgressListener redundant.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19750 from vanzin/SPARK-20650.
2017-11-29 14:34:41 -08:00
Yinan Li e9b2070ab2 [SPARK-18278][SCHEDULER] Spark on Kubernetes - Basic Scheduler Backend
## What changes were proposed in this pull request?

This is a stripped down version of the `KubernetesClusterSchedulerBackend` for Spark with the following components:
- Static Allocation of Executors
- Executor Pod Factory
- Executor Recovery Semantics

It's step 1 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
This addition is covered by the [SPIP vote](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) which passed on Aug 31 .

## How was this patch tested?

- The patch contains unit tests which are passing.
- Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
- It is a **subset** of the entire changelist hosted in http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
- There is integration testing enabled in the fork currently [hosted by PepperData](spark-k8s-jenkins.pepperdata.org:8080) which is being moved over to RiseLAB CI.
- Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah ash211 ssuchter varunkatta kimoonkim erikerlandson liyinan926 tnachen ifilonenko

Author: Yinan Li <liyinan926@gmail.com>
Author: foxish <ramanathana@google.com>
Author: mcheah <mcheah@palantir.com>

Closes #19468 from foxish/spark-kubernetes-3.
2017-11-28 23:02:09 -08:00
GuoChenzhao efd0036ec8 [SPARK-22537][CORE] Aggregation of map output statistics on driver faces single point bottleneck
## What changes were proposed in this pull request?

In adaptive execution, the map output statistics of all mappers will be aggregated after previous stage is successfully executed. Driver takes the aggregation job while it will get slow when the number of `mapper * shuffle partitions` is large, since it only uses single thread to compute. This PR uses multi-thread to deal with this single point bottleneck.

## How was this patch tested?

Test cases are in `MapOutputTrackerSuite.scala`

Author: GuoChenzhao <chenzhao.guo@intel.com>
Author: gczsjdy <gczsjdy1994@gmail.com>

Closes #19763 from gczsjdy/single_point_mapstatistics.
2017-11-24 15:09:43 +01:00
Wang Gengliang 449e26ecdc [SPARK-22559][CORE] history server: handle exception on opening corrupted listing.ldb
## What changes were proposed in this pull request?
Currently history server v2 failed to start if `listing.ldb` is corrupted.
This patch get rid of the corrupted `listing.ldb` and re-create it.
The exception handling follows [opening disk store for app](0ffa7c488f/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala (L307))
## How was this patch tested?
manual test

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #19786 from gengliangwang/listingException.
2017-11-24 15:07:43 +01:00
Marcelo Vanzin c13b60e019 [SPARK-22533][CORE] Handle deprecated names in ConfigEntry.
This change hooks up the config reader to `SparkConf.getDeprecatedConfig`,
so that config constants with deprecated names generate the proper warnings.
It also changes two deprecated configs from the new "alternatives" system to
the old deprecation system, since they're not yet hooked up to each other.

Added a few unit tests to verify the desired behavior.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19760 from vanzin/SPARK-22533.
2017-11-20 12:45:21 +01:00
Kazuaki Ishizaki 3c3eebc873 [SPARK-20101][SQL] Use OffHeapColumnVector when "spark.sql.columnVector.offheap.enable" is set to "true"
This PR enables to use ``OffHeapColumnVector`` when ``spark.sql.columnVector.offheap.enable`` is set to ``true``. While ``ColumnVector`` has two implementations ``OnHeapColumnVector`` and ``OffHeapColumnVector``, only ``OnHeapColumnVector`` is always used.

This PR implements the followings
- Pass ``OffHeapColumnVector`` to ``ColumnarBatch.allocate()`` when ``spark.sql.columnVector.offheap.enable`` is set to ``true``
- Free all of off-heap memory regions by ``OffHeapColumnVector.close()``
- Ensure to call ``OffHeapColumnVector.close()``

Use existing tests

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #17436 from kiszk/SPARK-20101.
2017-11-20 12:40:26 +01:00
hyukjinkwon 57c5514de9 [SPARK-22554][PYTHON] Add a config to control if PySpark should use daemon or not for workers
## What changes were proposed in this pull request?

This PR proposes to add a flag to control if PySpark should use daemon or not.

Actually, SparkR already has a flag for useDaemon:
478fbc866f/core/src/main/scala/org/apache/spark/api/r/RRunner.scala (L362)

It'd be great if we have this flag too. It makes easier to debug Windows specific issue.

## How was this patch tested?

Manually tested.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19782 from HyukjinKwon/use-daemon-flag.
2017-11-20 13:34:06 +09:00
Dongjoon Hyun b10837ab1a [SPARK-22557][TEST] Use ThreadSignaler explicitly
## What changes were proposed in this pull request?

ScalaTest 3.0 uses an implicit `Signaler`. This PR makes it sure all Spark tests uses `ThreadSignaler` explicitly which has the same default behavior of interrupting a thread on the JVM like ScalaTest 2.2.x. This will reduce potential flakiness.

## How was this patch tested?

This is testsuite-only update. This should passes the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19784 from dongjoon-hyun/use_thread_signaler.
2017-11-20 13:32:01 +09:00
Li Jin 7d039e0c0a [SPARK-22409] Introduce function type argument in pandas_udf
## What changes were proposed in this pull request?

* Add a "function type" argument to pandas_udf.
* Add a new public enum class `PandasUdfType` in pyspark.sql.functions
* Refactor udf related code from pyspark.sql.functions to pyspark.sql.udf
* Merge "PythonUdfType" and "PythonEvalType" into a single enum class "PythonEvalType"

Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType

pandas_udf('double', PandasUDFType.SCALAR):
def plus_one(v):
    return v + 1
```

## Design doc
https://docs.google.com/document/d/1KlLaa-xJ3oz28xlEJqXyCAHU3dwFYkFs_ixcUXrJNTc/edit

## How was this patch tested?

Added PandasUDFTests

## TODO:
* [x] Implement proper enum type for `PandasUDFType`
* [x] Update documentation
* [x] Add more tests in PandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19630 from icexelloss/spark-22409-pandas-udf-type.
2017-11-17 16:43:08 +01:00
yucai d00b55d4b2 [SPARK-22540][SQL] Ensure HighlyCompressedMapStatus calculates correct avgSize
## What changes were proposed in this pull request?

Ensure HighlyCompressedMapStatus calculates correct avgSize

## How was this patch tested?

New unit test added.

Author: yucai <yucai.yu@intel.com>

Closes #19765 from yucai/avgsize.
2017-11-17 07:53:53 -06:00
Shixiong Zhu 03f2b7bff7 [SPARK-22535][PYSPARK] Sleep before killing the python worker in PythonRunner.MonitorThread
## What changes were proposed in this pull request?

`PythonRunner.MonitorThread` should give the task a little time to finish before forcibly killing the python worker. This will reduce the chance of the race condition a lot. I also improved the log a bit to find out the task to blame when it's stuck.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19762 from zsxwing/SPARK-22535.
2017-11-16 14:22:25 +09:00
ArtRand 1e82335413 [SPARK-21842][MESOS] Support Kerberos ticket renewal and creation in Mesos
## What changes were proposed in this pull request?
tl;dr: Add a class, `MesosHadoopDelegationTokenManager` that updates delegation tokens on a schedule on the behalf of Spark Drivers. Broadcast renewed credentials to the executors.

## The problem
We recently added Kerberos support to Mesos-based Spark jobs as well as Secrets support to the Mesos Dispatcher (SPARK-16742, SPARK-20812, respectively). However the delegation tokens have a defined expiration. This poses a problem for long running Spark jobs (e.g. Spark Streaming applications). YARN has a solution for this where a thread is scheduled to renew the tokens they reach 75% of their way to expiration. It then writes the tokens to HDFS for the executors to find (uses a monotonically increasing suffix).

## This solution
We replace the current method in `CoarseGrainedSchedulerBackend` which used to discard the token renewal time with a protected method `fetchHadoopDelegationTokens`. Now the individual cluster backends are responsible for overriding this method to fetch and manage token renewal. The delegation tokens themselves, are still part of the `CoarseGrainedSchedulerBackend` as before.
In the case of Mesos renewed Credentials are broadcasted to the executors. This maintains all transfer of Credentials within Spark (as opposed to Spark-to-HDFS). It also does not require any writing of Credentials to disk. It also does not require any GC of old files.

## How was this patch tested?
Manually against a Kerberized HDFS cluster.

Thank you for the reviews.

Author: ArtRand <arand@soe.ucsc.edu>

Closes #19272 from ArtRand/spark-21842-450-kerberos-ticket-renewal.
2017-11-15 15:53:05 -08:00
osatici 2014e7a789 [SPARK-22479][SQL] Exclude credentials from SaveintoDataSourceCommand.simpleString
## What changes were proposed in this pull request?

Do not include jdbc properties which may contain credentials in logging a logical plan with `SaveIntoDataSourceCommand` in it.

## How was this patch tested?

building locally and trying to reproduce (per the steps in https://issues.apache.org/jira/browse/SPARK-22479):
```
== Parsed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Analyzed Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Optimized Logical Plan ==
SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
   +- Range (0, 100, step=1, splits=Some(8))

== Physical Plan ==
Execute SaveIntoDataSourceCommand
   +- SaveIntoDataSourceCommand org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider570127fa, Map(dbtable -> test20, driver -> org.postgresql.Driver, url -> *********(redacted), password -> *********(redacted)), ErrorIfExists
         +- Range (0, 100, step=1, splits=Some(8))
```

Author: osatici <osatici@palantir.com>

Closes #19708 from onursatici/os/redact-jdbc-creds.
2017-11-15 14:08:51 -08:00
Marcelo Vanzin 39b3f10dda [SPARK-20649][CORE] Simplify REST API resource structure.
With the new UI store, the API resource classes have a lot less code,
since there's no need for complicated translations between the UI
types and the API types. So the code ended up with a bunch of files
with a single method declared in them.

This change re-structures the API code so that it uses less classes;
mainly, most sub-resources were removed, and the code to deal with
single-attempt and multi-attempt apps was simplified.

The only change was the addition of a method to return a single
attempt's information; that was missing in the old API, so trying
to retrieve "/v1/applications/appId/attemptId" would result in a
404 even if the attempt existed (and URIs under that one would
return valid data).

The streaming API resources also overtook the same treatment, even
though the data is not stored in the new UI store.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19748 from vanzin/SPARK-20649.
2017-11-15 15:41:53 -06:00
Marcelo Vanzin 0ffa7c488f [SPARK-20652][SQL] Store SQL UI data in the new app status store.
This change replaces the SQLListener with a new implementation that
saves the data to the same store used by the SparkContext's status
store. For that, the types used by the old SQLListener had to be
updated a bit so that they're more serialization-friendly.

The interface for getting data from the store was abstracted into
a new class, SQLAppStatusStore (following the convention used in
core).

Another change is the way that the SQL UI hooks up into the core
UI or the SHS. The old "SparkHistoryListenerFactory" was replaced
with a new "AppStatePlugin" that more explicitly differentiates
between the two use cases: processing events, and showing the UI.
Both live apps and the SHS use this new API (previously, it was
restricted to the SHS).

Note on the above: this causes a slight change of behavior for
live apps; the SQL tab will only show up after the first execution
is started.

The metrics gathering code was re-worked a bit so that the types
used are less memory hungry and more serialization-friendly. This
reduces memory usage when using in-memory stores, and reduces load
times when using disk stores.

Tested with existing and added unit tests. Note one unit test was
disabled because it depends on SPARK-20653, which isn't in yet.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19681 from vanzin/SPARK-20652.
2017-11-14 15:28:22 -06:00
Marcelo Vanzin 4741c07809 [SPARK-20648][CORE] Port JobsTab and StageTab to the new UI backend.
This change is a little larger because there's a whole lot of logic
behind these pages, all really tied to internal types and listeners,
and some of that logic had to be implemented in the new listener and
the needed data exposed through the API types.

- Added missing StageData and ExecutorStageSummary fields which are
  used by the UI. Some json golden files needed to be updated to account
  for new fields.

- Save RDD graph data in the store. This tries to re-use existing types as
  much as possible, so that the code doesn't need to be re-written. So it's
  probably not very optimal.

- Some old classes (e.g. JobProgressListener) still remain, since they're used
  in other parts of the code; they're not used by the UI anymore, though, and
  will be cleaned up in a separate change.

- Save information about active pools in the store. This data is not really used
  in the SHS, but it's not a lot of data so it's still recorded when replaying
  applications.

- Because the new store sorts things slightly differently from the previous
  code, some json golden files had some elements within them shuffled around.

- The retention unit test in UISeleniumSuite was disabled because the code
  to throw away old stages / tasks hasn't been added yet.

- The job description field in the API tries to follow the old behavior, which
  makes it be empty most of the time, even though there's information to fill it
  in. For stages, a new field was added to hold the description (which is basically
  the job description), so that the UI can be rendered in the old way.

- A new stage status ("SKIPPED") was added to account for the fact that the API
  couldn't represent that state before. Without this, the stage would show up as
  "PENDING" in the UI, which is now based on API types.

- The API used to expose "executorRunTime" as the value of the task's duration,
  which wasn't really correct (also because that value was easily available
  from the metrics object); this change fixes that by storing the correct duration,
  which also means a few expectation files needed to be updated to account for
  the new durations and sorting differences due to the changed values.

- Added changes to implement SPARK-20713 and SPARK-21922 in the new code.

Tested with existing unit tests (and by using the UI a lot).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19698 from vanzin/SPARK-20648.
2017-11-14 10:34:32 -06:00
Xianyang Liu 176ae4d53e [MINOR][CORE] Using bufferedInputStream for dataDeserializeStream
## What changes were proposed in this pull request?

Small fix. Using bufferedInputStream for dataDeserializeStream.

## How was this patch tested?

Existing UT.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #19735 from ConeyLiu/smallfix.
2017-11-13 06:19:13 -06:00
Xianyang Liu 1c923d7d65 [SPARK-22450][CORE][MLLIB] safely register class for mllib
## What changes were proposed in this pull request?

There are still some algorithms based on mllib, such as KMeans. For now, many mllib common class (such as: Vector, DenseVector, SparseVector, Matrix, DenseMatrix, SparseMatrix) are not registered in Kryo. So there are some performance issues for those object serialization or deserialization.
Previously dicussed: https://github.com/apache/spark/pull/19586

## How was this patch tested?

New test case.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #19661 from ConeyLiu/register_vector.
2017-11-10 12:43:29 +01:00
Nathan Kronenfeld b57ed2245c [SPARK-22308][TEST-MAVEN] Support alternative unit testing styles in external applications
Continuation of PR#19528 (https://github.com/apache/spark/pull/19529#issuecomment-340252119)

The problem with the maven build in the previous PR was the new tests.... the creation of a spark session outside the tests meant there was more than one spark session around at a time.
I was using the spark session outside the tests so that the tests could share data; I've changed it so that each test creates the data anew.

Author: Nathan Kronenfeld <nicole.oresme@gmail.com>
Author: Nathan Kronenfeld <nkronenfeld@uncharted.software>

Closes #19705 from nkronenfeld/alternative-style-tests-2.
2017-11-09 19:11:30 -08:00
Srinivasa Reddy Vundela 9eb7096c47 [SPARK-22483][CORE] Exposing java.nio bufferedPool memory metrics to Metric System
## What changes were proposed in this pull request?

Adds java.nio bufferedPool memory metrics to metrics system which includes both direct and mapped memory.

## How was this patch tested?
Manually tested and checked direct and mapped memory metrics too available in metrics system using Console sink.

Here is the sample console output

application_1509655862825_0016.2.jvm.direct.capacity
             value = 19497
application_1509655862825_0016.2.jvm.direct.count
             value = 6
application_1509655862825_0016.2.jvm.direct.used
             value = 19498

application_1509655862825_0016.2.jvm.mapped.capacity
             value = 0
application_1509655862825_0016.2.jvm.mapped.count
             value = 0
application_1509655862825_0016.2.jvm.mapped.used
             value = 0

Author: Srinivasa Reddy Vundela <vsr@cloudera.com>

Closes #19709 from vundela/SPARK-22483.
2017-11-09 16:05:47 -08:00
Marcelo Vanzin 6ae12715c7 [SPARK-20647][CORE] Port StorageTab to the new UI backend.
This required adding information about StreamBlockId to the store,
which is not available yet via the API. So an internal type was added
until there's a need to expose that information in the API.

The UI only lists RDDs that have cached partitions, and that information
wasn't being correctly captured in the listener, so that's also fixed,
along with some minor (internal) API adjustments so that the UI can
get the correct data.

Because of the way partitions are cached, some optimizations w.r.t. how
often the data is flushed to the store could not be applied to this code;
because of that, some different ways to make the code more performant
were added to the data structures tracking RDD blocks, with the goal of
avoiding expensive copies when lots of blocks are being updated.

Tested with existing and updated unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19679 from vanzin/SPARK-20647.
2017-11-09 15:46:16 -06:00
Srinivasa Reddy Vundela 6b19c0735d [MINOR][CORE] Fix nits in MetricsSystemSuite
## What changes were proposed in this pull request?
Fixing nits in MetricsSystemSuite file
1) Using Sink instead of Source while casting
2) Using meaningful naming for variables, which reflect their usage

## How was this patch tested?
Ran the tests locally and all of them passing

Author: Srinivasa Reddy Vundela <vsr@cloudera.com>

Closes #19699 from vundela/master.
2017-11-09 09:53:41 -08:00
Sean Owen 51debf8b1f [SPARK-14540][BUILD] Support Scala 2.12 closures and Java 8 lambdas in ClosureCleaner (step 0)
## What changes were proposed in this pull request?

Preliminary changes to get ClosureCleaner to work with Scala 2.12. Makes many usages just work, but not all. This does _not_ resolve the JIRA.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19675 from srowen/SPARK-14540.0.
2017-11-08 10:24:40 +00:00
Marcelo Vanzin 11eea1a4ce [SPARK-20646][CORE] Port executors page to new UI backend.
The executors page is built on top of the REST API, so the page itself
was easy to hook up to the new code.

Some other pages depend on the `ExecutorListener` class that is being
removed, though, so they needed to be modified to use data from the
new store. Fortunately, all they seemed to need is the map of executor
logs, so that was somewhat easy too.

The executor timeline graph required adding some properties to the
ExecutorSummary API type. Instead of following the previous code,
which stored all the listener events in memory, the timeline is
now created based on the data available from the API.

I had to change some of the test golden files because the old code would
return executors in "random" order (since it used a mutable Map instead
of something that returns a sorted list), and the new code returns executors
in id order.

Tested with existing unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19678 from vanzin/SPARK-20646.
2017-11-07 23:14:29 -06:00
Marcelo Vanzin 7475a9655c [SPARK-20645][CORE] Port environment page to new UI backend.
This change modifies the status listener to collect the information
needed to render the envionment page, and populates that page and the
API with information collected by the listener.

Tested with existing and added unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19677 from vanzin/SPARK-20645.
2017-11-07 16:03:24 -06:00
Alexander Istomin 14a32a647a [SPARK-22330][CORE] Linear containsKey operation for serialized maps
…alization.

## What changes were proposed in this pull request?

Use non-linear containsKey operation for serialized maps, lookup into underlying map.

## How was this patch tested?

unit tests

Author: Alexander Istomin <istomin@rutarget.ru>

Closes #19553 from Whoosh/SPARK-22330.
2017-11-07 00:47:16 +01:00
Marcelo Vanzin c7f38e5adb [SPARK-20644][core] Initial ground work for kvstore UI backend.
There are two somewhat unrelated things going on in this patch, but
both are meant to make integration of individual UI pages later on
much easier.

The first part is some tweaking of the code in the listener so that
it does less updates of the kvstore for data that changes fast; for
example, it avoids writing changes down to the store for every
task-related event, since those can arrive very quickly at times.
Instead, for these kinds of events, it chooses to only flush things
if a certain interval has passed. The interval is based on how often
the current spark-shell code updates the progress bar for jobs, so
that users can get reasonably accurate data.

The code also delays as much as possible hitting the underlying kvstore
when replaying apps in the history server. This is to avoid unnecessary
writes to disk.

The second set of changes prepare the history server and SparkUI for
integrating with the kvstore. A new class, AppStatusStore, is used
for translating between the stored data and the types used in the
UI / API. The SHS now populates a kvstore with data loaded from
event logs when an application UI is requested.

Because this store can hold references to disk-based resources, the
code was modified to retrieve data from the store under a read lock.
This allows the SHS to detect when the store is still being used, and
only update it (e.g. because an updated event log was detected) when
there is no other thread using the store.

This change ended up creating a lot of churn in the ApplicationCache
code, which was cleaned up a lot in the process. I also removed some
metrics which don't make too much sense with the new code.

Tested with existing and added unit tests, and by making sure the SHS
still works on a real cluster.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19582 from vanzin/SPARK-20644.
2017-11-06 08:45:40 -06:00
Kazuaki Ishizaki bc1e101039 [SPARK-22254][CORE] Fix the arrayMax in BufferHolder
## What changes were proposed in this pull request?

This PR replaces the old the maximum array size (`Int.MaxValue`) with the new one (`ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH`).
This PR also refactor the code to calculate the new array size to easily understand why we have to use `newSize - 2` for allocating a new array.

## How was this patch tested?

Used the existing test

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #19650 from kiszk/SPARK-22254.
2017-11-03 23:35:57 -07:00
zhoukang 51145f1376 [SPARK-22407][WEB-UI] Add rdd id column on storage page to speed up navigating
## What changes were proposed in this pull request?

Add rdd id column on storage page to speed up navigating.
Example has attached on [SPARK-22407](https://issues.apache.org/jira/browse/SPARK-22407)
An example below:
![add-rddid](https://user-images.githubusercontent.com/26762018/32361127-da0758ac-c097-11e7-9f8c-0ea7ffb87e12.png)
![rdd-cache](https://user-images.githubusercontent.com/26762018/32361128-da3c1574-c097-11e7-8ab1-2def66466f33.png)
## How was this patch tested?

Current unit test and manually deploy an history server for testing

Author: zhoukang <zhoukang199191@gmail.com>

Closes #19625 from caneGuy/zhoukang/add-rddid.
2017-11-03 12:20:17 +00:00
Stavros Kontopoulos b2463fad71 [SPARK-22145][MESOS] fix supervise with checkpointing on mesos
## What changes were proposed in this pull request?

- Fixes the issue with the frameworkId being recovered by checkpointed data overwriting the one sent by the dipatcher.
- Keeps submission driver id as the only index for all data structures in the dispatcher.
Allocates a different task id per driver retry to satisfy the mesos requirements. Check the relevant ticket for the details on that.
## How was this patch tested?

Manually tested this with DC/OS 1.10. Launched a streaming job with checkpointing to hdfs, made the driver fail several times and observed behavior:
![image](https://user-images.githubusercontent.com/7945591/30940500-f7d2a744-a3e9-11e7-8c56-f2ccbb271e80.png)

![image](https://user-images.githubusercontent.com/7945591/30940550-19bc15de-a3ea-11e7-8a11-f48abfe36720.png)

![image](https://user-images.githubusercontent.com/7945591/30940524-083ea308-a3ea-11e7-83ae-00d3fa17b928.png)

![image](https://user-images.githubusercontent.com/7945591/30940579-2f0fb242-a3ea-11e7-82f9-86179da28b8c.png)

![image](https://user-images.githubusercontent.com/7945591/30940591-3b561b0e-a3ea-11e7-9dbd-e71912bb2ef3.png)

![image](https://user-images.githubusercontent.com/7945591/30940605-49c810ca-a3ea-11e7-8af5-67930851fd38.png)

![image](https://user-images.githubusercontent.com/7945591/30940631-59f4a288-a3ea-11e7-88cb-c3741b72bb13.png)

![image](https://user-images.githubusercontent.com/7945591/30940642-62346c9e-a3ea-11e7-8935-82e494925f67.png)

![image](https://user-images.githubusercontent.com/7945591/30940653-6c46d53c-a3ea-11e7-8dd1-5840d484d28c.png)

Author: Stavros Kontopoulos <st.kontopoulos@gmail.com>

Closes #19374 from skonto/fix_retry.
2017-11-02 13:25:48 +00:00
LucaCanali 1ffe03d9e8 [SPARK-22190][CORE] Add Spark executor task metrics to Dropwizard metrics
## What changes were proposed in this pull request?

This proposed patch is about making Spark executor task metrics available as Dropwizard metrics. This is intended to be of aid in monitoring Spark jobs and when drilling down on performance troubleshooting issues.

## How was this patch tested?

Manually tested on a Spark cluster (see JIRA for an example screenshot).

Author: LucaCanali <luca.canali@cern.ch>

Closes #19426 from LucaCanali/SparkTaskMetricsDropWizard.
2017-11-01 15:40:25 +01:00
Sital Kedia 444bce1c98 [SPARK-19112][CORE] Support for ZStandard codec
## What changes were proposed in this pull request?

Using zstd compression for Spark jobs spilling 100s of TBs of data, we could reduce the amount of data written to disk by as much as 50%. This translates to significant latency gain because of reduced disk io operations. There is a degradation CPU time by 2 - 5% because of zstd compression overhead, but for jobs which are bottlenecked by disk IO, this hit can be taken.

## Benchmark
Please note that this benchmark is using real world compute heavy production workload spilling TBs of data to disk

|         | zstd performance as compred to LZ4   |
| ------------- | -----:|
| spill/shuffle bytes    | -48% |
| cpu time    |    + 3% |
| cpu reservation time       |    -40%|
| latency     |     -40% |

## How was this patch tested?

Tested by running few jobs spilling large amount of data on the cluster and amount of intermediate data written to disk reduced by as much as 50%.

Author: Sital Kedia <skedia@fb.com>

Closes #18805 from sitalkedia/skedia/upstream_zstd.
2017-11-01 14:54:08 +01:00
Devaraj K 96798d14f0 [SPARK-22172][CORE] Worker hangs when the external shuffle service port is already in use
## What changes were proposed in this pull request?

Handling the NonFatal exceptions while starting the external shuffle service, if there are any NonFatal exceptions it logs and continues without the external shuffle service.

## How was this patch tested?

I verified it manually, it logs the exception and continues to serve without external shuffle service when BindException occurs.

Author: Devaraj K <devaraj@apache.org>

Closes #19396 from devaraj-kavali/SPARK-22172.
2017-11-01 18:07:39 +08:00
Zheng RuiFeng 556b5d2151 [SPARK-5484][FOLLOWUP] PeriodicRDDCheckpointer doc cleanup
## What changes were proposed in this pull request?
PeriodicRDDCheckpointer was already moved out of mllib in Spark-5484

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #19618 from zhengruifeng/checkpointer_doc.
2017-11-01 08:45:11 +00:00
Sital Kedia 7986cc09b1 [SPARK-11334][CORE] Fix bug in Executor allocation manager in running tasks calculation
## What changes were proposed in this pull request?

We often see the issue of Spark jobs stuck because the Executor Allocation Manager does not ask for any executor even if there are pending tasks in case dynamic allocation is turned on. Looking at the logic in Executor Allocation Manager, which calculates the running tasks, it can happen that the calculation will be wrong and the number of running tasks can become negative.

## How was this patch tested?

Added unit test

Author: Sital Kedia <skedia@fb.com>

Closes #19580 from sitalkedia/skedia/fix_stuck_job.
2017-10-31 09:49:58 -07:00