You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by Nirmal Kumar <ni...@impetus.co.in> on 2013/11/07 14:05:18 UTC

Running Samza on multi node

All,

I was able to run the hello-samza application on a single node machine.
Now I am trying to run the hello-samza application on  a 2 node setup.

Node1 has a Resource Manager
Node2 has a Node Manager

The NM gets registered with the RM successfully as seen in rm.log of the RM node:
13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.
13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack
13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093
13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING
13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>

I am submitting the job from the RM machine using the command line:
bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties

However, I am getting the following exception after submitting the job to YARN:

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}
2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032
2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in
2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi
2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.
2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0
2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093
2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest
2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0
2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-
2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/
2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0
2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093
2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000
2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)
2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing
2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092
2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751
2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog
2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032
2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF
2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir
2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0:39152
2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF
2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir
2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0:26751
2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.
2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e failed to shutdown.
java.lang.reflect.UndeclaredThrowableException
         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)
         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)
         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)
         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
         at scala.collection.immutable.List.foreach(List.scala:45)
         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)
         at $Proxy12.finishApplicationMaster(Unknown Source)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)
         ... 9 more
Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)
         ... 11 more
Caused by: java.net.ConnectException: Connection refused
         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
         ... 12 more


I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:

<property>
                <name>yarn.resourcemanager.scheduler.address</name>
                <value>192.168.145.37:8030</value>
</property>
<property>
                <name>yarn.resourcemanager.resource-tracker.address</name>
                <value>192.168.145.37:8031</value>
</property>
<property>
                <name>yarn.resourcemanager.address</name>
                <value>192.168.145.37:8032</value>
</property>
<property>
                <name>yarn.resourcemanager.admin.address</name>
                <value>192.168.145.37:8033</value>
</property>
<property>
                <name>yarn.resourcemanager.webapp.address</name>
                <value>192.168.145.37:8088</value>
</property>



These properties are reflected on the UI screen as well:

[cid:image001.png@01CEDBE8.1B2F1890]

But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.
I still get:
Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

Nestat on the RM machine shows me:
tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java

Nestat on the NM machine shows me:
tcp        0      0 :::8040                     :::*                        LISTEN      1331/java
tcp        0      0 :::8042                     :::*                        LISTEN      1331/java
tcp        0      0 :::56877                    :::*                        LISTEN      1331/java

Kindly help me how to rectify this error.

Regards,
-Nirmal

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

RE: Running Samza on multi node

Posted by Nirmal Kumar <ni...@impetus.co.in>.
Hi Chris,

I implemented the Consumer part in Storm and Samza. The same Java program was used for publishing the messages to Kafka.

I used the following configs on the 3 Node Cluster deployment(refer below mail):

KAFKA
======
* num.partitions=8
* batch.num.messages=200
* log.flush.interval.messages=20000
* log.flush.interval.ms=600000
* log.segment.bytes=1073741824
* log.cleanup.interval.mins=30
* request.required.acks=0
* serializer.class=kafka.serializer.StringEncoder
* producer.type=async
* queue.buffering.max.ms=11000000
* queue.buffering.max.messages=2000000
* fetch.message.max.bytes=10485760

Samza Consumer:
=============
job.factory.class=org.apache.samza.job.yarn.YarnJobFactory
job.name=test-Consumer
yarn.package.path=file:/home/bda/nirmal/samza-job-package-0.8.3-dist.tar.gz
task.class=samza.examples.wikipedia.task.TestConsumer
task.inputs=kafka.storm-sentence
task.window.ms=10000
serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory
systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager
systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory
systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory
systems.kafka.samza.msg.serde=json
systems.kafka.consumer.zookeeper.connect=IMPETUS-DSRV05.impetus.co.in:2181/
systems.kafka.consumer.auto.offset.reset=largest
systems.kafka.producer.metadata.broker.list=IMPETUS-DSRV08.impetus.co.in:9092,IMPETUS-DSRV08.impetus.co.in:9093
yarn.container.count=2

Storm
======
The topology was deployed with the following configuration:
* Num_tasks:4
* Num_workers:2

Kafka Producer
============
Same standalone Java Program implemented using the Java Kafka0.8 Producer API was used to publish messages to the Kafka.
Message size: 8KB(String)

Results:
======
# of Messages Published: 5 million
Total Time taken by  Storm + Kafka0.8 to consume messages: 8 mins 23 secs
Total Time taken by Samza to consume messages: 8 mins 4 secs

Let me know if the configs seems to be fine for the comparison.

Thanks,
-Nirmal

-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Saturday, November 16, 2013 12:14 AM
To: dev@samza.incubator.apache.org
Subject: Re: Running Samza on multi node

Hey Nirmal

This seems reasonable. A few notes:

An equivalent to the 4 tasks 2 wrkers configuration for Storm is to set your Samza job to have yarn.contaner.count=2, and make sure that your input topic has 4 partitions. This wil give you 4 Samza tasks in 2 containers.

Also, if your Samza job is produing messages, you should tune your Kafka producers accordingly. Things like num.acks, sync vs. async, etc have a large effect on throughput in Samza. If your Samza job is not producing messages, just conuming, then

You should also take care to use the same serialization inboth tests (i.e. String serialization in both, etc).

I'd love to hear our results, as well. If you could post your Storm/Samza job configs/messge size/etc, and the results, it'd be cool to see. We haven't done much peformance testing.

Cheers,
Chris

On 11/14/13 1:30 AM, "Nirmal Kumar" <ni...@impetus.co.in> wrote:

>Hello Chris,
>
>
>
>Thanks for your help.
>
>
>
>I am comparing Storm + Kafka0.8 with Samza.
>
>
>
>I have used the following deployment in a 3 node cluster:
>
>
>Node
>
>omponents
>
>Node 1
>
>Zookeeper
>Storm Nimbus\YARN RM (Master)
>Storm I
>Kafka Broker-1
>Kafka Broker-2
>Kafka Producer
>
>Node 2
>
>
>Storm upervisor\YARN NM (Worker)
>
>
>Node 3
>
>Storm Supervisor\YARN NM (Worke)
>
>
>
>The Storm topology is deployed with the following configuration:

