incubator-hama-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ChiaHung Lin" <chl...@nuk.edu.tw>
Subject Re: Awesome bench results after removing Thread.sleep in sync() method.
Date Thu, 22 Sep 2011 08:25:05 GMT
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

Mime
View raw message