[ https://issues.apache.org/jira/browse/SPARK-7700?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14550436#comment-14550436 ]
Kaveen Raajan commented on SPARK-7700: -------------------------------------- Hi [~srowen], Thanks for the update. Now I able to run Spark job successfully on both yarn-client and yarn-cluster mode. Here I changed following in Spark code. Replacing single quotes as double quotes *Line Number - 163* {code:title=YarnSparkHadoopUtil.scala|borderStyle=solid} def escapeForShell(arg: String): String = { if (arg != null) { val escaped = new StringBuilder("'") for (i <- 0 to arg.length() - 1) { arg.charAt(i) match { case '$' => escaped.append("\\$") case '"' => escaped.append("\\\"") case '\'' => escaped.append("'\\''") case c => escaped.append(c) } } escaped.append("'").toString() } else { arg } } {code} After changing above changes, we faced ISSUE – {color:red}Error: Could not find or load main class PWD.Syncfusion.BigDataSDK.2.1.0.70.SDK.Hadoop.logs.userlogs.application_1431950105623_0001.container_1431950105623_0001_01_000004{color} To resolve this we identified the issue reproducing location from Spark source as, Remove this " -XX:OnOutOfMemoryError='kill %p'" from commands. *Line Number - 213* {code:title=ExecutorRunnable.scala|borderStyle=solid} val commands = prefixEnv ++ Seq( YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", "-server", // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in // an inconsistent state. // TODO: If the OOM is not recoverable by rescheduling it on different node, then do // 'something' to fail job ... akin to blacklisting trackers in mapred ? " -XX:OnOutOfMemoryError='kill %p'") ++ javaOpts ++ Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", "--driver-url", masterAddress.toString, "--executor-id", slaveId.toString, "--hostname", hostname.toString, "--cores", executorCores.toString, "--app-id", appId) ++ userClassPath ++ Seq( "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") {code} Now We able to run all spark job in both yarn-cluster and yarn-client mode. I have a random question that Is there any equivalent patch available for following changes? why windows not accepted single quotes? and What the reason for giving this line " -XX:OnOutOfMemoryError='kill %p'"? > Spark 1.3.0 on YARN: Application failed 2 times due to AM Container > ------------------------------------------------------------------- > > Key: SPARK-7700 > URL: https://issues.apache.org/jira/browse/SPARK-7700 > Project: Spark > Issue Type: Story > Components: Build > Affects Versions: 1.3.1 > Environment: windows 8 Single language > Hadoop-2.5.2 > Protocol Buffer-2.5.0 > Scala-2.11 > Reporter: Kaveen Raajan > > I build SPARK on yarn mode by giving following command. Build got succeeded. > {panel} > mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-0.12.0 > -Phive-thriftserver -DskipTests clean package > {panel} > I set following property at spark-env.cmd file > {panel} > SET SPARK_JAR=hdfs://master:9000/user/spark/jar > {panel} > *Note:* spark jar files are moved to hdfs specified location. Also spark > classpath are added to hadoop-config.cmd and HADOOP_CONF_DIR are set at > enviroment variable. > I tried to execute following SparkPi example in yarn-cluster mode. > {panel} > spark-submit --class org.apache.spark.examples.SparkPi --master yarn-cluster > --num-executors 3 --driver-memory 4g --executor-memory 2g --executor-cores 1 > --queue default > S:\Hadoop\Spark\spark-1.3.1\examples\target\spark-examples_2.10-1.3.1.jar 10 > {panel} > My job able to submit at hadoop cluster, but it always in accepted state and > Failed with following error > {panel} > 15/05/14 13:00:51 INFO client.RMProxy: Connecting to ResourceManager at > /0.0.0.0:8032 > 15/05/14 13:00:51 INFO yarn.Client: Requesting a new application from cluster > with 1 NodeManagers > 15/05/14 13:00:51 INFO yarn.Client: Verifying our application has not > requestedmore than the maximum memory capability of the cluster (8048 MB per > container) > 15/05/14 13:00:51 INFO yarn.Client: Will allocate AM container, with 4480 MB > memory including 384 MB overhead > 15/05/14 13:00:51 INFO yarn.Client: Setting up container launch context for > ourAM > 15/05/14 13:00:51 INFO yarn.Client: Preparing resources for our AM container > 15/05/14 13:00:52 INFO yarn.Client: Source and destination file systems are > thesame. Not copying hdfs://master:9000/user/spark/jar > 15/05/14 13:00:52 INFO yarn.Client: Uploading resource > file:/S:/Hadoop/Spark/spark-1.3.1/examples/target/spark-examples_2.10-1.3.1.jar > -> > hdfs://master:9000/user/HDFS/.sparkStaging/application_1431587916618_0003/spark-examples_2.10-1.3.1.jar > 15/05/14 13:00:52 INFO yarn.Client: Setting up the launch environment for our > AM container > 15/05/14 13:00:52 INFO spark.SecurityManager: Changing view acls to: HDFS > 15/05/14 13:00:52 INFO spark.SecurityManager: Changing modify acls to: HDFS > 15/05/14 13:00:52 INFO spark.SecurityManager: SecurityManager: authentication > disabled; ui acls disabled; users with view permissions: Set(HDFS); users > with modify permissions: Set(HDFS) > 15/05/14 13:00:52 INFO yarn.Client: Submitting application 3 to > ResourceManager > 15/05/14 13:00:52 INFO impl.YarnClientImpl: Submitted application > application_1431587916618_0003 > 15/05/14 13:00:53 INFO yarn.Client: Application report for > application_1431587916618_0003 (state: ACCEPTED) > 15/05/14 13:00:53 INFO yarn.Client: > client token: N/A > diagnostics: N/A > ApplicationMaster host: N/A > ApplicationMaster RPC port: -1 > queue: default > start time: 1431588652790 > final status: UNDEFINED > tracking URL: > http://master:8088/proxy/application_1431587916618_0003/ > user: HDFS > 15/05/14 13:00:54 INFO yarn.Client: Application report for > application_1431587916618_0003 (state: ACCEPTED) > 15/05/14 13:00:55 INFO yarn.Client: Application report for > application_1431587916618_0003 (state: ACCEPTED) > 15/05/14 13:00:56 INFO yarn.Client: Application report for > application_1431587916618_0003 (state: ACCEPTED) > 15/05/14 13:00:57 INFO yarn.Client: Application report for > application_1431587916618_0003 (state: ACCEPTED) > 15/05/14 13:00:58 INFO yarn.Client: Application report for > application_1431587916618_0003 (state: ACCEPTED) > 15/05/14 13:00:59 INFO yarn.Client: Application report for > application_1431587916618_0003 (state: FAILED) > 15/05/14 13:00:59 INFO yarn.Client: > client token: N/A > diagnostics: Application application_1431587916618_0003 failed 2 > times > due to AM Container for appattempt_1431587916618_0003_000002 exited with > exitCode: 1 > For more detailed output, check application tracking > page:http://master:8088/proxy/application_1431587916618_0003/Then, click on > links to logs of each attempt. > Diagnostics: Exception from container-launch. > Container id: container_1431587916618_0003_02_000001 > Exit code: 1 > Stack trace: ExitCodeException exitCode=1: > at org.apache.hadoop.util.Shell.runCommand(Shell.java:538) > at org.apache.hadoop.util.Shell.run(Shell.java:455) > at > org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:715) > at > org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:211) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82) > at java.util.concurrent.FutureTask.run(FutureTask.java:262) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Shell output: 1 file(s) moved. > Container exited with a non-zero exit code 1 > Failing this attempt. Failing the application. > ApplicationMaster host: N/A > ApplicationMaster RPC port: -1 > queue: default > start time: 1431588652790 > final status: FAILED > tracking URL: > http://master:8088/cluster/app/application_1431587916618_0003 > user: HDFS > Exception in thread "main" org.apache.spark.SparkException: Application > finished with failed status > at > org.apache.spark.deploy.yarn.ClientBase$class.run(ClientBase.scala:522) > at org.apache.spark.deploy.yarn.Client.run(Client.scala:35) > at org.apache.spark.deploy.yarn.Client$.main(Client.scala:139) > at org.apache.spark.deploy.yarn.Client.main(Client.scala) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:360) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:76) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > {panel} > While debugging launch-container.cmd file I found that spark jar are not able > to access by AM. > {color:red} > Error: Could not find or load main class '-Dspark.executor.memory=2g' > {color} > If I tried to *run spark-shell in yarn-client* mean same issue found with > error message _yarn application already ended,might be killed or not able to > launch application._ > Similar Issues are available at > [http://blog.csdn.net/sunflower_cao/article/details/38046341] > [http://stackoverflow.com/questions/29392318/spark-1-3-0-on-yarn-application-failed-2-times-due-to-am-container] > Please guide me where I made a mistake to run hadoop on yarn mode. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org