>o    Num_tasks:4
>
>o    Num_workers:2
>
>
>
>I am using the same Java Prgram for publishing the messages.
>
>On the Storm\Samza side I am simply consuming the messages from the
>Kafka topic.
>
>
>
>Is this is a correct way to do the comparison? Are there an other
>things that need to be taken care of?
>
>
>
>Thanks,
>
>-Nirmal>
>
>
>-----Original Message-----
>From: Chris Riccomini [mailto:criccoini@linkedin.com]
>Sent: Wednesday, November 13, 2013 2:30 AM
>To: dev@saza.incubator.apache.org
>Subject: Re: Running Samza on multi node
>
>
>
>Hy Nirmal,
>
>
>
>Glad to hear you've made progress.
>
>
>
>When trying to rn the .tar.gz packages in a cluster, you'll want to
>keep them on HDFS or an HTTP server. At LinkedIn, we use HTTP servers,
>and the yarn.package.path becomes an http:// address. YARN manages the
>download and unzip of these packages.
>
>
>
>Cheers,
>
>Chris
>
>
>
>From: Nirmal Kumar
><nirmal.kumar@impetus.co.in<mailto:nirmal.kumar@impetus.co.in<mailto:ni
>rma l.kumar@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in>>>
>
>Reply-To:
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><
>mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org
>%3e>"
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<m
>ail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.o
>rg>
>>>
>
>Date: Monday, November 11, 2013 10:46 PM
>
>To:
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><
>mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org
>%3e>"
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<m
>ail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.o
>rg>
>>>
>
>Subject: RE: Running Samza on multi node
>
>
>
>Hi Chris,
>
>
>
>I updated the yarn-default.xml in hadoop-yarn-common-2.0.5-alpha.jar
>present in samza-job-package-0.7.0-dist.tar.gz and finally got the job
>running.
>
>
>
>I updated the following in yarn-default.xml:
>
><property>
>
>    <description>The address of the scheduler interface.</description>
>
>    <name>yarn.resourcemanager.scheduler.address</name>
>
>    <value>192.168.145.37:8030</value>
>
>  </property>
>
><property>
>
>    <name>yarn.resourcemanager.resource-tracker.address</name>
>
>    <value>192.168.145.37:8031</value>
>
> </property>
>
><property>
>
>    <description>The address of the applications manager interface in
>the RM.</description>
>
>    <name>yarn.resourcemanager.address</name>
>
>    <value>192.168.145.37:8032</value>
>
>  </property>
>
><property>
>
>    <description>The address of the RM admin interface.</description>
>
>    <name>yarn.resourcemanager.admin.address</name>
>
>    <value>192.168.145.37:8033</value>
>
>  </property>
>
>
>
>Earlier these had 0.0.0.0 values. Is there any other workaround?
>
>
>
>My question is on a cluster environment where do I need to keep the
>samza-job-package-0.7.0-dist.tar.gz file.
>
>I am submitting the job from the RM node and I have to keep the
>samza-job-package-0.7.0-dist.tar.gz file on both the RM and NM machines.
>
>Otherwise I receives the samza-job-package-0.7.0-dist.tar.gz file not
>found.
>
>
>
>
>
>
>
>Keeping the samza job file at both the NM and RM machine at a similar
>location gives me exception of different timestamps as the machines has
>different time zones:
>
>
>
>
>
>Application application_1384196041399_0002 failed 1 times due to AM
>Container for appattempt_1384196041399_0002_000001 exited with exitCode:
>-1000 due to: RemoteTrace:
>
>java.io.IOException: Resource
>file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src
>filesystem (expected 1384196169000, was 1383911494000
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>at
>org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at
>java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.j
>ava
>:908)
>
>at java.lang.Thread.run(Thread.java:662)
>
>at LocalTrace:
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl:
>Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz
>changed on src filesystem (expected 1384196169000, was 1383911494000
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.
>jav
>a:217)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocali
>zat
>ionService.java:491)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localizat
>ion
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:
>46)
>
>at
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolS
>erv
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(
>Pro
>tobufRpcEngine.java:454)
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>I am attaching the NM and RM logs for reference.
>
>
>
>Thanks,
>
>-Nirmal
>
>
>
>From: Nirmal Kumar
>
>Sent: Tuesday, November 12, 2013 1:14 AM
>
>To:
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<ma
>ilt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.or
>g>>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>Hi Chris,
>
>
>
>
>
>
>
>Thanks for the advice.
>
>
>
>
>
>
>
>I put my yarn-site.xml in hello-samza/deploy/yarn/conf directory and
>also put the following echo in hello-samza/deploy/yarn/etc/hadoop/
>yarn-env.sh
>file:
>
>
>
>
>
>
>
>I am using the commands:
>
>
>
>nohup bin/yarn resourcemanager > logs/rm.log 2>&1 &
>
>
>
>nohup bin/yarn nodemanager > logs/nm.log 2>&1 &
>
>
>
>
>
>
>
>#export YARN_CONF_DIR="${YARN_CONF_DIR:-$HADOOP_YARN_HOME/conf}"
>
>
>
>export YARN_CONF_DIR="/home/bda/nirmal/hello-samza/deploy/yarn/conf"
>
>
>
>echo "YARN_CONF_DIR in sh: $YARN_CONF_DIR"
>
>
>
>
>
>
>
>Now in my nm.log and rm.log I see
>
>
>
>
>
>
>
>nm.log
>
>
>
>--------
>
>
>
>nohup: ignoring input
>
>
>
>YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf
>
>
>
>13/11/11 19:52:01 INFO nodemanager.NodeManager: STARTUP_MSG:
>
>
>
>/************************************************************
>
>
>
>STARTUP_MSG: Starting NodeManager
>
>
>
>STARTUP_MSG:   host = IMPETUS-DSRV05.impetus.co.in/192.168.145.195
>
>
>
>STARTUP_MSG:   args = []
>
>
>
>STARTUP_MSG:   version = 2.0.5-alpha
>
>
>
>STARTUP_MSG:   classpath =
>
>
>
>
>
>
>
>rm.log
>
>
>
>--------
>
>
>
>nohup: ignoring input
>
>
>
>YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf
>
>
>
>13/11/12 00:24:01 INFO resourcemanager.ResourceManager: STARTUP_MSG:
>
>
>
>/************************************************************
>
>
>
>STARTUP_MSG: Starting ResourceManager
>
>
>
>STARTUP_MSG:   host = IMPETUS-DSRV08.impetus.co.in/192.168.145.37
>
>
>
>STARTUP_MSG:   args = []
>
>
>
>STARTUP_MSG:   version = 2.0.5-alpha
>
>
>
>STARTUP_MSG:   classpath =
>
>
>
>
>
>
>
>This rm.log and nm.log are fine.
>
>
>
>
>
>
>
>hello-samza/deploy/yarn/conf/yarn-site.xml in both the machines is:
>
>
>
><configuration>
>
>
>
>  <property>
>
>
>
>    <name>yarn.scheduler.minimum-allocation-mb</name>
>
>
>
>    <value>128</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
>  <property>
>
>
>
>    <name>yarn.nodemanager.vmem-pmem-ratio</name>
>
>
>
>    <value>10</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.hostname</name>
>
>
>
>    <value>192.168.145.37</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>    <value>192.168.145.37:8031</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.address</name>
>
>
>
>    <value>192.168.145.37:8032</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>   <name>yarn.resourcemanager.admin.address</name>
>
>
>
>   <value>192.168.145.37:8033</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>        <name>yarn.resourcemanager.scheduler.address</name>
>
>
>
>        <value>192.168.145.37:8030</value>
>
>
>
></property>
>
>
>
></configuration>
>
>
>
>
>
>
>
>But when I submit the job using the following command :
>
>
>
>bin/run-job.sh
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
>ory
>
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/tes
>t-c
>onsumer.properties
>
>
>
>
>
>
>
>After submitting the job file successfully I see the following under
>stdout:
>
>
>
>
>
>2013-11-11 19:52:24 ClientHelper [INFO] trying to connect to RM
>/0.0.0.0:8032
>
>
>
>2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005
>
>
>
>2013-11-11 19:52:24 log [INFO] Extract
>jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/applicatio
>n_1
>384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.
>0-d
>ist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB
>-IN F/ to /tmp/Jetty_0_0_0_0_38376_scalate____.8y41jd/webinf/WEB-INF
>
>
>
>2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template
>engine using working directory:
>/tmp/scalate-7888530459139045883-workdir
>
>
>
>2013-11-11 19:52:24 log [INFO] Started
>SelectChannelConnector@0.0.0.0:38376<mailto:SelectChannelConnector@0.0.
>0.0
>:38376<mailto:SelectChannelConnector@0.0.0.0:38376%3cmailto:SelectChann
>elC
>onnector@0.0.0.0:38376>>
>
>
>
>2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005
>
>
>
>2013-11-11 19:52:24 log [INFO] Extract
>jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/applicatio
>n_1
>384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.
>0-d
>ist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB
>-IN F/ to /tmp/Jetty_0_0_0_0_28858_scalate____.sc28cx/webinf/WEB-INF
>
>
>
>2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template
>engine using working directory:
>/tmp/scalate-3192563436096941762-workdir
>
>
>
>2013-11-11 19:52:24 log [INFO] Started
>SelectChannelConnector@0.0.0.0:28858<mailto:SelectChannelConnector@0.0.
>0.0
>:28858<mailto:SelectChannelConnector@0.0.0.0:28858%3cmailto:SelectChann
>elC
>onnector@0.0.0.0:28858>>
>
>
>
>2013-11-11 19:52:34 SamzaAppMasterLifecycle [INFO] Shutting down.
>
>
>
>2013-11-11 19:52:44 YarnAppMaster [WARN] Listener
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec<mailto:org.a
>pac
>he.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec<mailto:org.apache.sa
>mza
>.job.yarn.SamzaAppMasterLifecycle@4095c5ec%3cmailto:org.apache.samza.jo
>b.y arn.SamzaAppMasterLifecycle@4095c5ec>> failed to shutdown.
>
>
>
>java.lang.reflect.UndeclaredThrowableException
>
>
>
>                at
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unw
>rap
>AndThrowException(YarnRemoteExceptionPBImpl.java:135)
>
>
>
>                at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.fini
>shA
>pplicationMaster(AMRMProtocolPBClientImpl.java:90)
>
>
>
>                at
>org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaste
>r(A
>MRMClientImpl.java:244)
>
>
>
>                at
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMa
>ste
>rLifecycle.scala:68)
>
>
>
>                at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMas
>ter
>.scala:70)
>
>
>
>                at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMas
>ter
>.scala:69)
>
>
>
>                at
>scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.sc
>ala
>:61)
>
>
>
>                at
> scala.collection.immutable.List.foreach(List.scala:45)
>
>
>
>                at
>org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
>
>
>
>                at
>org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
>
>
>
>                at
>org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
>
>
>
>Caused by: com.google.protobuf.ServiceException:
>java.net.ConnectException: Call From
>IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>                at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngin
>e.j
>ava:212)
>
>
>
>                at $Proxy12.finishApplicationMaster(Unknown Source)
>
>
>
>                at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.fini
>shA
>pplicationMaster(AMRMProtocolPBClientImpl.java:87)
>
>
>
>                ... 9 more
>
>
>
>Caused by: java.net.ConnectException: Call From
>IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>                at
>sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>
>
>
>                at
>sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
>Acc
>essorImpl.java:39)
>
>
>
>                at
>sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
>str
>uctorAccessorImpl.java:27)
>
>
>
>                at
>java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
>
>
>
>                at org.apache.hadoop.ipc.Client.call(Client.java:1239)
>
>
>
>                at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngin
>e.j
>ava:202)
>
>
>
>                ... 11 more
>
>
>
>Caused by: java.net.ConnectException: Connection refused
>
>
>
>                at sun.nio.ch.SocketChannelImpl.checkConnect(Native
>Method)
>
>
>
>                at
>sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:567)
>
>
>
>                at
>org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.j
>ava
>:206)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
>
>
>
>                at
>org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508
>)
>
>
>
>                at
>org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
>
>
>
>                at
>org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
>
>
>
>                at
>org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
>
>
>
>                at org.apache.hadoop.ipc.Client.call(Client.java:1206)
>
>
>
>                ... 12 more
>
>
>
>
>
>
>
>In the stdout I see the following jar is used when invoking
>org.apache.samza.job.yarn.SamzaAppMaster:
>
>
>
>/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179
>355
>066_0001/container_1384179355066_0001_01_000001/__package/lib/hadoop-ya
>rn-
>common-2.0.5-alpha.jar
>
>
>
>
>
>
>
>This jar too contains the yarn-default.xml. Is this the file that's
>being picked up?
>
>
>
>
>
>
>
>
>
>
>
>Also where do I need to keep the job file ?
>
>
>
>I am setting the job file name in a test-consumer.properties file:
>
>
>
>yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.
>gz
>
>
>
>
>
>
>
>And submitting the job from the RM node:
>
>
>
>bin/run-job.sh
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
>ory
>
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/tes
>t-c
>onsumer.properties
>
>
>
>
>
>
>
>But that way do I need to keep the job file in both NM and RM machines
>on the same location(/home/temptest/ i.e. I have created same
>directories on both the machines)?
>
>
>
>
>
>
>
>Keeping the samza job file at both the NM and RM machine at a similar
>location gives me exception:
>
>
>
>
>
>Application application_1384196041399_0002 failed 1 times due to AM
>Container for appattempt_1384196041399_0002_000001 exited with exitCode:
>-1000 due to: RemoteTrace:
>
>java.io.IOException: Resource
>file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src
>filesystem (expected 1384196169000, was 1383911494000
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>at
>org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at
>java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.j
>ava
>:908)
>
>at java.lang.Thread.run(Thread.java:662)
>
>at LocalTrace:
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl:
>Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz
>changed on src filesystem (expected 1384196169000, was 1383911494000
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.
>jav
>a:217)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocali
>zat
>ionService.java:491)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localizat
>ion
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:
>46)
>
>at
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolS
>erv
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(
>Pro
>tobufRpcEngine.java:454)
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>
>
>
>
>
>
>Regards,
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>
>
>
>
>-----Original Message-----
>
>From: Chris Riccomini [mailto:criccomini@linkedin.com]
>
>Sent: Sunday, November 10, 2013 11:00 AM
>
>To:
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<ma
>ilt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.or
>g>>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>
>
>Hey Nirmal,
>
>
>
>
>
>
>
>It sounds like the yarn-site.xml is being ignored for some reason.
>Things to do:
>
>
>
>
>
>
>
>1. Could you please send full log files for the RM and NM?
>
>
>
>
>
>
>
>2. You might also try putting your yarn-site.xml in
>hello-samza/deploy/yarn/conf, and explicitly setting the
>HADOOP_YARN_HOME environment variable to:
>
>
>
>
>
>
>
>export HADOOP_YARN_HOME=<path to>/hello-samza/deploy/yarn
>
>
>
>
>
>
>
>Then try running bin/grid start yarn.
>
>
>
>
>
>
>
>3. Try staring yarn WITHOUT bin/grid. This can be done with:
>
>
>
>
>
>
>
>deploy/yarn/bin/yarn resourcemanager
>
>
>
>deploy/yarn/bin/yarn nodemanager
>
>
>
>
>
>
>
>Cheers,
>
>
>
>Chris
>
>
>
>________________________________
>
>
>
>From: Nirmal Kumar
>[nirmal.kumar@impetus.co.in<ma...@impetus.co.in>]
>
>
>
>Sent: Friday, November 08, 2013 5:01 AM
>
>
>
>To:
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<ma
>ilt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.or
>g>>
>
>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>
>
>Hi Chris,
>
>
>
>
>
>
>
>The below exception is gone if the job file timestamp is same:
>
>
>
>
>
>
>
>
>
>
>
>Application application_1383907331443_0002 failed 1 times due to AM
>Container for appattempt_1383907331443_0002_000001 exited with exitCode:
>-1000 due to: RemoteTrace:
>
>
>
>
>
>
>
>java.io.IOException: Resource
>file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src
>filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at
>java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>
>
>
>
>
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.j
>ava
>:908)
>
>
>
>
>
>
>
>at java.lang.Thread.run(Thread.java:662)
>
>
>
>
>
>
>
>at LocalTrace:
>
>
>
>
>
>
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl:
>Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz
>changed on src filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.
>jav
>a:217)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocali
>zat
>ionService.java:491)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localizat
>ion
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:
>46)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolS
>erv
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(
>Pro
>tobufRpcEngine.java:454)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>
>
>PFA the exception that's coming. I am still struggling with the same
>exception i.e. NM trying to connect to 0.0.0.0:8030 I don't know from
>where the NM is picking up this 0.0.0.0:8030 value. Overriding the
>yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.
>
>
>
>
>
>
>
>I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on
>both RM and NM machines:
>
>
>
>
>
>
>
><configuration>
>
>
>
>  <property>
>
>
>
>    <name>yarn.scheduler.minimum-allocation-mb</name>
>
>
>
>    <value>128</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
>  <property>
>
>
>
>    <name>yarn.nodemanager.vmem-pmem-ratio</name>
>
>
>
>    <value>10</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.hostname</name>
>
>
>
>   <value>192.168.145.37</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>     <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>     <value>192.168.145.37:8031</value>
>
>
>
></property>
>
>
>
></configuration>
>
>
>
>
>
>
>
>
>
>
>
>Regards,
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>From: Nirmal Kumar
>
>
>
>Sent: Friday, November 08, 2013 6:11 PM
>
>
>
>To:
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<ma
>ilt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.or
>g>>
>
>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>
>
>
>
>
>
>Hi Chris,
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Using just the yarn.resourcemanager.hostname property gives me the
>following exception on the NM:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Caused by: com.google.protobuf.ServiceException:
>java.net.ConnectException: Call From
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I then added the following property as well:
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>   <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>
>
>
>
>   <value>192.168.145.37:8031</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>After this my RM and NM were up and the NM got registered as well:
>
>
>
>
>
>
>
>13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager
>is started.
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO util.RackResolver: Resolved
>IMPETUS-DSRV14.impetus.co.in to /default-rack
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService:
>NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948
>httpPort: 8042) registered with capability: <memory:8192, vCores:16>,
>assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO rmnode.RMNodeImpl:
>IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to
>RUNNING
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node
>IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192,
>vCores:16>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>When submitting the job I'm still getting the same exception:
>
>
>
>
>
>
>
>
>
>
>
>YarnAppMaster [WARN] Listener
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.a
>pac
>he.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.sa
>mza
>.job.yarn.SamzaAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.jo
>b.y
>arn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.samza.job.yarn.S
>amz
>aAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAp
>pMa
>sterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMaste
>rLi
>fecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMasterLifec
>ycl
>e@500c954e>>> failed to shutdown.
>
>
>
>java.lang.reflect.UndeclaredThrowableException
>
>
>
>         at
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unw
>rap
>AndThrowException(YarnRemoteExceptionPBImpl.java:135)
>
>
>
>         at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.fini
>shA
>pplicationMaster(AMRMProtocolPBClientImpl.java:90)
>
>
>
>         at
>org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaste
>r(A
>MRMClientImpl.java:244)
>
>
>
>         at
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMa
>ste
>rLifecycle.scala:68)
>
>
>
>         at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMas
>ter
>.scala:70)
>
>
>
>         at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMas
>ter
>.scala:69)
>
>
>
>         at
>scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.sc
>ala
>:61)
>
>
>
>         at scala.collection.immutable.List.foreach(List.scala:45)
>
>
>
>         at
>org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
>
>
>
>         at
>org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
>
>
>
>         at
>org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
>
>
>
>Caused by: com.google.protobuf.ServiceException:
>java.net.ConnectException: Call From
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>         at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngin
>e.j
>ava:212)
>
>
>
>         at $Proxy12.finishApplicationMaster(Unknown Source)
>
>
>
>         at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.fini
>shA
>pplicationMaster(AMRMProtocolPBClientImpl.java:87)
>
>
>
>         ... 9 more
>
>
>
>Caused by: java.net.ConnectException: Call From
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>         at
>sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
>Method)
>
>
>
>         at
>sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
>Acc
>essorImpl.java:39)
>
>
>
>         at
>sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
>str
>uctorAccessorImpl.java:27)
>
>
>
>         at
>java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>
>
>         at
>org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
>
>
>
>         at
>org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
>
>
>
>         at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngin
>e.j
>ava:202)
>
>
>
>         ... 11 more
>
>
>
>Caused by: java.net.ConnectException: Connection refused
>
>
>
>         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>
>
>
>         at
>sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
>
>
>
>         at
>org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.j
>ava
>:206)
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
>
>
>
>         at
>org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508
>)
>
>
>
>         at
>org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
>
>
>
>         at
>org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
>
>
>
>         at
> org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
>
>
>
>         ... 12 more
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Where do I need to keep the job file ?
>
>
>
>
>
>
>
>I am setting the job file name in a test-consumer.properties file:
>
>
>
>
>
>
>
>yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.
>gz
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>And submitting the job from :
>
>
>
>
>
>
>
>bin/run-job.sh
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
>ory
>
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/tes
>t-c
>onsumer.properties
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>But that way do I need to keep the job file in both NM and RM machines
>on the same location?
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I tried submitting the job several time with different properties in
>yarn-site.xml and now I am getting some strange exception. This is
>probably due to the different timestamps.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Application application_1383907331443_0002 failed 1 times due to AM
>Container for appattempt_1383907331443_0002_000001 exited with exitCode:
>-1000 due to: RemoteTrace:
>
>
>
>
>
>
>
>java.io.IOException: Resource
>file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src
>filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at
>java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>
>
>
>
>
>
>at
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.j
>ava
>:908)
>
>
>
>
>
>
>
>at java.lang.Thread.run(Thread.java:662)
>
>
>
>
>
>
>
>at LocalTrace:
>
>
>
>
>
>
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl:
>Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz
>changed on src filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.
>jav
>a:217)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.L
>oca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocali
>zat
>ionService.java:491)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Re
>sou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localizat
>ion
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:
>46)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolS
>erv
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(
>Pro
>tobufRpcEngine.java:454)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>
>
>
>
>
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>When I am using NM and RM on a single node all is running fine. PFA the
>logs for the job.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Other questions:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>1. Both your NM and RM are running YARN 2.2.0, right?
>
>
>
>
>
>
>
>I am using same YARN on both NM and RM that was downloaded as part of
>hello-samza application.
>
>
>
>
>
>
>
>2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?
>
>
>
>
>
>
>
>I am forcibly killing the java processes using kill -9 pid command.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Thanks,
>
>
>
>
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>-----Original Message-----
>
>
>
>From: Chris Riccomini [mailto:criccomini@linkedin.com]
>
>
>
>Sent: Thursday, November 07, 2013 9:19 PM
>
>
>
>To:
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<ma
>ilt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.or
>g>>
>
>
>
>Subject: Re: Running Samza on multi node
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Hey Nirmal,
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Thanks for this detailed report! It makes things much easier to figure
>out. The problem appears to be that the Samza AM is trying to connect
>to
>0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which
>is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM
>host). This is causing a timeout, since 8030 isn't open on localhost
>for the Samza AM, which is running on the NM's box.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>It is somewhat interesting that the NM does connect to the RM for the
>capacity scheduler. Rather than setting each individual host/port pair,
>as you've done, I recommend just setting:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>  <property>
>
>
>
>
>
>
>
>    <name>yarn.resourcemanager.hostname</name>
>
>
>
>
>
>
>
>    <value>192.168.145.37</value>
>
>
>
>
>
>
>
>  </property>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Your netstat reports look fine - as expected.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Other questions:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>1. Both your NM and RM are running YARN 2.2.0, right?
>
>
>
>
>
>
>
>2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Regarding (2), it appears that the AM never tries to register. This
>normally happens. I'm wondering if another failure is being triggered,
>which is then causing the AM to try and shut itself down. Could you
>turn on debugging for your Samza job (in log4j.xml), and re-run? I'm
>curious if the web-service that's starting up, or the registration
>itself is failing. In a normal execution, you would expect to see:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>    info("Got AM register response. The YARN RM supports container
>requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I don't see this in your logs, which means the AM is failing (and
>triggering a shutdown) before it even tries to register.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Cheers,
>
>
>
>
>
>
>
>Chris
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>From: Nirmal Kumar
><nirmal.kumar@impetus.co.in<mailto:nirmal.kumar@impetus.co.in<mailto:ni
>rma
>l.kumar@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in<mailto:nirmal
>.ku
>mar@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.k
>uma
>r@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in<mailto:nirmal.kumar
>@im
>petus.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.kumar@i
>mpe
>tus.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.kumar@imp
>etu
>s.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.kumar@impetus.
>co.in%3cmailto:nirmal.kumar@impetus.co.in>>>>>
>
>
>
>
>
>
>
>Reply-To:
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><
>mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org
>%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator
>.ap
>ache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.
>inc
>ubator.apache.org%3e%3e><mailto:dev@samza.incubator.apache.org%3cmailto
>:de
>v@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org
>%3c
>mailto:dev@samza.incubator.apache.org%3e%3e%3cmailto:dev@samza.incubato
>r.a
>pache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza
>.in
>cubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3e>"
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<m
>ail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.o
>rg<
>mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubato
>r.a
>pache.org<mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incu
>bat
>or.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samz
>a.i
>ncubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:de
>v@s
>amza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmai
>lto
>:dev@samza.incubator.apache.org>>>>>
>
>
>
>
>
>
>
>Date: Thursday, November 7, 2013 5:05 AM
>
>
>
>
>
>
>
>To:
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><
>mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org
>%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator
>.ap
>ache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.
>inc
>ubator.apache.org%3e%3e><mailto:dev@samza.incubator.apache.org%3cmailto
>:de
>v@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org
>%3c
>mailto:dev@samza.incubator.apache.org%3e%3e%3cmailto:dev@samza.incubato
>r.a
>pache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza
>.in
>cubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3e>"
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<m
>ail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.o
>rg<
>mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubato
>r.a
>pache.org<mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incu
>bat
>or.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samz
>a.i
>ncubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:de
>v@s
>amza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmai
>lto
>:dev@samza.incubator.apache.org>>>>>
>
>
>
>
>
>
>
>Subject: Running Samza on multi node
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>All,
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I was able to run the hello-samza application on a single node machine.
>
>
>
>
>
>
>
>Now I am trying to run the hello-samza application on  a 2 node setup.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Node1 has a Resource Manager
>
>
>
>
>
>
>
>Node2 has a Node Manager
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>The NM gets registered with the RM successfully as seen in rm.log of
>the RM node:
>
>
>
>
>
>
>
>13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager
>is started.
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO util.RackResolver: Resolved
>IMPETUS-DSRV14.impetus.co.in to /default-rack
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService:
>NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093
>httpPort: 8042) registered with capability: <memory:8192, vCores:16>,
>assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO rmnode.RMNodeImpl:
>IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to
>RUNNING
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node
>IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192,
>vCores:16>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I am submitting the job from the RM machine using the command line:
>
>
>
>
>
>
>
>bin/run-job.sh
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFact
>ory
>
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/tes
>t-c
>onsumer.properties
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>However, I am getting the following exception after submitting the job
>to
>YARN:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id:
>container_1383816757258_0001_01_000001
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id:
>appattempt_1383816757258_0001_000001
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host:
>IMPETUS-DSRV14
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port:
>8042
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config:
>{task.inputs=kafka.storm-sentence,
>job.factory.class=org.apache.samza.job.yarn.YarnJobFactory,
>systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFa
>cto
>ry, job.name=test-Consumer,
>systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/,
>systems.kafka.consumer.auto.offset.reset=largest,
>systems.kafka.samza.msg.serde=json,
>serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeF
>act
>ory,
>systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartition
>Man
>ager, task.window.ms=10000,
>task.class=samza.examples.wikipedia.task.TestConsumer,
>yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job
>-pa ckage/target/samza-job-package-0.7.0-dist.tar.gz,
>systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFa
>cto
>ry,
>systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.16
>8.1
>45.195:9093}
>
>
>
>
>
>
>
>2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM
>/0.0.0.0:8032
>
>
>
>
>
>
>
>2013-11-07 15:05:57 JmxServer [INFO] According to
>InetAddress.getLocalHost.getHostName we are
>IMPETUS-DSRV14.impetus.co.in
>
>
>
>
>
>
>
>2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115
>url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jm
>xrm
>i
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No
>yarn.container.count specified. Defaulting to one container.
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is
>overridden to samza_admin-test_Consumer-1-1383816957797-0
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property
>metadata.broker.list is overridden to
>192.168.145.195:9092,192.168.145.195:9093
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property
>auto.offset.reset is overridden to largest
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is
>overridden to samza_admin-test_Consumer-1-1383816957797-0
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is
>overridden to undefined-samza-consumer-group-
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property
>zookeeper.connect is overridden to 192.168.145.195:2181/
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is
>overridden to samza_admin-test_Consumer-1-1383816957797-0
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property
>metadata.broker.list is overridden to
>192.168.145.195:9092,192.168.145.195:9093
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property
>request.timeout.ms is overridden to 6000
>
>
>
>
>
>
>
>2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker
>id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1
>topic(s)
>Set(storm-sentence)
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SyncProducer [INFO] Connected to
>192.168.145.195:9092 for producing
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from
>192.168.145.195:9092
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc
>39152, tracking port 26751
>
>
>
>
>
>
>
>2013-11-07 15:05:57 log [INFO] Logging to
>org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via
>org.eclipse.jetty.util.log.Slf4jLog
>
>
>
>
>
>
>
>2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM
>/0.0.0.0:8032
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Extract
>jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/applicat
>ion
>_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7
>.0-
>dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WE
>B-I NF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF
>
>
>
>
>
>
>
>2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template
>engine using working directory:
>/tmp/scalate-5279562760844696556-workdir
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Started
>SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:3
>915
>2<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.
>0.0.0%3e:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectCha
>nne
>lConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cm
>ail
>to:SelectChannelConnector@0.0.0.0%3e:39152<mailto:SelectChannelConnecto
>r@0
>.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:Select
>Cha
>nnelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%
>3cm
>ailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.
>0.0
>.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChan
>nel
>Connector@0.0.0.0%3e:39152>>>
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Extract
>jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/applicat
>ion
>_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7
>.0-
>dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WE
>B-I NF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF
>
>
>
>
>
>
>
>2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template
>engine using working directory:
>/tmp/scalate-5582747144249485577-workdir
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Started
>SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:2
>675
>1<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.
>0.0.0%3e:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectCha
>nne
>lConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cm
>ail
>to:SelectChannelConnector@0.0.0.0%3e:26751<mailto:SelectChannelConnecto
>r@0
>.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:Select
>Cha
>nnelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%
>3cm
>ailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.
>0.0
>.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChan
>nel
>Connector@0.0.0.0%3e:26751>>>
>
>
>
>
>
>
>
>2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.
>
>
>
>
>
>
>
>2013-11-07 15:06:18 YarnAppMaster [WARN] Listener
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.a
>pac
>he.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.sa
>mza
>.job.yarn.SamzaAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.jo
>b.y
>arn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.samza.job.yarn.S
>amz
>aAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAp
>pMa
>sterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMaste
>rLi
>fecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMasterLifec
>ycl
>e@500c954e>>> failed to shutdown.
>
>
>
>
>
>
>
>java.lang.reflect.UndeclaredThrowableException
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unw
>rap
>AndThrowException(YarnRemoteExceptionPBImpl.java:135)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.fini
>shA
>pplicationMaster(AMRMProtocolPBClientImpl.java:90)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaste
>r(A
>MRMClientImpl.java:244)
>
>
>
>
>
>
>
>         at
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMa
>ste
>rLifecycle.scala:68)
>
>
>
>
>
>
>
>         at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMas
>ter
>.scala:70)
>
>
>
>
>
>
>
>         at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMas
>ter
>.scala:69)
>
>
>
>
>
>
>
>         at
>scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.sc
>ala
>:61)
>
>
>
>
>
>
>
>         at scala.collection.immutable.List.foreach(List.scala:45)
>
>
>
>
>
>
>
>         at
>org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
>
>
>
>
>
>
>
>         at
>org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
>
>
>
>
>
>
>
>         at
>org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
>
>
>
>
>
>
>
>Caused by: com.google.protobuf.ServiceException:
>java.net.ConnectException: Call From
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngin
>e.j
>ava:212)
>
>
>
>
>
>
>
>         at $Proxy12.finishApplicationMaster(Unknown Source)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.fini
>shA
>pplicationMaster(AMRMProtocolPBClientImpl.java:87)
>
>
>
>
>
>
>
>         ... 9 more
>
>
>
>
>
>
>
>Caused by: java.net.ConnectException: Call From
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>         at
>sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
>Method)
>
>
>
>
>
>
>
>         at
>sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructor
>Acc
>essorImpl.java:39)
>
>
>
>
>
>
>
>         at
>sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCon
>str
>uctorAccessorImpl.java:27)
>
>
>
>
>
>
>
>         at
>java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
>
>
>
>
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngin
>e.j
>ava:202)
>
>
>
>
>
>
>
>         ... 11 more
>
>
>
>
>
>
>
>Caused by: java.net.ConnectException: Connection refused
>
>
>
>
>
>
>
>         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>
>
>
>
>
>
>
>         at
>sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.j
>ava
>:206)
>
>
>
>
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
>
>
>
>
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508
>)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
>
>
>
>
>
>
>
>         at
>org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
>
>
>
>
>
>
>
>         at
> org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
>
>
>
>
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
>
>
>
>
>
>
>
>         ... 12 more
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I have changed the following properties in the
>hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager
>machine:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.scheduler.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8030</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>
> <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8031</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8032</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.admin.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8033</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.webapp.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8088</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>These properties are reflected on the UI screen as well:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>[<ci...@01CEDBE8.1B2F1890>cid:image001.png@01CEDBE8.1B2F1890
>]
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>But this overriding of the yarn.resourcemanager.scheduler.address to
>192.168.145.37:8030 does not rectify the error.
>
>
>
>
>
>
>
>I still get:
>
>
>
>
>
>
>
>Caused by: com.google.protobuf.ServiceException:
>java.net.ConnectException: Call From
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused;
>For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Nestat on the RM machine shows me:
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8088  :::*
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8030  :::*
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8031  :::*
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8032  :::*
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8033  :::*
>  LISTEN      14595/java
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Nestat on the NM machine shows me:
>
>
>
>
>
>
>
>tcp        0      0 :::8040                     :::*
>  LISTEN      1331/java
>
>
>
>
>
>
>
>tcp        0      0 :::8042                     :::*
>  LISTEN      1331/java
>
>
>
>
>
>
>
>tcp        0      0 :::56877                    :::*
>  LISTEN      1331/java
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Kindly help me how to rectify this error.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Regards,
>
>
>
>
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential,
>proprietary, privileged or otherwise protected by law. The message is
>intended solely for the named addressee. If received in error, please
>destroy and notify the sender. Any use of this email is prohibited when
>received in error. Impetus does not represent, warrant and/or
>guarantee, that the integrity of this communication has been maintained
>nor that the communication is free of errors, virus, interception or interference.
>
>
>
>
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential,
>proprietary, privileged or otherwise protected by law. The message is
>intended solely for the named addressee. If received in error, please
>destroy and notify the sender. Any use of this email is prohibited when
>received in error. Impetus does not represent, warrant and/or
>guarantee, that the integrity of this communication has been maintained
>nor that the communication is free of errors, virus, interception or interference.
>
>
>
>
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential,
>proprietary, privileged or otherwise protected by law. The message is
>intended solely for the named addressee. If received in error, please
>destroy and notify the sender. Any use of this email is prohibited when
>received in error. Impetus does not represent, warrant and/or
>guarantee, that the integrity of this communication has been maintained
>nor that the communication is free of errors, virus, interception or interference.
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential,
>proprietary, privileged or otherwise protected by law. The message is
>intended solely for the named addressee. If received in error, please
>destroy and notify the sender. Any use of this email is prohibited when
>received in error. Impetus does not represent, warrant and/or
>guarantee, that the integrity of this communication has been maintained
>nor that the communication is free of errors, virus, interception or interference.
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential,
>proprietary, privileged or otherwise protected by law. The message is
>intended solely for the named addressee. If received in error, please
>destroy and notify the sender. Any use of this email is prohibited when
>received in error. Impetus does not represent, warrant and/or
>guarantee, that the integrity of this communication has been maintained
>nor that the communication is free of errors, virus, interception or interference.
>
>________________________________
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential,
>proprietary, privileged or otherwise protected by law. The message is
>intended solely for the named addressee. If received in error, please
>destroy and notify the sender. Any use of this email is prohibited when
>received in error. Impetus does not represent, warrant and/or
>guarantee, that the integrity of this communication has been maintained
>nor that the communication is free of errors, virus, interception or interference.


________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

Re: Running Samza on multi node

Posted by Chris Riccomini <cr...@linkedin.com>.
Hey Nirmal

This seems reasonable. A few notes:

An equivalent to the 4 tasks 2 wrkers configuration for Storm is to set
your Samza job to have yarn.contaner.count=2, and make sure that your
input topic has 4 partitions. This wil give you 4 Samza tasks in 2
containers.

Also, if your Samza job is produing messages, you should tune your Kafka
producers accordingly. Things like num.acks, sync vs. async, etc have a
large effect on throughput in Samza. If your Samza job is not producing
messages, just conuming, then

You should also take care to use the same serialization inboth tests
(i.e. String serialization in both, etc).

I'd love to hear our results, as well. If you could post your Storm/Samza
job configs/messge size/etc, and the results, it'd be cool to see. We
haven't done much peformance testing.

Cheers,
Chris

On 11/14/13 1:30 AM, "Nirmal Kumar" <ni...@impetus.co.in> wrote:

>Hello Chris,
>
>
>
>Thanks for your help.
>
>
>
>I am comparing Storm + Kafka0.8 with Samza.
>
>
>
>I have used the following deployment in a 3 node cluster:
>
>
>Node
>
>omponents
>
>Node 1
>
>Zookeeper
>Storm Nimbus\YARN RM (Master)
>Storm I
>Kafka Broker-1
>Kafka Broker-2
>Kafka Producer
>
>Node 2
>
>
>Storm upervisor\YARN NM (Worker)
>
>
>Node 3
>
>Storm Supervisor\YARN NM (Worke)
>
>
>
>The Storm topology is deployed with the following configuration:

>o    Num_tasks:4
>
>o    Num_workers:2
>
>
>
>I am using the same Java Prgram for publishing the messages.
>
>On the Storm\Samza side I am simply consuming the messages from the Kafka
>topic.
>
>
>
>Is this is a correct way to do the comparison? Are there an other things
>that need to be taken care of?
>
>
>
>Thanks,
>
>-Nirmal>
>
>
>-----Original Message-----
>From: Chris Riccomini [mailto:criccoini@linkedin.com]
>Sent: Wednesday, November 13, 2013 2:30 AM
>To: dev@saza.incubator.apache.org
>Subject: Re: Running Samza on multi node
>
>
>
>Hy Nirmal,
>
>
>
>Glad to hear you've made progress.
>
>
>
>When trying to rn the .tar.gz packages in a cluster, you'll want to keep
>them on HDFS or an HTTP server. At LinkedIn, we use HTTP servers, and the
>yarn.package.path becomes an http:// address. YARN manages the download
>and unzip of these packages.
>
>
>
>Cheers,
>
>Chris
>
>
>
>From: Nirmal Kumar
><nirmal.kumar@impetus.co.in<mailto:nirmal.kumar@impetus.co.in<mailto:nirma
>l.kumar@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in>>>
>
>Reply-To: 
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org
>%3e>" 
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org>
>>>
>
>Date: Monday, November 11, 2013 10:46 PM
>
>To: 
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org
>%3e>" 
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org>
>>>
>
>Subject: RE: Running Samza on multi node
>
>
>
>Hi Chris,
>
>
>
>I updated the yarn-default.xml in hadoop-yarn-common-2.0.5-alpha.jar
>present in samza-job-package-0.7.0-dist.tar.gz and finally got the job
>running.
>
>
>
>I updated the following in yarn-default.xml:
>
><property>
>
>    <description>The address of the scheduler interface.</description>
>
>    <name>yarn.resourcemanager.scheduler.address</name>
>
>    <value>192.168.145.37:8030</value>
>
>  </property>
>
><property>
>
>    <name>yarn.resourcemanager.resource-tracker.address</name>
>
>    <value>192.168.145.37:8031</value>
>
> </property>
>
><property>
>
>    <description>The address of the applications manager interface in the
>RM.</description>
>
>    <name>yarn.resourcemanager.address</name>
>
>    <value>192.168.145.37:8032</value>
>
>  </property>
>
><property>
>
>    <description>The address of the RM admin interface.</description>
>
>    <name>yarn.resourcemanager.admin.address</name>
>
>    <value>192.168.145.37:8033</value>
>
>  </property>
>
>
>
>Earlier these had 0.0.0.0 values. Is there any other workaround?
>
>
>
>My question is on a cluster environment where do I need to keep the
>samza-job-package-0.7.0-dist.tar.gz file.
>
>I am submitting the job from the RM node and I have to keep the
>samza-job-package-0.7.0-dist.tar.gz file on both the RM and NM machines.
>
>Otherwise I receives the samza-job-package-0.7.0-dist.tar.gz file not
>found.
>
>
>
>
>
>
>
>Keeping the samza job file at both the NM and RM machine at a similar
>location gives me exception of different timestamps as the machines has
>different time zones:
>
>
>
>
>
>Application application_1384196041399_0002 failed 1 times due to AM
>Container for appattempt_1384196041399_0002_000001 exited with exitCode:
>-1000 due to: RemoteTrace:
>
>java.io.IOException: Resource
>file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src
>filesystem (expected 1384196169000, was 1383911494000
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java
>:908)
>
>at java.lang.Thread.run(Thread.java:662)
>
>at LocalTrace:
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl:
>Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed
>on src filesystem (expected 1384196169000, was 1383911494000
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.jav
>a:217)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizat
>ionService.java:491)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localization
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
>
>at 
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolServ
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(Pro
>tobufRpcEngine.java:454)
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>I am attaching the NM and RM logs for reference.
>
>
>
>Thanks,
>
>-Nirmal
>
>
>
>From: Nirmal Kumar
>
>Sent: Tuesday, November 12, 2013 1:14 AM
>
>To: 
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mailt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org>>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>Hi Chris,
>
>
>
>
>
>
>
>Thanks for the advice.
>
>
>
>
>
>
>
>I put my yarn-site.xml in hello-samza/deploy/yarn/conf directory and also
>put the following echo in hello-samza/deploy/yarn/etc/hadoop/ yarn-env.sh
>file:
>
>
>
>
>
>
>
>I am using the commands:
>
>
>
>nohup bin/yarn resourcemanager > logs/rm.log 2>&1 &
>
>
>
>nohup bin/yarn nodemanager > logs/nm.log 2>&1 &
>
>
>
>
>
>
>
>#export YARN_CONF_DIR="${YARN_CONF_DIR:-$HADOOP_YARN_HOME/conf}"
>
>
>
>export YARN_CONF_DIR="/home/bda/nirmal/hello-samza/deploy/yarn/conf"
>
>
>
>echo "YARN_CONF_DIR in sh: $YARN_CONF_DIR"
>
>
>
>
>
>
>
>Now in my nm.log and rm.log I see
>
>
>
>
>
>
>
>nm.log
>
>
>
>--------
>
>
>
>nohup: ignoring input
>
>
>
>YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf
>
>
>
>13/11/11 19:52:01 INFO nodemanager.NodeManager: STARTUP_MSG:
>
>
>
>/************************************************************
>
>
>
>STARTUP_MSG: Starting NodeManager
>
>
>
>STARTUP_MSG:   host = IMPETUS-DSRV05.impetus.co.in/192.168.145.195
>
>
>
>STARTUP_MSG:   args = []
>
>
>
>STARTUP_MSG:   version = 2.0.5-alpha
>
>
>
>STARTUP_MSG:   classpath =
>
>
>
>
>
>
>
>rm.log
>
>
>
>--------
>
>
>
>nohup: ignoring input
>
>
>
>YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf
>
>
>
>13/11/12 00:24:01 INFO resourcemanager.ResourceManager: STARTUP_MSG:
>
>
>
>/************************************************************
>
>
>
>STARTUP_MSG: Starting ResourceManager
>
>
>
>STARTUP_MSG:   host = IMPETUS-DSRV08.impetus.co.in/192.168.145.37
>
>
>
>STARTUP_MSG:   args = []
>
>
>
>STARTUP_MSG:   version = 2.0.5-alpha
>
>
>
>STARTUP_MSG:   classpath =
>
>
>
>
>
>
>
>This rm.log and nm.log are fine.
>
>
>
>
>
>
>
>hello-samza/deploy/yarn/conf/yarn-site.xml in both the machines is:
>
>
>
><configuration>
>
>
>
>  <property>
>
>
>
>    <name>yarn.scheduler.minimum-allocation-mb</name>
>
>
>
>    <value>128</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
>  <property>
>
>
>
>    <name>yarn.nodemanager.vmem-pmem-ratio</name>
>
>
>
>    <value>10</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.hostname</name>
>
>
>
>    <value>192.168.145.37</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>    <value>192.168.145.37:8031</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.address</name>
>
>
>
>    <value>192.168.145.37:8032</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>   <name>yarn.resourcemanager.admin.address</name>
>
>
>
>   <value>192.168.145.37:8033</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>        <name>yarn.resourcemanager.scheduler.address</name>
>
>
>
>        <value>192.168.145.37:8030</value>
>
>
>
></property>
>
>
>
></configuration>
>
>
>
>
>
>
>
>But when I submit the job using the following command :
>
>
>
>bin/run-job.sh 
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> 
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-c
>onsumer.properties
>
>
>
>
>
>
>
>After submitting the job file successfully I see the following under
>stdout:
>
>
>
>
>
>2013-11-11 19:52:24 ClientHelper [INFO] trying to connect to RM
>/0.0.0.0:8032
>
>
>
>2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005
>
>
>
>2013-11-11 19:52:24 log [INFO] Extract
>jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1
>384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-d
>ist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-IN
>F/ to /tmp/Jetty_0_0_0_0_38376_scalate____.8y41jd/webinf/WEB-INF
>
>
>
>2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine
>using working directory: /tmp/scalate-7888530459139045883-workdir
>
>
>
>2013-11-11 19:52:24 log [INFO] Started
>SelectChannelConnector@0.0.0.0:38376<mailto:SelectChannelConnector@0.0.0.0
>:38376<mailto:SelectChannelConnector@0.0.0.0:38376%3cmailto:SelectChannelC
>onnector@0.0.0.0:38376>>
>
>
>
>2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005
>
>
>
>2013-11-11 19:52:24 log [INFO] Extract
>jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1
>384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-d
>ist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-IN
>F/ to /tmp/Jetty_0_0_0_0_28858_scalate____.sc28cx/webinf/WEB-INF
>
>
>
>2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine
>using working directory: /tmp/scalate-3192563436096941762-workdir
>
>
>
>2013-11-11 19:52:24 log [INFO] Started
>SelectChannelConnector@0.0.0.0:28858<mailto:SelectChannelConnector@0.0.0.0
>:28858<mailto:SelectChannelConnector@0.0.0.0:28858%3cmailto:SelectChannelC
>onnector@0.0.0.0:28858>>
>
>
>
>2013-11-11 19:52:34 SamzaAppMasterLifecycle [INFO] Shutting down.
>
>
>
>2013-11-11 19:52:44 YarnAppMaster [WARN] Listener
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec<mailto:org.apac
>he.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec<mailto:org.apache.samza
>.job.yarn.SamzaAppMasterLifecycle@4095c5ec%3cmailto:org.apache.samza.job.y
>arn.SamzaAppMasterLifecycle@4095c5ec>> failed to shutdown.
>
>
>
>java.lang.reflect.UndeclaredThrowableException
>
>
>
>                at
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrap
>AndThrowException(YarnRemoteExceptionPBImpl.java:135)
>
>
>
>                at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishA
>pplicationMaster(AMRMProtocolPBClientImpl.java:90)
>
>
>
>                at
>org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(A
>MRMClientImpl.java:244)
>
>
>
>                at
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMaste
>rLifecycle.scala:68)
>
>
>
>                at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster
>.scala:70)
>
>
>
>                at
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster
>.scala:69)
>
>
>
>                at
>scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala
>:61)
>
>
>
>                at scala.collection.immutable.List.foreach(List.scala:45)
>
>
>
>                at
>org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
>
>
>
>                at
>org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
>
>
>
>                at
>org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
>
>
>
>Caused by: com.google.protobuf.ServiceException:
>java.net.ConnectException: Call From
>IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused; For
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>                at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.j
>ava:212)
>
>
>
>                at $Proxy12.finishApplicationMaster(Unknown Source)
>
>
>
>                at
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishA
>pplicationMaster(AMRMProtocolPBClientImpl.java:87)
>
>
>
>                ... 9 more
>
>
>
>Caused by: java.net.ConnectException: Call From
>IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on
>connection exception: java.net.ConnectException: Connection refused; For
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>                at
>sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>
>
>
>                at
>sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAcc
>essorImpl.java:39)
>
>
>
>                at
>sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstr
>uctorAccessorImpl.java:27)
>
>
>
>                at
>java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
>
>
>
>                at org.apache.hadoop.ipc.Client.call(Client.java:1239)
>
>
>
>                at
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.j
>ava:202)
>
>
>
>                ... 11 more
>
>
>
>Caused by: java.net.ConnectException: Connection refused
>
>
>
>                at sun.nio.ch.SocketChannelImpl.checkConnect(Native
>Method)
>
>
>
>                at
>sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:567)
>
>
>
>                at
>org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java
>:206)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
>
>
>
>                at
>org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
>
>
>
>                at
>org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
>
>
>
>                at
>org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
>
>
>
>                at
>org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
>
>
>
>                at
>org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
>
>
>
>                at org.apache.hadoop.ipc.Client.call(Client.java:1206)
>
>
>
>                ... 12 more
>
>
>
>
>
>
>
>In the stdout I see the following jar is used when invoking
>org.apache.samza.job.yarn.SamzaAppMaster:
>
>
>
>/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355
>066_0001/container_1384179355066_0001_01_000001/__package/lib/hadoop-yarn-
>common-2.0.5-alpha.jar
>
>
>
>
>
>
>
>This jar too contains the yarn-default.xml. Is this the file that's being
>picked up?
>
>
>
>
>
>
>
>
>
>
>
>Also where do I need to keep the job file ?
>
>
>
>I am setting the job file name in a test-consumer.properties file:
>
>
>
>yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz
>
>
>
>
>
>
>
>And submitting the job from the RM node:
>
>
>
>bin/run-job.sh 
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> 
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-c
>onsumer.properties
>
>
>
>
>
>
>
>But that way do I need to keep the job file in both NM and RM machines on
>the same location(/home/temptest/ i.e. I have created same directories on
>both the machines)?
>
>
>
>
>
>
>
>Keeping the samza job file at both the NM and RM machine at a similar
>location gives me exception:
>
>
>
>
>
>Application application_1384196041399_0002 failed 1 times due to AM
>Container for appattempt_1384196041399_0002_000001 exited with exitCode:
>-1000 due to: RemoteTrace:
>
>java.io.IOException: Resource
>file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src
>filesystem (expected 1384196169000, was 1383911494000
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java
>:908)
>
>at java.lang.Thread.run(Thread.java:662)
>
>at LocalTrace:
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl:
>Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed
>on src filesystem (expected 1384196169000, was 1383911494000
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.jav
>a:217)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizat
>ionService.java:491)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localization
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
>
>at 
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolServ
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(Pro
>tobufRpcEngine.java:454)
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>
>
>
>
>
>
>Regards,
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>
>
>
>
>-----Original Message-----
>
>From: Chris Riccomini [mailto:criccomini@linkedin.com]
>
>Sent: Sunday, November 10, 2013 11:00 AM
>
>To: 
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mailt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org>>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>
>
>Hey Nirmal,
>
>
>
>
>
>
>
>It sounds like the yarn-site.xml is being ignored for some reason. Things
>to do:
>
>
>
>
>
>
>
>1. Could you please send full log files for the RM and NM?
>
>
>
>
>
>
>
>2. You might also try putting your yarn-site.xml in
>hello-samza/deploy/yarn/conf, and explicitly setting the HADOOP_YARN_HOME
>environment variable to:
>
>
>
>
>
>
>
>export HADOOP_YARN_HOME=<path to>/hello-samza/deploy/yarn
>
>
>
>
>
>
>
>Then try running bin/grid start yarn.
>
>
>
>
>
>
>
>3. Try staring yarn WITHOUT bin/grid. This can be done with:
>
>
>
>
>
>
>
>deploy/yarn/bin/yarn resourcemanager
>
>
>
>deploy/yarn/bin/yarn nodemanager
>
>
>
>
>
>
>
>Cheers,
>
>
>
>Chris
>
>
>
>________________________________
>
>
>
>From: Nirmal Kumar
>[nirmal.kumar@impetus.co.in<ma...@impetus.co.in>]
>
>
>
>Sent: Friday, November 08, 2013 5:01 AM
>
>
>
>To: 
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mailt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org>>
>
>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>
>
>Hi Chris,
>
>
>
>
>
>
>
>The below exception is gone if the job file timestamp is same:
>
>
>
>
>
>
>
>
>
>
>
>Application application_1383907331443_0002 failed 1 times due to AM
>Container for appattempt_1383907331443_0002_000001 exited with exitCode:
>-1000 due to: RemoteTrace:
>
>
>
>
>
>
>
>java.io.IOException: Resource
>file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src
>filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>
>
>
>
>
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java
>:908)
>
>
>
>
>
>
>
>at java.lang.Thread.run(Thread.java:662)
>
>
>
>
>
>
>
>at LocalTrace:
>
>
>
>
>
>
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl:
>Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed
>on src filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.jav
>a:217)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizat
>ionService.java:491)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localization
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolServ
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(Pro
>tobufRpcEngine.java:454)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>
>
>PFA the exception that's coming. I am still struggling with the same
>exception i.e. NM trying to connect to 0.0.0.0:8030 I don't know from
>where the NM is picking up this 0.0.0.0:8030 value. Overriding the
>yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.
>
>
>
>
>
>
>
>I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on
>both RM and NM machines:
>
>
>
>
>
>
>
><configuration>
>
>
>
>  <property>
>
>
>
>    <name>yarn.scheduler.minimum-allocation-mb</name>
>
>
>
>    <value>128</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
>  <property>
>
>
>
>    <name>yarn.nodemanager.vmem-pmem-ratio</name>
>
>
>
>    <value>10</value>
>
>
>
>  </property>
>
>
>
>
>
>
>
><property>
>
>
>
>    <name>yarn.resourcemanager.hostname</name>
>
>
>
>   <value>192.168.145.37</value>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>     <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>     <value>192.168.145.37:8031</value>
>
>
>
></property>
>
>
>
></configuration>
>
>
>
>
>
>
>
>
>
>
>
>Regards,
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>From: Nirmal Kumar
>
>
>
>Sent: Friday, November 08, 2013 6:11 PM
>
>
>
>To: 
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mailt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org>>
>
>
>
>Subject: RE: Running Samza on multi node
>
>
>
>
>
>
>
>
>
>
>
>Hi Chris,
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Using just the yarn.resourcemanager.hostname property gives me the
>following exception on the NM:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Caused by: com.google.protobuf.ServiceException:
>java.net.ConnectException: Call From
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on
>connection exception: java.net.ConnectException: Connection refused; For
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I then added the following property as well:
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>   <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>
>
>
>
>   <value>192.168.145.37:8031</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>After this my RM and NM were up and the NM got registered as well:
>
>
>
>
>
>
>
>13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager
>is started.
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO util.RackResolver: Resolved
>IMPETUS-DSRV14.impetus.co.in to /default-rack
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService:
>NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948
>httpPort: 8042) registered with capability: <memory:8192, vCores:16>,
>assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO rmnode.RMNodeImpl:
>IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING
>
>
>
>
>
>
>
>13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node
>IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192,
>vCores:16>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>When submitting the job I'm still getting the same exception:
>
>
>
>
>
>
>
>
>
>
>
>YarnAppMaster [WARN] Listener
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.apac
>he.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.samza
>.job.yarn.SamzaAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.job.y
>arn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.samza.job.yarn.Samz
>aAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMa
>sterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMasterLi
>fecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMasterLifecycl
>e@500c954e>>> failed to shutdown.
>
>
>
>java.lang.reflect.UndeclaredThrowableException
>
>
>
>         at 
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrap
>AndThrowException(YarnRemoteExceptionPBImpl.java:135)
>
>
>
>         at 
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishA
>pplicationMaster(AMRMProtocolPBClientImpl.java:90)
>
>
>
>         at 
>org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(A
>MRMClientImpl.java:244)
>
>
>
>         at 
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMaste
>rLifecycle.scala:68)
>
>
>
>         at 
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster
>.scala:70)
>
>
>
>         at 
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster
>.scala:69)
>
>
>
>         at 
>scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala
>:61)
>
>
>
>         at scala.collection.immutable.List.foreach(List.scala:45)
>
>
>
>         at 
>org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
>
>
>
>         at 
>org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
>
>
>
>         at 
>org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
>
>
>
>Caused by: com.google.protobuf.ServiceException: 
>java.net.ConnectException: Call From 
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on 
>connection exception: java.net.ConnectException: Connection refused; For 
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>         at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.j
>ava:212)
>
>
>
>         at $Proxy12.finishApplicationMaster(Unknown Source)
>
>
>
>         at 
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishA
>pplicationMaster(AMRMProtocolPBClientImpl.java:87)
>
>
>
>         ... 9 more
>
>
>
>Caused by: java.net.ConnectException: Call From 
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on 
>connection exception: java.net.ConnectException: Connection refused; For 
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
>Method)
>
>
>
>         at 
>sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAcc
>essorImpl.java:39)
>
>
>
>         at 
>sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstr
>uctorAccessorImpl.java:27)
>
>
>
>         at 
>java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>
>
>         at 
>org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
>
>
>
>         at 
>org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
>
>
>
>         at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.j
>ava:202)
>
>
>
>         ... 11 more
>
>
>
>Caused by: java.net.ConnectException: Connection refused
>
>
>
>         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>
>
>
>         at 
>sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
>
>
>
>         at 
>org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java
>:206)
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
>
>
>
>         at 
>org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
>
>
>
>         at 
>org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
>
>
>
>         at 
>org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
>
>
>
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
>
>
>
>         ... 12 more
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Where do I need to keep the job file ?
>
>
>
>
>
>
>
>I am setting the job file name in a test-consumer.properties file:
>
>
>
>
>
>
>
>yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>And submitting the job from :
>
>
>
>
>
>
>
>bin/run-job.sh 
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> 
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-c
>onsumer.properties
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>But that way do I need to keep the job file in both NM and RM machines on 
>the same location?
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I tried submitting the job several time with different properties in 
>yarn-site.xml and now I am getting some strange exception. This is 
>probably due to the different timestamps.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Application application_1383907331443_0002 failed 1 times due to AM 
>Container for appattempt_1383907331443_0002_000001 exited with exitCode: 
>-1000 due to: RemoteTrace:
>
>
>
>
>
>
>
>java.io.IOException: Resource 
>file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src 
>filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
>
>
>
>
>
>
>
>at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>
>
>
>
>
>
>
>at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>
>
>
>
>
>
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.
>java:886)
>
>
>
>
>
>
>
>at 
>java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java
>:908)
>
>
>
>
>
>
>
>at java.lang.Thread.run(Thread.java:662)
>
>
>
>
>
>
>
>at LocalTrace:
>
>
>
>
>
>
>
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: 
>Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed 
>on src filesystem (expected 1383904942000, was 1382550495000
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.jav
>a:217)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loca
>lResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.
>java:819)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizat
>ionService.java:491)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.Resou
>rceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.Localization
>ProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolServ
>ice$2.callBlockingMethod(LocalizationProtocol.java:57)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(Pro
>tobufRpcEngine.java:454)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
>
>
>
>
>
>
>
>at java.security.AccessController.doPrivileged(Native Method)
>
>
>
>
>
>
>
>at javax.security.auth.Subject.doAs(Subject.java:396)
>
>
>
>
>
>
>
>at 
>org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.
>java:1478)
>
>
>
>
>
>
>
>at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
>
>
>
>
>
>
>
>.Failing this attempt.. Failing the application.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>When I am using NM and RM on a single node all is running fine. PFA the 
>logs for the job.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Other questions:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>1. Both your NM and RM are running YARN 2.2.0, right?
>
>
>
>
>
>
>
>I am using same YARN on both NM and RM that was downloaded as part of 
>hello-samza application.
>
>
>
>
>
>
>
>2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?
>
>
>
>
>
>
>
>I am forcibly killing the java processes using kill -9 pid command.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Thanks,
>
>
>
>
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>-----Original Message-----
>
>
>
>From: Chris Riccomini [mailto:criccomini@linkedin.com]
>
>
>
>Sent: Thursday, November 07, 2013 9:19 PM
>
>
>
>To: 
>dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mailt
>o:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org>>
>
>
>
>Subject: Re: Running Samza on multi node
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Hey Nirmal,
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Thanks for this detailed report! It makes things much easier to figure 
>out. The problem appears to be that the Samza AM is trying to connect to 
>0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is 
>running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM 
>host). This is causing a timeout, since 8030 isn't open on localhost for 
>the Samza AM, which is running on the NM's box.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>It is somewhat interesting that the NM does connect to the RM for the 
>capacity scheduler. Rather than setting each individual host/port pair, 
>as you've done, I recommend just setting:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>  <property>
>
>
>
>
>
>
>
>    <name>yarn.resourcemanager.hostname</name>
>
>
>
>
>
>
>
>    <value>192.168.145.37</value>
>
>
>
>
>
>
>
>  </property>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Your netstat reports look fine - as expected.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Other questions:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>1. Both your NM and RM are running YARN 2.2.0, right?
>
>
>
>
>
>
>
>2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Regarding (2), it appears that the AM never tries to register. This 
>normally happens. I'm wondering if another failure is being triggered, 
>which is then causing the AM to try and shut itself down. Could you turn 
>on debugging for your Samza job (in log4j.xml), and re-run? I'm curious 
>if the web-service that's starting up, or the registration itself is 
>failing. In a normal execution, you would expect to see:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>    info("Got AM register response. The YARN RM supports container 
>requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I don't see this in your logs, which means the AM is failing (and 
>triggering a shutdown) before it even tries to register.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Cheers,
>
>
>
>
>
>
>
>Chris
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>From: Nirmal Kumar 
><nirmal.kumar@impetus.co.in<mailto:nirmal.kumar@impetus.co.in<mailto:nirma
>l.kumar@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in<mailto:nirmal.ku
>mar@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.kuma
>r@impetus.co.in%3cmailto:nirmal.kumar@impetus.co.in<mailto:nirmal.kumar@im
>petus.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.kumar@impe
>tus.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.kumar@impetu
>s.co.in%3cmailto:nirmal.kumar@impetus.co.in%3cmailto:nirmal.kumar@impetus.
>co.in%3cmailto:nirmal.kumar@impetus.co.in>>>>>
>
>
>
>
>
>
>
>Reply-To: 
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org
>%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.ap
>ache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.inc
>ubator.apache.org%3e%3e><mailto:dev@samza.incubator.apache.org%3cmailto:de
>v@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3c
>mailto:dev@samza.incubator.apache.org%3e%3e%3cmailto:dev@samza.incubator.a
>pache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.in
>cubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3e>" 
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org<
>mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.a
>pache.org<mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubat
>or.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.i
>ncubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@s
>amza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto
>:dev@samza.incubator.apache.org>>>>>
>
>
>
>
>
>
>
>Date: Thursday, November 7, 2013 5:05 AM
>
>
>
>
>
>
>
>To: 
>"dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mai
>lto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org
>%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.ap
>ache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.inc
>ubator.apache.org%3e%3e><mailto:dev@samza.incubator.apache.org%3cmailto:de
>v@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3c
>mailto:dev@samza.incubator.apache.org%3e%3e%3cmailto:dev@samza.incubator.a
>pache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.in
>cubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3e>" 
><dev@samza.incubator.apache.org<mailto:dev@samza.incubator.apache.org<mail
>to:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org<
>mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.
>org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.a
>pache.org<mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubat
>or.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.i
>ncubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@s
>amza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3cmailto
>:dev@samza.incubator.apache.org>>>>>
>
>
>
>
>
>
>
>Subject: Running Samza on multi node
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>All,
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I was able to run the hello-samza application on a single node machine.
>
>
>
>
>
>
>
>Now I am trying to run the hello-samza application on  a 2 node setup.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Node1 has a Resource Manager
>
>
>
>
>
>
>
>Node2 has a Node Manager
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>The NM gets registered with the RM successfully as seen in rm.log of the 
>RM node:
>
>
>
>
>
>
>
>13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager 
>is started.
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO util.RackResolver: Resolved 
>IMPETUS-DSRV14.impetus.co.in to /default-rack
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: 
>NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 
>httpPort: 8042) registered with capability: <memory:8192, vCores:16>, 
>assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO rmnode.RMNodeImpl: 
>IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING
>
>
>
>
>
>
>
>13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node 
>IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, 
>vCores:16>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I am submitting the job from the RM machine using the command line:
>
>
>
>
>
>
>
>bin/run-job.sh 
>--config-factory=org.apache.samza.config.factories.PropertiesConfigFactory
> 
>--config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-c
>onsumer.properties
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>However, I am getting the following exception after submitting the job to 
>YARN:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: 
>container_1383816757258_0001_01_000001
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: 
>appattempt_1383816757258_0001_000001
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: 
>IMPETUS-DSRV14
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 
>8042
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: 
>{task.inputs=kafka.storm-sentence, 
>job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, 
>systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFacto
>ry, job.name=test-Consumer, 
>systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, 
>systems.kafka.consumer.auto.offset.reset=largest, 
>systems.kafka.samza.msg.serde=json, 
>serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFact
>ory, 
>systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionMan
>ager, task.window.ms=10000, 
>task.class=samza.examples.wikipedia.task.TestConsumer, 
>yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-pa
>ckage/target/samza-job-package-0.7.0-dist.tar.gz, 
>systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFacto
>ry, 
>systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.1
>45.195:9093}
>
>
>
>
>
>
>
>2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM 
>/0.0.0.0:8032
>
>
>
>
>
>
>
>2013-11-07 15:05:57 JmxServer [INFO] According to 
>InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in
>
>
>
>
>
>
>
>2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 
>url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrm
>i
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No 
>yarn.container.count specified. Defaulting to one container.
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is 
>overridden to samza_admin-test_Consumer-1-1383816957797-0
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property 
>metadata.broker.list is overridden to 
>192.168.145.195:9092,192.168.145.195:9093
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property 
>auto.offset.reset is overridden to largest
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is 
>overridden to samza_admin-test_Consumer-1-1383816957797-0
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is 
>overridden to undefined-samza-consumer-group-
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property 
>zookeeper.connect is overridden to 192.168.145.195:2181/
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is 
>overridden to samza_admin-test_Consumer-1-1383816957797-0
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property 
>metadata.broker.list is overridden to 
>192.168.145.195:9092,192.168.145.195:9093
>
>
>
>
>
>
>
>2013-11-07 15:05:57 VerifiableProperties [INFO] Property 
>request.timeout.ms is overridden to 6000
>
>
>
>
>
>
>
>2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker 
>id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) 
>Set(storm-sentence)
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 
>for producing
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 
>192.168.145.195:9092
>
>
>
>
>
>
>
>2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 
>39152, tracking port 26751
>
>
>
>
>
>
>
>2013-11-07 15:05:57 log [INFO] Logging to 
>org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via 
>org.eclipse.jetty.util.log.Slf4jLog
>
>
>
>
>
>
>
>2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM 
>/0.0.0.0:8032
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Extract 
>jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application
>_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-
>dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-I
>NF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF
>
>
>
>
>
>
>
>2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine 
>using working directory: /tmp/scalate-5279562760844696556-workdir
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Started 
>SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:3915
>2<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.
>0.0.0%3e:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChanne
>lConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmail
>to:SelectChannelConnector@0.0.0.0%3e:39152<mailto:SelectChannelConnector@0
>.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectCha
>nnelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cm
>ailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0
>.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannel
>Connector@0.0.0.0%3e:39152>>>
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Extract 
>jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application
>_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-
>dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-I
>NF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF
>
>
>
>
>
>
>
>2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine 
>using working directory: /tmp/scalate-5582747144249485577-workdir
>
>
>
>
>
>
>
>2013-11-07 15:05:58 log [INFO] Started 
>SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:2675
>1<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.
>0.0.0%3e:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChanne
>lConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmail
>to:SelectChannelConnector@0.0.0.0%3e:26751<mailto:SelectChannelConnector@0
>.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectCha
>nnelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cm
>ailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0
>.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannel
>Connector@0.0.0.0%3e:26751>>>
>
>
>
>
>
>
>
>2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.
>
>
>
>
>
>
>
>2013-11-07 15:06:18 YarnAppMaster [WARN] Listener 
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.apac
>he.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.samza
>.job.yarn.SamzaAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.job.y
>arn.SamzaAppMasterLifecycle@500c954e<mailto:org.apache.samza.job.yarn.Samz
>aAppMasterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMa
>sterLifecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMasterLi
>fecycle@500c954e%3cmailto:org.apache.samza.job.yarn.SamzaAppMasterLifecycl
>e@500c954e>>> failed to shutdown.
>
>
>
>
>
>
>
>java.lang.reflect.UndeclaredThrowableException
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrap
>AndThrowException(YarnRemoteExceptionPBImpl.java:135)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishA
>pplicationMaster(AMRMProtocolPBClientImpl.java:90)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(A
>MRMClientImpl.java:244)
>
>
>
>
>
>
>
>         at 
>org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMaste
>rLifecycle.scala:68)
>
>
>
>
>
>
>
>         at 
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster
>.scala:70)
>
>
>
>
>
>
>
>         at 
>org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster
>.scala:69)
>
>
>
>
>
>
>
>         at 
>scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala
>:61)
>
>
>
>
>
>
>
>         at scala.collection.immutable.List.foreach(List.scala:45)
>
>
>
>
>
>
>
>         at 
>org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
>
>
>
>
>
>
>
>         at 
>org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
>
>
>
>
>
>
>
>         at 
>org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
>
>
>
>
>
>
>
>Caused by: com.google.protobuf.ServiceException: 
>java.net.ConnectException: Call From 
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on 
>connection exception: java.net.ConnectException: Connection refused; For 
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.j
>ava:212)
>
>
>
>
>
>
>
>         at $Proxy12.finishApplicationMaster(Unknown Source)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishA
>pplicationMaster(AMRMProtocolPBClientImpl.java:87)
>
>
>
>
>
>
>
>         ... 9 more
>
>
>
>
>
>
>
>Caused by: java.net.ConnectException: Call From 
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on 
>connection exception: java.net.ConnectException: Connection refused; For 
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
>Method)
>
>
>
>
>
>
>
>         at 
>sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAcc
>essorImpl.java:39)
>
>
>
>
>
>
>
>         at 
>sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstr
>uctorAccessorImpl.java:27)
>
>
>
>
>
>
>
>         at 
>java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
>
>
>
>
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.j
>ava:202)
>
>
>
>
>
>
>
>         ... 11 more
>
>
>
>
>
>
>
>Caused by: java.net.ConnectException: Connection refused
>
>
>
>
>
>
>
>         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>
>
>
>
>
>
>
>         at 
>sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java
>:206)
>
>
>
>
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
>
>
>
>
>
>
>
>         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
>
>
>
>
>
>
>
>         at 
>org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
>
>
>
>
>
>
>
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
>
>
>
>
>
>
>
>         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
>
>
>
>
>
>
>
>         ... 12 more
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>I have changed the following properties in the 
>hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager 
>machine:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.scheduler.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8030</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.resource-tracker.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8031</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8032</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.admin.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8033</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
><property>
>
>
>
>
>
>
>
>                <name>yarn.resourcemanager.webapp.address</name>
>
>
>
>
>
>
>
>                <value>192.168.145.37:8088</value>
>
>
>
>
>
>
>
></property>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>These properties are reflected on the UI screen as well:
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>[<ci...@01CEDBE8.1B2F1890>cid:image001.png@01CEDBE8.1B2F1890]
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>But this overriding of the yarn.resourcemanager.scheduler.address to 
>192.168.145.37:8030 does not rectify the error.
>
>
>
>
>
>
>
>I still get:
>
>
>
>
>
>
>
>Caused by: com.google.protobuf.ServiceException: 
>java.net.ConnectException: Call From 
>IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on 
>connection exception: java.net.ConnectException: Connection refused; For 
>more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Nestat on the RM machine shows me:
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8088  :::*                      
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8030  :::*                      
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8031  :::*                      
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8032  :::*                      
>  LISTEN      14595/java
>
>
>
>
>
>
>
>tcp        0      0 ::ffff:192.168.145.37:8033  :::*                      
>  LISTEN      14595/java
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Nestat on the NM machine shows me:
>
>
>
>
>
>
>
>tcp        0      0 :::8040                     :::*                      
>  LISTEN      1331/java
>
>
>
>
>
>
>
>tcp        0      0 :::8042                     :::*                      
>  LISTEN      1331/java
>
>
>
>
>
>
>
>tcp        0      0 :::56877                    :::*                      
>  LISTEN      1331/java
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Kindly help me how to rectify this error.
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>Regards,
>
>
>
>
>
>
>
>-Nirmal
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential, 
>proprietary, privileged or otherwise protected by law. The message is 
>intended solely for the named addressee. If received in error, please 
>destroy and notify the sender. Any use of this email is prohibited when 
>received in error. Impetus does not represent, warrant and/or guarantee, 
>that the integrity of this communication has been maintained nor that the 
>communication is free of errors, virus, interception or interference.
>
>
>
>
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential, 
>proprietary, privileged or otherwise protected by law. The message is 
>intended solely for the named addressee. If received in error, please 
>destroy and notify the sender. Any use of this email is prohibited when 
>received in error. Impetus does not represent, warrant and/or guarantee, 
>that the integrity of this communication has been maintained nor that the 
>communication is free of errors, virus, interception or interference.
>
>
>
>
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential, 
>proprietary, privileged or otherwise protected by law. The message is 
>intended solely for the named addressee. If received in error, please 
>destroy and notify the sender. Any use of this email is prohibited when 
>received in error. Impetus does not represent, warrant and/or guarantee, 
>that the integrity of this communication has been maintained nor that the 
>communication is free of errors, virus, interception or interference.
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential, 
>proprietary, privileged or otherwise protected by law. The message is 
>intended solely for the named addressee. If received in error, please 
>destroy and notify the sender. Any use of this email is prohibited when 
>received in error. Impetus does not represent, warrant and/or guarantee, 
>that the integrity of this communication has been maintained nor that the 
>communication is free of errors, virus, interception or interference.
>
>
>
>________________________________
>
>
>
>
>
>
>
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential, 
>proprietary, privileged or otherwise protected by law. The message is 
>intended solely for the named addressee. If received in error, please 
>destroy and notify the sender. Any use of this email is prohibited when 
>received in error. Impetus does not represent, warrant and/or guarantee, 
>that the integrity of this communication has been maintained nor that the 
>communication is free of errors, virus, interception or interference.
>
>________________________________
>
>
>
>
>
>
>NOTE: This message may contain information that is confidential, 
>proprietary, privileged or otherwise protected by law. The message is 
>intended solely for the named addressee. If received in error, please 
>destroy and notify the sender. Any use of this email is prohibited when 
>received in error. Impetus does not represent, warrant and/or guarantee, 
>that the integrity of this communication has been maintained nor that the 
>communication is free of errors, virus, interception or interference.


RE: Running Samza on multi node

Posted by Nirmal Kumar <ni...@impetus.co.in>.
Hello Chris,



Thanks for your help.



I am comparing Storm + Kafka0.8 with Samza.



I have used the following deployment in a 3 node cluster:


Node

Components

Node 1

Zookeeper
Storm Nimbus\YARN RM (Master)
Storm UI
Kafka Broker-1
Kafka Broker-2
Kafka Producer

Node 2


Storm Supervisor\YARN NM (Worker)


Node 3

Storm Supervisor\YARN NM (Worker)



The Storm topology is deployed with the following configuration:

o    Num_tasks:4

o    Num_workers:2



I am using the same Java Program for publishing the messages.

On the Storm\Samza side I am simply consuming the messages from the Kafka topic.



Is this is a correct way to do the comparison? Are there any other things that need to be taken care of?



Thanks,

-Nirmal



-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Wednesday, November 13, 2013 2:30 AM
To: dev@samza.incubator.apache.org
Subject: Re: Running Samza on multi node



Hey Nirmal,



Glad to hear you've made progress.



When trying to run the .tar.gz packages in a cluster, you'll want to keep them on HDFS or an HTTP server. At LinkedIn, we use HTTP servers, and the yarn.package.path becomes an http:// address. YARN manages the download and unzip of these packages.



Cheers,

Chris



From: Nirmal Kumar <ni...@impetus.co.in>>>

Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Date: Monday, November 11, 2013 10:46 PM

To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Subject: RE: Running Samza on multi node



Hi Chris,



I updated the yarn-default.xml in hadoop-yarn-common-2.0.5-alpha.jar present in samza-job-package-0.7.0-dist.tar.gz and finally got the job running.



I updated the following in yarn-default.xml:

<property>

    <description>The address of the scheduler interface.</description>

    <name>yarn.resourcemanager.scheduler.address</name>

    <value>192.168.145.37:8030</value>

  </property>

<property>

    <name>yarn.resourcemanager.resource-tracker.address</name>

    <value>192.168.145.37:8031</value>

 </property>

<property>

    <description>The address of the applications manager interface in the RM.</description>

    <name>yarn.resourcemanager.address</name>

    <value>192.168.145.37:8032</value>

  </property>

<property>

    <description>The address of the RM admin interface.</description>

    <name>yarn.resourcemanager.admin.address</name>

    <value>192.168.145.37:8033</value>

  </property>



Earlier these had 0.0.0.0 values. Is there any other workaround?



My question is on a cluster environment where do I need to keep the samza-job-package-0.7.0-dist.tar.gz file.

I am submitting the job from the RM node and I have to keep the samza-job-package-0.7.0-dist.tar.gz file on both the RM and NM machines.

Otherwise I receives the samza-job-package-0.7.0-dist.tar.gz file not found.







Keeping the samza job file at both the NM and RM machine at a similar location gives me exception of different timestamps as the machines has different time zones:





Application application_1384196041399_0002 failed 1 times due to AM Container for appattempt_1384196041399_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:

java.io.IOException: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000

at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)

at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)

at java.lang.Thread.run(Thread.java:662)

at LocalTrace:

org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)

at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)

at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)

at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)

at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.





I am attaching the NM and RM logs for reference.



Thanks,

-Nirmal



From: Nirmal Kumar

Sent: Tuesday, November 12, 2013 1:14 AM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>>

Subject: RE: Running Samza on multi node





Hi Chris,







Thanks for the advice.







I put my yarn-site.xml in hello-samza/deploy/yarn/conf directory and also put the following echo in hello-samza/deploy/yarn/etc/hadoop/ yarn-env.sh file:







I am using the commands:



nohup bin/yarn resourcemanager > logs/rm.log 2>&1 &



nohup bin/yarn nodemanager > logs/nm.log 2>&1 &







#export YARN_CONF_DIR="${YARN_CONF_DIR:-$HADOOP_YARN_HOME/conf}"



export YARN_CONF_DIR="/home/bda/nirmal/hello-samza/deploy/yarn/conf"



echo "YARN_CONF_DIR in sh: $YARN_CONF_DIR"







Now in my nm.log and rm.log I see







nm.log



--------



nohup: ignoring input



YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf



13/11/11 19:52:01 INFO nodemanager.NodeManager: STARTUP_MSG:



/************************************************************



STARTUP_MSG: Starting NodeManager



STARTUP_MSG:   host = IMPETUS-DSRV05.impetus.co.in/192.168.145.195



STARTUP_MSG:   args = []



STARTUP_MSG:   version = 2.0.5-alpha



STARTUP_MSG:   classpath =







rm.log



--------



nohup: ignoring input



YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf



13/11/12 00:24:01 INFO resourcemanager.ResourceManager: STARTUP_MSG:



/************************************************************



STARTUP_MSG: Starting ResourceManager



STARTUP_MSG:   host = IMPETUS-DSRV08.impetus.co.in/192.168.145.37



STARTUP_MSG:   args = []



STARTUP_MSG:   version = 2.0.5-alpha



STARTUP_MSG:   classpath =







This rm.log and nm.log are fine.







hello-samza/deploy/yarn/conf/yarn-site.xml in both the machines is:



<configuration>



  <property>



    <name>yarn.scheduler.minimum-allocation-mb</name>



    <value>128</value>



  </property>







  <property>



    <name>yarn.nodemanager.vmem-pmem-ratio</name>



    <value>10</value>



  </property>







<property>



    <name>yarn.resourcemanager.hostname</name>



    <value>192.168.145.37</value>



</property>







<property>



    <name>yarn.resourcemanager.resource-tracker.address</name>



    <value>192.168.145.37:8031</value>



</property>







<property>



    <name>yarn.resourcemanager.address</name>



    <value>192.168.145.37:8032</value>



</property>







<property>



   <name>yarn.resourcemanager.admin.address</name>



   <value>192.168.145.37:8033</value>



</property>







<property>



        <name>yarn.resourcemanager.scheduler.address</name>



        <value>192.168.145.37:8030</value>



</property>



</configuration>







But when I submit the job using the following command :



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







After submitting the job file successfully I see the following under stdout:





2013-11-11 19:52:24 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032



2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005



2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_38376_scalate____.8y41jd/webinf/WEB-INF



2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-7888530459139045883-workdir



2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:38376<mailto:SelectChannelConnector@0.0.0.0:38376<mailto:SelectChannelConnector@0.0.0.0:38376%3cmailto:SelectChannelConnector@0.0.0.0:38376>>



2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005



2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_28858_scalate____.sc28cx/webinf/WEB-INF



2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-3192563436096941762-workdir



2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:28858<mailto:SelectChannelConnector@0.0.0.0:28858<mailto:SelectChannelConnector@0.0.0.0:28858%3cmailto:SelectChannelConnector@0.0.0.0:28858>>



2013-11-11 19:52:34 SamzaAppMasterLifecycle [INFO] Shutting down.



2013-11-11 19:52:44 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec<ma...@4095c5ec>> failed to shutdown.



java.lang.reflect.UndeclaredThrowableException



                at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)



                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)



                at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)



                at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)



                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)



                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)



                at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)



                at scala.collection.immutable.List.foreach(List.scala:45)



                at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)



                at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)



                at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)



                at $Proxy12.finishApplicationMaster(Unknown Source)



                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)



                ... 9 more



Caused by: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



                at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)



                at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)



                at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)



                at java.lang.reflect.Constructor.newInstance(Constructor.java:513)



                at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)



                at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)



                at org.apache.hadoop.ipc.Client.call(Client.java:1239)



                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)



                ... 11 more



Caused by: java.net.ConnectException: Connection refused



                at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)



                at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:567)



                at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)



                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)



                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)



                at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)



                at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)



                at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)



                at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)



                at org.apache.hadoop.ipc.Client.call(Client.java:1206)



                ... 12 more







In the stdout I see the following jar is used when invoking org.apache.samza.job.yarn.SamzaAppMaster:



/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/container_1384179355066_0001_01_000001/__package/lib/hadoop-yarn-common-2.0.5-alpha.jar







This jar too contains the yarn-default.xml. Is this the file that's being picked up?











Also where do I need to keep the job file ?



I am setting the job file name in a test-consumer.properties file:



yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz







And submitting the job from the RM node:



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







But that way do I need to keep the job file in both NM and RM machines on the same location(/home/temptest/ i.e. I have created same directories on both the machines)?







Keeping the samza job file at both the NM and RM machine at a similar location gives me exception:





Application application_1384196041399_0002 failed 1 times due to AM Container for appattempt_1384196041399_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:

java.io.IOException: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000

at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)

at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)

at java.lang.Thread.run(Thread.java:662)

at LocalTrace:

org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)

at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)

at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)

at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)

at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.











Regards,



-Nirmal











-----Original Message-----

From: Chris Riccomini [mailto:criccomini@linkedin.com]

Sent: Sunday, November 10, 2013 11:00 AM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>>

Subject: RE: Running Samza on multi node







Hey Nirmal,







It sounds like the yarn-site.xml is being ignored for some reason. Things to do:







1. Could you please send full log files for the RM and NM?







2. You might also try putting your yarn-site.xml in hello-samza/deploy/yarn/conf, and explicitly setting the HADOOP_YARN_HOME environment variable to:







export HADOOP_YARN_HOME=<path to>/hello-samza/deploy/yarn







Then try running bin/grid start yarn.







3. Try staring yarn WITHOUT bin/grid. This can be done with:







deploy/yarn/bin/yarn resourcemanager



deploy/yarn/bin/yarn nodemanager







Cheers,



Chris



________________________________



From: Nirmal Kumar [nirmal.kumar@impetus.co.in<ma...@impetus.co.in>]



Sent: Friday, November 08, 2013 5:01 AM



To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>>



Subject: RE: Running Samza on multi node







Hi Chris,







The below exception is gone if the job file timestamp is same:











Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:







java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000







at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)







at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)







at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)







at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)







at java.security.AccessController.doPrivileged(Native Method)







at javax.security.auth.Subject.doAs(Subject.java:396)







at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)







at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)







at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)







at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)







at java.util.concurrent.FutureTask.run(FutureTask.java:138)







at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)







at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)







at java.util.concurrent.FutureTask.run(FutureTask.java:138)







at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)







at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)







at java.lang.Thread.run(Thread.java:662)







at LocalTrace:







org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000







at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)







at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)







at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)







at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)







at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)







at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)







at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)







at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)







at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)







at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)







at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)







at java.security.AccessController.doPrivileged(Native Method)







at javax.security.auth.Subject.doAs(Subject.java:396)







at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)







at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)



.Failing this attempt.. Failing the application.







PFA the exception that's coming. I am still struggling with the same exception i.e. NM trying to connect to 0.0.0.0:8030 I don't know from where the NM is picking up this 0.0.0.0:8030 value. Overriding the  yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.







I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on both RM and NM machines:







<configuration>



  <property>



    <name>yarn.scheduler.minimum-allocation-mb</name>



    <value>128</value>



  </property>







  <property>



    <name>yarn.nodemanager.vmem-pmem-ratio</name>



    <value>10</value>



  </property>







<property>



    <name>yarn.resourcemanager.hostname</name>



   <value>192.168.145.37</value>



</property>







<property>



     <name>yarn.resourcemanager.resource-tracker.address</name>



     <value>192.168.145.37:8031</value>



</property>



</configuration>











Regards,



-Nirmal







From: Nirmal Kumar



Sent: Friday, November 08, 2013 6:11 PM



To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>>



Subject: RE: Running Samza on multi node











Hi Chris,















Using just the yarn.resourcemanager.hostname property gives me the following exception on the NM:















Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused















I then added the following property as well:







<property>







   <name>yarn.resourcemanager.resource-tracker.address</name>







   <value>192.168.145.37:8031</value>







</property>















After this my RM and NM were up and the NM got registered as well:







13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager is started.







13/11/08 16:12:19 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack







13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948







13/11/08 16:12:19 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING







13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192, vCores:16>















When submitting the job I'm still getting the same exception:











YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>>> failed to shutdown.



java.lang.reflect.UndeclaredThrowableException



         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)



         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)



         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)



         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)



         at scala.collection.immutable.List.foreach(List.scala:45)



         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)



         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)



         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)



         at $Proxy12.finishApplicationMaster(Unknown Source)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)



         ... 9 more



Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)



         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)



         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)



         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)



         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)



         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)



         at org.apache.hadoop.ipc.Client.call(Client.java:1239)



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)



         ... 11 more



Caused by: java.net.ConnectException: Connection refused



         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)



         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)



         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)



         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)



         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)



         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)



         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)



         at org.apache.hadoop.ipc.Client.call(Client.java:1206)



         ... 12 more















Where do I need to keep the job file ?







I am setting the job file name in a test-consumer.properties file:







yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz















And submitting the job from :







bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties















But that way do I need to keep the job file in both NM and RM machines on the same location?















I tried submitting the job several time with different properties in yarn-site.xml and now I am getting some strange exception. This is probably due to the different timestamps.















Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:







java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000







at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)







at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)







at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)







at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)







at java.security.AccessController.doPrivileged(Native Method)







at javax.security.auth.Subject.doAs(Subject.java:396)







at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)







at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)







at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)







at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)







at java.util.concurrent.FutureTask.run(FutureTask.java:138)







at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)







at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)







at java.util.concurrent.FutureTask.run(FutureTask.java:138)







at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)







at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)







at java.lang.Thread.run(Thread.java:662)







at LocalTrace:







org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000







at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)







at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)







at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)







at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)







at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)







at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)







at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)







at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)







at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)







at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)







at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)







at java.security.AccessController.doPrivileged(Native Method)







at javax.security.auth.Subject.doAs(Subject.java:396)







at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)







at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)







.Failing this attempt.. Failing the application.















When I am using NM and RM on a single node all is running fine. PFA the logs for the job.















Other questions:















1. Both your NM and RM are running YARN 2.2.0, right?







I am using same YARN on both NM and RM that was downloaded as part of hello-samza application.







2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?







I am forcibly killing the java processes using kill -9 pid command.















Thanks,







-Nirmal















-----Original Message-----



From: Chris Riccomini [mailto:criccomini@linkedin.com]



Sent: Thursday, November 07, 2013 9:19 PM



To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>>



Subject: Re: Running Samza on multi node















Hey Nirmal,















Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.















It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:















  <property>







    <name>yarn.resourcemanager.hostname</name>







    <value>192.168.145.37</value>







  </property>















Your netstat reports look fine - as expected.















Other questions:















1. Both your NM and RM are running YARN 2.2.0, right?







2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?















Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:























    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))















I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.















Cheers,







Chris















From: Nirmal Kumar <ni...@impetus.co.in>>>>>







Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3e>" <de...@samza.incubator.apache.org>>>>>







Date: Thursday, November 7, 2013 5:05 AM







To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e%3e>" <de...@samza.incubator.apache.org>>>>>







Subject: Running Samza on multi node















All,















I was able to run the hello-samza application on a single node machine.







Now I am trying to run the hello-samza application on  a 2 node setup.















Node1 has a Resource Manager







Node2 has a Node Manager















The NM gets registered with the RM successfully as seen in rm.log of the RM node:







13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.







13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack







13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093







13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING







13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>















I am submitting the job from the RM machine using the command line:







bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties















However, I am getting the following exception after submitting the job to YARN:















2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}







2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032







2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in







2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi







2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.







2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties







2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0







2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093







2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties







2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest







2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0







2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-







2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/







2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties







2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0







2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093







2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000







2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)







2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing







2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092







2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751







2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog







2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032







2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005







2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF







2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir







2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152>>>







2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005







2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF







2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir







2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751>>>







2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.







2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>>> failed to shutdown.







java.lang.reflect.UndeclaredThrowableException







         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)







         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)







         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)







         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)







         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)







         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)







         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)







         at scala.collection.immutable.List.foreach(List.scala:45)







         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)







         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)







         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)







Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)







         at $Proxy12.finishApplicationMaster(Unknown Source)







         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)







         ... 9 more







Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)







         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)







         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)







         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)







         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)







         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)







         at org.apache.hadoop.ipc.Client.call(Client.java:1239)







         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)







         ... 11 more







Caused by: java.net.ConnectException: Connection refused







         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)







         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)







         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)







         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)







         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)







         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)







         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)







         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)







         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)







         at org.apache.hadoop.ipc.Client.call(Client.java:1206)







         ... 12 more























I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:















<property>







                <name>yarn.resourcemanager.scheduler.address</name>







                <value>192.168.145.37:8030</value>







</property>







<property>







                <name>yarn.resourcemanager.resource-tracker.address</name>







                <value>192.168.145.37:8031</value>







</property>







<property>







                <name>yarn.resourcemanager.address</name>







                <value>192.168.145.37:8032</value>







</property>







<property>







                <name>yarn.resourcemanager.admin.address</name>







                <value>192.168.145.37:8033</value>







</property>







<property>







                <name>yarn.resourcemanager.webapp.address</name>







                <value>192.168.145.37:8088</value>







</property>































These properties are reflected on the UI screen as well:















[<ci...@01CEDBE8.1B2F1890>cid:image001.png@01CEDBE8.1B2F1890]















But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.







I still get:







Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused















Nestat on the RM machine shows me:







tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java







tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java







tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java







tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java







tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java















Nestat on the NM machine shows me:







tcp        0      0 :::8040                     :::*                        LISTEN      1331/java







tcp        0      0 :::8042                     :::*                        LISTEN      1331/java







tcp        0      0 :::56877                    :::*                        LISTEN      1331/java















Kindly help me how to rectify this error.















Regards,







-Nirmal















________________________________























































NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.







________________________________



























NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.







________________________________



























NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

Re: Running Samza on multi node

Posted by Chris Riccomini <cr...@linkedin.com>.
Hey Nirmal,

Glad to hear you've made progress.

When trying to run the .tar.gz packages in a cluster, you'll want to keep them on HDFS or an HTTP server. At LinkedIn, we use HTTP servers, and the yarn.package.path becomes an http:// address. YARN manages the download and unzip of these packages.

Cheers,
Chris

From: Nirmal Kumar <ni...@impetus.co.in>>
Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>" <de...@samza.incubator.apache.org>>
Date: Monday, November 11, 2013 10:46 PM
To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>" <de...@samza.incubator.apache.org>>
Subject: RE: Running Samza on multi node

Hi Chris,

I updated the yarn-default.xml in hadoop-yarn-common-2.0.5-alpha.jar present in samza-job-package-0.7.0-dist.tar.gz and finally got the job running.

I updated the following in yarn-default.xml:
<property>
    <description>The address of the scheduler interface.</description>
    <name>yarn.resourcemanager.scheduler.address</name>
    <value>192.168.145.37:8030</value>
  </property>
<property>
    <name>yarn.resourcemanager.resource-tracker.address</name>
    <value>192.168.145.37:8031</value>
  </property>
<property>
    <description>The address of the applications manager interface in the RM.</description>
    <name>yarn.resourcemanager.address</name>
    <value>192.168.145.37:8032</value>
  </property>
<property>
    <description>The address of the RM admin interface.</description>
    <name>yarn.resourcemanager.admin.address</name>
    <value>192.168.145.37:8033</value>
  </property>

Earlier these had 0.0.0.0 values. Is there any other workaround?

My question is on a cluster environment where do I need to keep the samza-job-package-0.7.0-dist.tar.gz file.
I am submitting the job from the RM node and I have to keep the samza-job-package-0.7.0-dist.tar.gz file on both the RM and NM machines.
Otherwise I receives the samza-job-package-0.7.0-dist.tar.gz file not found.



Keeping the samza job file at both the NM and RM machine at a similar location gives me exception of different timestamps as the machines has different time zones:


Application application_1384196041399_0002 failed 1 times due to AM Container for appattempt_1384196041399_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:
java.io.IOException: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)
at LocalTrace:
org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)
at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
.Failing this attempt.. Failing the application.


I am attaching the NM and RM logs for reference.

Thanks,
-Nirmal

From: Nirmal Kumar
Sent: Tuesday, November 12, 2013 1:14 AM
To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>
Subject: RE: Running Samza on multi node


Hi Chris,



Thanks for the advice.



I put my yarn-site.xml in hello-samza/deploy/yarn/conf directory and also put the following echo in hello-samza/deploy/yarn/etc/hadoop/ yarn-env.sh file:



I am using the commands:

nohup bin/yarn resourcemanager > logs/rm.log 2>&1 &

nohup bin/yarn nodemanager > logs/nm.log 2>&1 &



#export YARN_CONF_DIR="${YARN_CONF_DIR:-$HADOOP_YARN_HOME/conf}"

export YARN_CONF_DIR="/home/bda/nirmal/hello-samza/deploy/yarn/conf"

echo "YARN_CONF_DIR in sh: $YARN_CONF_DIR"



Now in my nm.log and rm.log I see



nm.log

--------

nohup: ignoring input

YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf

13/11/11 19:52:01 INFO nodemanager.NodeManager: STARTUP_MSG:

/************************************************************

STARTUP_MSG: Starting NodeManager

STARTUP_MSG:   host = IMPETUS-DSRV05.impetus.co.in/192.168.145.195

STARTUP_MSG:   args = []

STARTUP_MSG:   version = 2.0.5-alpha

STARTUP_MSG:   classpath =



rm.log

--------

nohup: ignoring input

YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf

13/11/12 00:24:01 INFO resourcemanager.ResourceManager: STARTUP_MSG:

/************************************************************

STARTUP_MSG: Starting ResourceManager

STARTUP_MSG:   host = IMPETUS-DSRV08.impetus.co.in/192.168.145.37

STARTUP_MSG:   args = []

STARTUP_MSG:   version = 2.0.5-alpha

STARTUP_MSG:   classpath =



This rm.log and nm.log are fine.



hello-samza/deploy/yarn/conf/yarn-site.xml in both the machines is:

<configuration>

  <property>

    <name>yarn.scheduler.minimum-allocation-mb</name>

    <value>128</value>

  </property>



  <property>

    <name>yarn.nodemanager.vmem-pmem-ratio</name>

    <value>10</value>

  </property>



<property>

    <name>yarn.resourcemanager.hostname</name>

    <value>192.168.145.37</value>

</property>



<property>

    <name>yarn.resourcemanager.resource-tracker.address</name>

    <value>192.168.145.37:8031</value>

</property>



<property>

    <name>yarn.resourcemanager.address</name>

    <value>192.168.145.37:8032</value>

</property>



<property>

   <name>yarn.resourcemanager.admin.address</name>

   <value>192.168.145.37:8033</value>

</property>



<property>

        <name>yarn.resourcemanager.scheduler.address</name>

        <value>192.168.145.37:8030</value>

</property>

</configuration>



But when I submit the job using the following command :

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



After submitting the job file successfully I see the following under stdout:


2013-11-11 19:52:24 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005

2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_38376_scalate____.8y41jd/webinf/WEB-INF

2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-7888530459139045883-workdir

2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:38376<mailto:SelectChannelConnector@0.0.0.0:38376>

2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005

2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_28858_scalate____.sc28cx/webinf/WEB-INF

2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-3192563436096941762-workdir

2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:28858<mailto:SelectChannelConnector@0.0.0.0:28858>

2013-11-11 19:52:34 SamzaAppMasterLifecycle [INFO] Shutting down.

2013-11-11 19:52:44 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec<ma...@4095c5ec> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

                at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

                at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

                at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

                at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

                at scala.collection.immutable.List.foreach(List.scala:45)

                at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

                at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

                at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

                at $Proxy12.finishApplicationMaster(Unknown Source)

                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

                ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

                at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

                at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

                at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

                at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

                at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

                at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

                at org.apache.hadoop.ipc.Client.call(Client.java:1239)

                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

                ... 11 more

Caused by: java.net.ConnectException: Connection refused

                at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

                at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:567)

                at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

                at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

                at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

                at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

                at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

                at org.apache.hadoop.ipc.Client.call(Client.java:1206)

                ... 12 more



In the stdout I see the following jar is used when invoking org.apache.samza.job.yarn.SamzaAppMaster:

/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/container_1384179355066_0001_01_000001/__package/lib/hadoop-yarn-common-2.0.5-alpha.jar



This jar too contains the yarn-default.xml. Is this the file that’s being picked up?





Also where do I need to keep the job file ?

I am setting the job file name in a test-consumer.properties file:

yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz



And submitting the job from the RM node:

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



But that way do I need to keep the job file in both NM and RM machines on the same location(/home/temptest/ i.e. I have created same directories on both the machines)?



Keeping the samza job file at both the NM and RM machine at a similar location gives me exception:


Application application_1384196041399_0002 failed 1 times due to AM Container for appattempt_1384196041399_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:
java.io.IOException: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)
at LocalTrace:
org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)
at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
.Failing this attempt.. Failing the application.





Regards,

-Nirmal





-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Sunday, November 10, 2013 11:00 AM
To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>
Subject: RE: Running Samza on multi node



Hey Nirmal,



It sounds like the yarn-site.xml is being ignored for some reason. Things to do:



1. Could you please send full log files for the RM and NM?



2. You might also try putting your yarn-site.xml in hello-samza/deploy/yarn/conf, and explicitly setting the HADOOP_YARN_HOME environment variable to:



export HADOOP_YARN_HOME=<path to>/hello-samza/deploy/yarn



Then try running bin/grid start yarn.



3. Try staring yarn WITHOUT bin/grid. This can be done with:



deploy/yarn/bin/yarn resourcemanager

deploy/yarn/bin/yarn nodemanager



Cheers,

Chris

________________________________

From: Nirmal Kumar [nirmal.kumar@impetus.co.in<ma...@impetus.co.in>]

Sent: Friday, November 08, 2013 5:01 AM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: RE: Running Samza on multi node



Hi Chris,



The below exception is gone if the job file timestamp is same:





Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:



java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)



at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)



at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



at java.lang.Thread.run(Thread.java:662)



at LocalTrace:



org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)



at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)



at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)



at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)



at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.



PFA the exception that’s coming. I am still struggling with the same exception i.e. NM trying to connect to 0.0.0.0:8030 I don’t know from where the NM is picking up this 0.0.0.0:8030 value. Overriding the  yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.



I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on both RM and NM machines:



<configuration>

  <property>

    <name>yarn.scheduler.minimum-allocation-mb</name>

    <value>128</value>

  </property>



  <property>

    <name>yarn.nodemanager.vmem-pmem-ratio</name>

    <value>10</value>

  </property>



<property>

    <name>yarn.resourcemanager.hostname</name>

   <value>192.168.145.37</value>

</property>



<property>

     <name>yarn.resourcemanager.resource-tracker.address</name>

     <value>192.168.145.37:8031</value>

</property>

</configuration>





Regards,

-Nirmal



From: Nirmal Kumar

Sent: Friday, November 08, 2013 6:11 PM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: RE: Running Samza on multi node





Hi Chris,







Using just the yarn.resourcemanager.hostname property gives me the following exception on the NM:







Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







I then added the following property as well:



<property>



   <name>yarn.resourcemanager.resource-tracker.address</name>



   <value>192.168.145.37:8031</value>



</property>







After this my RM and NM were up and the NM got registered as well:



13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager is started.



13/11/08 16:12:19 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack



13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948



13/11/08 16:12:19 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING



13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192, vCores:16>







When submitting the job I'm still getting the same exception:





YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

         at scala.collection.immutable.List.foreach(List.scala:45)

         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

         at $Proxy12.finishApplicationMaster(Unknown Source)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

         ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

         at org.apache.hadoop.ipc.Client.call(Client.java:1239)

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

         ... 11 more

Caused by: java.net.ConnectException: Connection refused

         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)

         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

         at org.apache.hadoop.ipc.Client.call(Client.java:1206)

         ... 12 more







Where do I need to keep the job file ?



I am setting the job file name in a test-consumer.properties file:



yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz







And submitting the job from :



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







But that way do I need to keep the job file in both NM and RM machines on the same location?







I tried submitting the job several time with different properties in yarn-site.xml and now I am getting some strange exception. This is probably due to the different timestamps.







Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:



java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)



at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)



at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



at java.lang.Thread.run(Thread.java:662)



at LocalTrace:



org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)



at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)



at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)



at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)



at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)



.Failing this attempt.. Failing the application.







When I am using NM and RM on a single node all is running fine. PFA the logs for the job.







Other questions:







1. Both your NM and RM are running YARN 2.2.0, right?



I am using same YARN on both NM and RM that was downloaded as part of hello-samza application.



2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?



I am forcibly killing the java processes using kill -9 pid command.







Thanks,



-Nirmal







-----Original Message-----

From: Chris Riccomini [mailto:criccomini@linkedin.com]

Sent: Thursday, November 07, 2013 9:19 PM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: Re: Running Samza on multi node







Hey Nirmal,







Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.







It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:







  <property>



    <name>yarn.resourcemanager.hostname</name>



    <value>192.168.145.37</value>



  </property>







Your netstat reports look fine – as expected.







Other questions:







1. Both your NM and RM are running YARN 2.2.0, right?



2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?







Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:











    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))







I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.







Cheers,



Chris







From: Nirmal Kumar <ni...@impetus.co.in>>>>



Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e>" <de...@samza.incubator.apache.org>>>>



Date: Thursday, November 7, 2013 5:05 AM



To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e>" <de...@samza.incubator.apache.org>>>>



Subject: Running Samza on multi node







All,







I was able to run the hello-samza application on a single node machine.



Now I am trying to run the hello-samza application on  a 2 node setup.







Node1 has a Resource Manager



Node2 has a Node Manager







The NM gets registered with the RM successfully as seen in rm.log of the RM node:



13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.



13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack



13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093



13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING



13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>







I am submitting the job from the RM machine using the command line:



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







However, I am getting the following exception after submitting the job to YARN:







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}



2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032



2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in



2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi



2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-



2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093



2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000



2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)



2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing



2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092



2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751



2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog



2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032



2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005



2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF



2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir



2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152>>



2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005



2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF



2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir



2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751>>



2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.



2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>> failed to shutdown.



java.lang.reflect.UndeclaredThrowableException



         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)



         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)



         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)



         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)



         at scala.collection.immutable.List.foreach(List.scala:45)



         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)



         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)



         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)



         at $Proxy12.finishApplicationMaster(Unknown Source)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)



         ... 9 more



Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)



         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)



         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)



         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)



         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)



         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)



         at org.apache.hadoop.ipc.Client.call(Client.java:1239)



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)



         ... 11 more



Caused by: java.net.ConnectException: Connection refused



         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)



         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)



         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)



         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)



         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)



         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)



         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)



         at org.apache.hadoop.ipc.Client.call(Client.java:1206)



         ... 12 more











I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:







<property>



                <name>yarn.resourcemanager.scheduler.address</name>



                <value>192.168.145.37:8030</value>



</property>



<property>



                <name>yarn.resourcemanager.resource-tracker.address</name>



                <value>192.168.145.37:8031</value>



</property>



<property>



                <name>yarn.resourcemanager.address</name>



                <value>192.168.145.37:8032</value>



</property>



<property>



                <name>yarn.resourcemanager.admin.address</name>



                <value>192.168.145.37:8033</value>



</property>



<property>



                <name>yarn.resourcemanager.webapp.address</name>



                <value>192.168.145.37:8088</value>



</property>















These properties are reflected on the UI screen as well:







[<ci...@01CEDBE8.1B2F1890>cid:image001.png@01CEDBE8.1B2F1890]







But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.



I still get:



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







Nestat on the RM machine shows me:



tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java







Nestat on the NM machine shows me:



tcp        0      0 :::8040                     :::*                        LISTEN      1331/java



tcp        0      0 :::8042                     :::*                        LISTEN      1331/java



tcp        0      0 :::56877                    :::*                        LISTEN      1331/java







Kindly help me how to rectify this error.







Regards,



-Nirmal







________________________________



























NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

RE: Running Samza on multi node

Posted by Nirmal Kumar <ni...@impetus.co.in>.
Hi Chris,

I updated the yarn-default.xml in hadoop-yarn-common-2.0.5-alpha.jar present in samza-job-package-0.7.0-dist.tar.gz and finally got the job running.

I updated the following in yarn-default.xml:
<property>
    <description>The address of the scheduler interface.</description>
    <name>yarn.resourcemanager.scheduler.address</name>
    <value>192.168.145.37:8030</value>
  </property>
<property>
    <name>yarn.resourcemanager.resource-tracker.address</name>
    <value>192.168.145.37:8031</value>
  </property>
<property>
    <description>The address of the applications manager interface in the RM.</description>
    <name>yarn.resourcemanager.address</name>
    <value>192.168.145.37:8032</value>
  </property>
<property>
    <description>The address of the RM admin interface.</description>
    <name>yarn.resourcemanager.admin.address</name>
    <value>192.168.145.37:8033</value>
  </property>

Earlier these had 0.0.0.0 values. Is there any other workaround?

My question is on a cluster environment where do I need to keep the samza-job-package-0.7.0-dist.tar.gz file.
I am submitting the job from the RM node and I have to keep the samza-job-package-0.7.0-dist.tar.gz file on both the RM and NM machines.
Otherwise I receives the samza-job-package-0.7.0-dist.tar.gz file not found.



Keeping the samza job file at both the NM and RM machine at a similar location gives me exception of different timestamps as the machines has different time zones:


Application application_1384196041399_0002 failed 1 times due to AM Container for appattempt_1384196041399_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:
java.io.IOException: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)
at LocalTrace:
org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)
at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
.Failing this attempt.. Failing the application.


I am attaching the NM and RM logs for reference.

Thanks,
-Nirmal

From: Nirmal Kumar
Sent: Tuesday, November 12, 2013 1:14 AM
To: dev@samza.incubator.apache.org
Subject: RE: Running Samza on multi node


Hi Chris,



Thanks for the advice.



I put my yarn-site.xml in hello-samza/deploy/yarn/conf directory and also put the following echo in hello-samza/deploy/yarn/etc/hadoop/ yarn-env.sh file:



I am using the commands:

nohup bin/yarn resourcemanager > logs/rm.log 2>&1 &

nohup bin/yarn nodemanager > logs/nm.log 2>&1 &



#export YARN_CONF_DIR="${YARN_CONF_DIR:-$HADOOP_YARN_HOME/conf}"

export YARN_CONF_DIR="/home/bda/nirmal/hello-samza/deploy/yarn/conf"

echo "YARN_CONF_DIR in sh: $YARN_CONF_DIR"



Now in my nm.log and rm.log I see



nm.log

--------

nohup: ignoring input

YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf

13/11/11 19:52:01 INFO nodemanager.NodeManager: STARTUP_MSG:

/************************************************************

STARTUP_MSG: Starting NodeManager

STARTUP_MSG:   host = IMPETUS-DSRV05.impetus.co.in/192.168.145.195

STARTUP_MSG:   args = []

STARTUP_MSG:   version = 2.0.5-alpha

STARTUP_MSG:   classpath =



rm.log

--------

nohup: ignoring input

YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf

13/11/12 00:24:01 INFO resourcemanager.ResourceManager: STARTUP_MSG:

/************************************************************

STARTUP_MSG: Starting ResourceManager

STARTUP_MSG:   host = IMPETUS-DSRV08.impetus.co.in/192.168.145.37

STARTUP_MSG:   args = []

STARTUP_MSG:   version = 2.0.5-alpha

STARTUP_MSG:   classpath =



This rm.log and nm.log are fine.



hello-samza/deploy/yarn/conf/yarn-site.xml in both the machines is:

<configuration>

  <property>

    <name>yarn.scheduler.minimum-allocation-mb</name>

    <value>128</value>

  </property>



  <property>

    <name>yarn.nodemanager.vmem-pmem-ratio</name>

    <value>10</value>

  </property>



<property>

    <name>yarn.resourcemanager.hostname</name>

    <value>192.168.145.37</value>

</property>



<property>

    <name>yarn.resourcemanager.resource-tracker.address</name>

    <value>192.168.145.37:8031</value>

</property>



<property>

    <name>yarn.resourcemanager.address</name>

    <value>192.168.145.37:8032</value>

</property>



<property>

   <name>yarn.resourcemanager.admin.address</name>

   <value>192.168.145.37:8033</value>

</property>



<property>

        <name>yarn.resourcemanager.scheduler.address</name>

        <value>192.168.145.37:8030</value>

</property>

</configuration>



But when I submit the job using the following command :

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



After submitting the job file successfully I see the following under stdout:


2013-11-11 19:52:24 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005

2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_38376_scalate____.8y41jd/webinf/WEB-INF

2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-7888530459139045883-workdir

2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:38376<mailto:SelectChannelConnector@0.0.0.0:38376>

2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005

2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_28858_scalate____.sc28cx/webinf/WEB-INF

2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-3192563436096941762-workdir

2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:28858<mailto:SelectChannelConnector@0.0.0.0:28858>

2013-11-11 19:52:34 SamzaAppMasterLifecycle [INFO] Shutting down.

2013-11-11 19:52:44 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec<ma...@4095c5ec> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

                at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

                at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

                at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

                at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

                at scala.collection.immutable.List.foreach(List.scala:45)

                at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

                at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

                at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

                at $Proxy12.finishApplicationMaster(Unknown Source)

                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

                ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

                at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

                at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

                at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

                at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

                at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

                at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

                at org.apache.hadoop.ipc.Client.call(Client.java:1239)

                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

                ... 11 more

Caused by: java.net.ConnectException: Connection refused

                at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

                at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:567)

                at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

                at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

                at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

                at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

                at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

                at org.apache.hadoop.ipc.Client.call(Client.java:1206)

                ... 12 more



In the stdout I see the following jar is used when invoking org.apache.samza.job.yarn.SamzaAppMaster:

/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/container_1384179355066_0001_01_000001/__package/lib/hadoop-yarn-common-2.0.5-alpha.jar



This jar too contains the yarn-default.xml. Is this the file that's being picked up?





Also where do I need to keep the job file ?

I am setting the job file name in a test-consumer.properties file:

yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz



And submitting the job from the RM node:

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



But that way do I need to keep the job file in both NM and RM machines on the same location(/home/temptest/ i.e. I have created same directories on both the machines)?



Keeping the samza job file at both the NM and RM machine at a similar location gives me exception:


Application application_1384196041399_0002 failed 1 times due to AM Container for appattempt_1384196041399_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:
java.io.IOException: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)
at LocalTrace:
org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)
at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
.Failing this attempt.. Failing the application.





Regards,

-Nirmal





-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Sunday, November 10, 2013 11:00 AM
To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>
Subject: RE: Running Samza on multi node



Hey Nirmal,



It sounds like the yarn-site.xml is being ignored for some reason. Things to do:



1. Could you please send full log files for the RM and NM?



2. You might also try putting your yarn-site.xml in hello-samza/deploy/yarn/conf, and explicitly setting the HADOOP_YARN_HOME environment variable to:



export HADOOP_YARN_HOME=<path to>/hello-samza/deploy/yarn



Then try running bin/grid start yarn.



3. Try staring yarn WITHOUT bin/grid. This can be done with:



deploy/yarn/bin/yarn resourcemanager

deploy/yarn/bin/yarn nodemanager



Cheers,

Chris

________________________________

From: Nirmal Kumar [nirmal.kumar@impetus.co.in]

Sent: Friday, November 08, 2013 5:01 AM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: RE: Running Samza on multi node



Hi Chris,



The below exception is gone if the job file timestamp is same:





Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:



java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)



at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)



at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



at java.lang.Thread.run(Thread.java:662)



at LocalTrace:



org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)



at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)



at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)



at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)



at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.



PFA the exception that's coming. I am still struggling with the same exception i.e. NM trying to connect to 0.0.0.0:8030 I don't know from where the NM is picking up this 0.0.0.0:8030 value. Overriding the  yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.



I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on both RM and NM machines:



<configuration>

  <property>

    <name>yarn.scheduler.minimum-allocation-mb</name>

    <value>128</value>

  </property>



  <property>

    <name>yarn.nodemanager.vmem-pmem-ratio</name>

    <value>10</value>

  </property>



<property>

    <name>yarn.resourcemanager.hostname</name>

   <value>192.168.145.37</value>

</property>



<property>

     <name>yarn.resourcemanager.resource-tracker.address</name>

     <value>192.168.145.37:8031</value>

</property>

</configuration>





Regards,

-Nirmal



From: Nirmal Kumar

Sent: Friday, November 08, 2013 6:11 PM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: RE: Running Samza on multi node





Hi Chris,







Using just the yarn.resourcemanager.hostname property gives me the following exception on the NM:







Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







I then added the following property as well:



<property>



   <name>yarn.resourcemanager.resource-tracker.address</name>



   <value>192.168.145.37:8031</value>



</property>







After this my RM and NM were up and the NM got registered as well:



13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager is started.



13/11/08 16:12:19 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack



13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948



13/11/08 16:12:19 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING



13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192, vCores:16>







When submitting the job I'm still getting the same exception:





YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

         at scala.collection.immutable.List.foreach(List.scala:45)

         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

         at $Proxy12.finishApplicationMaster(Unknown Source)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

         ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

         at org.apache.hadoop.ipc.Client.call(Client.java:1239)

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

         ... 11 more

Caused by: java.net.ConnectException: Connection refused

         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)

         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

         at org.apache.hadoop.ipc.Client.call(Client.java:1206)

         ... 12 more







Where do I need to keep the job file ?



I am setting the job file name in a test-consumer.properties file:



yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz







And submitting the job from :



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







But that way do I need to keep the job file in both NM and RM machines on the same location?







I tried submitting the job several time with different properties in yarn-site.xml and now I am getting some strange exception. This is probably due to the different timestamps.







Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:



java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)



at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)



at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



at java.lang.Thread.run(Thread.java:662)



at LocalTrace:



org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)



at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)



at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)



at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)



at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)



.Failing this attempt.. Failing the application.







When I am using NM and RM on a single node all is running fine. PFA the logs for the job.







Other questions:







1. Both your NM and RM are running YARN 2.2.0, right?



I am using same YARN on both NM and RM that was downloaded as part of hello-samza application.



2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?



I am forcibly killing the java processes using kill -9 pid command.







Thanks,



-Nirmal







-----Original Message-----

From: Chris Riccomini [mailto:criccomini@linkedin.com]

Sent: Thursday, November 07, 2013 9:19 PM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: Re: Running Samza on multi node







Hey Nirmal,







Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.







It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:







  <property>



    <name>yarn.resourcemanager.hostname</name>



    <value>192.168.145.37</value>



  </property>







Your netstat reports look fine - as expected.







Other questions:







1. Both your NM and RM are running YARN 2.2.0, right?



2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?







Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:











    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))







I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.







Cheers,



Chris







From: Nirmal Kumar <ni...@impetus.co.in>>>>



Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e>" <de...@samza.incubator.apache.org>>>>



Date: Thursday, November 7, 2013 5:05 AM



To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e>" <de...@samza.incubator.apache.org>>>>



Subject: Running Samza on multi node







All,







I was able to run the hello-samza application on a single node machine.



Now I am trying to run the hello-samza application on  a 2 node setup.







Node1 has a Resource Manager



Node2 has a Node Manager







The NM gets registered with the RM successfully as seen in rm.log of the RM node:



13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.



13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack



13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093



13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING



13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>







I am submitting the job from the RM machine using the command line:



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







However, I am getting the following exception after submitting the job to YARN:







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}



2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032



2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in



2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi



2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-



2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093



2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000



2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)



2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing



2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092



2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751



2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog



2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032



2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005



2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF



2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir



2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152>>



2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005



2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF



2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir



2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751>>



2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.



2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>> failed to shutdown.



java.lang.reflect.UndeclaredThrowableException



         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)



         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)



         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)



         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)



         at scala.collection.immutable.List.foreach(List.scala:45)



         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)



         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)



         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)



         at $Proxy12.finishApplicationMaster(Unknown Source)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)



         ... 9 more



Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)



         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)



         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)



         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)



         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)



         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)



         at org.apache.hadoop.ipc.Client.call(Client.java:1239)



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)



         ... 11 more



Caused by: java.net.ConnectException: Connection refused



         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)



         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)



         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)



         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)



         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)



         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)



         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)



         at org.apache.hadoop.ipc.Client.call(Client.java:1206)



         ... 12 more











I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:







<property>



                <name>yarn.resourcemanager.scheduler.address</name>



                <value>192.168.145.37:8030</value>



</property>



<property>



                <name>yarn.resourcemanager.resource-tracker.address</name>



                <value>192.168.145.37:8031</value>



</property>



<property>



                <name>yarn.resourcemanager.address</name>



                <value>192.168.145.37:8032</value>



</property>



<property>



                <name>yarn.resourcemanager.admin.address</name>



                <value>192.168.145.37:8033</value>



</property>



<property>



                <name>yarn.resourcemanager.webapp.address</name>



                <value>192.168.145.37:8088</value>



</property>















These properties are reflected on the UI screen as well:







[cid:image001.png@01CEDBE8.1B2F1890]







But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.



I still get:



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







Nestat on the RM machine shows me:



tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java







Nestat on the NM machine shows me:



tcp        0      0 :::8040                     :::*                        LISTEN      1331/java



tcp        0      0 :::8042                     :::*                        LISTEN      1331/java



tcp        0      0 :::56877                    :::*                        LISTEN      1331/java







Kindly help me how to rectify this error.







Regards,



-Nirmal







________________________________



























NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

RE: Running Samza on multi node

Posted by Nirmal Kumar <ni...@impetus.co.in>.
Hi Chris,



Thanks for the advice.



I put my yarn-site.xml in hello-samza/deploy/yarn/conf directory and also put the following echo in hello-samza/deploy/yarn/etc/hadoop/ yarn-env.sh file:



I am using the commands:

nohup bin/yarn resourcemanager > logs/rm.log 2>&1 &

nohup bin/yarn nodemanager > logs/nm.log 2>&1 &



#export YARN_CONF_DIR="${YARN_CONF_DIR:-$HADOOP_YARN_HOME/conf}"

export YARN_CONF_DIR="/home/bda/nirmal/hello-samza/deploy/yarn/conf"

echo "YARN_CONF_DIR in sh: $YARN_CONF_DIR"



Now in my nm.log and rm.log I see



nm.log

--------

nohup: ignoring input

YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf

13/11/11 19:52:01 INFO nodemanager.NodeManager: STARTUP_MSG:

/************************************************************

STARTUP_MSG: Starting NodeManager

STARTUP_MSG:   host = IMPETUS-DSRV05.impetus.co.in/192.168.145.195

STARTUP_MSG:   args = []

STARTUP_MSG:   version = 2.0.5-alpha

STARTUP_MSG:   classpath =



rm.log

--------

nohup: ignoring input

YARN_CONF_DIR in sh: /home/bda/nirmal/hello-samza/deploy/yarn/conf

13/11/12 00:24:01 INFO resourcemanager.ResourceManager: STARTUP_MSG:

/************************************************************

STARTUP_MSG: Starting ResourceManager

STARTUP_MSG:   host = IMPETUS-DSRV08.impetus.co.in/192.168.145.37

STARTUP_MSG:   args = []

STARTUP_MSG:   version = 2.0.5-alpha

STARTUP_MSG:   classpath =



This rm.log and nm.log are fine.



hello-samza/deploy/yarn/conf/yarn-site.xml in both the machines is:

<configuration>

  <property>

    <name>yarn.scheduler.minimum-allocation-mb</name>

    <value>128</value>

  </property>



  <property>

    <name>yarn.nodemanager.vmem-pmem-ratio</name>

    <value>10</value>

  </property>



<property>

    <name>yarn.resourcemanager.hostname</name>

    <value>192.168.145.37</value>

</property>



<property>

    <name>yarn.resourcemanager.resource-tracker.address</name>

    <value>192.168.145.37:8031</value>

</property>



<property>

    <name>yarn.resourcemanager.address</name>

    <value>192.168.145.37:8032</value>

</property>



<property>

   <name>yarn.resourcemanager.admin.address</name>

   <value>192.168.145.37:8033</value>

</property>



<property>

        <name>yarn.resourcemanager.scheduler.address</name>

        <value>192.168.145.37:8030</value>

</property>

</configuration>



But when I submit the job using the following command :

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



After submitting the job file successfully I see the following under stdout:


2013-11-11 19:52:24 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005

2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_38376_scalate____.8y41jd/webinf/WEB-INF

2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-7888530459139045883-workdir

2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:38376

2013-11-11 19:52:24 log [INFO] jetty-7.0.0.v20091005

2013-11-11 19:52:24 log [INFO] Extract jar:file:/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/filecache/-3746405862769211445/samza-job-package-0.8.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_28858_scalate____.sc28cx/webinf/WEB-INF

2013-11-11 19:52:24 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-3192563436096941762-workdir

2013-11-11 19:52:24 log [INFO] Started SelectChannelConnector@0.0.0.0:28858

2013-11-11 19:52:34 SamzaAppMasterLifecycle [INFO] Shutting down.

2013-11-11 19:52:44 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@4095c5ec failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

                at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

                at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

                at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

                at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

                at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

                at scala.collection.immutable.List.foreach(List.scala:45)

                at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

                at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

                at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

                at $Proxy12.finishApplicationMaster(Unknown Source)

                at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

                ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV05.impetus.co.in/192.168.145.195 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

                at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

                at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

                at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

                at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

                at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

                at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

                at org.apache.hadoop.ipc.Client.call(Client.java:1239)

                at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

                ... 11 more

Caused by: java.net.ConnectException: Connection refused

                at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

                at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:567)

                at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

                at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

                at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

                at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

                at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

                at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

                at org.apache.hadoop.ipc.Client.call(Client.java:1206)

                ... 12 more



In the stdout I see the following jar is used when invoking org.apache.samza.job.yarn.SamzaAppMaster:

/tmp/hadoop-bda/nm-local-dir/usercache/bda/appcache/application_1384179355066_0001/container_1384179355066_0001_01_000001/__package/lib/hadoop-yarn-common-2.0.5-alpha.jar



This jar too contains the yarn-default.xml. Is this the file that's being picked up?





Also where do I need to keep the job file ?

I am setting the job file name in a test-consumer.properties file:

yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz



And submitting the job from the RM node:

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



But that way do I need to keep the job file in both NM and RM machines on the same location(/home/temptest/ i.e. I have created same directories on both the machines)?



Keeping the samza job file at both the NM and RM machine at a similar location gives me exception:


Application application_1384196041399_0002 failed 1 times due to AM Container for appattempt_1384196041399_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:
java.io.IOException: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)
at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)
at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)
at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:662)
at LocalTrace:
org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.8.0-dist.tar.gz changed on src filesystem (expected 1384196169000, was 1383911494000
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)
at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)
at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)
at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)
at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)
at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
.Failing this attempt.. Failing the application.





Regards,

-Nirmal





-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Sunday, November 10, 2013 11:00 AM
To: dev@samza.incubator.apache.org
Subject: RE: Running Samza on multi node



Hey Nirmal,



It sounds like the yarn-site.xml is being ignored for some reason. Things to do:



1. Could you please send full log files for the RM and NM?



2. You might also try putting your yarn-site.xml in hello-samza/deploy/yarn/conf, and explicitly setting the HADOOP_YARN_HOME environment variable to:



export HADOOP_YARN_HOME=<path to>/hello-samza/deploy/yarn



Then try running bin/grid start yarn.



3. Try staring yarn WITHOUT bin/grid. This can be done with:



deploy/yarn/bin/yarn resourcemanager

deploy/yarn/bin/yarn nodemanager



Cheers,

Chris

________________________________

From: Nirmal Kumar [nirmal.kumar@impetus.co.in]

Sent: Friday, November 08, 2013 5:01 AM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: RE: Running Samza on multi node



Hi Chris,



The below exception is gone if the job file timestamp is same:





Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:



java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)



at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)



at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



at java.lang.Thread.run(Thread.java:662)



at LocalTrace:



org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)



at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)



at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)



at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)



at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.



PFA the exception that's coming. I am still struggling with the same exception i.e. NM trying to connect to 0.0.0.0:8030 I don't know from where the NM is picking up this 0.0.0.0:8030 value. Overriding the  yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.



I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on both RM and NM machines:



<configuration>

  <property>

    <name>yarn.scheduler.minimum-allocation-mb</name>

    <value>128</value>

  </property>



  <property>

    <name>yarn.nodemanager.vmem-pmem-ratio</name>

    <value>10</value>

  </property>



<property>

    <name>yarn.resourcemanager.hostname</name>

   <value>192.168.145.37</value>

</property>



<property>

     <name>yarn.resourcemanager.resource-tracker.address</name>

     <value>192.168.145.37:8031</value>

</property>

</configuration>





Regards,

-Nirmal



From: Nirmal Kumar

Sent: Friday, November 08, 2013 6:11 PM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: RE: Running Samza on multi node





Hi Chris,







Using just the yarn.resourcemanager.hostname property gives me the following exception on the NM:







Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







I then added the following property as well:



<property>



   <name>yarn.resourcemanager.resource-tracker.address</name>



   <value>192.168.145.37:8031</value>



</property>







After this my RM and NM were up and the NM got registered as well:



13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager is started.



13/11/08 16:12:19 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack



13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948



13/11/08 16:12:19 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING



13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192, vCores:16>







When submitting the job I'm still getting the same exception:





YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

         at scala.collection.immutable.List.foreach(List.scala:45)

         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

         at $Proxy12.finishApplicationMaster(Unknown Source)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

         ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

         at org.apache.hadoop.ipc.Client.call(Client.java:1239)

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

         ... 11 more

Caused by: java.net.ConnectException: Connection refused

         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)

         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

         at org.apache.hadoop.ipc.Client.call(Client.java:1206)

         ... 12 more







Where do I need to keep the job file ?



I am setting the job file name in a test-consumer.properties file:



yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz







And submitting the job from :



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







But that way do I need to keep the job file in both NM and RM machines on the same location?







I tried submitting the job several time with different properties in yarn-site.xml and now I am getting some strange exception. This is probably due to the different timestamps.







Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:



java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)



at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)



at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)



at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)



at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)



at java.util.concurrent.FutureTask.run(FutureTask.java:138)



at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)



at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



at java.lang.Thread.run(Thread.java:662)



at LocalTrace:



org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)



at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)



at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)



at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)



at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)



at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)



at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)



at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)



at java.security.AccessController.doPrivileged(Native Method)



at javax.security.auth.Subject.doAs(Subject.java:396)



at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)



at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)



.Failing this attempt.. Failing the application.







When I am using NM and RM on a single node all is running fine. PFA the logs for the job.







Other questions:







1. Both your NM and RM are running YARN 2.2.0, right?



I am using same YARN on both NM and RM that was downloaded as part of hello-samza application.



2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?



I am forcibly killing the java processes using kill -9 pid command.







Thanks,



-Nirmal







-----Original Message-----

From: Chris Riccomini [mailto:criccomini@linkedin.com]

Sent: Thursday, November 07, 2013 9:19 PM

To: dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>

Subject: Re: Running Samza on multi node







Hey Nirmal,







Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.







It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:







  <property>



    <name>yarn.resourcemanager.hostname</name>



    <value>192.168.145.37</value>



  </property>







Your netstat reports look fine - as expected.







Other questions:







1. Both your NM and RM are running YARN 2.2.0, right?



2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?







Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:











    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))







I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.







Cheers,



Chris







From: Nirmal Kumar <ni...@impetus.co.in>>>>



Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e>" <de...@samza.incubator.apache.org>>>>



Date: Thursday, November 7, 2013 5:05 AM



To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3cmailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e%3e>" <de...@samza.incubator.apache.org>>>>



Subject: Running Samza on multi node







All,







I was able to run the hello-samza application on a single node machine.



Now I am trying to run the hello-samza application on  a 2 node setup.







Node1 has a Resource Manager



Node2 has a Node Manager







The NM gets registered with the RM successfully as seen in rm.log of the RM node:



13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.



13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack



13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093



13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING



13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>







I am submitting the job from the RM machine using the command line:



bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties







However, I am getting the following exception after submitting the job to YARN:







2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}



2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032



2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in



2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi



2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-



2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/



2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties



2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0



2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093



2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000



2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)



2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing



2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092



2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751



2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog



2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032



2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005



2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF



2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir



2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152>>



2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005



2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF



2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir



2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751%3cmailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751>>



2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.



2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e>> failed to shutdown.



java.lang.reflect.UndeclaredThrowableException



         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)



         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)



         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)



         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)



         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)



         at scala.collection.immutable.List.foreach(List.scala:45)



         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)



         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)



         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)



         at $Proxy12.finishApplicationMaster(Unknown Source)



         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)



         ... 9 more



Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)



         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)



         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)



         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)



         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)



         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)



         at org.apache.hadoop.ipc.Client.call(Client.java:1239)



         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)



         ... 11 more



Caused by: java.net.ConnectException: Connection refused



         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)



         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)



         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)



         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)



         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)



         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)



         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)



         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)



         at org.apache.hadoop.ipc.Client.call(Client.java:1206)



         ... 12 more











I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:







<property>



                <name>yarn.resourcemanager.scheduler.address</name>



                <value>192.168.145.37:8030</value>



</property>



<property>



                <name>yarn.resourcemanager.resource-tracker.address</name>



                <value>192.168.145.37:8031</value>



</property>



<property>



                <name>yarn.resourcemanager.address</name>



                <value>192.168.145.37:8032</value>



</property>



<property>



                <name>yarn.resourcemanager.admin.address</name>



                <value>192.168.145.37:8033</value>



</property>



<property>



                <name>yarn.resourcemanager.webapp.address</name>



                <value>192.168.145.37:8088</value>



</property>















These properties are reflected on the UI screen as well:







[cid:image001.png@01CEDBE8.1B2F1890]







But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.



I still get:



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused







Nestat on the RM machine shows me:



tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java



tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java







Nestat on the NM machine shows me:



tcp        0      0 :::8040                     :::*                        LISTEN      1331/java



tcp        0      0 :::8042                     :::*                        LISTEN      1331/java



tcp        0      0 :::56877                    :::*                        LISTEN      1331/java







Kindly help me how to rectify this error.







Regards,



-Nirmal







________________________________



























NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

RE: Running Samza on multi node

Posted by Chris Riccomini <cr...@linkedin.com>.
Hey Nirmal,

It sounds like the yarn-site.xml is being ignored for some reason. Things to do:

1. Could you please send full log files for the RM and NM?

2. You might also try putting your yarn-site.xml in hello-samza/deploy/yarn/conf, and explicitly setting the HADOOP_YARN_HOME environment variable to:

export HADOOP_YARN_HOME=<path to>/hello-samza/deploy/yarn

Then try running bin/grid start yarn.

3. Try staring yarn WITHOUT bin/grid. This can be done with:

deploy/yarn/bin/yarn resourcemanager
deploy/yarn/bin/yarn nodemanager

Cheers,
Chris
________________________________
From: Nirmal Kumar [nirmal.kumar@impetus.co.in]
Sent: Friday, November 08, 2013 5:01 AM
To: dev@samza.incubator.apache.org
Subject: RE: Running Samza on multi node

Hi Chris,

The below exception is gone if the job file timestamp is same:


Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:

java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)

at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)

at java.lang.Thread.run(Thread.java:662)

at LocalTrace:

org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)

at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)

at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)

at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)

at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
.Failing this attempt.. Failing the application.

PFA the exception that’s coming. I am still struggling with the same exception i.e. NM trying to connect to 0.0.0.0:8030
I don’t know from where the NM is picking up this 0.0.0.0:8030 value. Overriding the  yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.

I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on both RM and NM machines:

<configuration>
  <property>
    <name>yarn.scheduler.minimum-allocation-mb</name>
    <value>128</value>
  </property>

  <property>
    <name>yarn.nodemanager.vmem-pmem-ratio</name>
    <value>10</value>
  </property>

<property>
    <name>yarn.resourcemanager.hostname</name>
   <value>192.168.145.37</value>
 </property>

<property>
     <name>yarn.resourcemanager.resource-tracker.address</name>
     <value>192.168.145.37:8031</value>
 </property>
</configuration>


Regards,
-Nirmal

From: Nirmal Kumar
Sent: Friday, November 08, 2013 6:11 PM
To: dev@samza.incubator.apache.org
Subject: RE: Running Samza on multi node


Hi Chris,



Using just the yarn.resourcemanager.hostname property gives me the following exception on the NM:



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



I then added the following property as well:

<property>

   <name>yarn.resourcemanager.resource-tracker.address</name>

   <value>192.168.145.37:8031</value>

</property>



After this my RM and NM were up and the NM got registered as well:

13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager is started.

13/11/08 16:12:19 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack

13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948

13/11/08 16:12:19 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING

13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192, vCores:16>



When submitting the job I'm still getting the same exception:


YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e> failed to shutdown.
java.lang.reflect.UndeclaredThrowableException
         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)
         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)
         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)
         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
         at scala.collection.immutable.List.foreach(List.scala:45)
         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)
         at $Proxy12.finishApplicationMaster(Unknown Source)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)
         ... 9 more
Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)
         ... 11 more
Caused by: java.net.ConnectException: Connection refused
         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
         ... 12 more



Where do I need to keep the job file ?

I am setting the job file name in a test-consumer.properties file:

yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz



And submitting the job from :

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



But that way do I need to keep the job file in both NM and RM machines on the same location?



I tried submitting the job several time with different properties in yarn-site.xml and now I am getting some strange exception. This is probably due to the different timestamps.



Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:

java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)

at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)

at java.lang.Thread.run(Thread.java:662)

at LocalTrace:

org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)

at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)

at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)

at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)

at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.



When I am using NM and RM on a single node all is running fine. PFA the logs for the job.



Other questions:



1. Both your NM and RM are running YARN 2.2.0, right?

I am using same YARN on both NM and RM that was downloaded as part of hello-samza application.

2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?

I am forcibly killing the java processes using kill -9 pid command.



Thanks,

-Nirmal



-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Thursday, November 07, 2013 9:19 PM
To: dev@samza.incubator.apache.org
Subject: Re: Running Samza on multi node



Hey Nirmal,



Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.



It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:



  <property>

    <name>yarn.resourcemanager.hostname</name>

    <value>192.168.145.37</value>

  </property>



Your netstat reports look fine – as expected.



Other questions:



1. Both your NM and RM are running YARN 2.2.0, right?

2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?



Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:





    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))



I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.



Cheers,

Chris



From: Nirmal Kumar <ni...@impetus.co.in>>>

Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Date: Thursday, November 7, 2013 5:05 AM

To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Subject: Running Samza on multi node



All,



I was able to run the hello-samza application on a single node machine.

Now I am trying to run the hello-samza application on  a 2 node setup.



Node1 has a Resource Manager

Node2 has a Node Manager



The NM gets registered with the RM successfully as seen in rm.log of the RM node:

13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.

13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack

13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093

13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING

13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>



I am submitting the job from the RM machine using the command line:

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



However, I am getting the following exception after submitting the job to YARN:



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}

2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in

2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi

2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-

2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093

2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000

2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)

2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing

2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092

2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751

2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog

2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005

2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF

2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir

2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152>

2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005

2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF

2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir

2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751>

2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.

2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

         at scala.collection.immutable.List.foreach(List.scala:45)

         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

         at $Proxy12.finishApplicationMaster(Unknown Source)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

         ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

         at org.apache.hadoop.ipc.Client.call(Client.java:1239)

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

         ... 11 more

Caused by: java.net.ConnectException: Connection refused

         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)

         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

         at org.apache.hadoop.ipc.Client.call(Client.java:1206)

         ... 12 more





I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:



<property>

                <name>yarn.resourcemanager.scheduler.address</name>

                <value>192.168.145.37:8030</value>

</property>

<property>

                <name>yarn.resourcemanager.resource-tracker.address</name>

                <value>192.168.145.37:8031</value>

</property>

<property>

                <name>yarn.resourcemanager.address</name>

                <value>192.168.145.37:8032</value>

</property>

<property>

                <name>yarn.resourcemanager.admin.address</name>

                <value>192.168.145.37:8033</value>

</property>

<property>

                <name>yarn.resourcemanager.webapp.address</name>

                <value>192.168.145.37:8088</value>

</property>







These properties are reflected on the UI screen as well:



[cid:image001.png@01CEDBE8.1B2F1890]



But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.

I still get:

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



Nestat on the RM machine shows me:

tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java



Nestat on the NM machine shows me:

tcp        0      0 :::8040                     :::*                        LISTEN      1331/java

tcp        0      0 :::8042                     :::*                        LISTEN      1331/java

tcp        0      0 :::56877                    :::*                        LISTEN      1331/java



Kindly help me how to rectify this error.



Regards,

-Nirmal



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

RE: Running Samza on multi node

Posted by Nirmal Kumar <ni...@impetus.co.in>.
Hi Chris,

The below exception is gone if the job file timestamp is same:


Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:

java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)

at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)

at java.lang.Thread.run(Thread.java:662)

at LocalTrace:

org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)

at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)

at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)

at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)

at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)
.Failing this attempt.. Failing the application.

PFA the exception that's coming. I am still struggling with the same exception i.e. NM trying to connect to 0.0.0.0:8030
I don't know from where the NM is picking up this 0.0.0.0:8030 value. Overriding the  yarn.resourcemanager.scheduler.address in yarn-site.xml is not working.

I am using the same hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on both RM and NM machines:

<configuration>
  <property>
    <name>yarn.scheduler.minimum-allocation-mb</name>
    <value>128</value>
  </property>

  <property>
    <name>yarn.nodemanager.vmem-pmem-ratio</name>
    <value>10</value>
  </property>

<property>
    <name>yarn.resourcemanager.hostname</name>
   <value>192.168.145.37</value>
 </property>

<property>
     <name>yarn.resourcemanager.resource-tracker.address</name>
     <value>192.168.145.37:8031</value>
 </property>
</configuration>


Regards,
-Nirmal

From: Nirmal Kumar
Sent: Friday, November 08, 2013 6:11 PM
To: dev@samza.incubator.apache.org
Subject: RE: Running Samza on multi node


Hi Chris,



Using just the yarn.resourcemanager.hostname property gives me the following exception on the NM:



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



I then added the following property as well:

<property>

   <name>yarn.resourcemanager.resource-tracker.address</name>

   <value>192.168.145.37:8031</value>

</property>



After this my RM and NM were up and the NM got registered as well:

13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager is started.

13/11/08 16:12:19 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack

13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948

13/11/08 16:12:19 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING

13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192, vCores:16>



When submitting the job I'm still getting the same exception:


YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e> failed to shutdown.
java.lang.reflect.UndeclaredThrowableException
         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)
         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)
         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)
         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
         at scala.collection.immutable.List.foreach(List.scala:45)
         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)
         at $Proxy12.finishApplicationMaster(Unknown Source)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)
         ... 9 more
Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)
         ... 11 more
Caused by: java.net.ConnectException: Connection refused
         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
         ... 12 more



Where do I need to keep the job file ?

I am setting the job file name in a test-consumer.properties file:

yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz



And submitting the job from :

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



But that way do I need to keep the job file in both NM and RM machines on the same location?



I tried submitting the job several time with different properties in yarn-site.xml and now I am getting some strange exception. This is probably due to the different timestamps.



Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:

java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)

at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)

at java.lang.Thread.run(Thread.java:662)

at LocalTrace:

org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)

at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)

at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)

at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)

at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.



When I am using NM and RM on a single node all is running fine. PFA the logs for the job.



Other questions:



1. Both your NM and RM are running YARN 2.2.0, right?

I am using same YARN on both NM and RM that was downloaded as part of hello-samza application.

2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?

I am forcibly killing the java processes using kill -9 pid command.



Thanks,

-Nirmal



-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Thursday, November 07, 2013 9:19 PM
To: dev@samza.incubator.apache.org
Subject: Re: Running Samza on multi node



Hey Nirmal,



Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.



It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:



  <property>

    <name>yarn.resourcemanager.hostname</name>

    <value>192.168.145.37</value>

  </property>



Your netstat reports look fine - as expected.



Other questions:



1. Both your NM and RM are running YARN 2.2.0, right?

2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?



Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:





    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))



I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.



Cheers,

Chris



From: Nirmal Kumar <ni...@impetus.co.in>>>

Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Date: Thursday, November 7, 2013 5:05 AM

To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Subject: Running Samza on multi node



All,



I was able to run the hello-samza application on a single node machine.

Now I am trying to run the hello-samza application on  a 2 node setup.



Node1 has a Resource Manager

Node2 has a Node Manager



The NM gets registered with the RM successfully as seen in rm.log of the RM node:

13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.

13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack

13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093

13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING

13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>



I am submitting the job from the RM machine using the command line:

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



However, I am getting the following exception after submitting the job to YARN:



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}

2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in

2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi

2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-

2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093

2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000

2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)

2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing

2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092

2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751

2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog

2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005

2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF

2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir

2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152>

2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005

2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF

2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir

2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751>

2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.

2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

         at scala.collection.immutable.List.foreach(List.scala:45)

         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

         at $Proxy12.finishApplicationMaster(Unknown Source)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

         ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

         at org.apache.hadoop.ipc.Client.call(Client.java:1239)

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

         ... 11 more

Caused by: java.net.ConnectException: Connection refused

         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)

         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

         at org.apache.hadoop.ipc.Client.call(Client.java:1206)

         ... 12 more





I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:



<property>

                <name>yarn.resourcemanager.scheduler.address</name>

                <value>192.168.145.37:8030</value>

</property>

<property>

                <name>yarn.resourcemanager.resource-tracker.address</name>

                <value>192.168.145.37:8031</value>

</property>

<property>

                <name>yarn.resourcemanager.address</name>

                <value>192.168.145.37:8032</value>

</property>

<property>

                <name>yarn.resourcemanager.admin.address</name>

                <value>192.168.145.37:8033</value>

</property>

<property>

                <name>yarn.resourcemanager.webapp.address</name>

                <value>192.168.145.37:8088</value>

</property>







These properties are reflected on the UI screen as well:



[cid:image001.png@01CEDBE8.1B2F1890]



But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.

I still get:

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



Nestat on the RM machine shows me:

tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java



Nestat on the NM machine shows me:

tcp        0      0 :::8040                     :::*                        LISTEN      1331/java

tcp        0      0 :::8042                     :::*                        LISTEN      1331/java

tcp        0      0 :::56877                    :::*                        LISTEN      1331/java



Kindly help me how to rectify this error.



Regards,

-Nirmal



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

RE: Running Samza on multi node

Posted by Nirmal Kumar <ni...@impetus.co.in>.
Hi Chris,



Using just the yarn.resourcemanager.hostname property gives me the following exception on the NM:



Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8031 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



I then added the following property as well:

<property>

   <name>yarn.resourcemanager.resource-tracker.address</name>

   <value>192.168.145.37:8031</value>

</property>



After this my RM and NM were up and the NM got registered as well:

13/11/08 16:12:12 INFO service.AbstractService: Service:ResourceManager is started.

13/11/08 16:12:19 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack

13/11/08 16:12:19 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 32948 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:32948

13/11/08 16:12:19 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:32948 Node Transitioned from NEW to RUNNING

13/11/08 16:12:19 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:32948 clusterResource: <memory:8192, vCores:16>



When submitting the job I'm still getting the same exception:


YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e> failed to shutdown.
java.lang.reflect.UndeclaredThrowableException
         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)
         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)
         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)
         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
         at scala.collection.immutable.List.foreach(List.scala:45)
         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)
         at $Proxy12.finishApplicationMaster(Unknown Source)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)
         ... 9 more
Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)
         ... 11 more
Caused by: java.net.ConnectException: Connection refused
         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
         ... 12 more



Where do I need to keep the job file ?

I am setting the job file name in a test-consumer.properties file:

yarn.package.path=file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz



And submitting the job from :

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



But that way do I need to keep the job file in both NM and RM machines on the same location?



I tried submitting the job several time with different properties in yarn-site.xml and now I am getting some strange exception. This is probably due to the different timestamps.



Application application_1383907331443_0002 failed 1 times due to AM Container for appattempt_1383907331443_0002_000001 exited with exitCode: -1000 due to: RemoteTrace:

java.io.IOException: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:178)

at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:51)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:284)

at org.apache.hadoop.yarn.util.FSDownload$1.run(FSDownload.java:282)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:280)

at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:51)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)

at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)

at java.util.concurrent.FutureTask.run(FutureTask.java:138)

at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)

at java.lang.Thread.run(Thread.java:662)

at LocalTrace:

org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: Resource file:/home/temptest/samza-job-package-0.7.0-dist.tar.gz changed on src filesystem (expected 1383904942000, was 1382550495000

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.convertFromProtoFormat(LocalResourceStatusPBImpl.java:217)

at org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.LocalResourceStatusPBImpl.getException(LocalResourceStatusPBImpl.java:147)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerRunner.update(ResourceLocalizationService.java:819)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService$LocalizerTracker.processHeartbeat(ResourceLocalizationService.java:491)

at org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.heartbeat(ResourceLocalizationService.java:218)

at org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.service.LocalizationProtocolPBServiceImpl.heartbeat(LocalizationProtocolPBServiceImpl.java:46)

at org.apache.hadoop.yarn.proto.LocalizationProtocol$LocalizationProtocolService$2.callBlockingMethod(LocalizationProtocol.java:57)

at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:454)

at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1014)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1741)

at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1737)

at java.security.AccessController.doPrivileged(Native Method)

at javax.security.auth.Subject.doAs(Subject.java:396)

at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1478)

at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1735)

.Failing this attempt.. Failing the application.



When I am using NM and RM on a single node all is running fine. PFA the logs for the job.



Other questions:



1. Both your NM and RM are running YARN 2.2.0, right?

I am using same YARN on both NM and RM that was downloaded as part of hello-samza application.

2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?

I am forcibly killing the java processes using kill -9 pid command.



Thanks,

-Nirmal



-----Original Message-----
From: Chris Riccomini [mailto:criccomini@linkedin.com]
Sent: Thursday, November 07, 2013 9:19 PM
To: dev@samza.incubator.apache.org
Subject: Re: Running Samza on multi node



Hey Nirmal,



Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.



It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:



  <property>

    <name>yarn.resourcemanager.hostname</name>

    <value>192.168.145.37</value>

  </property>



Your netstat reports look fine - as expected.



Other questions:



1. Both your NM and RM are running YARN 2.2.0, right?

2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?



Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:





    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))



I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.



Cheers,

Chris



From: Nirmal Kumar <ni...@impetus.co.in>>>

Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Date: Thursday, November 7, 2013 5:05 AM

To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org><mailto:dev@samza.incubator.apache.org%3cmailto:dev@samza.incubator.apache.org%3e>" <de...@samza.incubator.apache.org>>>

Subject: Running Samza on multi node



All,



I was able to run the hello-samza application on a single node machine.

Now I am trying to run the hello-samza application on  a 2 node setup.



Node1 has a Resource Manager

Node2 has a Node Manager



The NM gets registered with the RM successfully as seen in rm.log of the RM node:

13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.

13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack

13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093

13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING

13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>



I am submitting the job from the RM machine using the command line:

bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties



However, I am getting the following exception after submitting the job to YARN:



2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}

2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in

2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi

2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-

2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/

2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties

2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0

2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093

2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000

2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)

2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing

2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092

2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751

2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog

2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032

2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005

2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF

2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir

2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:39152>

2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005

2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF

2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir

2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751<mailto:SelectChannelConnector@0.0.0.0%3cmailto:SelectChannelConnector@0.0.0.0%3e:26751>

2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.

2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e<ma...@500c954e> failed to shutdown.

java.lang.reflect.UndeclaredThrowableException

         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)

         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)

         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)

         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)

         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)

         at scala.collection.immutable.List.foreach(List.scala:45)

         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)

         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)

         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)

         at $Proxy12.finishApplicationMaster(Unknown Source)

         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)

         ... 9 more

Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)

         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)

         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)

         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)

         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)

         at org.apache.hadoop.ipc.Client.call(Client.java:1239)

         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)

         ... 11 more

Caused by: java.net.ConnectException: Connection refused

         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)

         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)

         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)

         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)

         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)

         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)

         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)

         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)

         at org.apache.hadoop.ipc.Client.call(Client.java:1206)

         ... 12 more





I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:



<property>

                <name>yarn.resourcemanager.scheduler.address</name>

                <value>192.168.145.37:8030</value>

</property>

<property>

                <name>yarn.resourcemanager.resource-tracker.address</name>

                <value>192.168.145.37:8031</value>

</property>

<property>

                <name>yarn.resourcemanager.address</name>

                <value>192.168.145.37:8032</value>

</property>

<property>

                <name>yarn.resourcemanager.admin.address</name>

                <value>192.168.145.37:8033</value>

</property>

<property>

                <name>yarn.resourcemanager.webapp.address</name>

                <value>192.168.145.37:8088</value>

</property>







These properties are reflected on the UI screen as well:



[cid:image001.png@01CEDBE8.1B2F1890]



But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.

I still get:

Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused



Nestat on the RM machine shows me:

tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java

tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java



Nestat on the NM machine shows me:

tcp        0      0 :::8040                     :::*                        LISTEN      1331/java

tcp        0      0 :::8042                     :::*                        LISTEN      1331/java

tcp        0      0 :::56877                    :::*                        LISTEN      1331/java



Kindly help me how to rectify this error.



Regards,

-Nirmal



________________________________













NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.

Re: Running Samza on multi node

Posted by Chris Riccomini <cr...@linkedin.com>.
Hey Nirmal,

Thanks for this detailed report! It makes things much easier to figure out. The problem appears to be that the Samza AM is trying to connect to 0.0.0.0:8030 when trying to talk to the RM. This is an RM port, which is running on 192.168.145.37 (the RM host), not 192.168.145.43 (the NM host). This is causing a timeout, since 8030 isn't open on localhost for the Samza AM, which is running on the NM's box.

It is somewhat interesting that the NM does connect to the RM for the capacity scheduler. Rather than setting each individual host/port pair, as you've done, I recommend just setting:

  <property>
    <name>yarn.resourcemanager.hostname</name>
    <value>192.168.145.37</value>
  </property>

Your netstat reports look fine – as expected.

Other questions:

1. Both your NM and RM are running YARN 2.2.0, right?
2. It appears that your AM shuts down. Did you run kill-job.sh to kill it?

Regarding (2), it appears that the AM never tries to register. This normally happens. I'm wondering if another failure is being triggered, which is then causing the AM to try and shut itself down. Could you turn on debugging for your Samza job (in log4j.xml), and re-run? I'm curious if the web-service that's starting up, or the registration itself is failing. In a normal execution, you would expect to see:


    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))

I don't see this in your logs, which means the AM is failing (and triggering a shutdown) before it even tries to register.

Cheers,
Chris

From: Nirmal Kumar <ni...@impetus.co.in>>
Reply-To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>" <de...@samza.incubator.apache.org>>
Date: Thursday, November 7, 2013 5:05 AM
To: "dev@samza.incubator.apache.org<ma...@samza.incubator.apache.org>" <de...@samza.incubator.apache.org>>
Subject: Running Samza on multi node

All,

I was able to run the hello-samza application on a single node machine.
Now I am trying to run the hello-samza application on  a 2 node setup.

Node1 has a Resource Manager
Node2 has a Node Manager

The NM gets registered with the RM successfully as seen in rm.log of the RM node:
13/11/07 11:44:29 INFO service.AbstractService: Service:ResourceManager is started.
13/11/07 11:48:30 INFO util.RackResolver: Resolved IMPETUS-DSRV14.impetus.co.in to /default-rack
13/11/07 11:48:30 INFO resourcemanager.ResourceTrackerService: NodeManager from node IMPETUS-DSRV14.impetus.co.in(cmPort: 56093 httpPort: 8042) registered with capability: <memory:8192, vCores:16>, assigned nodeId IMPETUS-DSRV14.impetus.co.in:56093
13/11/07 11:48:30 INFO rmnode.RMNodeImpl: IMPETUS-DSRV14.impetus.co.in:56093 Node Transitioned from NEW to RUNNING
13/11/07 11:48:30 INFO capacity.CapacityScheduler: Added node IMPETUS-DSRV14.impetus.co.in:56093 clusterResource: <memory:8192, vCores:16>

I am submitting the job from the RM machine using the command line:
bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file:/home/bda/nirmal/hello-samza/deploy/samza/config/test-consumer.properties

However, I am getting the following exception after submitting the job to YARN:

2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got container id: container_1383816757258_0001_01_000001
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got app attempt id: appattempt_1383816757258_0001_000001
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager host: IMPETUS-DSRV14
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager port: 59828
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got node manager http port: 8042
2013-11-07 15:05:57 SamzaAppMaster$ [INFO] got config: {task.inputs=kafka.storm-sentence, job.factory.class=org.apache.samza.job.yarn.YarnJobFactory, systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory, job.name=test-Consumer, systems.kafka.consumer.zookeeper.connect=192.168.145.195:2181/, systems.kafka.consumer.auto.offset.reset=largest, systems.kafka.samza.msg.serde=json, serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory, systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager, task.window.ms=10000, task.class=samza.examples.wikipedia.task.TestConsumer, yarn.package.path=file:/home/temptest/samza+storm/hello-samza/samza-job-package/target/samza-job-package-0.7.0-dist.tar.gz, systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory, systems.kafka.producer.metadata.broker.list=192.168.145.195:9092,192.168.145.195:9093}
2013-11-07 15:05:57 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032
2013-11-07 15:05:57 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are IMPETUS-DSRV14.impetus.co.in
2013-11-07 15:05:57 JmxServer [INFO] Started JmxServer port=47115 url=service:jmx:rmi:///jndi/rmi://IMPETUS-DSRV14.impetus.co.in:47115/jmxrmi
2013-11-07 15:05:57 SamzaAppMasterTaskManager [INFO] No yarn.container.count specified. Defaulting to one container.
2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0
2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093
2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
2013-11-07 15:05:57 VerifiableProperties [INFO] Property auto.offset.reset is overridden to largest
2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0
2013-11-07 15:05:57 VerifiableProperties [INFO] Property group.id is overridden to undefined-samza-consumer-group-
2013-11-07 15:05:57 VerifiableProperties [INFO] Property zookeeper.connect is overridden to 192.168.145.195:2181/
2013-11-07 15:05:57 VerifiableProperties [INFO] Verifying properties
2013-11-07 15:05:57 VerifiableProperties [INFO] Property client.id is overridden to samza_admin-test_Consumer-1-1383816957797-0
2013-11-07 15:05:57 VerifiableProperties [INFO] Property metadata.broker.list is overridden to 192.168.145.195:9092,192.168.145.195:9093
2013-11-07 15:05:57 VerifiableProperties [INFO] Property request.timeout.ms is overridden to 6000
2013-11-07 15:05:57 ClientUtils$ [INFO] Fetching metadata from broker id:0,host:192.168.145.195,port:9092 with correlation id 0 for 1 topic(s) Set(storm-sentence)
2013-11-07 15:05:57 SyncProducer [INFO] Connected to 192.168.145.195:9092 for producing
2013-11-07 15:05:57 SyncProducer [INFO] Disconnecting from 192.168.145.195:9092
2013-11-07 15:05:57 SamzaAppMasterService [INFO] Starting webapp at rpc 39152, tracking port 26751
2013-11-07 15:05:57 log [INFO] Logging to org.slf4j.impl.Log4jLoggerAdapter(org.eclipse.jetty.util.log) via org.eclipse.jetty.util.log.Slf4jLog
2013-11-07 15:05:58 ClientHelper [INFO] trying to connect to RM /0.0.0.0:8032
2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_39152_scalate____xveaws/webinf/WEB-INF
2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5279562760844696556-workdir
2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:39152
2013-11-07 15:05:58 log [INFO] jetty-7.0.0.v20091005
2013-11-07 15:05:58 log [INFO] Extract jar:file:/tmp/hadoop-vuser/nm-local-dir/usercache/bda/appcache/application_1383816757258_0001/filecache/8004956396276725272/samza-job-package-0.7.0-dist.tar.gz/lib/samza-yarn_2.8.1-0.7.0-yarn-2.0.5-alpha.jar!/scalate/WEB-INF/ to /tmp/Jetty_0_0_0_0_26751_scalate____.dr19qj/webinf/WEB-INF
2013-11-07 15:05:58 ServletTemplateEngine [INFO] Scalate template engine using working directory: /tmp/scalate-5582747144249485577-workdir
2013-11-07 15:05:58 log [INFO] Started SelectChannelConnector@0.0.0.0<ma...@0.0.0.0>:26751
2013-11-07 15:06:08 SamzaAppMasterLifecycle [INFO] Shutting down.
2013-11-07 15:06:18 YarnAppMaster [WARN] Listener org.apache.samza.job.yarn.SamzaAppMasterLifecycle@500c954e failed to shutdown.
java.lang.reflect.UndeclaredThrowableException
         at org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl.unwrapAndThrowException(YarnRemoteExceptionPBImpl.java:135)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:90)
         at org.apache.hadoop.yarn.client.AMRMClientImpl.unregisterApplicationMaster(AMRMClientImpl.java:244)
         at org.apache.samza.job.yarn.SamzaAppMasterLifecycle.onShutdown(SamzaAppMasterLifecycle.scala:68)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:70)
         at org.apache.samza.job.yarn.YarnAppMaster$$anonfun$run$9.apply(YarnAppMaster.scala:69)
         at scala.collection.LinearSeqOptimized$class.foreach(LinearSeqOptimized.scala:61)
         at scala.collection.immutable.List.foreach(List.scala:45)
         at org.apache.samza.job.yarn.YarnAppMaster.run(YarnAppMaster.scala:69)
         at org.apache.samza.job.yarn.SamzaAppMaster$.main(SamzaAppMaster.scala:78)
         at org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster.scala)
Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:212)
         at $Proxy12.finishApplicationMaster(Unknown Source)
         at org.apache.hadoop.yarn.api.impl.pb.client.AMRMProtocolPBClientImpl.finishApplicationMaster(AMRMProtocolPBClientImpl.java:87)
         ... 9 more
Caused by: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
         at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
         at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
         at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
         at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:780)
         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:727)
         at org.apache.hadoop.ipc.Client.call(Client.java:1239)
         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)
         ... 11 more
Caused by: java.net.ConnectException: Connection refused
         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:599)
         at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:526)
         at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:490)
         at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:508)
         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:603)
         at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:253)
         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1288)
         at org.apache.hadoop.ipc.Client.call(Client.java:1206)
         ... 12 more


I have changed the following properties in the hello-samza/deploy/yarn/etc/hadoop/yarn-site.xml on the Node Manager machine:

<property>
                <name>yarn.resourcemanager.scheduler.address</name>
                <value>192.168.145.37:8030</value>
</property>
<property>
                <name>yarn.resourcemanager.resource-tracker.address</name>
                <value>192.168.145.37:8031</value>
</property>
<property>
                <name>yarn.resourcemanager.address</name>
                <value>192.168.145.37:8032</value>
</property>
<property>
                <name>yarn.resourcemanager.admin.address</name>
                <value>192.168.145.37:8033</value>
</property>
<property>
                <name>yarn.resourcemanager.webapp.address</name>
                <value>192.168.145.37:8088</value>
</property>



These properties are reflected on the UI screen as well:

[cid:image001.png@01CEDBE8.1B2F1890]

But this overriding of the yarn.resourcemanager.scheduler.address to 192.168.145.37:8030 does not rectify the error.
I still get:
Caused by: com.google.protobuf.ServiceException: java.net.ConnectException: Call From IMPETUS-DSRV14.impetus.co.in/192.168.145.43 to 0.0.0.0:8030 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused

Nestat on the RM machine shows me:
tcp        0      0 ::ffff:192.168.145.37:8088  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8030  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8031  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8032  :::*                        LISTEN      14595/java
tcp        0      0 ::ffff:192.168.145.37:8033  :::*                        LISTEN      14595/java

Nestat on the NM machine shows me:
tcp        0      0 :::8040                     :::*                        LISTEN      1331/java
tcp        0      0 :::8042                     :::*                        LISTEN      1331/java
tcp        0      0 :::56877                    :::*                        LISTEN      1331/java

Kindly help me how to rectify this error.

Regards,
-Nirmal

________________________________






NOTE: This message may contain information that is confidential, proprietary, privileged or otherwise protected by law. The message is intended solely for the named addressee. If received in error, please destroy and notify the sender. Any use of this email is prohibited when received in error. Impetus does not represent, warrant and/or guarantee, that the integrity of this communication has been maintained nor that the communication is free of errors, virus, interception or interference.