Commit graph

359 commits

Author SHA1 Message Date
WangTaoTheTonic b65bad65c3 [SPARK-3591][YARN]fire and forget for YARN cluster mode
https://issues.apache.org/jira/browse/SPARK-3591

The output after this patch:
>doggie153:/opt/oss/spark-1.3.0-bin-hadoop2.4/bin # ./spark-submit  --class org.apache.spark.examples.SparkPi --master yarn-cluster ../lib/spark-examples*.jar
15/03/31 21:15:25 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
15/03/31 21:15:25 INFO RMProxy: Connecting to ResourceManager at doggie153/10.177.112.153:8032
15/03/31 21:15:25 INFO Client: Requesting a new application from cluster with 4 NodeManagers
15/03/31 21:15:25 INFO Client: Verifying our application has not requested more than the maximum memory capability of the cluster (8192 MB per container)
15/03/31 21:15:25 INFO Client: Will allocate AM container, with 896 MB memory including 384 MB overhead
15/03/31 21:15:25 INFO Client: Setting up container launch context for our AM
15/03/31 21:15:25 INFO Client: Preparing resources for our AM container
15/03/31 21:15:26 INFO Client: Uploading resource file:/opt/oss/spark-1.3.0-bin-hadoop2.4/lib/spark-assembly-1.4.0-SNAPSHOT-hadoop2.4.1.jar -> hdfs://doggie153:9000/user/root/.sparkStaging/application_1427257505534_0016/spark-assembly-1.4.0-SNAPSHOT-hadoop2.4.1.jar
15/03/31 21:15:27 INFO Client: Uploading resource file:/opt/oss/spark-1.3.0-bin-hadoop2.4/lib/spark-examples-1.3.0-hadoop2.4.0.jar -> hdfs://doggie153:9000/user/root/.sparkStaging/application_1427257505534_0016/spark-examples-1.3.0-hadoop2.4.0.jar
15/03/31 21:15:28 INFO Client: Setting up the launch environment for our AM container
15/03/31 21:15:28 INFO SecurityManager: Changing view acls to: root
15/03/31 21:15:28 INFO SecurityManager: Changing modify acls to: root
15/03/31 21:15:28 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(root); users with modify permissions: Set(root)
15/03/31 21:15:28 INFO Client: Submitting application 16 to ResourceManager
15/03/31 21:15:28 INFO YarnClientImpl: Submitted application application_1427257505534_0016
15/03/31 21:15:28 INFO Client: ... waiting before polling ResourceManager for application state
15/03/31 21:15:33 INFO Client: ... polling ResourceManager for application state
15/03/31 21:15:33 INFO Client: Application report for application_1427257505534_0016 (state: RUNNING)
15/03/31 21:15:33 INFO Client:
         client token: N/A
         diagnostics: N/A
         ApplicationMaster host: doggie157
         ApplicationMaster RPC port: 0
         queue: default
         start time: 1427807728307
         final status: UNDEFINED
         tracking URL: http://doggie153:8088/proxy/application_1427257505534_0016/
         user: root

/cc andrewor14

Author: WangTaoTheTonic <wangtao111@huawei.com>

Closes #5297 from WangTaoTheTonic/SPARK-3591 and squashes the following commits:

c76d232 [WangTaoTheTonic] wrap lines
16c90a8 [WangTaoTheTonic] move up lines to avoid duplicate
fea390d [WangTaoTheTonic] log failed/killed report, style and comment
be1cc2e [WangTaoTheTonic] reword
f0bc54f [WangTaoTheTonic] minor: expose appid in excepiton messages
ba9b22b [WangTaoTheTonic] wrong config name
e1a4013 [WangTaoTheTonic] revert to the old version and do some robust
19706c0 [WangTaoTheTonic] add a config to control whether to forget
0cbdce8 [WangTaoTheTonic] fire and forget for YARN cluster mode
2015-04-07 08:36:25 -05:00
zsxwing f15806a8f8 [SPARK-6602][Core] Replace direct use of Akka with Spark RPC interface - part 1
This PR replaced the following `Actor`s to `RpcEndpoint`:

1. HeartbeatReceiver
1. ExecutorActor
1. BlockManagerMasterActor
1. BlockManagerSlaveActor
1. CoarseGrainedExecutorBackend and subclasses
1. CoarseGrainedSchedulerBackend.DriverActor

This is the first PR. I will split the work of SPARK-6602 to several PRs for code review.

Author: zsxwing <zsxwing@gmail.com>

Closes #5268 from zsxwing/rpc-rewrite and squashes the following commits:

287e9f8 [zsxwing] Fix the code style
26c56b7 [zsxwing] Merge branch 'master' into rpc-rewrite
9cc825a [zsxwing] Rmove setupThreadSafeEndpoint and add ThreadSafeRpcEndpoint
30a9036 [zsxwing] Make self return null after stopping RpcEndpointRef; fix docs and error messages
705245d [zsxwing] Fix some bugs after rebasing the changes on the master
003cf80 [zsxwing] Update CoarseGrainedExecutorBackend and CoarseGrainedSchedulerBackend to use RpcEndpoint
7d0e6dc [zsxwing] Update BlockManagerSlaveActor to use RpcEndpoint
f5d6543 [zsxwing] Update BlockManagerMaster to use RpcEndpoint
30e3f9f [zsxwing] Update ExecutorActor to use RpcEndpoint
478b443 [zsxwing] Update HeartbeatReceiver to use RpcEndpoint
2015-04-04 11:52:05 -07:00
Davies Liu 9b40c17ab1 [SPARK-6700] disable flaky test
Author: Davies Liu <davies@databricks.com>

Closes #5356 from davies/flaky and squashes the following commits:

08955f4 [Davies Liu] disable flaky test
2015-04-03 15:22:21 -07:00
Reynold Xin 82701ee25f [SPARK-6428] Turn on explicit type checking for public methods.
This builds on my earlier pull requests and turns on the explicit type checking in scalastyle.

Author: Reynold Xin <rxin@databricks.com>

Closes #5342 from rxin/SPARK-6428 and squashes the following commits:

