Hi Dotan, Thank you. You will get more logs by looking at YARN's log. In your situation, you will find more information in /var/log/hadoop-yarn/container or by running *yarn logs -applicationId yourAppId*. Let me know if you find anything. Thank you. Cheers,
Fang, Yan [email protected] +1 (206) 849-4108 On Wed, Jun 25, 2014 at 8:23 AM, Dotan Patrich <[email protected]> wrote: > hi, > > Thank you very much for the help, I did try and compiled samza with yarn > version 2.3.0 and managed to progress a bit. > Now the application master seems to be loaded but the task still does run. > > This is the error logged to the stdout of the application container, I > don't really know how to debug it so any idea and help will be appreciated > > 18:17:51.816 [main] INFO o.a.s.j.y.SamzaAppMasterTaskManager - > Container container_1403682668290_0013_01_000003 failed with exit code > 1 - Exception from container-launch: > org.apache.hadoop.util.Shell$ExitCodeException: > org.apache.hadoop.util.Shell$ExitCodeException: > at org.apache.hadoop.util.Shell.runCommand(Shell.java:505) > at org.apache.hadoop.util.Shell.run(Shell.java:418) > at > org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:650) > at > org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:195) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:300) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:81) > 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:745) > > > Container exited with a non-zero exit code 1 > > > Thanks! > > > > On Wed, Jun 25, 2014 at 1:08 PM, Zhijie Shen <[email protected]> > wrote: > > > I think one solution is to compile samza from source, and in > > {path_to_samza}/gradle/dependency-versions.gradle, change yarnVersion to > > 2.3.0. > > > > > > On Wed, Jun 25, 2014 at 5:50 PM, Dotan Patrich <[email protected]> > > wrote: > > > > > Thank you very much for the answer, I was afraid this was the reason... > > > > > > Is there any way to support YARN 2.3.0? for example, implementing my > own > > > YarnAppMaster? > > > Did anyone managed to run samza on CDH 4 or CDH 5 ? > > > > > > > > > On Wed, Jun 25, 2014 at 12:46 PM, Zhijie Shen <[email protected]> > > > wrote: > > > > > > > The problem might be as follows: > > > > > > > > 1. On the one hand, Samza 0.7.0 was compiled against YARN 2.2.0; > > > > 2. CDH 5.0.1 is mainly based on YARN 2.3.0, and > > > > AllocateRequest.setIncreaseRequests > > > > is new method introduced into YARN since 2.3.0 > > > > > > > > Unfortunately, you're running into a compatibility issue. > > > > > > > > > > > > On Wed, Jun 25, 2014 at 5:26 PM, Dotan Patrich <[email protected] > > > > > > wrote: > > > > > > > > > Hi, > > > > > I'm running on version 0.7.0 > > > > > > > > > > > > > > > On Wed, Jun 25, 2014 at 12:24 PM, Zhijie Shen < > [email protected] > > > > > > > > wrote: > > > > > > > > > > > What version of samza are you using now? > > > > > > > > > > > > > > > > > > On Wed, Jun 25, 2014 at 4:55 PM, Dotan Patrich < > > [email protected] > > > > > > > > > > wrote: > > > > > > > > > > > > > Hi, > > > > > > > > > > > > > > I'm trying to run a very simple samza task over CDH 5.0.1 YARN. > > > > > > > The streaming task actually does nothing... just wanted to test > > it > > > > > before > > > > > > > deploying a task that uses state and generate metrics. > > > > > > > > > > > > > > The task fails after 20 seconds of submitting it to YARN. > > > > > > > > > > > > > > This is the task configuration: > > > > > > > # Job > > > > > > > job.factory.class=org.apache.samza.job.yarn.YarnJobFactory > > > > > > > job.name=foo-task > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > yarn.package.path=file:///home/cloudera/xxxx/xxx-streaming-1.0.0-SNAPSHOT-dist.tar.gz > > > > > > > > > > > > > > # Task > > > > > > > task.class=fortscale.streaming.task.FooTask > > > > > > > task.inputs=kafka.fortscale-4769-event-score > > > > > > > > > > > > > > # Serializers > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > serializers.registry.integer.class=org.apache.samza.serializers.IntegerSerdeFactory > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > serializers.registry.metrics.class=org.apache.samza.serializers.MetricsSnapshotSerdeFactory > > > > > > > > > > > > > > > > > > > > > # Systems > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory > > > > > > > systems.kafka.samza.msg.serde=string > > > > > > > systems.kafka.consumer.zookeeper.connect=localhost:2181 > > > > > > > systems.kafka.consumer.auto.offset.reset=smallest > > > > > > > systems.kafka.producer.metadata.broker.list=localhost:9092 > > > > > > > systems.kafka.producer.producer.type=sync > > > > > > > # Normally, we'd set this much higher, but we want things to > look > > > > > snappy > > > > > > in > > > > > > > the demo. > > > > > > > systems.kafka.producer.batch.num.messages=1 > > > > > > > > > > > > > > > > > > > > > The only error I see in the stderr logs is that: > > > > > > > > > > > > > > Null identity service, trying login service: null > > > > > > > Finding identity service: null > > > > > > > Null identity service, trying login service: null > > > > > > > Finding identity service: null > > > > > > > Exception in thread "main" java.lang.AbstractMethodError: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest.setIncreaseRequests(Ljava/util/List;)V > > > > > > > at > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest.newInstance(AllocateRequest.java:81) > > > > > > > at > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest.newInstance(AllocateRequest.java:64) > > > > > > > at > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl.allocate(AMRMClientImpl.java:237) > > > > > > > at > > > > > > > > > org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:51) > > > > > > > at > > > > > > > > > > > > org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:81) > > > > > > > at > > > > > > > > > org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala) > > > > > > > > > > > > > > > > > > > > > Any idea what can be wrong ? > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > Zhijie Shen > > > > > > Hortonworks Inc. > > > > > > http://hortonworks.com/ > > > > > > > > > > > > -- > > > > > > CONFIDENTIALITY NOTICE > > > > > > NOTICE: This message is intended for the use of the individual or > > > > entity > > > > > to > > > > > > which it is addressed and may contain information that is > > > confidential, > > > > > > privileged and exempt from disclosure under applicable law. If > the > > > > reader > > > > > > of this message is not the intended recipient, you are hereby > > > notified > > > > > that > > > > > > any printing, copying, dissemination, distribution, disclosure or > > > > > > forwarding of this communication is strictly prohibited. If you > > have > > > > > > received this communication in error, please contact the sender > > > > > immediately > > > > > > and delete it from your system. Thank You. > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > Zhijie Shen > > > > Hortonworks Inc. > > > > http://hortonworks.com/ > > > > > > > > -- > > > > CONFIDENTIALITY NOTICE > > > > NOTICE: This message is intended for the use of the individual or > > entity > > > to > > > > which it is addressed and may contain information that is > confidential, > > > > privileged and exempt from disclosure under applicable law. If the > > reader > > > > of this message is not the intended recipient, you are hereby > notified > > > that > > > > any printing, copying, dissemination, distribution, disclosure or > > > > forwarding of this communication is strictly prohibited. If you have > > > > received this communication in error, please contact the sender > > > immediately > > > > and delete it from your system. Thank You. > > > > > > > > > > > > > > > -- > > Zhijie Shen > > Hortonworks Inc. > > http://hortonworks.com/ > > > > -- > > CONFIDENTIALITY NOTICE > > NOTICE: This message is intended for the use of the individual or entity > to > > which it is addressed and may contain information that is confidential, > > privileged and exempt from disclosure under applicable law. If the reader > > of this message is not the intended recipient, you are hereby notified > that > > any printing, copying, dissemination, distribution, disclosure or > > forwarding of this communication is strictly prohibited. If you have > > received this communication in error, please contact the sender > immediately > > and delete it from your system. Thank You. > > >
