hama-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Edward J. Yoon" <edwardy...@apache.org>
Subject Re: Hadoop RPC as a default
Date Wed, 26 Sep 2012 02:24:10 GMT
Sorry, same on large cluster :( Let's fix 0.7 release.

On Wed, Sep 26, 2012 at 10:46 AM, Edward J. Yoon <edwardyoon@apache.org> wrote:
> P.S., ignore this. +      data = null;
>
> On Wed, Sep 26, 2012 at 10:16 AM, Edward J. Yoon <edwardyoon@apache.org> wrote:
>> FYI, the problem is fixed by below small patch.
>>
>> Index: src/main/java/org/apache/hama/bsp/message/AvroMessageManagerImpl.java
>> ===================================================================
>> --- src/main/java/org/apache/hama/bsp/message/AvroMessageManagerImpl.java       (revision
>> 1389695)
>> +++ src/main/java/org/apache/hama/bsp/message/AvroMessageManagerImpl.java       (working
>> copy)
>> @@ -135,12 +135,15 @@
>>        ByteArrayInputStream inArray = new ByteArrayInputStream(byteArray);
>>        DataInputStream in = new DataInputStream(inArray);
>>        msg.readFields(in);
>> +      in.close();
>> +      inArray.close();
>>      } else {
>>        peer.incrementCounter(BSPPeerImpl.PeerCounter.COMPRESSED_BYTES_RECEIVED,
>>            byteArray.length);
>>        msg = compressor.decompressBundle(new BSPCompressedBundle(byteArray));
>>      }
>>
>> +    byteArray = null;
>>      return msg;
>>    }
>>
>> @@ -154,12 +157,14 @@
>>        byte[] byteArray = outArray.toByteArray();
>>        peer.incrementCounter(BSPPeerImpl.PeerCounter.MESSAGE_BYTES_TRANSFERED,
>>            byteArray.length);
>> +      outArray.close();
>>        return ByteBuffer.wrap(byteArray);
>>      } else {
>>        BSPCompressedBundle compMsgBundle = compressor.compressBundle(msg);
>>        byte[] data = compMsgBundle.getData();
>>        peer.incrementCounter(BSPPeerImpl.PeerCounter.COMPRESSED_BYTES_SENT,
>>            data.length);
>> +      data = null;
>>        return ByteBuffer.wrap(data);
>>      }
>>    }
>>
>> On Thu, Sep 20, 2012 at 4:22 PM, Edward J. Yoon <edwardyoon@apache.org> wrote:
>>> It seems we didn't test 0.5 release thoroughly.
>>>
>>> Let's give ourselves more time.
>>>
>>> On Wed, Sep 19, 2012 at 8:00 PM, Thomas Jungblut
>>> <thomas.jungblut@gmail.com> wrote:
>>>> Wait, the problem had multiple roots. We just fixed a few. I will sit down
>>>> and take a look at the messaging.
>>>> If this is scalable we can remove the multi step partitioning.
>>>> Am 19.09.2012 12:21 schrieb "Edward J. Yoon" <edwardyoon@apache.org>:
>>>>
>>>>> P.S., Since memory issue of graph job will be fixed by Thomas's
>>>>> HAMA-642, I'll remove my dirty multi-step partitioning code in graph
>>>>> module if there's no problem w/ Hadoop RPC tomorrow.
>>>>>
>>>>> On Wed, Sep 19, 2012 at 5:53 PM, Thomas Jungblut
>>>>> <thomas.jungblut@gmail.com> wrote:
>>>>> > I will give you more details what I planned on the interface changes
once
>>>>> > I'm back from my lecture.
>>>>> >
>>>>> > 2012/9/19 Suraj Menon <surajsmenon@apache.org>
>>>>> >
>>>>> >> As a beginning we should have a spilling queue and the same
with
>>>>> combiner
>>>>> >> running in batch if possible.
>>>>> >> I have been looking into implementing the spilling queue. Chalking
out
>>>>> the
>>>>> >> requirements, we should look into the following:
>>>>> >>
>>>>> >> A queue should persist all the data if required by the framework
for
>>>>> fault
>>>>> >> tolerance. ( I feel it would be a bad idea for framework to
spend
>>>>> resource
>>>>> >> on making a separate copy of the file )
>>>>> >> Asynchronous communication is our next important feature required
for
>>>>> >> performance.Hence we would need this queue with combiner on
sender side
>>>>> to
>>>>> >> batch the messages before sending. This implies we need to support
both
>>>>> >> concurrent reads and writes.
>>>>> >>
>>>>> >> -Suraj
>>>>> >>
>>>>> >> On Wed, Sep 19, 2012 at 4:21 AM, Thomas Jungblut
>>>>> >> <thomas.jungblut@gmail.com>wrote:
>>>>> >>
>>>>> >> > Oh okay, very interesting. Just another argument for making
the
>>>>> messaging
>>>>> >> > more scalable ;)
>>>>> >> >
>>>>> >> > 2012/9/19 Edward J. Yoon <edward.yoon@oracle.com>
>>>>> >> >
>>>>> >> > > Didn't check memory usage because each machine's memory
is 48 GB,
>>>>> but I
>>>>> >> > > guess there's no big difference.
>>>>> >> > >
>>>>> >> > > In short, "bin/hama bench 16 10000 32" was maximum
capacity (See
>>>>> [1]).
>>>>> >> If
>>>>> >> > > message numbers or nodes are increased, job is always
fails. Hadoop
>>>>> RPC
>>>>> >> > is
>>>>> >> > > OK.
>>>>> >> > >
>>>>> >> > > Will need time to debug this.
>>>>> >> > >
>>>>> >> > > 1. http://wiki.apache.org/hama/**Benchmarks#Random_**
>>>>> >> > > Communication_Benchmark<
>>>>> >> > http://wiki.apache.org/hama/Benchmarks#Random_Communication_Benchmark
>>>>> >
>>>>> >> > >
>>>>> >> > > On 9/19/2012 4:34 PM, Thomas Jungblut wrote:
>>>>> >> > >
>>>>> >> > >> BTW after HAMA-642<https://issues.**
>>>>> apache.org/jira/browse/HAMA-**642
>>>>> >> <
>>>>> >> > https://issues.apache.org/jira/browse/HAMA-642>>
>>>>> >> > >>  I will
>>>>> >> > >>
>>>>> >> > >> redesign our messaging system to being completely
disk based with
>>>>> >> > caching.
>>>>> >> > >> I will formulate a followup issue for this. However
I plan to get
>>>>> rid
>>>>> >> of
>>>>> >> > >> the RPC anyway, I think it is more efficient to
stream the messages
>>>>> >> from
>>>>> >> > >> disk over network to the other host via NIO (we
can later replace
>>>>> it
>>>>> >> > with
>>>>> >> > >> netty). Also this saves us the time to do the
checkpointing,
>>>>> because
>>>>> >> > this
>>>>> >> > >> can be combined with it pretty well. RPC requires
the whole bundle
>>>>> to
>>>>> >> be
>>>>> >> > >> in
>>>>> >> > >> RAM, which is totally bad.
>>>>> >> > >> Will follow with more details later.
>>>>> >> > >>
>>>>> >> > >> 2012/9/19 Thomas Jungblut<thomas.jungblut@**gmail.com<
>>>>> >> > thomas.jungblut@gmail.com>
>>>>> >> > >> >:
>>>>> >> > >>
>>>>> >> > >>> What is more memory efficient?
>>>>> >> > >>>
>>>>> >> > >>> Am 19.09.2012 08:23 schrieb "Edward J. Yoon"<
>>>>> edward.yoon@oracle.com
>>>>> >> >:
>>>>> >> > >>>
>>>>> >> > >>>  Let's change the default value of RPC in
hama-default.xml to
>>>>> Hadoop
>>>>> >> > RPC.
>>>>> >> > >>>>
>>>>> >> > >>> I
>>>>> >> > >>
>>>>> >> > >>> am testing Hadoop RPC and Avro RPC on 4 racks
cluster. Avro RPC is
>>>>> >> > >>>>
>>>>> >> > >>> criminal.
>>>>> >> > >>
>>>>> >> > >>> There's no significant performance difference.
>>>>> >> > >>>>
>>>>> >> > >>>> --
>>>>> >> > >>>> Best Regards, Edward J. Yoon
>>>>> >> > >>>> @eddieyoon
>>>>> >> > >>>>
>>>>> >> > >>>>
>>>>> >> > > --
>>>>> >> > > Best Regards, Edward J. Yoon
>>>>> >> > > @eddieyoon
>>>>> >> > >
>>>>> >> > >
>>>>> >> >
>>>>> >>
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Best Regards, Edward J. Yoon
>>>>> @eddieyoon
>>>>>
>>>
>>>
>>>
>>> --
>>> Best Regards, Edward J. Yoon
>>> @eddieyoon
>>
>>
>>
>> --
>> Best Regards, Edward J. Yoon
>> @eddieyoon
>
>
>
> --
> Best Regards, Edward J. Yoon
> @eddieyoon



-- 
Best Regards, Edward J. Yoon
@eddieyoon

Mime
View raw message