incubator-hama-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Thomas Jungblut <thomas.jungb...@googlemail.com>
Subject Re: Awesome bench results after removing Thread.sleep in sync() method.
Date Fri, 23 Sep 2011 16:20:11 GMT
Hi all,

I've attached the new profile.


2011/9/23 Thomas Jungblut <thomas.jungblut@googlemail.com>

> Yes, it just can be about the exceptions. I check for myself with the
> profiler.
>
>
> 2011/9/23 ChiaHung Lin <chl501@nuk.edu.tw>
>
>> Log contents are all about NoNodeException and NoExistsException. This
>> does not prevent computation to proceed further. Can you help check if /tmp
>> is full or any other exception in addition to these 2 exceptions? Or start a
>> clean run to see if it still hangs. Maybe there are some other issues.
>>
>> -----Original message-----
>> From:Thomas Jungblut <thomas.jungblut@googlemail.com>
>> To:hama-dev@incubator.apache.org,chl501@nuk.edu.tw
>> Date:Thu, 22 Sep 2011 13:28:10 +0200
>> Subject:Re: Awesome bench results after removing Thread.sleep in sync()
>> method.
>>
>> Maybe I have made a mistake.
>> But I'm pasting you the logs.
>>
>> In the groom's log, I've filtered the noise of benchmarks task outputs.
>> Hope
>> it can help you. What additional information do you need?
>> groom: http://justpaste.it/i22
>> zookeeper: http://justpaste.it/i23
>> master shows nothing, just the normal outputs.
>>
>> thanks :)
>>
>> 2011/9/22 ChiaHung Lin <chl501@nuk.edu.tw>
>>
>> > Is there any "Ignore because znode may be deleted." sentence just above
>> the
>> > NoNodeException? This exception is thrown as warning which should not
>> stop
>> > the computation.
>> >
>> > Also, I test with pseudo-distributed mode as below
>> >
>> > for((i=0;i<20;i++)) ; do hama jar
>> > hama-examples-0.4.0-incubating-SNAPSHOT.jar pi; done
>> >
>> > It works ok.
>> > http://pastebin.com/CxGSfzHN
>> >
>> > And the log has exception which doesn't cause computation to hang
>> >
>> > http://pastebin.com/5HVwx6A1
>> >
>> > attempt_201109221848_0020_000000_0 11/09/22 18:57:37 WARN bsp.BSPPeer:
>> > Ignore because znode may be deleted.
>> > 2011-09-22 18:57:37,331 INFO org.apache.hama.bsp.TaskRunner:
>> > attempt_201109221848_0020_000000_0
>> > org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode =
>> > NoNode for /bsp/job_201109221848_0020/0/ready
>> >
>> > Can we have the full log post? And how it is executed, env, etc. Maybe
>> the
>> > problem stems from somewhere else.
>> >
>> > -----Original message-----
>> > From:Thomas Jungblut <thomas.jungblut@googlemail.com>
>> > To:hama-dev@incubator.apache.org,chl501@nuk.edu.tw
>> > Date:Thu, 22 Sep 2011 10:43:13 +0200
>> > Subject:Re: Awesome bench results after removing Thread.sleep in sync()
>> > method.
>> >
>> > I think when just changing the log level, log4j will take care of the
>> > if(isEnabled) stuff, so we don't need to fragment our code.
>> > Yes the current rev in trunk contains this snippet. I give you the rest
>> of
>> > the exception:
>> >
>> > org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode =
>> > > NoNode for /bsp/job_201109220959_0001/224/ready
>> > >          at
>> > > org.apache.zookeeper.KeeperException.create(KeeperException.java:102)
>> > >          at
>> > > org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
>> > >          at org.apache.zookeeper.ZooKeeper.delete(ZooKeeper.java:728)
>> > >          at org.apache.hama.bsp.BSPPeer$1.process(BSPPeer.java:396)
>> > >          at
>> > > org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:488)
>> > >
>> >
>> > Here is the part of the log of our zookeeper deamon:
>> >
>> > > 2011-09-22 09:59:59,435 INFO
>> > > org.apache.zookeeper.server.PrepRequestProcessor: Got user-level
>> > > KeeperException when processing sessionid:0x1329025208e0003
>> type:delete
>> > > cxid:0xc01 zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a Error
>> > > Path:/bsp/job_201109220959_0001/222/ready Error:KeeperErrorCode =
>> NoNode
>> > for
>> > > /bsp/job_201109220959_0001/222/ready
>> > > 2011-09-22 09:59:59,499 INFO
>> > > org.apache.zookeeper.server.PrepRequestProcessor: Got user-level
>> > > KeeperException when processing sessionid:0x1329025208e0003
>> type:create
>> > > cxid:0xc0e zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a Error
>> > > Path:/bsp/job_201109220959_0001/223/ready Error:KeeperErrorCode =
>> > NodeExists
>> > > for /bsp/job_201109220959_0001/223/ready
>> > > 2011-09-22 09:59:59,627 INFO
>> > > org.apache.zookeeper.server.PrepRequestProcessor: Got user-level
>> > > KeeperException when processing sessionid:0x1329025208e0004
>> type:delete
>> > > cxid:0xc22 zxid:0xfffffffffffffffe txntype:unknown reqpath:n/a Error
>> > > Path:/bsp/job_201109220959_0001/224/ready Error:KeeperErrorCode =
>> NoNode
>> > for
>> > > /bsp/job_201109220959_0001/224/ready
>> > >
>> >
>> > 2011/9/22 ChiaHung Lin <chl501@nuk.edu.tw>
>> >
>> > > We might need to change log method by adding
>> > >
>> > > if(LOG.isInfoEnabled()){
>> > >  ...
>> > > }
>> > >
>> > > at least it can prevent string concatenation for performance
>> > optimization.
>> > > (debug can be changed to if(LOG.isDebugEnabled()){} for performance
>> > > optimization, too.)
>> > >
>> > > In addition, can you help check if enterBarrier() contains the
>> following
>> > > code snippet?
>> > >
>> > >   ...
>> > >   zk.exists(pathToSuperstepZnode+"/ready", new Watcher() {
>> > >      @Override
>> > >      public void process(WatchedEvent event) {
>> > >          // check if /ready znode exists, then delete it.
>> > >          ...
>> > >          } catch(KeeperException.NoNodeException nne) {
>> > >            LOG.warn("Ignore because znode may be deleted.", nne);
>> > >          }...
>> > >      }
>> > >    });
>> > >    zk.create(getNodeName(), null, Ids.OPEN_ACL_UNSAFE,
>> > > CreateMode.EPHEMERAL);
>> > >    ...
>> > >
>> > > It looks like bsp peer is trying to remove /ready znode which may have
>> > > already been removed by other bsp peer. Or stack trace in log would be
>> > > helpful.
>> > >
>> > >
>> > > -----Original message-----
>> > > From:Thomas Jungblut <thomas.jungblut@googlemail.com>
>> > > To:hama-dev@incubator.apache.org
>> > > Date:Thu, 22 Sep 2011 10:05:52 +0200
>> > > Subject:Re: Awesome bench results after removing Thread.sleep in
>> sync()
>> > > method.
>> > >
>> > > You're going to laugh, but we spend 80% of the time, logging the
>> > messages.
>> > > Let's change the log level to debug or remove the logging in the bench
>> > > example.
>> > >
>> > > Sadly I still receive
>> > >
>> > > org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode
>> =
>> > > > NoNode for /bsp/job_201109220959_0001/224/ready
>> > > >
>> > >
>> > > and it hangs forever. Current version is after you committed
>> ChiaHung's
>> > > patch.
>> > > I'm in pseudo-distributed mode with 3 tasks.
>> > >
>> > > Are you going to bench this without the logging? That would be
>> > interesting
>> > > though ;D
>> > >
>> > > 2011/9/22 Thomas Jungblut <thomas.jungblut@googlemail.com>
>> > >
>> > > > That is great. I think we can push this under 200s.
>> > > > I attach a profiler and send you a list of hotspots.
>> > > >
>> > > > lg.
>> > > >
>> > > > 2011/9/22 Edward J. Yoon <edwardyoon@apache.org>
>> > > >
>> > > > By ChiaHung's HAMA-387.patch, hang problem is fixed.
>> > > >>
>> > > >> And also, on same environment (1 rack, 256 cores), a bench example
>> > > >> result is dramatically improved. (184.076 seconds from 307.129
>> > > >> seconds)
>> > > >>
>> > > >> ----
>> > > >> # core/bin/hama jar
>> > > >> examples/target/hama-examples-0.4.0-incubating-SNAPSHOT.jar bench
>> 16
>> > > >> 1000 512
>> > > >> ..
>> > > >> 11/09/22 10:27:32 INFO bsp.BSPJobClient: Current supersteps number:
>> > 504
>> > > >> 11/09/22 10:27:35 INFO bsp.BSPJobClient: Current supersteps number:
>> > 508
>> > > >> 11/09/22 10:27:38 INFO bsp.BSPJobClient: Current supersteps number:
>> > 512
>> > > >> 11/09/22 10:27:38 INFO bsp.BSPJobClient: The total number of
>> > supersteps:
>> > > >> 512
>> > > >> Job Finished in 184.076 seconds
>> > > >>
>> > > >> Hama 0.4 (r.1163903) was:
>> > > >>
>> > > >> 16 bytes | 1000 | 512 | 307.129 seconds
>> > > >>
>> > > >> --
>> > > >> Best Regards, Edward J. Yoon
>> > > >> @eddieyoon
>> > > >>
>> > > >
>> > > >
>> > > >
>> > > > --
>> > > > Thomas Jungblut
>> > > > Berlin
>> > > >
>> > > > mobile: 0170-3081070
>> > > >
>> > > > business: thomas.jungblut@testberichte.de
>> > > > private: thomas.jungblut@gmail.com
>> > > >
>> > >
>> > >
>> > >
>> > > --
>> > > Thomas Jungblut
>> > > Berlin
>> > >
>> > > mobile: 0170-3081070
>> > >
>> > > business: thomas.jungblut@testberichte.de
>> > > private: thomas.jungblut@gmail.com
>> > >
>> > >
>> > > --
>> > > ChiaHung Lin
>> > > Department of Information Management
>> > > National University of Kaohsiung
>> > > Taiwan
>> > >
>> >
>> >
>> >
>> > --
>> > Thomas Jungblut
>> > Berlin
>> >
>> > mobile: 0170-3081070
>> >
>> > business: thomas.jungblut@testberichte.de
>> > private: thomas.jungblut@gmail.com
>> >
>> >
>> > --
>> > ChiaHung Lin
>> > Department of Information Management
>> > National University of Kaohsiung
>> > Taiwan
>> >
>>
>>
>>
>> --
>> Thomas Jungblut
>> Berlin
>>
>> mobile: 0170-3081070
>>
>> business: thomas.jungblut@testberichte.de
>> private: thomas.jungblut@gmail.com
>>
>>
>> --
>> ChiaHung Lin
>> Department of Information Management
>> National University of Kaohsiung
>> Taiwan
>>
>
>
>
> --
> Thomas Jungblut
> Berlin
>
> mobile: 0170-3081070
>
> business: thomas.jungblut@testberichte.de
> private: thomas.jungblut@gmail.com
>



-- 
Thomas Jungblut
Berlin

mobile: 0170-3081070

business: thomas.jungblut@testberichte.de
private: thomas.jungblut@gmail.com

Mime
  • Unnamed multipart/mixed (inline, None, 0 bytes)
View raw message