7b531ab [Reynold Xin] import ordering
2d9a8a5 [Reynold Xin] jl
e668b1c [Reynold Xin] override
9b9e119 [Reynold Xin] Parenthesis.
82e0cf5 [Reynold Xin] [SPARK-6428] Turn on explicit type checking for public methods.
2015-04-03 01:25:02 -07:00
Marcelo Vanzin a74564591f [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5056 from vanzin/SPARK-6371 and squashes the following commits:

63220df [Marcelo Vanzin] Merge branch 'master' into SPARK-6371
6506f75 [Marcelo Vanzin] Use more fine-grained exclusion.
178ba71 [Marcelo Vanzin] Oops.
75b2375 [Marcelo Vanzin] Exclude VertexRDD in MiMA.
a45a62c [Marcelo Vanzin] Work around MIMA warning.
1d8a670 [Marcelo Vanzin] Re-group jetty exclusion.
0e8e909 [Marcelo Vanzin] Ignore ml, don't ignore graphx.
cef4603 [Marcelo Vanzin] Indentation.
296cf82 [Marcelo Vanzin] [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.
2015-03-20 18:43:57 +00:00
Sean Owen 6f80c3e888 SPARK-6338 [CORE] Use standard temp dir mechanisms in tests to avoid orphaned temp files
Use `Utils.createTempDir()` to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify

Author: Sean Owen <sowen@cloudera.com>

Closes #5029 from srowen/SPARK-6338 and squashes the following commits:

27b740a [Sean Owen] Fix hive-thriftserver tests that don't expect an existing dir
4a212fa [Sean Owen] Standardize a bit more temp dir management
9004081 [Sean Owen] Revert some added recursive-delete calls
57609e4 [Sean Owen] Use Utils.createTempDir() to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify
2015-03-20 14:16:21 +00:00
Marcelo Vanzin 981fbafa2a [SPARK-6325] [core,yarn] Do not change target executor count when killing executors.
The dynamic execution code has two ways to reduce the number of executors: one
where it reduces the total number of executors it wants, by asking for an absolute
number of executors that is lower than the previous one. The second is by
explicitly killing idle executors.

YarnAllocator was mixing those up and lowering the target number of executors
when a kill was issued. Instead, trust the frontend knows what it's doing, and kill
executors without messing with other accounting. That means that if the frontend
kills an executor without lowering the target, it will get a new executor shortly.

The one situation where both actions (lower the target and kill executor) need to
happen together is when user code explicitly calls `SparkContext.killExecutors`.
In that case, issue two calls to the backend to achieve the goal.

I also did some minor cleanup in related code:
- avoid sending a request for executors when target is unchanged, to avoid log
  spam in the AM
- avoid printing misleading log messages in the AM when there are no requests
  to cancel
- fix a slow memory leak plus misleading error message on the driver caused by
  failing to completely unregister the executor.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5018 from vanzin/SPARK-6325 and squashes the following commits:

2e782a3 [Marcelo Vanzin] Avoid redundant logging on the AM side.
a3567cd [Marcelo Vanzin] Add parentheses.
a363926 [Marcelo Vanzin] Update logic.
a158101 [Marcelo Vanzin] [SPARK-6325] [core,yarn] Disallow reducing executor count past running count.
2015-03-18 09:18:28 -04:00
Steve Loughran e09c852d6b SPARK-6389 YARN app diagnostics report doesn't report NPEs
Trivial patch to implicitly call `Exception.toString()` over `Exception.getMessage()` —this defaults to including the exception class & any non-null message; some subclasses include more.

No test.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #5070 from steveloughran/stevel/patches/SPARK-6389-NPE-reporting and squashes the following commits:

8239d85 [Steve Loughran] SPARK-6389 cull use of getMessage over toString in the container launcher
6fbaf6a [Steve Loughran] SPARK-6389 YARN app diagnostics report doesn't report NPEs
2015-03-18 09:09:32 -04:00
Sean Owen c9cfba0ceb SPARK-6182 [BUILD] spark-parent pom needs to be published for both 2.10 and 2.11
Option 1 of 2: Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11

Author: Sean Owen <sowen@cloudera.com>

Closes #4912 from srowen/SPARK-6182.1 and squashes the following commits:

eff60de [Sean Owen] Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11
2015-03-05 11:31:48 -08:00
tedyu 8d3e2414d4 SPARK-6085 Increase default value for memory overhead
Author: tedyu <yuzhihong@gmail.com>

Closes #4836 from tedyu/master and squashes the following commits:

d65b495 [tedyu] SPARK-6085 Increase default value for memory overhead
1fdd4df [tedyu] SPARK-6085 Increase default value for memory overhead
2015-03-04 11:00:52 +00:00
Andrew Or 258d154c9f [SPARK-6048] SparkConf should not translate deprecated configs on set
There are multiple issues with translating on set outlined in the JIRA.

This PR reverts the translation logic added to `SparkConf`. In the future, after the 1.3.0 release we will figure out a way to reorganize the internal structure more elegantly. For now, let's preserve the existing semantics of `SparkConf` since it's a public interface. Unfortunately this means duplicating some code for now, but this is all internal and we can always clean it up later.

Author: Andrew Or <andrew@databricks.com>

Closes #4799 from andrewor14/conf-set-translate and squashes the following commits:

11c525b [Andrew Or] Move warning to driver
10e77b5 [Andrew Or] Add documentation for deprecation precedence
a369cb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into conf-set-translate
c26a9e3 [Andrew Or] Revert all translate logic in SparkConf
fef6c9c [Andrew Or] Restore deprecation logic for spark.executor.userClassPathFirst
94b4dfa [Andrew Or] Translate on get, not set
2015-03-02 16:36:42 -08:00
Marcelo Vanzin 6b348d90f4 [SPARK-6050] [yarn] Relax matching of vcore count in received containers.
Some YARN configurations return a vcore count for allocated
containers that does not match the requested resource. That means
Spark would always ignore those containers. So relax the the matching
of the vcore count to allow the Spark jobs to run.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #4818 from vanzin/SPARK-6050 and squashes the following commits:

991c803 [Marcelo Vanzin] Remove config option, standardize on legacy behavior (no vcore matching).
8c9c346 [Marcelo Vanzin] Restrict lax matching to vcores only.
3359692 [Marcelo Vanzin] [SPARK-6050] [yarn] Add config option to do lax resource matching.
2015-03-02 16:41:43 -06:00
zsxwing 57566d0af3 [SPARK-6059][Yarn] Add volatile to ApplicationMaster's reporterThread and allocator
`ApplicationMaster.reporterThread` and `ApplicationMaster.allocator` are accessed in multiple threads, so they should be marked as `volatile`.

Author: zsxwing <zsxwing@gmail.com>

Closes #4814 from zsxwing/SPARK-6059 and squashes the following commits:

17d9386 [zsxwing] Add volatile to ApplicationMaster's reporterThread and allocator
2015-02-27 13:33:39 +00:00
zsxwing e747e98490 [SPARK-6058][Yarn] Log the user class exception in ApplicationMaster
Because ApplicationMaster doesn't set SparkUncaughtExceptionHandler, the exception in the user class won't be logged. This PR added a `logError` for it.

Author: zsxwing <zsxwing@gmail.com>

Closes #4813 from zsxwing/SPARK-6058 and squashes the following commits:

806c932 [zsxwing] Log the user class exception
2015-02-27 13:31:46 +00:00
mohit.goyal b38dec2ffd [SPARK-5951][YARN] Remove unreachable driver memory properties in yarn client mode
Remove unreachable driver memory properties in yarn client mode

Author: mohit.goyal <mohit.goyal@guavus.com>

Closes #4730 from zuxqoj/master and squashes the following commits:

977dc96 [mohit.goyal] remove not rechable deprecated variables in yarn client mode
2015-02-26 14:27:47 -08:00
Cheolsoo Park 5f3238b3b0 [SPARK-6018] [YARN] NoSuchMethodError in Spark app is swallowed by YARN AM
Author: Cheolsoo Park <cheolsoop@netflix.com>

Closes #4773 from piaozhexiu/SPARK-6018 and squashes the following commits:

2a919d5 [Cheolsoo Park] Rename e with cause to avoid duplicate names
1e71d2d [Cheolsoo Park] Replace placeholder with throwable
eb5750d [Cheolsoo Park] NoSuchMethodError in Spark app is swallowed by YARN AM
2015-02-26 13:53:49 -08:00
Hari Shreedharan 7138816abe [SPARK-5937][YARN] Fix ClientSuite to set YARN mode, so that the correct class is used in t...
...ests.

Without this SparkHadoopUtil is used by the Client instead of YarnSparkHadoopUtil.

Author: Hari Shreedharan <hshreedharan@apache.org>

Closes #4711 from harishreedharan/SPARK-5937 and squashes the following commits:

d154de6 [Hari Shreedharan] Use System.clearProperty() instead of setting the value of SPARK_YARN_MODE to empty string.
f729f70 [Hari Shreedharan] Fix ClientSuite to set YARN mode, so that the correct class is used in tests.
2015-02-21 10:01:01 -08:00
lianhuiwang 947b8bd82e [SPARK-5759][Yarn]ExecutorRunnable should catch YarnException while NMClient start contain...
some time since some reasons, it lead to some exception while NMClient start some containers.example:we do not config spark_shuffle on some machines, so it will throw a exception:
java.lang.Error: org.apache.hadoop.yarn.exceptions.InvalidAuxServiceException: The auxService:spark_shuffle does not exist.
because YarnAllocator use ThreadPoolExecutor to start Container, so we can not find which container or hostname throw exception. I think we should catch YarnException in ExecutorRunnable when start container. if there are some exceptions, we can know the container id or hostname of failed container.

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #4554 from lianhuiwang/SPARK-5759 and squashes the following commits:

caf5a99 [lianhuiwang] use SparkException to warp exception
c02140f [lianhuiwang] ExecutorRunnable should catch YarnException while NMClient start container
2015-02-12 14:51:06 -08:00
Marcelo Vanzin 20a6013106 [SPARK-2996] Implement userClassPathFirst for driver, yarn.
Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as
`spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it
modifies the system classpath, instead of restricting the changes to the user's class
loader. So this change implements the behavior of the latter for Yarn, and deprecates
the more dangerous choice.

To be able to achieve feature-parity, I also implemented the option for drivers (the existing
option only applies to executors). So now there are two options, each controlling whether
to apply userClassPathFirst to the driver or executors. The old option was deprecated, and
aliased to the new one (`spark.executor.userClassPathFirst`).

The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it
was also doing some things that ended up causing JVM errors depending on how things
were being called.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3233 from vanzin/SPARK-2996 and squashes the following commits:

9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation.
fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically.
a8c69f1 [Marcelo Vanzin] Review feedback.
cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test.
0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful.
fe970a7 [Marcelo Vanzin] Review feedback.
25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent.
fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks.
2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation.
b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
a10f379 [Marcelo Vanzin] Some feedback.
3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
7b57cba [Marcelo Vanzin] Remove now outdated message.
5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996
d1273b2 [Marcelo Vanzin] Add test file to rat exclude.
fa1aafa [Marcelo Vanzin] Remove write check on user jars.
89d8072 [Marcelo Vanzin] Cleanups.
a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode.
50afa5f [Marcelo Vanzin] Fix Yarn executor command line.
7d14397 [Marcelo Vanzin] Register user jars in executor up front.
7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst.
20373f5 [Marcelo Vanzin] Fix ClientBaseSuite.
55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit.
0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option.
4a84d87 [Marcelo Vanzin] Fix the child-first class loader.
d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf.
46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst".
a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit.
91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation.
a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments.
89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode.
2015-02-09 21:17:28 -08:00
Sean Owen de7806048a SPARK-4267 [YARN] Failing to launch jobs on Spark on YARN with Hadoop 2.5.0 or later
Before passing to YARN, escape arguments in "extraJavaOptions" args, in order to correctly handle cases like -Dfoo="one two three". Also standardize how these args are handled and ensure that individual args are treated as stand-alone args, not one string.

vanzin andrewor14

Author: Sean Owen <sowen@cloudera.com>

Closes #4452 from srowen/SPARK-4267.2 and squashes the following commits:

c8297d2 [Sean Owen] Before passing to YARN, escape arguments in "extraJavaOptions" args, in order to correctly handle cases like -Dfoo="one two three". Also standardize how these args are handled and ensure that individual args are treated as stand-alone args, not one string.
2015-02-09 10:33:57 -08:00
Kashish Jain ca66159a4f SPARK-5613: Catch the ApplicationNotFoundException exception to avoid thread from getting killed on yarn restart.
[SPARK-5613] Added a  catch block to catch the ApplicationNotFoundException. Without this catch block the thread gets killed on occurrence of this exception. This Exception occurs when yarn restarts and tries to find an application id for a spark job which got interrupted due to yarn getting stopped.
See the stacktrace in the bug for more details.

Author: Kashish Jain <kashish.jain@guavus.com>

Closes #4392 from kasjain/branch-1.2 and squashes the following commits:

4831000 [Kashish Jain] SPARK-5613: Catch the ApplicationNotFoundException exception to avoid thread from getting killed on yarn restart.
2015-02-06 13:59:11 -08:00
Kostas Sakellis 32e964c410 SPARK-2450 Adds executor log links to Web UI
Adds links to stderr/stdout in the executor tab of the webUI for:
1) Standalone
2) Yarn client
3) Yarn cluster

This tries to add the log url support in a general way so as to make it easy to add support for all the
cluster managers. This is done by using environment variables to pass to the executor the log urls. The
SPARK_LOG_URL_ prefix is used and so additional logs besides stderr/stdout can also be added.

To propagate this information to the UI we use the onExecutorAdded spark listener event.

Although this commit doesn't add log urls when running on a mesos cluster, it should be possible to add using the same mechanism.

Author: Kostas Sakellis <kostas@cloudera.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #3486 from ksakellis/kostas-spark-2450 and squashes the following commits:

d190936 [Josh Rosen] Fix a few minor style / formatting nits. Reset listener after each test Don't null listener out at end of main().
8673fe1 [Kostas Sakellis] CR feedback. Hide the log column if there are no logs available
5bf6952 [Kostas Sakellis] [SPARK-2450] [CORE] Adds exeuctor log links to Web UI
2015-02-06 11:13:00 -08:00
Sandy Ryza 1a88f20de7 SPARK-4337. [YARN] Add ability to cancel pending requests
Author: Sandy Ryza <sandy@cloudera.com>

Closes #4141 from sryza/sandy-spark-4337 and squashes the following commits:

a98bd20 [Sandy Ryza] Andrew's comments
cdaab7f [Sandy Ryza] SPARK-4337. Add ability to cancel pending requests to YARN
2015-02-06 10:53:16 -08:00
lianhuiwang cc6e53119d [SPARK-5653][YARN] In ApplicationMaster rename isDriver to isClusterMode
in ApplicationMaster rename isDriver to isClusterMode,because in Client it uses isClusterMode,ApplicationMaster should keep consistent with it and uses isClusterMode.Also isClusterMode is easier to understand.
andrewor14 sryza

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #4430 from lianhuiwang/am-isDriver-rename and squashes the following commits:

f9f3ed0 [lianhuiwang] rename isDriver to isClusterMode
2015-02-06 10:48:31 -08:00
Kousuke Saruta 24dbc50b93 [SPARK-5157][YARN] Configure more JVM options properly when we use ConcMarkSweepGC for AM.
When we set `SPARK_USE_CONC_INCR_GC`, ConcurrentMarkSweepGC works on the AM.
Actually, if ConcurrentMarkSweepGC is set for the JVM, following JVM options are set automatically and implicitly.

* MaxTenuringThreshold=0
* SurvivorRatio=1024

Those can not be proper value for most cases.
See also http://www.oracle.com/technetwork/java/tuning-139912.html

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #3956 from sarutak/SPARK-5157 and squashes the following commits:

c15da4e [Kousuke Saruta] Set more JVM options for AM when enabling CMS
2015-02-06 09:39:12 +00:00
Patrick Wendell 7930d2bef0 SPARK-3996: Add jetty servlet and continuations.
These are needed transitively from the other Jetty libraries
we include. It was not picked up by unit tests because we
disable the UI.

Author: Patrick Wendell <patrick@databricks.com>

Closes #4323 from pwendell/jetty and squashes the following commits:

d8669da [Patrick Wendell] SPARK-3996: Add jetty servlet and continuations.
2015-02-02 21:01:36 -08:00
Jacek Lewandowski cfea30037f Spark 3883: SSL support for HttpServer and Akka
SPARK-3883: SSL support for Akka connections and Jetty based file servers.

This story introduced the following changes:
- Introduced SSLOptions object which holds the SSL configuration and can build the appropriate configuration for Akka or Jetty. SSLOptions can be created by parsing SparkConf entries at a specified namespace.
- SSLOptions is created and kept by SecurityManager
- All Akka actor address creation snippets based on interpolated strings were replaced by a dedicated methods from AkkaUtils. Those methods select the proper Akka protocol - whether akka.tcp or akka.ssl.tcp
- Added tests cases for AkkaUtils, FileServer, SSLOptions and SecurityManager
- Added a way to use node local SSL configuration by executors and driver in standalone mode. It can be done by specifying spark.ssl.useNodeLocalConf in SparkConf.
- Made CoarseGrainedExecutorBackend not overwrite the settings which are executor startup configuration - they are passed anyway from Worker

Refer to https://github.com/apache/spark/pull/3571 for discussion and details

Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>
Author: Jacek Lewandowski <jacek.lewandowski@datastax.com>

Closes #3571 from jacek-lewandowski/SPARK-3883-master and squashes the following commits:

9ef4ed1 [Jacek Lewandowski] Merge pull request #2 from jacek-lewandowski/SPARK-3883-docs2
fb31b49 [Jacek Lewandowski] SPARK-3883: Added SSL setup documentation
2532668 [Jacek Lewandowski] SPARK-3883: Refactored AkkaUtils.protocol method to not use Try
90a8762 [Jacek Lewandowski] SPARK-3883: Refactored methods to resolve Akka address and made it possible to easily configure multiple communication layers for SSL
72b2541 [Jacek Lewandowski] SPARK-3883: A reference to the fallback SSLOptions can be provided when constructing SSLOptions
93050f4 [Jacek Lewandowski] SPARK-3883: SSL support for HttpServer and Akka
2015-02-02 17:27:26 -08:00
Patrick Wendell 2321dd1ef9 [HOTFIX] Add jetty references to build for YARN module. 2015-02-02 14:00:49 -08:00
Xutingjun 62a93a1698 [SPARK-5530] Add executor container to executorIdToContainer
when call killExecutor method, it will only go to the else branch, because  the variable executorIdToContainer never be put any value.

Author: Xutingjun <1039320815@qq.com>

Closes #4309 from XuTingjun/dynamicAllocator and squashes the following commits:

c823418 [Xutingjun] fix bugwq
2015-02-02 12:37:51 -08:00
lianhuiwang f5e63751f0 [SPARK-5173]support python application running on yarn cluster mode
now when we run python application on yarn cluster mode through spark-submit, spark-submit does not support python application on yarn cluster mode. so i modify code of submit and yarn's AM in order to support it.
through specifying .py file or primaryResource file via spark-submit, we can make pyspark run in yarn-cluster mode.
example:spark-submit --master yarn-master --num-executors 1 --driver-memory 1g --executor-memory 1g  xx.py --primaryResource yy.conf
this config is same as pyspark on yarn-client mode.
firstly,we put local path of .py or primaryResource to yarn's dist.files.that can be distributed on slave nodes.and then in spark-submit we transfer --py-files and --primaryResource to yarn.Client and use "org.apache.spark.deploy.PythonRunner" to user class that can run .py files on ApplicationMaster.
in yarn.Client we transfer --py-files and --primaryResource to  ApplicationMaster.
in ApplicationMaster, user's class is org.apache.spark.deploy.PythonRunner, and user's args is primaryResource and -py-files. so that can make pyspark run on ApplicationMaster.
JoshRosen tgravescs sryza

Author: lianhuiwang <lianhuiwang09@gmail.com>
Author: Wang Lianhui <lianhuiwang09@gmail.com>

Closes #3976 from lianhuiwang/SPARK-5173 and squashes the following commits:

28a8a58 [lianhuiwang] fix variable name
67f8cee [lianhuiwang] update with andrewor's comments
0319ae3 [lianhuiwang] address with sryza's comments
2385ef6 [lianhuiwang] address with sryza's comments
03640ab [lianhuiwang] add sparkHome to env
47d2fc3 [lianhuiwang] fix test
2adc8f5 [lianhuiwang] add spark.test.home
d60bc60 [lianhuiwang] fix test
5b30064 [lianhuiwang] add test
097a5ec [lianhuiwang] fix line length exceeds 100
905a106 [lianhuiwang] update with sryza and andrewor 's comments
f1f55b6 [lianhuiwang] when yarn-cluster, all python files can be non-local
172eec1 [Wang Lianhui] fix a min submit's bug
9c941bc [lianhuiwang] support python application running on yarn cluster mode
2015-02-02 12:32:28 -08:00
Sandy Ryza b2047b55c5 SPARK-4585. Spark dynamic executor allocation should use minExecutors as...
... initial number

Author: Sandy Ryza <sandy@cloudera.com>

Closes #4051 from sryza/sandy-spark-4585 and squashes the following commits:

d1dd039 [Sandy Ryza] Add spark.dynamicAllocation.initialNumExecutors and make min and max not required
b7c59dc [Sandy Ryza] SPARK-4585. Spark dynamic executor allocation should use minExecutors as initial number
2015-02-02 12:27:08 -08:00
Masayoshi TSUZUKI 7712ed5b16 [SPARK-1825] Make Windows Spark client work fine with Linux YARN cluster
Modified environment strings and path separators to platform-independent style if possible.

Author: Masayoshi TSUZUKI <tsudukim@oss.nttdata.co.jp>

Closes #3943 from tsudukim/feature/SPARK-1825 and squashes the following commits:

ec4b865 [Masayoshi TSUZUKI] Rebased and modified as comments.
f8a1d5a [Masayoshi TSUZUKI] Merge branch 'master' of github.com:tsudukim/spark into feature/SPARK-1825
3d03d35 [Masayoshi TSUZUKI] [SPARK-1825] Make Windows Spark client work fine with Linux YARN cluster
2015-02-01 18:28:55 -08:00
Sandy Ryza 254eaa4d35 SPARK-5393. Flood of util.RackResolver log messages after SPARK-1714
Previously I had tried to solve this with by adding a line in Spark's log4j-defaults.properties.

The issue with the message in log4j-defaults.properties was that the log4j.properties packaged inside Hadoop was getting picked up instead. While it would be ideal to fix that as well, we still want to quiet this in situations where a user supplies their own custom log4j properties.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #4192 from sryza/sandy-spark-5393 and squashes the following commits:

4d5dedc [Sandy Ryza] Only set log level if unset
46e07c5 [Sandy Ryza] SPARK-5393. Flood of util.RackResolver log messages after SPARK-1714
2015-01-30 11:31:54 -06:00
lianhuiwang 81f8f34062 [SPARK-4955]With executor dynamic scaling enabled,executor shoude be added or killed in yarn-cluster mode.
With executor dynamic scaling enabled, executor number shoude be added or killed in yarn-cluster mode.so in yarn-cluster mode, ApplicationMaster start a AMActor that add or kill a executor. then YarnSchedulerActor  in YarnSchedulerBackend send message to am's AMActor.
andrewor14 ChengXiangLi tdas

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #3962 from lianhuiwang/SPARK-4955 and squashes the following commits:

48d9ebb [lianhuiwang] update with andrewor14's comments
12426af [lianhuiwang] refactor am's code
45da3b0 [lianhuiwang] remove unrelated code
9318fc1 [lianhuiwang] update with andrewor14's comments
08ba473 [lianhuiwang] address andrewor14's comments
265c36d [lianhuiwang] fix small change
f43bda8 [lianhuiwang] fix address andrewor14's comments
7a7767a [lianhuiwang] fix address andrewor14's comments
bbc4d5a [lianhuiwang] address andrewor14's comments
1b029a4 [lianhuiwang] fix bug
7d33791 [lianhuiwang] in AM create a new actorSystem
2164ea8 [lianhuiwang] fix a min bug
6dfeeec [lianhuiwang] in yarn-cluster mode,executor number can be added or killed.
2015-01-28 12:51:15 -08:00
Sandy Ryza 820ce03597 SPARK-5370. [YARN] Remove some unnecessary synchronization in YarnAlloca...
...tor

Author: Sandy Ryza <sandy@cloudera.com>

Closes #4164 from sryza/sandy-spark-5370 and squashes the following commits:

0c8d736 [Sandy Ryza] SPARK-5370. [YARN] Remove some unnecessary synchronization in YarnAllocator
2015-01-22 13:49:35 -06:00
Sandy Ryza 2eeada373e SPARK-1714. Take advantage of AMRMClient APIs to simplify logic in YarnA...
...llocator

The goal of this PR is to simplify YarnAllocator as much as possible and get it up to the level of code quality we see in the rest of Spark.

In service of this, it does a few things:
* Uses AMRMClient APIs for matching containers to requests.
* Adds calls to AMRMClient.removeContainerRequest so that, when we use a container, we don't end up requesting it again.
* Removes YarnAllocator's host->rack cache. YARN's RackResolver already does this caching, so this is redundant.
* Adds tests for basic YarnAllocator functionality.
* Breaks up the allocateResources method, which was previously nearly 300 lines.
* A little bit of stylistic cleanup.
* Fixes a bug that causes three times the requests to be filed when preferred host locations are given.

The patch is lossy. In particular, it loses the logic for trying to avoid containers bunching up on nodes. As I understand it, the logic that's gone is:

* If, in a single response from the RM, we receive a set of containers on a node, and prefer some number of containers on that node greater than 0 but less than the number we received, give back the delta between what we preferred and what we received.

This seems like a weird way to avoid bunching E.g. it does nothing to avoid bunching when we don't request containers on particular nodes.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3765 from sryza/sandy-spark-1714 and squashes the following commits:

32a5942 [Sandy Ryza] Muffle RackResolver logs
74f56dd [Sandy Ryza] Fix a couple comments and simplify requestTotalExecutors
60ea4bd [Sandy Ryza] Fix scalastyle
ca35b53 [Sandy Ryza] Simplify further
e9cf8a6 [Sandy Ryza] Fix YarnClusterSuite
257acf3 [Sandy Ryza] Remove locality stuff and more cleanup
59a3c5e [Sandy Ryza] Take out rack stuff
5f72fd5 [Sandy Ryza] Further documentation and cleanup
89edd68 [Sandy Ryza] SPARK-1714. Take advantage of AMRMClient APIs to simplify logic in YarnAllocator
2015-01-21 10:31:54 -06:00
WangTao 8c06a5faac [SPARK-5336][YARN]spark.executor.cores must not be less than spark.task.cpus
https://issues.apache.org/jira/browse/SPARK-5336

Author: WangTao <barneystinson@aliyun.com>
Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #4123 from WangTaoTheTonic/SPARK-5336 and squashes the following commits:

6c9676a [WangTao] Update ClientArguments.scala
9632d3a [WangTaoTheTonic] minor comment fix
d03d6fa [WangTaoTheTonic] import ordering should be alphabetical'
3112af9 [WangTao] spark.executor.cores must not be less than spark.task.cpus
2015-01-21 09:42:30 -06:00
Patrick Wendell ad16da1bcc [HOTFIX]: Minor clean up regarding skipped artifacts in build files.
There are two relevant 'skip' configurations in the build, the first
is for "mvn install" and the second is for "mvn deploy". As of 1.2,
we actually use "mvn install" to generate our deployed artifcts,
because we have some customization of the nexus upload due to having
to cross compile for Scala 2.10 and 2.11.

There is no reason to have differents settings for these values,
this patch simply cleans this up for the repl/ and yarn/
projects.

Author: Patrick Wendell <patrick@databricks.com>

Closes #4080 from pwendell/master and squashes the following commits:

e21b78b [Patrick Wendell] [HOTFIX]: Minor clean up regarding skipped artifacts in build files.
2015-01-17 23:15:12 -08:00
WangTaoTheTonic 2be82b1e66 [SPARK-1507][YARN]specify # cores for ApplicationMaster
Based on top of changes in https://github.com/apache/spark/pull/3806.

https://issues.apache.org/jira/browse/SPARK-1507

`--driver-cores` and `spark.driver.cores` for all cluster modes and `spark.yarn.am.cores` for yarn client mode.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #4018 from WangTaoTheTonic/SPARK-1507 and squashes the following commits:

01419d3 [WangTaoTheTonic] amend the args name
b255795 [WangTaoTheTonic] indet thing
d86557c [WangTaoTheTonic] some comments amend
43c9392 [WangTao] fix compile error
b39a100 [WangTao] specify # cores for ApplicationMaster
2015-01-16 09:16:56 -08:00
WangTaoTheTonic 9dea64e53a [SPARK-4697][YARN]System properties should override environment variables
I found some arguments in yarn module take environment variables before system properties while the latter override the former in core module.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3557 from WangTaoTheTonic/SPARK4697 and squashes the following commits:

836b9ef [WangTaoTheTonic] fix type mismatch
e3e486a [WangTaoTheTonic] remove the comma
1262d57 [WangTaoTheTonic] handle spark.app.name and SPARK_YARN_APP_NAME in SparkSubmitArguments
bee9447 [WangTaoTheTonic] wrong brace
81833bb [WangTaoTheTonic] rebase
40934b4 [WangTaoTheTonic] just switch blocks
5f43f45 [WangTao] System property can override environment variable
2015-01-13 09:50:14 -08:00
WangTaoTheTonic f7741a9a72 [SPARK-5006][Deploy]spark.port.maxRetries doesn't work
https://issues.apache.org/jira/browse/SPARK-5006

I think the issue is produced in https://github.com/apache/spark/pull/1777.

Not digging mesos's backend yet. Maybe should add same logic either.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits:

8cdf96d [WangTao] indent thing
2d86d65 [WangTaoTheTonic] fix line length
7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor
61a370d [WangTaoTheTonic] some minor fixes
bc6e1ec [WangTaoTheTonic] rebase
67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments
f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg
29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable
396c226 [WangTaoTheTonic] make the grammar more like scala
191face [WangTaoTheTonic] invalid value name
62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work
2015-01-13 09:29:25 -08:00
WangTaoTheTonic e966452060 [SPARK-1953][YARN]yarn client mode Application Master memory size is same as driver memory...
... size

Ways to set Application Master's memory on yarn-client mode:
1.  `spark.yarn.am.memory` in SparkConf or System Properties
2.  default value 512m

Note: this arguments is only available in yarn-client mode.

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3607 from WangTaoTheTonic/SPARK4181 and squashes the following commits:

d5ceb1b [WangTaoTheTonic] spark.driver.memeory is used in both modes
6c1b264 [WangTaoTheTonic] rebase
b8410c0 [WangTaoTheTonic] minor optiminzation
ddcd592 [WangTaoTheTonic] fix the bug produced in rebase and some improvements
3bf70cc [WangTaoTheTonic] rebase and give proper hint
987b99d [WangTaoTheTonic] disable --driver-memory in client mode
2b27928 [WangTaoTheTonic] inaccurate description
b7acbb2 [WangTaoTheTonic] incorrect method invoked
2557c5e [WangTaoTheTonic] missing a single blank
42075b0 [WangTaoTheTonic] arrange the args and warn logging
69c7dba [WangTaoTheTonic] rebase
1960d16 [WangTaoTheTonic] fix wrong comment
7fa9e2e [WangTaoTheTonic] log a warning
f6bee0e [WangTaoTheTonic] docs issue
d619996 [WangTaoTheTonic] Merge branch 'master' into SPARK4181
b09c309 [WangTaoTheTonic] use code format
ab16bb5 [WangTaoTheTonic] fix bug and add comments
44e48c2 [WangTaoTheTonic] minor fix
6fd13e1 [WangTaoTheTonic] add overhead mem and remove some configs
0566bb8 [WangTaoTheTonic] yarn client mode Application Master memory size is same as driver memory size
2015-01-09 13:23:13 -08:00
WangTaoTheTonic f3da4bd728 [SPARK-5169][YARN]fetch the correct max attempts
Soryy for fetching the wrong max attempts in this commit 8fdd48959c.
We need to fix it now.

tgravescs

If we set an spark.yarn.maxAppAttempts which is larger than `yarn.resourcemanager.am.max-attempts` in yarn side, it will be overrided as described here:
>The maximum number of application attempts. It's a global setting for all application masters. Each application master can specify its individual maximum number of application attempts via the API, but the individual number cannot be more than the global upper bound. If it is, the resourcemanager will override it. The default number is set to 2, to allow at least one retry for AM.

http://hadoop.apache.org/docs/r2.6.0/hadoop-yarn/hadoop-yarn-common/yarn-default.xml

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3942 from WangTaoTheTonic/HOTFIX and squashes the following commits:

9ac16ce [WangTaoTheTonic] fetch the correct max attempts
2015-01-09 08:10:09 -06:00
Marcelo Vanzin 48cecf673c [SPARK-4048] Enhance and extend hadoop-provided profile.
This change does a few things to make the hadoop-provided profile more useful:

- Create new profiles for other libraries / services that might be provided by the infrastructure
- Simplify and fix the poms so that the profiles are only activated while building assemblies.
- Fix tests so that they're able to run when the profiles are activated
- Add a new env variable to be used by distributions that use these profiles to provide the runtime
  classpath for Spark jobs and daemons.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2982 from vanzin/SPARK-4048 and squashes the following commits:

82eb688 [Marcelo Vanzin] Add a comment.
eb228c0 [Marcelo Vanzin] Fix borked merge.
4e38f4e [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
9ef79a3 [Marcelo Vanzin] Alternative way to propagate test classpath to child processes.
371ebee [Marcelo Vanzin] Review feedback.
52f366d [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
83099fc [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
7377e7b [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
322f882 [Marcelo Vanzin] Fix merge fail.
f24e9e7 [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
8b00b6a [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
9640503 [Marcelo Vanzin] Cleanup child process log message.
115fde5 [Marcelo Vanzin] Simplify a comment (and make it consistent with another pom).
e3ab2da [Marcelo Vanzin] Fix hive-thriftserver profile.
7820d58 [Marcelo Vanzin] Fix CliSuite with provided profiles.
1be73d4 [Marcelo Vanzin] Restore flume-provided profile.
d1399ed [Marcelo Vanzin] Restore jetty dependency.
82a54b9 [Marcelo Vanzin] Remove unused profile.
5c54a25 [Marcelo Vanzin] Fix HiveThriftServer2Suite with *-provided profiles.
1fc4d0b [Marcelo Vanzin] Update dependencies for hive-thriftserver.
f7b3bbe [Marcelo Vanzin] Add snappy to hadoop-provided list.
9e4e001 [Marcelo Vanzin] Remove duplicate hive profile.
d928d62 [Marcelo Vanzin] Redirect child stderr to parent's log.
4d67469 [Marcelo Vanzin] Propagate SPARK_DIST_CLASSPATH on Yarn.
417d90e [Marcelo Vanzin] Introduce "SPARK_DIST_CLASSPATH".
2f95f0d [Marcelo Vanzin] Propagate classpath to child processes during testing.
1adf91c [Marcelo Vanzin] Re-enable maven-install-plugin for a few projects.
284dda6 [Marcelo Vanzin] Rework the "hadoop-provided" profile, add new ones.
2015-01-08 17:15:13 -08:00
Sandy Ryza 8d45834deb SPARK-5087. [YARN] Merge yarn.Client and yarn.ClientBase
Author: Sandy Ryza <sandy@cloudera.com>

Closes #3896 from sryza/sandy-spark-5087 and squashes the following commits:

65611d0 [Sandy Ryza] Review feedback
3294176 [Sandy Ryza] SPARK-5087. [YARN] Merge yarn.Client and yarn.ClientBase
2015-01-08 09:25:43 -08:00
WangTaoTheTonic 8fdd48959c [SPARK-2165][YARN]add support for setting maxAppAttempts in the ApplicationSubmissionContext
...xt

https://issues.apache.org/jira/browse/SPARK-2165

I still have 2 questions:
* If this config is not set, we should use yarn's corresponding value or a default value(like 2) on spark side?
* Is the config name best? Or "spark.yarn.am.maxAttempts"?

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #3878 from WangTaoTheTonic/SPARK-2165 and squashes the following commits:

1416c83 [WangTaoTheTonic] use the name spark.yarn.maxAppAttempts
202ac85 [WangTaoTheTonic] rephrase some
afdfc99 [WangTaoTheTonic] more detailed description
91562c6 [WangTaoTheTonic] add support for setting maxAppAttempts in the ApplicationSubmissionContext
2015-01-07 08:14:39 -06:00
huangzhaowei 5fde66163f [YARN][SPARK-4929] Bug fix: fix the yarn-client code to support HA
Nowadays, yarn-client will exit directly when the HA change happens no matter how many times the am should retry.
The reason may be that the default final status only considerred the sys.exit, and the yarn-client HA cann't benefit from this.
So we should distinct the default final status between client and cluster, because the SUCCEEDED status may cause the HA failed in client mode and UNDEFINED may cause the error reporter in cluster when using sys.exit.

Author: huangzhaowei <carlmartinmax@gmail.com>

Closes #3771 from SaintBacchus/YarnHA and squashes the following commits:

c02bfcc [huangzhaowei] Improve the comment of the funciton 'getDefaultFinalStatus'
0e69924 [huangzhaowei] Bug fix: fix the yarn-client code to support HA
2015-01-07 08:10:42 -06:00
Sean Owen 4cba6eb420 SPARK-4159 [CORE] Maven build doesn't run JUnit test suites
This PR:

- Reenables `surefire`, and copies config from `scalatest` (which is itself an old fork of `surefire`, so similar)
- Tells `surefire` to test only Java tests
- Enables `surefire` and `scalatest` for all children, and in turn eliminates some duplication.

For me this causes the Scala and Java tests to be run once each, it seems, as desired. It doesn't affect the SBT build but works for Maven. I still need to verify that all of the Scala tests and Java tests are being run.

Author: Sean Owen <sowen@cloudera.com>

Closes #3651 from srowen/SPARK-4159 and squashes the following commits:

2e8a0af [Sean Owen] Remove specialized SPARK_HOME setting for REPL, YARN tests as it appears to be obsolete
12e4558 [Sean Owen] Append to unit-test.log instead of overwriting, so that both surefire and scalatest output is preserved. Also standardize/correct comments a bit.
e6f8601 [Sean Owen] Reenable Java tests by reenabling surefire with config cloned from scalatest; centralize test config in the parent
2015-01-06 12:02:08 -08:00
Kostas Sakellis 451546aa6d SPARK-4843 [YARN] Squash ExecutorRunnableUtil and ExecutorRunnable
ExecutorRunnableUtil is a parent of ExecutorRunnable because of the yarn-alpha and yarn-stable split. Now that yarn-alpha is gone, this commit squashes the unnecessary hierarchy. The methods from ExecutorRunnableUtil are added as private.

Author: Kostas Sakellis <kostas@cloudera.com>

Closes #3696 from ksakellis/kostas-spark-4843 and squashes the following commits:

486716f [Kostas Sakellis] Moved prepareEnvironment call to after yarnConf declaration
470e22e [Kostas Sakellis] Fixed indentation and renamed sparkConf variable
9b1b1c9 [Kostas Sakellis] SPARK-4843 [YARN] Squash ExecutorRunnableUtil and ExecutorRunnable
2015-01-05 23:26:33 -08:00
meiyoula 14fa87bdf4 [SPARK-4966][YARN]The MemoryOverhead value is setted not correctly
Author: meiyoula <1039320815@qq.com>

Closes #3797 from XuTingjun/MemoryOverhead and squashes the following commits:

5a780fc [meiyoula] Update ClientArguments.scala
2014-12-29 08:20:30 -06:00
Kousuke Saruta 199e59aacd [SPARK-4881][Minor] Use SparkConf#getBoolean instead of get().toBoolean
It's really a minor issue.

In ApplicationMaster, there is code like as follows.

    val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean

I think, the code can be simplified like as follows.

    val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false)

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #3733 from sarutak/SPARK-4881 and squashes the following commits:

1771430 [Kousuke Saruta] Modified the code like sparkConf.get(...).toBoolean to sparkConf.getBoolean(...)
c63daa0 [Kousuke Saruta] Simplified code
2014-12-23 19:14:34 -08:00
Andrew Or 27c5399f4d [SPARK-4730][YARN] Warn against deprecated YARN settings
See https://issues.apache.org/jira/browse/SPARK-4730.

Author: Andrew Or <andrew@databricks.com>

Closes #3590 from andrewor14/yarn-settings and squashes the following commits:

36e0753 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-settings
dcd1316 [Andrew Or] Warn against deprecated YARN settings
2014-12-23 14:28:36 -08:00
Sandy Ryza d62da642ac SPARK-4447. Remove layers of abstraction in YARN code no longer needed after dropping yarn-alpha
Author: Sandy Ryza <sandy@cloudera.com>

Closes #3652 from sryza/sandy-spark-4447 and squashes the following commits:

2791158 [Sandy Ryza] Review feedback
c23507b [Sandy Ryza] Strip margin from client arguments help string
18be7ba [Sandy Ryza] SPARK-4447
2014-12-22 12:23:43 -08:00
Sandy Ryza 253b72b56f SPARK-3779. yarn spark.yarn.applicationMaster.waitTries config should be...
... changed to a time period

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3471 from sryza/sandy-spark-3779 and squashes the following commits:

20b9887 [Sandy Ryza] Deprecate old property
42b5df7 [Sandy Ryza] Review feedback
9a959a1 [Sandy Ryza] SPARK-3779. yarn spark.yarn.applicationMaster.waitTries config should be changed to a time period
2014-12-18 12:19:07 -06:00
Zhan Zhang 3b764699ff [SPARK-4461][YARN] pass extra java options to yarn application master
Currently, there is no way to pass yarn am specific java options. It cause some potential issues when reading classpath from hadoop configuration file. Hadoop configuration actually replace variables in its property with the system property passed in java options. How to specify the value depends on different hadoop distribution.

The new options are SPARK_YARN_JAVA_OPTS or spark.yarn.extraJavaOptions. I make it as spark global level, because typically we don't want user to specify this in their command line each time submitting spark job after it is setup in spark-defaults.conf.

In addition, with this new extra options enabled to be passed to AM, it provides more flexibility.

For example int the following valid mapred-site.xml file, we have the class path which specify values using system property. Hadoop can correctly handle it because it has java options passed in.

This is the example, currently spark will break due to hadoop.version is not passed in.
  <property>
    <name>mapreduce.application.classpath</name>
    <value>/etc/hadoop/${hadoop.version}/mapreduce/*</value>
  </property>

In the meantime, we cannot relies on  mapreduce.admin.map.child.java.opts in mapred-site.xml, because it has its own extra java options specified, which does not apply to Spark.

Author: Zhan Zhang <zhazhan@gmail.com>

Closes #3409 from zhzhan/Spark-4461 and squashes the following commits:

daec3d0 [Zhan Zhang] solve review comments
08f44a7 [Zhan Zhang] add warning in driver mode if spark.yarn.am.extraJavaOptions is configured
5a505d3 [Zhan Zhang] solve review comments
4ed43ad [Zhan Zhang] solve review comments
ad777ed [Zhan Zhang] Merge branch 'master' into Spark-4461
3e9e574 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
e3f9abe [Zhan Zhang] solve review comments
8963552 [Zhan Zhang] rebase
f8f6700 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
dea1692 [Zhan Zhang] change the option key name to client mode specific
90d5dff [Zhan Zhang] rebase
8ac9254 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
092a25f [Zhan Zhang] solve review comments
bc5a9ae [Zhan Zhang] solve review comments
782b014 [Zhan Zhang] add new configuration to docs/running-on-yarn.md and remove it from spark-defaults.conf.template
6faaa97 [Zhan Zhang] solve review comments
369863f [Zhan Zhang] clean up unnecessary var
733de9c [Zhan Zhang] Merge branch 'master' into Spark-4461
a68e7f0 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
864505a [Zhan Zhang] Add extra java options to be passed to Yarn application master
15830fc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
685d911 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
03ebad3 [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark
46d9e3d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
ebb213a [Zhan Zhang] revert
b983ef3 [Zhan Zhang] test
c4efb9b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
779d67b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
4daae6d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
12e1be5 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
ce0ca7b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
93f3081 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3764505 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
a9d372b [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark
a00f60f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
497b0f4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
4a2e36d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
a72c0d4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
301eb4a [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cedcc6f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
adf4924 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
d10bf00 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
7e0cc36 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
68deb11 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3ee3b2b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
2b0d513 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
1ccd7cc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
af9feb9 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
e4c1982 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
921e914 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
789ea21 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cb53a2c [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
f6a8a40 [Zhan Zhang] revert
ba14f28 [Zhan Zhang] test
2014-12-18 10:01:46 -06:00
Sandy Ryza 912563aa35 SPARK-4338. [YARN] Ditch yarn-alpha.
Sorry if this is a little premature with 1.2 still not out the door, but it will make other work like SPARK-4136 and SPARK-2089 a lot easier.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3215 from sryza/sandy-spark-4338 and squashes the following commits:

1c5ac08 [Sandy Ryza] Update building Spark docs and remove unnecessary newline
9c1421c [Sandy Ryza] SPARK-4338. Ditch yarn-alpha.
2014-12-09 11:02:43 -08:00
Andrew Or fd8525334c Revert "SPARK-2624 add datanucleus jars to the container in yarn-cluster"
This reverts commit a975dc3279.
2014-12-04 21:53:49 -08:00
Andrew Or 87437df036 Revert "[HOT FIX] [YARN] Check whether /lib exists before listing its files"
This reverts commit 90ec643e9a.
2014-12-04 21:53:38 -08:00
WangTaoTheTonic 8106b1e36b [SPARK-4253] Ignore spark.driver.host in yarn-cluster and standalone-cluster modes
In yarn-cluster and standalone-cluster modes, we don't know where driver will run until it is launched.  If the `spark.driver.host` property is set on the submitting machine and propagated to the driver through SparkConf then this will lead to errors when the driver launches.

This patch fixes this issue by dropping the `spark.driver.host` property in SparkSubmit when running in a cluster deploy mode.

Author: WangTaoTheTonic <barneystinson@aliyun.com>
Author: WangTao <barneystinson@aliyun.com>

Closes #3112 from WangTaoTheTonic/SPARK4253 and squashes the following commits:

ed1a25c [WangTaoTheTonic] revert unrelated formatting issue
02c4e49 [WangTao] add comment
32a3f3f [WangTaoTheTonic] ingore it in SparkSubmit instead of SparkContext
667cf24 [WangTaoTheTonic] document fix
ff8d5f7 [WangTaoTheTonic] also ignore it in standalone cluster mode
2286e6b [WangTao] ignore spark.driver.host in yarn-cluster mode
2014-12-04 11:53:23 -08:00
Andrew Or 90ec643e9a [HOT FIX] [YARN] Check whether /lib exists before listing its files
This is caused by a975dc3279

Author: Andrew Or <andrew@databricks.com>

Closes #3589 from andrewor14/yarn-hot-fix and squashes the following commits:

a4fad5f [Andrew Or] Check whether lib directory exists before listing its files
2014-12-03 13:56:23 -08:00
Jim Lim a975dc3279 SPARK-2624 add datanucleus jars to the container in yarn-cluster
If `spark-submit` finds the datanucleus jars, it adds them to the driver's classpath, but does not add it to the container.

This patch modifies the yarn deployment class to copy all `datanucleus-*` jars found in `[spark-home]/libs` to the container.

Author: Jim Lim <jim@quixey.com>

Closes #3238 from jimjh/SPARK-2624 and squashes the following commits:

3633071 [Jim Lim] SPARK-2624 update documentation and comments
fe95125 [Jim Lim] SPARK-2624 keep java imports together
6c31fe0 [Jim Lim] SPARK-2624 update documentation
6690fbf [Jim Lim] SPARK-2624 add tests
d28d8e9 [Jim Lim] SPARK-2624 add spark.yarn.datanucleus.dir option
84e6cba [Jim Lim] SPARK-2624 add datanucleus jars to the container in yarn-cluster
2014-12-03 11:16:29 -08:00
Marcelo Vanzin 915f8eeb3a [SPARK-4584] [yarn] Remove security manager from Yarn AM.
The security manager adds a lot of overhead to the runtime of the
app, and causes a severe performance regression. Even stubbing out
all unneeded methods (all except checkExit()) does not help.

So, instead, penalize users who do an explicit System.exit() by leaving
them in "undefined behavior" territory: if they do that, the Yarn
backend won't be able to report the final app status to the RM.
The result is that the final status of the application might not match
the user's expectations.

One side-effect of the change is that users who do an explicit
System.exit() will lose the AM retry functionality. Since there is
no way to know if the exit was because of success or failure, the
AM right now errs on the side of it being a successful exit.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3484 from vanzin/SPARK-4584 and squashes the following commits:

21f2502 [Marcelo Vanzin] Do not retry apps that use System.exit().
4198b3b [Marcelo Vanzin] [SPARK-4584] [yarn] Remove security manager from Yarn AM.
2014-11-28 15:16:05 -05:00
Marcelo Vanzin 397d3aae5b Bumping version to 1.3.0-SNAPSHOT.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3277 from vanzin/version-1.3 and squashes the following commits:

7c3c396 [Marcelo Vanzin] Added temp repo to sbt build.
5f404ff [Marcelo Vanzin] Add another exclusion.
19457e7 [Marcelo Vanzin] Update old version to 1.2, add temporary 1.2 repo.
3c8d705 [Marcelo Vanzin] Workaround for MIMA checks.
e940810 [Marcelo Vanzin] Bumping version to 1.3.0-SNAPSHOT.
2014-11-18 21:24:18 -08:00
Kousuke Saruta 7f3718842c [SPARK-4282][YARN] Stopping flag in YarnClientSchedulerBackend should be volatile
In YarnClientSchedulerBackend, a variable "stopping" is used as a flag and it's accessed by some threads so it should be volatile.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #3143 from sarutak/stopping-flag-volatile and squashes the following commits:

58fdcc9 [Kousuke Saruta] Marked stoppig flag as volatile
2014-11-11 12:33:53 -06:00
Andrew Or 96136f222a [SPARK-3797] Minor addendum to Yarn shuffle service
I did not realize there was a `network.util.JavaUtils` when I wrote this code. This PR moves the `ByteBuffer` string conversion to the appropriate place. I tested the changes on a stable yarn cluster.

Author: Andrew Or <andrew@databricks.com>

Closes #3144 from andrewor14/yarn-shuffle-util and squashes the following commits:

b6c08bf [Andrew Or] Remove unused import
94e205c [Andrew Or] Use netty Unpooled
85202a5 [Andrew Or] Use guava Charsets
057135b [Andrew Or] Reword comment
adf186d [Andrew Or] Move byte buffer String conversion logic to JavaUtils
2014-11-06 17:18:49 -08:00
Andrew Or 61a5cced04 [SPARK-3797] Run external shuffle service in Yarn NM
This creates a new module `network/yarn` that depends on `network/shuffle` recently created in #3001. This PR introduces a custom Yarn auxiliary service that runs the external shuffle service. As of the changes here this shuffle service is required for using dynamic allocation with Spark.

This is still WIP mainly because it doesn't handle security yet. I have tested this on a stable Yarn cluster.

Author: Andrew Or <andrew@databricks.com>

Closes #3082 from andrewor14/yarn-shuffle-service and squashes the following commits:

ef3ddae [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
0ee67a2 [Andrew Or] Minor wording suggestions
1c66046 [Andrew Or] Remove unused provided dependencies
0eb6233 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
6489db5 [Andrew Or] Try catch at the right places
7b71d8f [Andrew Or] Add detailed java docs + reword a few comments
d1124e4 [Andrew Or] Add security to shuffle service (INCOMPLETE)
5f8a96f [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
9b6e058 [Andrew Or] Address various feedback
f48b20c [Andrew Or] Fix tests again
f39daa6 [Andrew Or] Do not make network-yarn an assembly module
761f58a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service
15a5b37 [Andrew Or] Fix build for Hadoop 1.x
baff916 [Andrew Or] Fix tests
5bf9b7e [Andrew Or] Address a few minor comments
5b419b8 [Andrew Or] Add missing license header
804e7ff [Andrew Or] Include the Yarn shuffle service jar in the distribution
cd076a4 [Andrew Or] Require external shuffle service for dynamic allocation
ea764e0 [Andrew Or] Connect to Yarn shuffle service only if it's enabled
1bf5109 [Andrew Or] Use the shuffle service port specified through hadoop config
b4b1f0c [Andrew Or] 4 tabs -> 2 tabs
43dcb96 [Andrew Or] First cut integration of shuffle service with Yarn aux service
b54a0c4 [Andrew Or] Initial skeleton for Yarn shuffle service
2014-11-05 15:42:05 -08:00
andrewor14 087e31a7a0 [HOT FIX] Yarn stable tests don't compile
This is caused by this commit: acd4ac7c9a

Author: andrewor14 <andrew@databricks.com>
Author: Andrew Or <andrew@databricks.com>

Closes #3041 from andrewor14/yarn-hot-fix and squashes the following commits:

e5deba1 [andrewor14] Add new line at the end (minor)
aa998e8 [Andrew Or] Compilation hot fix
2014-10-31 14:36:55 -07:00
Sandy Ryza acd4ac7c9a SPARK-3837. Warn when YARN kills containers for exceeding memory limits
I triggered the issue and verified the message gets printed on a pseudo-distributed cluster.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #2744 from sryza/sandy-spark-3837 and squashes the following commits:

858a268 [Sandy Ryza] Review feedback
c937f00 [Sandy Ryza] SPARK-3837. Warn when YARN kills containers for exceeding memory limits
2014-10-31 08:43:06 -05:00
Andrew Or 5231a3f228 [Minor] A few typos in comments and log messages
Author: Andrew Or <andrewor14@gmail.com>
Author: Andrew Or <andrew@databricks.com>

Closes #3021 from andrewor14/typos and squashes the following commits:

daaf417 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos
4838ae4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos
026d426 [Andrew Or] Merge branch 'master' of github.com:andrewor14/spark into typos
a81ae8f [Andrew Or] Some typos
2014-10-30 15:32:11 -07:00
Andrew Or 26f092d4e3 [SPARK-4138][SPARK-4139] Improve dynamic allocation settings
This should be merged after #2746 (SPARK-3795).

**SPARK-4138**. If the user sets both the number of executors and `spark.dynamicAllocation.enabled`, we should throw an exception.

**SPARK-4139**. If the user sets `spark.dynamicAllocation.enabled`, we should use the max number of executors as the starting number of executors because the first job is likely to run immediately after application startup. If the latter is not set, throw an exception.

Author: Andrew Or <andrew@databricks.com>

Closes #3002 from andrewor14/yarn-set-executors and squashes the following commits:

c528fce [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-set-executors
55d4699 [Andrew Or] Bug fix: `isDynamicAllocationEnabled` was always false
2b0ccec [Andrew Or] Start the number of executors at the max
022bfde [Andrew Or] Guard against incompatible settings of number of executors
2014-10-30 15:31:23 -07:00
GuoQiang Li cd739bd756 [SPARK-1720][SPARK-1719] use LD_LIBRARY_PATH instead of -Djava.library.path
- [X] Standalone
- [X] YARN
- [X] Mesos
- [X]  Mac OS X
- [X] Linux
- [ ]  Windows

This is another implementation about #1031

Author: GuoQiang Li <witgo@qq.com>

Closes #2711 from witgo/SPARK-1719 and squashes the following commits:

c7b26f6 [GuoQiang Li] review commits
4488e41 [GuoQiang Li] Refactoring CommandUtils
a444094 [GuoQiang Li] review commits
40c0b4a [GuoQiang Li] Add buildLocalCommand method
c1a0ddd [GuoQiang Li] fix comments
156ce88 [GuoQiang Li] review commit
38aa377 [GuoQiang Li] Refactor CommandUtils.scala
4269e00 [GuoQiang Li] Refactor SparkSubmitDriverBootstrapper.scala
7a1d634 [GuoQiang Li] use LD_LIBRARY_PATH instead of -Djava.library.path
2014-10-29 23:02:58 -07:00
Andrew Or 8d59b37b02 [SPARK-3795] Heuristics for dynamically scaling executors
This is part of a bigger effort to provide elastic scaling of executors within a Spark application ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This PR does not provide any functionality by itself; it is a skeleton that is missing a mechanism to be added later in [SPARK-3822](https://issues.apache.org/jira/browse/SPARK-3822).

Comments and feedback are most welcome. For those of you reviewing this in detail, I highly recommend doing it through your favorite IDE instead of through the diff here.

Author: Andrew Or <andrewor14@gmail.com>
Author: Andrew Or <andrew@databricks.com>

Closes #2746 from andrewor14/scaling-heuristics and squashes the following commits:

8a4fdaa [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
e045df8 [Andrew Or] Add warning message (minor)
dfa31ec [Andrew Or] Fix tests
c0becc4 [Andrew Or] Merging with SPARK-3822
4784f93 [Andrew Or] Reword an awkward log message
181f27f [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
c79e907 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
4672b90 [Andrew Or] It's nano time.
a6a30f2 [Andrew Or] Do not allow min/max executors of 0
c60ec33 [Andrew Or] Rewrite test logic with clocks
b00b680 [Andrew Or] Fix style
c3caa65 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
7f9da14 [Andrew Or] Factor out logic to verify bounds on # executors (minor)
f279019 [Andrew Or] Add time mocking tests for polling loop
685e347 [Andrew Or] Factor out clock in polling loop to facilitate testing
3cea7f7 [Andrew Or] Use PrivateMethodTester to keep original class private
3156d81 [Andrew Or] Update comments and exception messages
92f36f9 [Andrew Or] Address minor review comments
abdea61 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
2aefd09 [Andrew Or] Correct listener behavior
9fe6e44 [Andrew Or] Rename variables and configs + update comments and log messages
149cc32 [Andrew Or] Fix style
254c958 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
5ff829b [Andrew Or] Add tests for ExecutorAllocationManager
19c6c4b [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
5896515 [Andrew Or] Move ExecutorAllocationManager out of scheduler package
9ca8945 [Andrew Or] Rewrite callbacks through the listener interface
5e336b9 [Andrew Or] Remove code from backend to avoid conflict with SPARK-3822
092d1fd [Andrew Or] Remove timeout logic for pending requests
1309fab [Andrew Or] Request executors by specifying the number pending
8bc0e9d [Andrew Or] Add logic to expire pending requests after timeouts
b750ee1 [Andrew Or] Express timers in terms of expiration times + remove retry logic
7f8dd47 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
9d516cc [Andrew Or] Bug fix: Actually trigger the add timer / add retry timer
44f1832 [Andrew Or] Rename configs to include time units
eaae7ef [Andrew Or] Address various review comments
6f8be6c [Andrew Or] Beef up comments on what each of the timers mean
baaa403 [Andrew Or] Simplify variable names (minor)
42beec8 [Andrew Or] Reset whether the add threshold is crossed on cancellation
9bcc0bc [Andrew Or] ExecutorScalingManager -> ExecutorAllocationManager
2784398 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics
5a97d9e [Andrew Or] Log retry attempts in INFO + clean up logging
2f55c9f [Andrew Or] Do not keep requesting executors even after max attempts
0acd1cb [Andrew Or] Rewrite timer logic with polling
b3c7d44 [Andrew Or] Start the retry timer for adding executors at the right time
9b5f2ea [Andrew Or] Wording changes in comments and log messages
c2203a5 [Andrew Or] Simplify code to access the scheduler backend
e519d08 [Andrew Or] Simplify initialization code
2cc87a7 [Andrew Or] Add retry logic for removing executors
d0b34a6 [Andrew Or] Add retry logic for adding executors
9cc4649 [Andrew Or] Simplifying synchronization logic
67c03c7 [Andrew Or] Correct semantics of adding executors + update comments
6c48ab0 [Andrew Or] Update synchronization comment
8901900 [Andrew Or] Simplify remove policy + change the semantics of add policy
1cc8444 [Andrew Or] Minor wording change
ae5b64a [Andrew Or] Add synchronization
20ec6b9 [Andrew Or] First cut implementation of removing executors dynamically
4077ae2 [Andrew Or] Minor code re-organization
6f1fa66 [Andrew Or] First cut implementation of adding executors dynamically
b2e6dcc [Andrew Or] Add skeleton interface for requesting / killing executors
2014-10-29 17:48:59 -07:00
Andrew Or 1df05a40eb [SPARK-3822] Executor scaling mechanism for Yarn
This is part of a broader effort to enable dynamic scaling of executors ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This is intended to work alongside SPARK-3795 (#2746), SPARK-3796 and SPARK-3797, but is functionally independently of these other issues.

The logic is built on top of PraveenSeluka's changes at #2798. This is different from the changes there in a few major ways: (1) the mechanism is implemented within the existing scheduler backend framework rather than in new `Actor` classes. This also introduces a parent abstract class `YarnSchedulerBackend` to encapsulate common logic to communicate with the Yarn `ApplicationMaster`. (2) The interface of requesting executors exposed to the `SparkContext` is the same, but the communication between the scheduler backend and the AM uses total number executors desired instead of an incremental number. This is discussed in #2746 and explained in the comments in the code.

I have tested this significantly on a stable Yarn cluster.

------------
A remaining task for this issue is to tone down the error messages emitted when an executor is removed.
Currently, `SparkContext` and its components react as if the executor has failed, resulting in many scary error messages and eventual timeouts. While it's not strictly necessary to fix this as of the first-cut implementation of this mechanism, it would be good to add logic to distinguish this case. I prefer to address this in a separate PR. I have filed a separate JIRA for this task at SPARK-4134.

Author: Andrew Or <andrew@databricks.com>
Author: Andrew Or <andrewor14@gmail.com>

Closes #2840 from andrewor14/yarn-scaling-mechanism and squashes the following commits:

485863e [Andrew Or] Minor log message changes
4920be8 [Andrew Or] Clarify that public API is only for Yarn mode for now
1c57804 [Andrew Or] Reword a few comments + other review comments
6321140 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
02836c0 [Andrew Or] Limit scope of synchronization
4e2ed7f [Andrew Or] Fix bug: keep track of removed executors properly
73ade46 [Andrew Or] Wording changes (minor)
2a7a6da [Andrew Or] Add `sc.killExecutor` as a shorthand (minor)
665f229 [Andrew Or] Mima excludes
79aa2df [Andrew Or] Simplify the request interface by asking for a total
04f625b [Andrew Or] Fix race condition that causes over-allocation of executors
f4783f8 [Andrew Or] Change the semantics of requesting executors
005a124 [Andrew Or] Fix tests
4628b16 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
db4a679 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
572f5c5 [Andrew Or] Unused import (minor)
f30261c [Andrew Or] Kill multiple executors rather than one at a time
de260d9 [Andrew Or] Simplify by skipping useless null check
9c52542 [Andrew Or] Simplify by skipping the TaskSchedulerImpl
97dd1a8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism
d987b3e [Andrew Or] Move addWebUIFilters to Yarn scheduler backend
7b76d0a [Andrew Or] Expose mechanism in SparkContext as developer API
47466cd [Andrew Or] Refactor common Yarn scheduler backend logic
c4dfaac [Andrew Or] Avoid thrashing when removing executors
53e8145 [Andrew Or] Start yarn actor early to listen for AM registration message
bbee669 [Andrew Or] Add mechanism in yarn client mode
2014-10-29 14:01:00 -07:00
Kousuke Saruta 247c529b35 [SPARK-3657] yarn alpha YarnRMClientImpl throws NPE appMasterRequest.setTrackingUrl starting spark-shell
tgravescs reported this issue.

Following is quoted from tgravescs' report.

YarnRMClientImpl.registerApplicationMaster can throw null pointer exception when setting the trackingurl if its empty:

    appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority())

I hit this just start spark-shell without the tracking url set.

14/09/23 16:18:34 INFO yarn.YarnRMClientImpl: Connecting to ResourceManager at kryptonitered-jt1.red.ygrid.yahoo.com/98.139.154.99:8030
Exception in thread "main" java.lang.NullPointerException
        at org.apache.hadoop.yarn.proto.YarnServiceProtos$RegisterApplicationMasterRequestProto$Builder.setTrackingUrl(YarnServiceProtos.java:710)
        at org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl.setTrackingUrl(RegisterApplicationMasterRequestPBImpl.java:132)
        at org.apache.spark.deploy.yarn.YarnRMClientImpl.registerApplicationMaster(YarnRMClientImpl.scala:102)
        at org.apache.spark.deploy.yarn.YarnRMClientImpl.register(YarnRMClientImpl.scala:55)
        at org.apache.spark.deploy.yarn.YarnRMClientImpl.register(YarnRMClientImpl.scala:38)
        at org.apache.spark.deploy.yarn.ApplicationMaster.registerAM(ApplicationMaster.scala:168)
        at org.apache.spark.deploy.yarn.ApplicationMaster.runExecutorLauncher(ApplicationMaster.scala:206)
        at org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:120)

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2981 from sarutak/SPARK-3657-2 and squashes the following commits:

e2fd6bc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3657
70b8882 [Kousuke Saruta] Fixed NPE thrown
2014-10-28 12:37:09 -07:00
WangTaoTheTonic 1ea3e3dc9d [SPARK-4096][YARN]let ApplicationMaster accept executor memory argument in same format as JVM memory strings
Here `ApplicationMaster` accept executor memory argument only in number format, we should let it accept JVM style memory strings as well.

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #2955 from WangTaoTheTonic/modifyDesc and squashes the following commits:

ab98c70 [WangTaoTheTonic] append parameter passed in
3779767 [WangTaoTheTonic] Update executor memory description in the help message
2014-10-28 12:31:42 -07:00
WangTaoTheTonic 0ac52e3055 [SPARK-4098][YARN]use appUIAddress instead of appUIHostPort in yarn-client mode
https://issues.apache.org/jira/browse/SPARK-4098

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #2958 from WangTaoTheTonic/useAddress and squashes the following commits:

29236e6 [WangTaoTheTonic] use appUIAddress instead of appUIHostPort in yarn-cluster mode
2014-10-28 09:51:44 -05:00
WangTaoTheTonic e8813be653 [SPARK-4095][YARN][Minor]extract val isLaunchingDriver in ClientBase
Instead of checking if `args.userClass` is null repeatedly, we extract it to an global val as in `ApplicationMaster`.

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #2954 from WangTaoTheTonic/MemUnit and squashes the following commits:

13bda20 [WangTaoTheTonic] extract val isLaunchingDriver in ClientBase
2014-10-28 08:53:10 -05:00
WangTaoTheTonic 47346cd029 [SPARK-4116][YARN]Delete the abandoned log4j-spark-container.properties
Since its name reduced at https://github.com/apache/spark/pull/560, the log4j-spark-container.properties was never used again.
And I have searched its name globally in code and found no cite.

Author: WangTaoTheTonic <barneystinson@aliyun.com>

Closes #2977 from WangTaoTheTonic/delLog4j and squashes the following commits:

fb2729f [WangTaoTheTonic] delete the log4j file obsoleted
2014-10-28 08:46:31 -05:00
Prashant Sharma c9e05ca27c [SPARK-4032] Deprecate YARN alpha support in Spark 1.2
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #2878 from ScrapCodes/SPARK-4032/deprecate-yarn-alpha and squashes the following commits:

17e9857 [Prashant Sharma] added deperecated comment to Client and ExecutorRunnable.
3a34b1e [Prashant Sharma] Updated docs...
4608dea [Prashant Sharma] [SPARK-4032] Deprecate YARN alpha support in Spark 1.2
2014-10-27 10:02:48 -07:00
Kousuke Saruta d2987e8f7a [SPARK-3900][YARN] ApplicationMaster's shutdown hook fails and IllegalStateException is thrown.
ApplicationMaster registers a shutdown hook and it calls ApplicationMaster#cleanupStagingDir.

cleanupStagingDir invokes FileSystem.get(yarnConf) and it invokes FileSystem.getInternal. FileSystem.getInternal also registers shutdown hook.
In FileSystem of hadoop 0.23, the shutdown hook registration does not consider whether shutdown is in progress or not (In 2.2, it's considered).

    // 0.23
    if (map.isEmpty() ) {
      ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY);
    }

    // 2.2
    if (map.isEmpty()
                && !ShutdownHookManager.get().isShutdownInProgress()) {
       ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY);
    }

Thus, in 0.23, another shutdown hook can be registered when ApplicationMaster's shutdown hook run.

This issue cause IllegalStateException as follows.

    java.lang.IllegalStateException: Shutdown in progress, cannot add a shutdownHook
            at org.apache.hadoop.util.ShutdownHookManager.addShutdownHook(ShutdownHookManager.java:152)
            at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2306)
            at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2278)
            at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:316)
            at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:162)
            at org.apache.spark.deploy.yarn.ApplicationMaster.org$apache$spark$deploy$yarn$ApplicationMaster$$cleanupStagingDir(ApplicationMaster.scala:307)
            at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$3.run(ApplicationMaster.scala:118)
            at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2924 from sarutak/SPARK-3900-2 and squashes the following commits:

9112817 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900-2
97018fa [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900
2c2850e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900
ee52db2 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900
a7d6c9b [Kousuke Saruta] Merge branch 'SPARK-3900' of github.com:sarutak/spark into SPARK-3900
1cdf03c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900
a5f6443 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900
57b397d [Kousuke Saruta] Fixed IllegalStateException caused by shutdown hook registration in another shutdown hook
2014-10-24 08:51:08 -05:00
zsxwing 137d942353 [SPARK-3877][YARN] Throw an exception when application is not successful so that the exit code wil be set to 1
When an yarn application fails (yarn-cluster mode), the exit code of spark-submit is still 0. It's hard for people to write some automatic scripts to run spark jobs in yarn because the failure can not be detected in these scripts.

This PR added a status checking after `monitorApplication`. If an application is not successful, `run()` will throw an `SparkException`, so that Client.scala will exit with code 1. Therefore, people can use the exit code of `spark-submit` to write some automatic scripts.

Author: zsxwing <zsxwing@gmail.com>

Closes #2732 from zsxwing/SPARK-3877 and squashes the following commits:

1f89fa5 [zsxwing] Fix the unit test
a0498e1 [zsxwing] Update the docs and the error message
e1cb9ef [zsxwing] Fix the hacky way of calling Client
ff16fec [zsxwing] Remove System.exit in Client.scala and add a test
6a2c103 [zsxwing] [SPARK-3877] Throw an exception when application is not successful so that the exit code wil be set to 1
2014-10-22 15:04:41 -07:00
Marcelo Vanzin 803e7f0877 [SPARK-3979] [yarn] Use fs's default replication.
This avoids issues when HDFS is configured in a way that would not
allow the hardcoded default replication of "3".

Note: getDefaultReplication(Path) was added in 0.23.3, and the oldest
one available on Maven Central is 0.23.7, so I chose to not add code
to access that method via reflection.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2831 from vanzin/SPARK-3979 and squashes the following commits:

b0e3a97 [Marcelo Vanzin] [SPARK-3979] [yarn] Use fs's default replication.
2014-10-17 13:45:10 -05:00
Andrew Or c86c976037 [HOTFIX] Fix compilation error for Yarn 2.0.*-alpha
This was reported in https://issues.apache.org/jira/browse/SPARK-3445. There are API differences between the 0.23.* and the 2.0.*-alpha branches that are not accounted for when this code was introduced.

Author: Andrew Or <andrewor14@gmail.com>

Closes #2776 from andrewor14/fix-yarn-alpha and squashes the following commits:

ec94752 [Andrew Or] Fix compilation error for 2.0.*-alpha
2014-10-12 15:41:27 -07:00
Sean Owen 363baacade SPARK-3811 [CORE] More robust / standard Utils.deleteRecursively, Utils.createTempDir
I noticed a few issues with how temp directories are created and deleted:

*Minor*

* Guava's `Files.createTempDir()` plus `File.deleteOnExit()` is used in many tests to make a temp dir, but `Utils.createTempDir()` seems to be the standard Spark mechanism
* Call to `File.deleteOnExit()` could be pushed into `Utils.createTempDir()` as well, along with this replacement
* _I messed up the message in an exception in `Utils` in SPARK-3794; fixed here_

*Bit Less Minor*

* `Utils.deleteRecursively()` fails immediately if any `IOException` occurs, instead of trying to delete any remaining files and subdirectories. I've observed this leave temp dirs around. I suggest changing it to continue in the face of an exception and throw one of the possibly several exceptions that occur at the end.
* `Utils.createTempDir()` will add a JVM shutdown hook every time the method is called. Even if the subdir is the parent of another parent dir, since this check is inside the hook. However `Utils` manages a set of all dirs to delete on shutdown already, called `shutdownDeletePaths`. A single hook can be registered to delete all of these on exit. This is how Tachyon temp paths are cleaned up in `TachyonBlockManager`.

I noticed a few other things that might be changed but wanted to ask first:

* Shouldn't the set of dirs to delete be `File`, not just `String` paths?
* `Utils` manages the set of `TachyonFile` that have been registered for deletion, but the shutdown hook is managed in `TachyonBlockManager`. Should this logic not live together, and not in `Utils`? it's more specific to Tachyon, and looks a slight bit odd to import in such a generic place.

Author: Sean Owen <sowen@cloudera.com>

Closes #2670 from srowen/SPARK-3811 and squashes the following commits:

071ae60 [Sean Owen] Update per @vanzin's review
da0146d [Sean Owen] Make Utils.deleteRecursively try to delete all paths even when an exception occurs; use one shutdown hook instead of one per method call to delete temp dirs
3a0faa4 [Sean Owen] Standardize on Utils.createTempDir instead of Files.createTempDir
2014-10-09 18:21:59 -07:00
Kousuke Saruta f18dd5962e [SPARK-3848] yarn alpha doesn't build on master
yarn alpha build was broken by #2432
as it added an argument to YarnAllocator but not to yarn/alpha YarnAllocationHandler
commit 79e45c9323

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2715 from sarutak/SPARK-3848 and squashes the following commits:

bafb8d1 [Kousuke Saruta] Fixed parameters for the default constructor of alpha/YarnAllocatorHandler.
2014-10-08 11:53:43 -05:00
Marcelo Vanzin 7fca8f41c8 [SPARK-3788] [yarn] Fix compareFs to do the right thing for HDFS namespaces.
HA and viewfs use namespaces instead of host names, so you can't
resolve them since that will fail. So be smarter to avoid doing
unnecessary work.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2649 from vanzin/SPARK-3788 and squashes the following commits:

fedbc73 [Marcelo Vanzin] Update comment.
c938845 [Marcelo Vanzin] Use Objects.equal() to avoid issues with ==.
9f7b571 [Marcelo Vanzin] [SPARK-3788] [yarn] Fix compareFs to do the right thing for HA, federation.
2014-10-08 08:48:55 -05:00
Marcelo Vanzin 35afdfd624 [SPARK-3710] Fix Yarn integration tests on Hadoop 2.2.
It seems some dependencies are not declared when pulling the 2.2
test dependencies, so we need to add them manually for the Yarn
cluster to come up.

These don't seem to be necessary for 2.3 and beyond, so restrict
them to the hadoop-2.2 profile.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2682 from vanzin/SPARK-3710 and squashes the following commits:

701d4fb [Marcelo Vanzin] Add comment.
0540bdf [Marcelo Vanzin] [SPARK-3710] Fix Yarn integration tests on Hadoop 2.2.
2014-10-07 23:26:24 -07:00
Thomas Graves 70e824f750 [SPARK-3627] - [yarn] - fix exit code and final status reporting to RM
See the description and whats handled in the jira comment: https://issues.apache.org/jira/browse/SPARK-3627?focusedCommentId=14150013&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14150013

This does not handle yarn client mode reporting of the driver to the AM.   I think that should be handled when we make it an unmanaged AM.

Author: Thomas Graves <tgraves@apache.org>

Closes #2577 from tgravescs/SPARK-3627 and squashes the following commits:

9c2efbf [Thomas Graves] review comments
e8cc261 [Thomas Graves] fix accidental typo during fixing comment
24c98e3 [Thomas Graves] rework
85f1901 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into SPARK-3627
fab166d [Thomas Graves] update based on review comments
32f4dfa [Thomas Graves] switch back
f0b6519 [Thomas Graves] change order of cleanup staging dir
d3cc800 [Thomas Graves] SPARK-3627 - yarn - fix exit code and final status reporting to RM
2014-10-07 09:51:37 -05:00
Kousuke Saruta 79e45c9323 [SPARK-3377] [SPARK-3610] Metrics can be accidentally aggregated / History server log name should not be based on user input
This PR is another solution for #2250

I'm using codahale base MetricsSystem of Spark with JMX or Graphite, and I saw following 2 problems.

(1) When applications which have same spark.app.name run on cluster at the same time, some metrics names are mixed. For instance, if 2+ application is running on the cluster at the same time, each application emits the same named metric like "SparkPi.DAGScheduler.stage.failedStages" and Graphite cannot distinguish the metrics is for which application.

(2) When 2+ executors run on the same machine, JVM metrics of each executors are mixed. For instance, 2+ executors running on the same node can emit the same named metric "jvm.memory" and Graphite cannot distinguish the metrics is from which application.

And there is an similar issue. The directory for event logs is named using application name.
Application name is defined by user and the name can includes illegal character for path names.
Further more, the directory name consists of application name and System.currentTimeMillis even though each application has unique Application ID so if we run jobs which have same name, it's difficult to identify which directory is for which application.

Closes #2250
Closes #1067

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2432 from sarutak/metrics-structure-improvement2 and squashes the following commits:

3288b2b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
39169e4 [Kousuke Saruta] Fixed style
6570494 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
817e4f0 [Kousuke Saruta] Simplified MetricsSystem#buildRegistryName
67fa5eb [Kousuke Saruta] Unified MetricsSystem#registerSources and registerSinks in start
10be654 [Kousuke Saruta] Fixed style.
990c078 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
f0c7fba [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
59cc2cd [Kousuke Saruta] Modified SparkContextSchedulerCreationSuite
f9b6fb3 [Kousuke Saruta] Modified style.
2cf8a0f [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
389090d [Kousuke Saruta] Replaced taskScheduler.applicationId() with getApplicationId in SparkContext#postApplicationStart
ff45c89 [Kousuke Saruta] Added some test cases to MetricsSystemSuite
69c46a6 [Kousuke Saruta] Added warning logging logic to MetricsSystem#buildRegistryName
5cca0d2 [Kousuke Saruta] Added Javadoc comment to SparkContext#getApplicationId
16a9f01 [Kousuke Saruta] Added data types to be returned to some methods
6434b06 [Kousuke Saruta] Reverted changes related to ApplicationId
0413b90 [Kousuke Saruta] Deleted ApplicationId.java and ApplicationIdSuite.java
a42300c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
0fc1b09 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
42bea55 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
248935d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
f6af132 [Kousuke Saruta] Modified SchedulerBackend and TaskScheduler to return System.currentTimeMillis as an unique Application Id
1b8b53e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
97cb85c [Kousuke Saruta] Modified confliction of MimExcludes
2cdd009 [Kousuke Saruta] Modified defailt implementation of applicationId
9aadb0b [Kousuke Saruta] Modified NetworkReceiverSuite to ensure "executor.start()" is finished in test "network receiver life cycle"
3011efc [Kousuke Saruta] Added ApplicationIdSuite.scala
d009c55 [Kousuke Saruta] Modified ApplicationId#equals to compare appIds
dfc83fd [Kousuke Saruta] Modified ApplicationId to implement Serializable
9ff4851 [Kousuke Saruta] Modified MimaExcludes.scala to ignore createTaskScheduler method in SparkContext
4567ffc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
6a91b14 [Kousuke Saruta] Modified SparkContextSchedulerCreationSuite, ExecutorRunnerTest and EventLoggingListenerSuite
0325caf [Kousuke Saruta] Added ApplicationId.scala
0a2fc14 [Kousuke Saruta] Modified style
eabda80 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
0f890e6 [Kousuke Saruta] Modified SparkDeploySchedulerBackend and Master to pass baseLogDir instead f eventLogDir
bcf25bf [Kousuke Saruta] Modified directory name for EventLogs
28d4d93 [Kousuke Saruta] Modified SparkContext and EventLoggingListener so that the directory for EventLogs is named same for Application ID
203634e [Kousuke Saruta] Modified comment in SchedulerBackend#applicationId and TaskScheduler#applicationId
424fea4 [Kousuke Saruta] Modified  the subclasses of TaskScheduler and SchedulerBackend so that they can return non-optional Unique Application ID
b311806 [Kousuke Saruta] Swapped last 2 arguments passed to CoarseGrainedExecutorBackend
8a2b6ec [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
086ee25 [Kousuke Saruta] Merge branch 'metrics-structure-improvement2' of github.com:sarutak/spark into metrics-structure-improvement2
e705386 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
36d2f7a [Kousuke Saruta] Added warning message for the situation we cannot get application id for the prefix for the name of metrics
eea6e19 [Kousuke Saruta] Modified CoarseGrainedMesosSchedulerBackend and MesosSchedulerBackend so that we can get Application ID
c229fbe [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
e719c39 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
4a93c7f [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2
4776f9e [Kousuke Saruta] Modified MetricsSystemSuite.scala
efcb6e1 [Kousuke Saruta] Modified to add application id to metrics name
2ec848a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
3ea7896 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
ead8966 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
08e627e [Kousuke Saruta] Revert "tmp"
7b67f5a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
45bd33d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
93e263a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
848819c [Kousuke Saruta] Merge branch 'metrics-structure-improvement' of github.com:sarutak/spark into metrics-structure-improvement
912a637 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
e4a4593 [Kousuke Saruta] tmp
3e098d8 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
4603a39 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
fa7175b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
15f88a3 [Kousuke Saruta] Modified MetricsSystem#buildRegistryName because conf.get does not return null when correspondin entry is absent
6f7dcd4 [Kousuke Saruta] Modified constructor of DAGSchedulerSource and BlockManagerSource because the instance of SparkContext is no longer used
6fc5560 [Kousuke Saruta] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource
4e057c9 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement
85ffc02 [Kousuke Saruta] Revert "Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource"
868e326 [Kousuke Saruta] Modified MetricsSystem to set registry name with unique application-id and driver/executor-id
71609f5 [Kousuke Saruta] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource
55debab [Kousuke Saruta] Modified SparkContext and Executor to set spark.executor.id to identifiers
4180993 [Kousuke Saruta] Modified SparkContext to retain spark.unique.app.name property in SparkConf
2014-10-03 13:48:56 -07:00
Marcelo Vanzin 30abef1547 [SPARK-3606] [yarn] Correctly configure AmIpFilter for Yarn HA.
The existing code only considered one of the RMs when running in
Yarn HA mode, so it was possible to get errors if the active RM
was not registered in the filter.

The change makes use of a new API added to Yarn that returns all
proxy addresses, and falls back to the old behavior if the API
is not present. While there, I also made a change to look for the
scheme (http or https) being used by Yarn when building the proxy
URIs.

Since, in the case of multiple RMs, Yarn uses commas as a separator,
it was not possible anymore to use spark.filter.params to propagate
this information (which used commas to delimit different config params).
Instead, I added a new param (spark.filter.jsonParams) which expects
a JSON string containing a map with the config data. I chose not to
add it to the documentation at this point since I don't believe users
will use it directly.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2469 from vanzin/SPARK-3606 and squashes the following commits:

aeb458a [Marcelo Vanzin] Undelete needed import.
65e400d [Marcelo Vanzin] Remove unused import.
d121883 [Marcelo Vanzin] Use separate config for each param instead of json.
04bc156 [Marcelo Vanzin] Review feedback.
4d4d6b9 [Marcelo Vanzin] [SPARK-3606] [yarn] Correctly configure AmIpFilter for Yarn HA.
2014-10-03 13:18:35 -07:00
Marcelo Vanzin fbe8e9856b [SPARK-2778] [yarn] Add workaround for race in MiniYARNCluster.
Sometimes the cluster's start() method returns before the configuration
having been updated, which is done by ClientRMService in, I assume, a
separate thread (otherwise there would be no race). That can cause tests
to fail if the old configuration data is read, since it will contain
the wrong RM address.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2605 from vanzin/SPARK-2778 and squashes the following commits:

8d02ce0 [Marcelo Vanzin] Minor cleanup.
5bebee7 [Marcelo Vanzin] [SPARK-2778] [yarn] Add workaround for race in MiniYARNCluster.
2014-10-03 11:36:24 -07:00
Nishkam Ravi b4fb7b80a0 Modify default YARN memory_overhead-- from an additive constant to a multiplier
Redone against the recent master branch (https://github.com/apache/spark/pull/1391)

Author: Nishkam Ravi <nravi@cloudera.com>
Author: nravi <nravi@c1704.halxg.cloudera.com>
Author: nishkamravi2 <nishkamravi@gmail.com>

Closes #2485 from nishkamravi2/master_nravi and squashes the following commits:

636a9ff [nishkamravi2] Update YarnAllocator.scala
8f76c8b [Nishkam Ravi] Doc change for yarn memory overhead
35daa64 [Nishkam Ravi] Slight change in the doc for yarn memory overhead
5ac2ec1 [Nishkam Ravi] Remove out
dac1047 [Nishkam Ravi] Additional documentation for yarn memory overhead issue
42c2c3d [Nishkam Ravi] Additional changes for yarn memory overhead issue
362da5e [Nishkam Ravi] Additional changes for yarn memory overhead
c726bd9 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
f00fa31 [Nishkam Ravi] Improving logging for AM memoryOverhead
1cf2d1e [nishkamravi2] Update YarnAllocator.scala
ebcde10 [Nishkam Ravi] Modify default YARN memory_overhead-- from an additive constant to a multiplier (redone to resolve merge conflicts)
2e69f11 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi
efd688a [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark
2b630f9 [nravi] Accept memory input as "30g", "512M" instead of an int value, to be consistent with rest of Spark
3bf8fad [nravi] Merge branch 'master' of https://github.com/apache/spark
5423a03 [nravi] Merge branch 'master' of https://github.com/apache/spark
eb663ca [nravi] Merge branch 'master' of https://github.com/apache/spark
df2aeb1 [nravi] Improved fix for ConcurrentModificationIssue (Spark-1097, Hadoop-10456)
6b840f0 [nravi] Undo the fix for SPARK-1758 (the problem is fixed)
5108700 [nravi] Fix in Spark for the Concurrent thread modification issue (SPARK-1097, HADOOP-10456)
681b36f [nravi] Fix for SPARK-1758: failing test org.apache.spark.JavaAPISuite.wholeTextFiles
2014-10-02 13:48:35 -05:00
Reynold Xin 3888ee2f38 [SPARK-3748] Log thread name in unit test logs
Thread names are useful for correlating failures.

Author: Reynold Xin <rxin@apache.org>

Closes #2600 from rxin/log4j and squashes the following commits:

83ffe88 [Reynold Xin] [SPARK-3748] Log thread name in unit test logs
2014-10-01 01:03:49 -07:00
Patrick Wendell 157e7d0f62 HOTFIX: Ignore flaky tests in YARN 2014-09-30 09:46:58 -07:00
Andrew Or 8da10bf146 [SPARK-3476] Remove outdated memory checks in Yarn
See description in [JIRA](https://issues.apache.org/jira/browse/SPARK-3476).

Author: Andrew Or <andrewor14@gmail.com>

Closes #2528 from andrewor14/yarn-memory-checks and squashes the following commits:

c5400cd [Andrew Or] Simplify checks
e30ffac [Andrew Or] Remove outdated memory checks
2014-09-26 11:50:48 -07:00
Marcelo Vanzin b8487713d3 [SPARK-2778] [yarn] Add yarn integration tests.
This patch adds a couple of, currently, very simple integration tests
to make sure both client and cluster modes are working. The tests don't
do much yet other than run a simple job, but the plan is to enhance
them after we get the framework in.

The cluster tests are noisy, so redirect all log output to a file
like other tests do. Copying the conf around sucks but it's less
work than messing with maven/sbt and having to clean up other
projects.

Note the test is only added for yarn-stable. The code compiles
against yarn-alpha but there are two issues I ran into that I
could not overcome:
- an old netty dependency kept creeping into the classpath and
  causing akka to not work, when using sbt; the old netty was
  correctly suppressed under maven.
- MiniYARNCluster kept failing to execute containers because it
  did not create the NM's local dir itself; this is apparently
  a known behavior, but I'm not sure how to work around it.

None of those issues are present with the stable Yarn.

Also, these tests are a little slow to run. Apparently Spark doesn't
yet tag tests (so that these could be isolated in a "slow" batch),
so this is something to keep in mind.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2257 from vanzin/yarn-tests and squashes the following commits:

6d5b84e [Marcelo Vanzin] Fix wrong system property being set.
8b0933d [Marcelo Vanzin] Merge branch 'master' into yarn-tests
5c2b56f [Marcelo Vanzin] Use custom log4j conf for Yarn containers.
ec73f17 [Marcelo Vanzin] More review feedback.
67f5b02 [Marcelo Vanzin] Review feedback.
f01517c [Marcelo Vanzin] Review feedback.
68fbbbf [Marcelo Vanzin] Use older constructor available in older Hadoop releases.
d07ef9a [Marcelo Vanzin] Merge branch 'master' into yarn-tests
add8416 [Marcelo Vanzin] [SPARK-2778] [yarn] Add yarn integration tests.
2014-09-24 23:10:26 -07:00
Kousuke Saruta 11c10df825 [SPARK-3304] [YARN] ApplicationMaster's Finish status is wrong when uncaught exception is thrown from ReporterThread
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2198 from sarutak/SPARK-3304 and squashes the following commits:

2696237 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
5b80363 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
4eb0a3e [Kousuke Saruta] Remoed the description about spark.yarn.scheduler.reporterThread.maxFailure
9741597 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
f7538d4 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
358ef8d [Kousuke Saruta] Merge branch 'SPARK-3304' of github.com:sarutak/spark into SPARK-3304
0d138c6 [Kousuke Saruta] Revert "tmp"
f8da10a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
b6e9879 [Kousuke Saruta] tmp
8d256ed [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
13b2652 [Kousuke Saruta] Merge branch 'SPARK-3304' of github.com:sarutak/spark into SPARK-3304
2711e15 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
c081f8e [Kousuke Saruta] Modified ApplicationMaster to handle exception in ReporterThread itself
0bbd3a6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304
a6982ad [Kousuke Saruta] Added ability handling uncaught exception thrown from Reporter thread
2014-09-23 11:40:14 -05:00
Andrew Or c4022dd52b [SPARK-3477] Clean up code in Yarn Client / ClientBase
This is part of a broader effort to clean up the Yarn integration code after #2020.

The high-level changes in this PR include:
- Removing duplicate code, especially across the alpha and stable APIs
- Simplify unnecessarily complex method signatures and hierarchies
- Rename unclear variable and method names
- Organize logging output produced when the user runs Spark on Yarn
- Extensively add documentation
- Privatize classes where possible

I have tested the stable API on a Hadoop 2.4 cluster. I tested submitting a jar that references classes in other jars in both client and cluster mode. I also made changes in the alpha API, though I do not have access to an alpha cluster. I have verified that it compiles, but it would be ideal if others can help test it.

For those interested in some examples in detail, please read on.

--------------------------------------------------------------------------------------------------------

***Appendix***

- The loop to `getApplicationReport` from the RM is duplicated in 4 places: in the stable `Client`, alpha `Client`, and twice in `YarnClientSchedulerBackend`. We should not have different loops for client and cluster deploy modes.
- There are many fragmented small helper methods that are only used once and should just be inlined. For instance, `ClientBase#getLocalPath` returns `null` on certain conditions, and its only caller `ClientBase#addFileToClasspath` checks whether the value returned is `null`. We could just have the caller check on that same condition to avoid passing `null`s around.
- In `YarnSparkHadoopUtil#addToEnvironment`, we take in an argument `classpathSeparator` that always has the same value upstream (i.e. `File.pathSeparator`). This argument is now removed from the signature and all callers of this method upstream.
- `ClientBase#copyRemoteFile` is now renamed to `copyFileToRemote`. It was unclear whether we are copying a remote file to our local file system, or copying a locally visible file to a remote file system. Also, even the content of the method has inaccurately named variables. We use `val remoteFs` to signify the file system of the locally visible file and `val fs` to signify the remote, destination file system. These are now renamed `srcFs` and `destFs` respectively.
- We currently log the AM container's environment and resource mappings directly as Scala collections. This is incredibly hard to read and probably too verbose for the average Spark user. In other modes (e.g. standalone), we also don't log the launch commands by default, so the logging level of these information is now set to `DEBUG`.
- None of these classes (`Client`, `ClientBase`, `YarnSparkHadoopUtil` etc.) is intended to be used by a Spark application (the user should go through Spark submit instead). At the very least they should be `private[spark]`.

Author: Andrew Or <andrewor14@gmail.com>

Closes #2350 from andrewor14/yarn-cleanup and squashes the following commits:

39e8c7b [Andrew Or] Address review comments
6619f9b [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-cleanup
2ca6d64 [Andrew Or] Improve logging in application monitor
a3b9693 [Andrew Or] Minor changes
7dd6298 [Andrew Or] Simplify ClientBase#monitorApplication
547487c [Andrew Or] Provide default values for null application report entries
a0ad1e9 [Andrew Or] Fix class not found error
1590141 [Andrew Or] Address review comments
45ccdea [Andrew Or] Remove usages of getAMMemory
d8e33b6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-cleanup
ed0b42d [Andrew Or] Fix alpha compilation error
c0587b4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-cleanup
6d74888 [Andrew Or] Minor comment changes
6573c1d [Andrew Or] Clean up, simplify and document code for setting classpaths
e4779b6 [Andrew Or] Clean up log messages + variable naming in ClientBase
8766d37 [Andrew Or] Heavily add documentation to Client* classes + various clean-ups
6c94d79 [Andrew Or] Various cleanups in ClientBase and ClientArguments
ef7069a [Andrew Or] Clean up YarnClientSchedulerBackend more
6de9072 [Andrew Or] Guard against potential NPE in debug logging mode
fabe4c4 [Andrew Or] Reuse more code in YarnClientSchedulerBackend
3f941dc [Andrew Or] First cut at simplifying the Client (stable and alpha)
2014-09-23 11:20:52 -05:00
peng.zhang 14f8c34040 [YARN] SPARK-2668: Add variable of yarn log directory for reference from the log4j configuration
Assign value of yarn container log directory to java opts "spark.yarn.app.container.log.dir", So user defined log4j.properties can reference this value and write log to YARN container's log directory.
Otherwise, user defined file appender will only write to container's CWD, and log files in CWD will not be displayed on YARN UI,and either cannot be aggregated to HDFS log directory after job finished.

User defined log4j.properties reference example:
log4j.appender.rolling_file.File = ${spark.yarn.app.container.log.dir}/spark.log

Author: peng.zhang <peng.zhang@xiaomi.com>

Closes #1573 from renozhang/yarn-log-dir and squashes the following commits:

16c5cb8 [peng.zhang] Update doc
f2b5e2a [peng.zhang] Change variable's name, and update running-on-yarn.md
503ea2d [peng.zhang] Support log4j log to yarn container dir
2014-09-23 08:45:56 -05:00
chesterxgchen 7d1a37239c SPARK-3177 (on Master Branch)
The JIRA and PR was original created for branch-1.1, and move to master branch now.
Chester

The Issue is due to that yarn-alpha and yarn have different APIs for certain class fields. In this particular case,  the ClientBase using reflection to to address this issue, and we need to different way to test the ClientBase's method.  Original ClientBaseSuite using getFieldValue() method to do this. But it doesn't work for yarn-alpha as the API returns an array of String instead of just String (which is the case for Yarn-stable API).

 To fix the test, I add a new method

  def getFieldValue2[A: ClassTag, A1: ClassTag, B](clazz: Class[_], field: String,
                                                      defaults: => B)
                              (mapTo:  A => B)(mapTo1: A1 => B) : B =
    Try(clazz.getField(field)).map(_.get(null)).map {
      case v: A => mapTo(v)
      case v1: A1 => mapTo1(v1)
      case _ => defaults
    }.toOption.getOrElse(defaults)

to handle the cases where the field type can be either type A or A1. In this new method the type A or A1 is pattern matched and corresponding mapTo function (mapTo or mapTo1) is used.

Author: chesterxgchen <chester@alpinenow.com>

Closes #2204 from chesterxgchen/SPARK-3177-master and squashes the following commits:

e72a6ea [chesterxgchen]  The Issue is due to that yarn-alpha and yarn have different APIs for certain class fields. In this particular case,  the ClientBase using reflection to to address this issue, and we need to different way to test the ClientBase's method.  Original ClientBaseSuite using getFieldValue() method to do this. But it doesn't work for yarn-alpha as the API returns an array of String instead of just String (which is the case for Yarn-stable API).
2014-09-17 10:25:52 -05:00