This lines up the HBase token logic with that done for Hive in SPARK-11265: reflection with only CFNE being swallowed.
There is a test, one which doesn't try to put HBase on the yarn/test class and really do the reflection (the way the hive introspection does). If people do want that then it could be added with careful POM work
+also: cut an incorrect comment from the Hive test case before copying it, and a couple of imports that may have been related to the hive test in the past.
Author: Steve Loughran <stevel@hortonworks.com>
Closes#10227 from steveloughran/stevel/patches/SPARK-12241-obtainTokenForHBase.
Because of AM failure, the target executor number between driver and AM will be different, which will lead to unexpected behavior in dynamic allocation. So when AM is re-registered with driver, state in `ExecutorAllocationManager` and `CoarseGrainedSchedulerBacked` should be reset.
This issue is originally addressed in #8737 , here re-opened again. Thanks a lot KaiXinXiaoLei for finding this issue.
andrewor14 and vanzin would you please help to review this, thanks a lot.
Author: jerryshao <sshao@hortonworks.com>
Closes#9963 from jerryshao/SPARK-10582.
Using Dynamic Allocation function, when a new AM is starting, and ExecutorAllocationManager send RequestExecutor message to AM. If the container allocator is not ready, the whole app will hang on
Author: meiyoula <1039320815@qq.com>
Closes#10138 from XuTingjun/patch-1.
`SynchronousQueue` cannot cache any task. This issue is similar to #9978. It's an easy fix. Just use the fixed `ThreadUtils.newDaemonCachedThreadPool`.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10108 from zsxwing/fix-threadpool.
This is purely the yarn/src/main and yarn/src/test bits of the YARN ATS integration: the extension model to load and run implementations of `SchedulerExtensionService` in the yarn cluster scheduler process —and to stop them afterwards.
There's duplication between the two schedulers, yarn-client and yarn-cluster, at least in terms of setting everything up, because the common superclass, `YarnSchedulerBackend` is in spark-core, and the extension services need the YARN app/attempt IDs.
If you look at how the the extension services are loaded, the case class `SchedulerExtensionServiceBinding` is used to pass in config info -currently just the spark context and the yarn IDs, of which one, the attemptID, will be null when running client-side. I'm passing in a case class to ensure that it would be possible in future to add extra arguments to the binding class, yet, as the method signature will not have changed, still be able to load existing services.
There's no functional extension service here, just one for testing. The real tests come in the bigger pull requests. At the same time, there's no restriction of this extension service purely to the ATS history publisher. Anything else that wants to listen to the spark context and publish events could use this, and I'd also consider writing one for the YARN-913 registry service, so that the URLs of the web UI would be locatable through that (low priority; would make more sense if integrated with a REST client).
There's no minicluster test. Given the test execution overhead of setting up minicluster tests, it'd probably be better to add an extension service into one of the existing tests.
Author: Steve Loughran <stevel@hortonworks.com>
Closes#9182 from steveloughran/stevel/feature/SPARK-1537-service.
Add label expression support for AM to restrict it runs on the specific set of nodes. I tested it locally and works fine.
sryza and vanzin please help to review, thanks a lot.
Author: jerryshao <sshao@hortonworks.com>
Closes#9800 from jerryshao/SPARK-7173.
Don't log ERROR messages when executors are explicitly killed or when
the exit reason is not yet known.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9780 from vanzin/SPARK-11789.
When we exceed the max memory tell users to increase both params instead of just the one.
Author: Holden Karau <holden@us.ibm.com>
Closes#9758 from holdenk/SPARK-11771-maximum-memory-in-yarn-is-controlled-by-two-params-have-both-in-error-msg.
Currently if dynamic allocation is enabled, explicitly killing executor will not get response, so the executor metadata is wrong in driver side. Which will make dynamic allocation on Yarn fail to work.
The problem is `disableExecutor` returns false for pending killing executors when `onDisconnect` is detected, so no further implementation is done.
One solution is to bypass these explicitly killed executors to use `super.onDisconnect` to remove executor. This is simple.
Another solution is still querying the loss reason for these explicitly kill executors. Since executor may get killed and informed in the same AM-RM communication, so current way of adding pending loss reason request is not worked (container complete is already processed), here we should store this loss reason for later query.
Here this PR chooses solution 2.
Please help to review. vanzin I think this part is changed by you previously, would you please help to review? Thanks a lot.
Author: jerryshao <sshao@hortonworks.com>
Closes#9684 from jerryshao/SPARK-11718.
See http://search-hadoop.com/m/q3RTtjpe8r1iRbTj2 for discussion.
Summary: addition of VisibleForTesting annotation resulted in spark-shell malfunctioning.
Author: tedyu <yuzhihong@gmail.com>
Closes#9585 from tedyu/master.
I tested the various options with both spark-submit and spark-class of specifying number of executors in both client and cluster mode where it applied.
--num-workers, --num-executors, spark.executor.instances, SPARK_EXECUTOR_INSTANCES, default nothing supplied
Author: Thomas Graves <tgraves@staydecay.corp.gq1.yahoo.com>
Closes#9523 from tgravescs/SPARK-11555.
In YARN mode, when preemption is enabled, we may leave executors in a
zombie state while we wait to retrieve the reason for which the executor
exited. This is so that we don't account for failed tasks that were
running on a preempted executor.
The issue is that while we wait for this information, the scheduler
might decide to schedule tasks on the executor, which will never be
able to run them. Other side effects include the block manager still
considering the executor available to cache blocks, for example.
So, when we know that an executor went down but we don't know why,
stop everything related to the executor, except its running tasks.
Only when we know the reason for the exit (or give up waiting for
it) we do update the running tasks.
This is achieved by a new `disableExecutor()` method in the
`Schedulable` interface. For managers that do not behave like this
(i.e. every one but YARN), the existing `executorLost()` method
will behave the same way it did before.
On top of that change, a few minor changes that made debugging easier,
and fixed some other minor issues:
- The cluster-mode AM was printing a misleading log message every
time an executor disconnected from the driver (because the akka
actor system was shared between driver and AM).
- Avoid sending unnecessary requests for an executor's exit reason
when we already know it was explicitly disabled / killed. This
avoids both multiple requests, and unnecessary requests that would
just cause warning messages on the AM (in the explicit kill case).
- Tone down a log message about the executor being lost when it
exited normally (e.g. preemption)
- Wake up the AM monitor thread when requests for executor loss
reasons arrive too, so that we can more quickly remove executors
from this zombie state.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#8887 from vanzin/SPARK-10622.
"Client mode" means the RPC env will not listen for incoming connections.
This allows certain processes in the Spark stack (such as Executors or
tha YARN client-mode AM) to act as pure clients when using the netty-based
RPC backend, reducing the number of sockets needed by the app and also the
number of open ports.
Client connections are also preferred when endpoints that actually have
a listening socket are involved; so, for example, if a Worker connects
to a Master and the Master needs to send a message to a Worker endpoint,
that client connection will be used, even though the Worker is also
listening for incoming connections.
With this change, the workaround for SPARK-10987 isn't necessary anymore, and
is removed. The AM connects to the driver in "client mode", and that connection
is used for all driver <-> AM communication, and so the AM is properly notified
when the connection goes down.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9210 from vanzin/SPARK-10997.
This is a follow-up PR to further improve the locality calculation by considering the pending container's request. Since the locality preferences of tasks may be shifted from time to time, current localities of pending container requests may not fully match the new preferences, this PR improve it by removing outdated, unmatched container requests and replace with new requests.
sryza please help to review, thanks a lot.
Author: jerryshao <sshao@hortonworks.com>
Closes#8100 from jerryshao/SPARK-9817.
Use standard JDK APIs for that (with a little help from Guava). Most of the
changes here are in test code, since there were no tests specific to that
part of the code.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9257 from vanzin/SPARK-11073.
This is a fix for SPARK-11265; the introspection code to get Hive delegation tokens failing on Spark 1.5.1+, due to changes in the Hive codebase
Author: Steve Loughran <stevel@hortonworks.com>
Closes#9232 from steveloughran/stevel/patches/SPARK-11265-hive-tokens.
Commit af3bc59d1f introduced new
functionality so that if an executor dies for a reason that's not
caused by one of the tasks running on the executor (e.g., due to
pre-emption), Spark doesn't count the failure towards the maximum
number of failures for the task. That commit introduced some vague
naming that this commit attempts to fix; in particular:
(1) The variable "isNormalExit", which was used to refer to cases where
the executor died for a reason unrelated to the tasks running on the
machine, has been renamed (and reversed) to "exitCausedByApp". The problem
with the existing name is that it's not clear (at least to me!) what it
means for an exit to be "normal"; the new name is intended to make the
purpose of this variable more clear.
(2) The variable "shouldEventuallyFailJob" has been renamed to
"countTowardsTaskFailures". This variable is used to determine whether
a task's failure should be counted towards the maximum number of failures
allowed for a task before the associated Stage is aborted. The problem
with the existing name is that it can be confused with implying that
the task's failure should immediately cause the stage to fail because it
is somehow fatal (this is the case for a fetch failure, for example: if
a task fails because of a fetch failure, there's no point in retrying,
and the whole stage should be failed).
Author: Kay Ousterhout <kayousterhout@gmail.com>
Closes#9164 from kayousterhout/SPARK-11178.
Currently log4j.properties file is not uploaded to executor's which is leading them to use the default values. This fix will make sure that file is always uploaded to distributed cache so that executor will use the latest settings.
If user specifies log configurations through --files then executors will be picking configs from --files instead of $SPARK_CONF_DIR/log4j.properties
Author: vundela <vsr@cloudera.com>
Author: Srinivasa Reddy Vundela <vsr@cloudera.com>
Closes#9118 from vundela/master.
I also added some information to container-failure error msgs about what host they failed on, which would have helped me identify the problem that lead me to this JIRA and PR sooner.
Author: Ryan Williams <ryan.blake.williams@gmail.com>
Closes#9147 from ryan-williams/dyn-exec-failures.
when spark.yarn.user.classpath.first=true and using 'spark-submit --jars hdfs://user/foo.jar', it can not put foo.jar to system classpath. so we need to put yarn's linkNames of jars to the system classpath. vanzin tgravescs
Author: Lianhui Wang <lianhuiwang09@gmail.com>
Closes#9045 from lianhuiwang/spark-11026.
Add application attempt window for Spark on Yarn to ignore old out of window failures, this is useful for long running applications to recover from failures.
Author: jerryshao <sshao@hortonworks.com>
Closes#8857 from jerryshao/SPARK-10739 and squashes the following commits:
36eabdc [jerryshao] change the doc
7f9b77d [jerryshao] Style change
1c9afd0 [jerryshao] Address the comments
caca695 [jerryshao] Add application attempt window for Spark on Yarn
The issue is that local paths on Windows, when provided with drive
letters or backslashes, are not valid URIs.
Instead of trying to figure out whether paths are URIs or not, use
Utils.resolveURI() which does that for us.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9049 from vanzin/SPARK-11023 and squashes the following commits:
77021f2 [Marcelo Vanzin] [SPARK-11023] [yarn] Avoid creating URIs from local paths directly.
This change adds an API that encapsulates information about an app
launched using the library. It also creates a socket-based communication
layer for apps that are launched as child processes; the launching
application listens for connections from launched apps, and once
communication is established, the channel can be used to send updates
to the launching app, or to send commands to the child app.
The change also includes hooks for local, standalone/client and yarn
masters.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#7052 from vanzin/SPARK-8673.
In YARN client mode, when the AM connects to the driver, it may be the case
that the driver never needs to send a message back to the AM (i.e., no
dynamic allocation or preemption). This triggers an issue in the netty rpc
backend where no disconnection event is sent to endpoints, and the AM never
exits after the driver shuts down.
The real fix is too complicated, so this is a quick hack to unblock YARN
client mode until we can work on the real fix. It forces the driver to
send a message to the AM when the AM registers, thus establishing that
connection and enabling the disconnection event when the driver goes
away.
Also, a minor side issue: when the executor is shutting down, it needs
to send an "ack" back to the driver when using the netty rpc backend; but
that "ack" wasn't being sent because the handler was shutting down the rpc
env before returning. So added a change to delay the shutdown a little bit,
allowing the ack to be sent back.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9021 from vanzin/SPARK-10987.
The `self` method returns null when called from the constructor;
instead, registration should happen in the `onStart` method, at
which point the `self` reference has already been initialized.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#9005 from vanzin/SPARK-10964.
A recent change to fix the referenced bug caused this exception in
the `SparkContext.stop()` path:
org.apache.spark.SparkException: YarnSparkHadoopUtil is not available in non-YARN mode!
at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$.get(YarnSparkHadoopUtil.scala:167)
at org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.stop(YarnClientSchedulerBackend.scala:182)
at org.apache.spark.scheduler.TaskSchedulerImpl.stop(TaskSchedulerImpl.scala:440)
at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1579)
at org.apache.spark.SparkContext$$anonfun$stop$7.apply$mcV$sp(SparkContext.scala:1730)
at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1185)
at org.apache.spark.SparkContext.stop(SparkContext.scala:1729)
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#8996 from vanzin/SPARK-10812.
This should go into 1.5.2 also.
The issue is we were no longer adding the __app__.jar to the system classpath.
Author: Thomas Graves <tgraves@staydecay.corp.gq1.yahoo.com>
Author: Tom Graves <tgraves@yahoo-inc.com>
Closes#8959 from tgravescs/SPARK-10901.
This makes YARN containers behave like all other processes launched by
Spark, which launch with a default perm gen size of 256m unless
overridden by the user (or not needed by the vm).
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#8970 from vanzin/SPARK-10916.
This PR just reverted 02144d6745 to remerge #6457 and also included the commits in #8905.
Author: zsxwing <zsxwing@gmail.com>
Closes#8944 from zsxwing/SPARK-6028.
This bug is introduced in [SPARK-9092](https://issues.apache.org/jira/browse/SPARK-9092), `targetExecutorNumber` should use `minExecutors` if `initialExecutors` is not set. Using 0 instead will meet the problem as mentioned in [SPARK-10790](https://issues.apache.org/jira/browse/SPARK-10790).
Also consolidate and simplify some similar code snippets to keep the consistent semantics.
Author: jerryshao <sshao@hortonworks.com>
Closes#8910 from jerryshao/SPARK-10790.
While this is likely not a huge issue for real production systems, for test systems which may setup a Spark Context and tear it down and stand up a Spark Context with a different master (e.g. some local mode & some yarn mode) tests this cane be an issue. Discovered during work on spark-testing-base on Spark 1.4.1, but seems like the logic that triggers it is present in master (see SparkHadoopUtil object). A valid work around for users encountering this issue is to fork a different JVM, however this can be heavy weight.
```
[info] SampleMiniClusterTest:
[info] Exception encountered when attempting to run a suite with class name: com.holdenkarau.spark.testing.SampleMiniClusterTest *** ABORTED ***
[info] java.lang.ClassCastException: org.apache.spark.deploy.SparkHadoopUtil cannot be cast to org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
[info] at org.apache.spark.deploy.yarn.YarnSparkHadoopUtil$.get(YarnSparkHadoopUtil.scala:163)
[info] at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:257)
[info] at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:561)
[info] at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:115)
[info] at org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.start(YarnClientSchedulerBackend.scala:57)
[info] at org.apache.spark.scheduler.TaskSchedulerImpl.start(TaskSchedulerImpl.scala:141)
[info] at org.apache.spark.SparkContext.<init>(SparkContext.scala:497)
[info] at com.holdenkarau.spark.testing.SharedMiniCluster$class.setup(SharedMiniCluster.scala:186)
[info] at com.holdenkarau.spark.testing.SampleMiniClusterTest.setup(SampleMiniClusterTest.scala:26)
[info] at com.holdenkarau.spark.testing.SharedMiniCluster$class.beforeAll(SharedMiniCluster.scala:103)
```
Author: Holden Karau <holden@pigscanfly.ca>
Closes#8911 from holdenk/SPARK-10812-spark-hadoop-util-support-switching-to-yarn.
This change does two things:
- tag a few tests and adds the mechanism in the build to be able to disable those tags,
both in maven and sbt, for both junit and scalatest suites.
- add some logic to run-tests.py to disable some tags depending on what files have
changed; that's used to disable expensive tests when a module hasn't explicitly
been changed, to speed up testing for changes that don't directly affect those
modules.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#8437 from vanzin/test-tags.
`ApplicationMaster` no longer has the `--num-executors` flag, and had an undocumented `--properties-file` configuration option.
cc srowen
Author: Erick Tryzelaar <erick.tryzelaar@gmail.com>
Closes#8754 from erickt/master.
The architecture is that, in YARN mode, if the driver detects that an executor has disconnected, it asks the ApplicationMaster why the executor died. If the ApplicationMaster is aware that the executor died because of preemption, all tasks associated with that executor are not marked as failed. The executor
is still removed from the driver's list of available executors, however.
There's a few open questions:
1. Should standalone mode have a similar "get executor loss reason" as well? I localized this change as much as possible to affect only YARN, but there could be a valid case to differentiate executor losses in standalone mode as well.
2. I make a pretty strong assumption in YarnAllocator that getExecutorLossReason(executorId) will only be called once per executor id; I do this so that I can remove the metadata from the in-memory map to avoid object accumulation. It's not clear if I'm being overly zealous to save space, however.
cc vanzin specifically for review because it collided with some earlier YARN scheduling work.
cc JoshRosen because it's similar to output commit coordination we did in the past
cc andrewor14 for our discussion on how to get executor exit codes and loss reasons
Author: mcheah <mcheah@palantir.com>
Closes#8007 from mccheah/feature/preemption-handling.
From Jira:
Running spark-submit with yarn with number-executors equal to 0 when not using dynamic allocation should error out.
In spark 1.5.0 it continues and ends up hanging.
yarn.ClientArguments still has the check so something else must have changed.
spark-submit --master yarn --deploy-mode cluster --class org.apache.spark.examples.SparkPi --num-executors 0 ....
spark 1.4.1 errors with:
java.lang.IllegalArgumentException:
Number of executors was 0, but must be at least 1
(or 0 if dynamic executor allocation is enabled).
Author: Holden Karau <holden@pigscanfly.ca>
Closes#8580 from holdenk/SPARK-10332-spark-submit-to-yarn-executors-0-message.
This change aims at speeding up the dev cycle a little bit, by making
sure that all tests behave the same w.r.t. where the code to be tested
is loaded from. Namely, that means that tests don't rely on the assembly
anymore, rather loading all needed classes from the build directories.
The main change is to make sure all build directories (classes and test-classes)
are added to the classpath of child processes when running tests.
YarnClusterSuite required some custom code since the executors are run
differently (i.e. not through the launcher library, like standalone and
Mesos do).
I also found a couple of tests that could leak a SparkContext on failure,
and added code to handle those.
With this patch, it's possible to run the following command from a clean
source directory and have all tests pass:
mvn -Pyarn -Phadoop-2.4 -Phive-thriftserver install
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#7629 from vanzin/SPARK-9284.
Current port number is fixed as default (7337) in test, this will introduce port contention exception, better to change to a random number in unit test.
squito , seems you're author of this unit test, mind taking a look at this fix? Thanks a lot.
```
[info] - executor state kept across NM restart *** FAILED *** (597 milliseconds)
[info] org.apache.hadoop.service.ServiceStateException: java.net.BindException: Address already in use
[info] at org.apache.hadoop.service.ServiceStateException.convert(ServiceStateException.java:59)
[info] at org.apache.hadoop.service.AbstractService.init(AbstractService.java:172)
[info] at org.apache.spark.network.yarn.YarnShuffleServiceSuite$$anonfun$1.apply$mcV$sp(YarnShuffleServiceSuite.scala:72)
[info] at org.apache.spark.network.yarn.YarnShuffleServiceSuite$$anonfun$1.apply(YarnShuffleServiceSuite.scala:70)
[info] at org.apache.spark.network.yarn.YarnShuffleServiceSuite$$anonfun$1.apply(YarnShuffleServiceSuite.scala:70)
[info] at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
[info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
[info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info] at org.scalatest.Transformer.apply(Transformer.scala:22)
[info] at org.scalatest.Transformer.apply(Transformer.scala:20)
[info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
[info] at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42)
...
```
Author: jerryshao <sshao@hortonworks.com>
Closes#8502 from jerryshao/avoid-hardcode-port.
Replace `JavaConversions` implicits with `JavaConverters`
Most occurrences I've seen so far are necessary conversions; a few have been avoidable. None are in critical code as far as I see, yet.
Author: Sean Owen <sowen@cloudera.com>
Closes#8033 from srowen/SPARK-9613.
https://issues.apache.org/jira/browse/SPARK-9439
In general, Yarn apps should be robust to NodeManager restarts. However, if you run spark with the external shuffle service on, after a NM restart all shuffles fail, b/c the shuffle service has lost some state with info on each executor. (Note the shuffle data is perfectly fine on disk across a NM restart, the problem is we've lost the small bit of state that lets us *find* those files.)
The solution proposed here is that the external shuffle service can write out its state to leveldb (backed by a local file) every time an executor is added. When running with yarn, that file is in the NM's local dir. Whenever the service is started, it looks for that file, and if it exists, it reads the file and re-registers all executors there.
Nothing is changed in non-yarn modes with this patch. The service is not given a place to save the state to, so it operates the same as before. This should make it easy to update other cluster managers as well, by just supplying the right file & the equivalent of yarn's `initializeApplication` -- I'm not familiar enough with those modes to know how to do that.
Author: Imran Rashid <irashid@cloudera.com>
Closes#7943 from squito/leveldb_external_shuffle_service_NM_restart and squashes the following commits:
0d285d3 [Imran Rashid] review feedback
70951d6 [Imran Rashid] Merge branch 'master' into leveldb_external_shuffle_service_NM_restart
5c71c8c [Imran Rashid] save executor to db before registering; style
2499c8c [Imran Rashid] explicit dependency on jackson-annotations
795d28f [Imran Rashid] review feedback
81f80e2 [Imran Rashid] Merge branch 'master' into leveldb_external_shuffle_service_NM_restart
594d520 [Imran Rashid] use json to serialize application executor info
1a7980b [Imran Rashid] version
8267d2a [Imran Rashid] style
e9f99e8 [Imran Rashid] cleanup the handling of bad dbs a little
9378ba3 [Imran Rashid] fail gracefully on corrupt leveldb files
acedb62 [Imran Rashid] switch to writing out one record per executor
79922b7 [Imran Rashid] rely on yarn to call stopApplication; assorted cleanup
12b6a35 [Imran Rashid] save registered executors when apps are removed; add tests
c878fbe [Imran Rashid] better explanation of shuffle service port handling
694934c [Imran Rashid] only open leveldb connection once per service
d596410 [Imran Rashid] store executor data in leveldb
59800b7 [Imran Rashid] Files.move in case renaming is unsupported
32fe5ae [Imran Rashid] Merge branch 'master' into external_shuffle_service_NM_restart
d7450f0 [Imran Rashid] style
f729e2b [Imran Rashid] debugging
4492835 [Imran Rashid] lol, dont use a PrintWriter b/c of scalastyle checks
0a39b98 [Imran Rashid] Merge branch 'master' into external_shuffle_service_NM_restart
55f49fc [Imran Rashid] make sure the service doesnt die if the registered executor file is corrupt; add tests
245db19 [Imran Rashid] style
62586a6 [Imran Rashid] just serialize the whole executors map
bdbbf0d [Imran Rashid] comments, remove some unnecessary changes
857331a [Imran Rashid] better tests & comments
bb9d1e6 [Imran Rashid] formatting
bdc4b32 [Imran Rashid] rename
86e0cb9 [Imran Rashid] for tests, shuffle service finds an open port
23994ff [Imran Rashid] style
7504de8 [Imran Rashid] style
a36729c [Imran Rashid] cleanup
efb6195 [Imran Rashid] proper unit test, and no longer leak if apps stop during NM restart
dd93dc0 [Imran Rashid] test for shuffle service w/ NM restarts
d596969 [Imran Rashid] cleanup imports
0e9d69b [Imran Rashid] better names
9eae119 [Imran Rashid] cleanup lots of duplication
1136f44 [Imran Rashid] test needs to have an actual shuffle
0b588bd [Imran Rashid] more fixes ...
ad122ef [Imran Rashid] more fixes
5e5a7c3 [Imran Rashid] fix build
c69f46b [Imran Rashid] maybe working version, needs tests & cleanup ...
bb3ba49 [Imran Rashid] minor cleanup
36127d3 [Imran Rashid] wip
b9d2ced [Imran Rashid] incomplete setup for external shuffle service tests