[jira] [Updated] (GIRAPH-259) TestBspBasic.testBspPageRank is broken

2012-08-07 Thread Maja Kabiljo (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Maja Kabiljo updated GIRAPH-259:


Attachment: GIRAPH-259-5.patch

 TestBspBasic.testBspPageRank is broken
 --

 Key: GIRAPH-259
 URL: https://issues.apache.org/jira/browse/GIRAPH-259
 Project: Giraph
  Issue Type: Bug
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo
 Attachments: GIRAPH-259-1.patch, GIRAPH-259-2.patch, 
 GIRAPH-259-3.patch, GIRAPH-259-4.patch, GIRAPH-259-5.patch


 Test crashes on line 152 in class SimplePageRankVertex in distributed mode.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-259) TestBspBasic.testBspPageRank is broken

2012-08-07 Thread Maja Kabiljo (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-259?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430368#comment-13430368
 ] 

Maja Kabiljo commented on GIRAPH-259:
-

Removed json. Passes mvn verify and test in pseudo distributed mode.
I updated Review Board also: https://reviews.apache.org/r/6134/diff/3/

This can later be improved by having aggregator types and not writing whole 
aggregator class name every time. Another thing to consider is having numbers 
instead of strings for aggregator ids (this would make more sense and decrease 
the amount of data sent, if we have a lot of aggregators with simple aggregated 
value objects). But for now, this makes aggregators work correctly.

 TestBspBasic.testBspPageRank is broken
 --

 Key: GIRAPH-259
 URL: https://issues.apache.org/jira/browse/GIRAPH-259
 Project: Giraph
  Issue Type: Bug
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo
 Attachments: GIRAPH-259-1.patch, GIRAPH-259-2.patch, 
 GIRAPH-259-3.patch, GIRAPH-259-4.patch, GIRAPH-259-5.patch


 Test crashes on line 152 in class SimplePageRankVertex in distributed mode.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




Re: [jira] [Commented] (GIRAPH-259) TestBspBasic.testBspPageRank is broken

2012-08-07 Thread Maja Kabiljo
Eli, the sum error I get in the pseudo distributed mode. I tried your
patch again, it passes mvn verify, and if I comment out the sum check (and
leave the 31 check) it also passes tests in pseudo distributed mode. You
can try running the tests again, I don't know why would it be any
different.

On 8/6/12 6:49 PM, Eli Reisman initialcont...@gmail.com wrote:

I think this might have happened another time when I was changing these IO
formats over in an earlier version of this patch. It occurs because I am
eliminating 2 redundant text output formats as per Jakob's request, and
plugging IdWithValueXXX into the tests that used the old version before. I
will take another look at IdWithValue, it maybe it is doing something in
that test the old output format didn't that increments that aggregated
value? But yes, its the 31 expected, 32 received value in the test.
Everything else seems to work fine. If you run the patch against trunk
with
'mvn verify' you should get the same result I did? i can run it again too.
Thanks for the feedback, let me know if I can help get this to work
somehow, or if you realize there's something wrong with IdWithValue that
needs to change for it to replace those other 2 out formats.


On Sat, Aug 4, 2012 at 12:42 PM, Maja Kabiljo (JIRA)
j...@apache.orgwrote:


 [
 
https://issues.apache.org/jira/browse/GIRAPH-259?page=com.atlassian.jira.
plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13428666#c
omment-13428666]

 Maja Kabiljo commented on GIRAPH-259:
 -

 Eli, as for the change 31 - 32 (TestBspBasic, lines 418-420), that test
 checks how many times each aggregator value was written to
 AggregatorWriter. Before this patch, in this test aggregators were not
 registered on master, they were registered only in WorkerContext, so in
 master superstep 0 master didn't have the aggregators so didn't write
them
 then. Removing the need for aggregators to be registered on worker, I
 register aggregators in MasterCompute so now they are written to
 AggregatorWriter even in superstep 0. That's why I had to make this
change.
 But are you sure you are getting that mistake, and not the wrong value
of
 SumAggreg: 30, should be: 15? Because those checks come first in the
test,
 and they don't work without this patch. I tried to run tests with your
 patch on GIRAPH-218 and I do get this sum error.

 Avery, I'll remove json then, since we are going to move aggregators
away
 from zookeeper anyway.

  TestBspBasic.testBspPageRank is broken
  --
 
  Key: GIRAPH-259
  URL: https://issues.apache.org/jira/browse/GIRAPH-259
  Project: Giraph
   Issue Type: Bug
 Reporter: Maja Kabiljo
 Assignee: Maja Kabiljo
  Attachments: GIRAPH-259-1.patch, GIRAPH-259-2.patch,
 GIRAPH-259-3.patch, GIRAPH-259-4.patch
 
 
  Test crashes on line 152 in class SimplePageRankVertex in distributed
 mode.

 --
 This message is automatically generated by JIRA.
 If you think it was sent incorrectly, please contact your JIRA
 administrators:
 https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
 For more information on JIRA, see:
http://www.atlassian.com/software/jira






[jira] [Created] (GIRAPH-287) Add option to limit the number of open requests

2012-08-07 Thread Maja Kabiljo (JIRA)
Maja Kabiljo created GIRAPH-287:
---

 Summary: Add option to limit the number of open requests
 Key: GIRAPH-287
 URL: https://issues.apache.org/jira/browse/GIRAPH-287
 Project: Giraph
  Issue Type: Improvement
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo


As I mentioned in the discussion on the mailing list, GIRAPH-45 patch itself 
wasn't enough to run jobs with any amount of messages data. The thing which can 
still happen is that we have too many send messages requests to which we didn't 
get replies yet, so these requests use all our memory. Adding limit on the 
number of open requests will fix that last case.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-287) Add option to limit the number of open requests

2012-08-07 Thread Alessandro Presta (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430422#comment-13430422
 ] 

Alessandro Presta commented on GIRAPH-287:
--

Just to be clear, this allows to pause the computation when the number of 
message requests we have sent (and that appear not to have been processed yet) 
is above a threshold (Maja, correct me if I'm wrong).
Patch looks good, and it's switched off by default, so this should be safe to 
include.
One small nit: I think you want to change the log message to waitSomeRequests 
too.
I'll wait to see if anyone has comments in the next couple of days.

 Add option to limit the number of open requests
 ---

 Key: GIRAPH-287
 URL: https://issues.apache.org/jira/browse/GIRAPH-287
 Project: Giraph
  Issue Type: Improvement
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo
 Attachments: GIRAPH-287.patch


 As I mentioned in the discussion on the mailing list, GIRAPH-45 patch itself 
 wasn't enough to run jobs with any amount of messages data. The thing which 
 can still happen is that we have too many send messages requests to which we 
 didn't get replies yet, so these requests use all our memory. Adding limit on 
 the number of open requests will fix that last case.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-275) Restore data locality to workers reading InputSplits where possible without querying NameNode, ZooKeeper

2012-08-07 Thread Eli Reisman (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430434#comment-13430434
 ] 

Eli Reisman commented on GIRAPH-275:


Will do. This is the only spot (including inside the vertex readers for some 
reason) where the data is available at all, and all rack and port info is 
stripped, its just a list of hosts names. The strings are split when they come 
from the InputSplit, and are concatenated into the znode by the master. It 
would be easy to test the concating/splitting to make sure its working right.

Initial tests are coming up very promising, roughly 60% speedup in 
INPUT_SUPERSTEP, and about 25% speedup on total job runtimes so far, and this 
is only with about 40% of the locality I hope to achieve on really big job 
runs. And (so far) I'm able to run the same data loads on less workers. 
GIRAPH-262 seems to complement/extend this performance nicely in the brief 
testing with it patched in that I've had time to run so far. If the nodes 
read/send too fast, it can overwhelm Netty. Still working up a good 
configuration profile for this as it behaves differently than without the patch 
on the same job runs/data loads. More to follow...

 Restore data locality to workers reading InputSplits where possible without 
 querying NameNode, ZooKeeper
 

 Key: GIRAPH-275
 URL: https://issues.apache.org/jira/browse/GIRAPH-275
 Project: Giraph
  Issue Type: Improvement
  Components: bsp, graph
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
 Fix For: 0.2.0

 Attachments: GIRAPH-275-1.patch, GIRAPH-275-2.patch, 
 GIRAPH-275-3.patch, GIRAPH-275-4.patch


 During INPUT_SUPERSTEP, workers wait on a barrier until the master has 
 created a complete list of available input splits. Once the barrier is past, 
 each worker iterates through this list of input splits, creating a znode to 
 lay claim to the next unprocessed split the worker encounters.
 For a brief moment while the master is creating the input split znodes each 
 worker iterates through, it has access to InputSplit objects that also 
 contain a list of hostnames on which the blocks of the file are hosted. By 
 including that list of locations in each znode pathname we can allow each 
 worker reading the list of available splits to sort it so that splits the 
 worker attempts to claim first are the ones that contain a block that is 
 local to that worker's host.
 This allows the possibility for many workers to end up reading at least one 
 split that is local to its own host. If the input split selected holds a 
 local block, the RecordReader Hadoop supplies us with will automatically read 
 from that block anyway. By supplying this locality data as part of the znode 
 name rather than info inside the znode, we avoid reading the data from each 
 znode while sorting, which is only currently done when a split is claimed and 
 which is IO intensive. Sorting the string path data is cheap and faster, and 
 making the final split znode's name longer doesn't seem to matter too much.
 By using the BspMaster's InputSplit data to include locality information in 
 the znode path directly, we also avoid having to access the 
 FileSystem/BlockLocations directly from either master or workers, which could 
 also flood the name node with queries. This is the only place I've found 
 where some locality information is already available to Giraph free of 
 additional cost.
 Finally, by sorting each worker's split list this way, we get the 
 contention-reduction of GIRAPH-250 for free, since only workers on the same 
 host will be likely to contend for a split instead of the current situation 
 in which all workers contend for the same input splits from the same list, 
 iterating from the same index. GIRAPH-250 has already been logged as reducing 
 pages of contention on the first pass (when using many 100's of workers) down 
 to 0-3 contentions before claiming a split to read.
 This passes 'mvn verify' etc. I will post results of cluster testing ASAP. If 
 anyone else could try this on an HDFS cluster where locality info is supplied 
 to InputSplit objects, I would be really interested to see other folks' 
 results.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-275) Restore data locality to workers reading InputSplits where possible without querying NameNode, ZooKeeper

2012-08-07 Thread Eli Reisman (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430440#comment-13430440
 ] 

Eli Reisman commented on GIRAPH-275:


This is a typical blow-up when I configure wrong. One unlucky worker read 3 
very large splits in 10 minutes, Netty couldn't keep up. Working on the right # 
of workers/transfer limits/Netty configs.

Aug 7, 2012 4:53:26 PM org.jboss.netty.channel.socket.nio.NioWorker
WARNING: Unexpected exception in the selector loop.
java.lang.OutOfMemoryError: GC overhead limit exceeded
at java.util.HashMap.newKeyIterator(HashMap.java:840)
at java.util.HashMap$KeySet.iterator(HashMap.java:874)
at java.util.HashSet.iterator(HashSet.java:153)
at sun.nio.ch.SelectorImpl.processDeregisterQueue(SelectorImpl.java:127)
at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:69)
at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
at 
org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:33)
at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:157)
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)


 Restore data locality to workers reading InputSplits where possible without 
 querying NameNode, ZooKeeper
 

 Key: GIRAPH-275
 URL: https://issues.apache.org/jira/browse/GIRAPH-275
 Project: Giraph
  Issue Type: Improvement
  Components: bsp, graph
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
 Fix For: 0.2.0

 Attachments: GIRAPH-275-1.patch, GIRAPH-275-2.patch, 
 GIRAPH-275-3.patch, GIRAPH-275-4.patch


 During INPUT_SUPERSTEP, workers wait on a barrier until the master has 
 created a complete list of available input splits. Once the barrier is past, 
 each worker iterates through this list of input splits, creating a znode to 
 lay claim to the next unprocessed split the worker encounters.
 For a brief moment while the master is creating the input split znodes each 
 worker iterates through, it has access to InputSplit objects that also 
 contain a list of hostnames on which the blocks of the file are hosted. By 
 including that list of locations in each znode pathname we can allow each 
 worker reading the list of available splits to sort it so that splits the 
 worker attempts to claim first are the ones that contain a block that is 
 local to that worker's host.
 This allows the possibility for many workers to end up reading at least one 
 split that is local to its own host. If the input split selected holds a 
 local block, the RecordReader Hadoop supplies us with will automatically read 
 from that block anyway. By supplying this locality data as part of the znode 
 name rather than info inside the znode, we avoid reading the data from each 
 znode while sorting, which is only currently done when a split is claimed and 
 which is IO intensive. Sorting the string path data is cheap and faster, and 
 making the final split znode's name longer doesn't seem to matter too much.
 By using the BspMaster's InputSplit data to include locality information in 
 the znode path directly, we also avoid having to access the 
 FileSystem/BlockLocations directly from either master or workers, which could 
 also flood the name node with queries. This is the only place I've found 
 where some locality information is already available to Giraph free of 
 additional cost.
 Finally, by sorting each worker's split list this way, we get the 
 contention-reduction of GIRAPH-250 for free, since only workers on the same 
 host will be likely to contend for a split instead of the current situation 
 in which all workers contend for the same input splits from the same list, 
 iterating from the same index. GIRAPH-250 has already been logged as reducing 
 pages of contention on the first pass (when using many 100's of workers) down 
 to 0-3 contentions before claiming a split to read.
 This passes 'mvn verify' etc. I will post results of cluster testing ASAP. If 
 anyone else could try this on an HDFS cluster where locality info is supplied 
 to InputSplit objects, I would be really interested to see other folks' 
 results.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (GIRAPH-26) Improve PseudoRandomVertexInputFormat to create a more realistic synthetic graph (e.g. power-law distributed vertex-cardinality).

2012-08-07 Thread Sean Choi (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-26?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430457#comment-13430457
 ] 

Sean Choi commented on GIRAPH-26:
-

Hello Eli

Yes, that's exactly what I am doing to pass in the arguments.

Thanks for the suggestion.

 Improve PseudoRandomVertexInputFormat to create a more realistic synthetic 
 graph (e.g. power-law distributed vertex-cardinality).
 -

 Key: GIRAPH-26
 URL: https://issues.apache.org/jira/browse/GIRAPH-26
 Project: Giraph
  Issue Type: Test
  Components: benchmark
Affects Versions: 0.2.0
Reporter: Jake Mannix
Assignee: Sean Choi
Priority: Minor
 Fix For: 0.2.0

 Attachments: GIRAPH-26-1.patch


 The PageRankBenchmark class, to be a proper benchmark, should run over graphs 
 which look more like data seen in the wild, and web link graphs, social 
 network graphs, and text corpora (represented as a bipartite graph) all have 
 power-law distributions, so benchmarking a synthetic graph which looks more 
 like this would be a nice test which would stress cases of uneven 
 split-distribution and bottlenecks of subclusters of the graph of heavily 
 connected vertices.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




Re: Review Request: GIRAPH-262 Netty optimizations and bandwidth tracking

2012-08-07 Thread Eugene Koontz


 On Aug. 6, 2012, 6:15 p.m., Eugene Koontz wrote:
  http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyClient.java,
   line 112
  https://reviews.apache.org/r/6008/diff/1/?file=123493#file123493line112
 
  For consistency with other pipeline components, use new ByteCounter() 
  here.
 
 Avery Ching wrote:
 I need the reference here so that I can print the metrics periodically in 
 the client.  See NettyClient.java:237.
 
 Eugene Koontz wrote:
 I see, makes sense, thanks.
 
 Avery Ching wrote:
 Thanks for taking a look at this Eugene.  Do you have any more comments, 
 or is this ready to go?

Hi Avery, 

Thanks a lot for rebasing on top of those 3 new JIRAs.

If you don't mind I feel like there are three JIRAs in here still :)

1. netty optimization #1 shortcircuit to local doRequest() calls
2. netty optimization #2 connection pools
3. bandwidth tracking

I would like to extract two of them and leave one behind as GIRAPH-262 (so it 
would be 1. (netty optimization #1))

The reason I'm being so picky about it is that I've modified a lot of the same 
code in my GIRAPH-211 patch. So I want to carefully merge your enhancements 
into there. 


I am happy to extract 2 and 3 out today as I did yesterday for GIRAPH-280-282, 
if you don't mind kindly rebasing your patch on them. 

-Eugene


- Eugene


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/6008/#review9890
---


On Aug. 7, 2012, 1:04 a.m., Avery Ching wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/6008/
 ---
 
 (Updated Aug. 7, 2012, 1:04 a.m.)
 
 
 Review request for giraph.
 
 
 Description
 ---
 
 * Makes netty the default instead of HadoopRPC
 * Added optimization to handle requests locally when possible rather than go 
 over the network
 * Added TimedLogger to print only within a given time period
 * Added optimization for using multiple channels between clients/servers when 
 bandwidth is limited per connection
 * Added ByteCounter to track bandwidth across Netty (can be later integrated 
 with GIRAPH-232).
 * Upgraded rat to 0.8 and excluded iml files (Intellij Idea)
 * Set 32 instead of 64 as the default maximum netty threads
 
 
 This addresses bug GIRAPH-262.
 https://issues.apache.org/jira/browse/GIRAPH-262
 
 
 Diffs
 -
 
   http://svn.apache.org/repos/asf/giraph/trunk/pom.xml 1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/ByteCounter.java
  PRE-CREATION 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/ChannelRotater.java
  PRE-CREATION 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyClient.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyServer.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyWorkerClient.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/NettyWorkerClientServer.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/RequestDecoder.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/ResponseClientHandler.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/comm/SendVertexRequest.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphJob.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
  1370086 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/main/java/org/apache/giraph/utils/TimedLogger.java
  PRE-CREATION 
   
 http://svn.apache.org/repos/asf/giraph/trunk/src/test/java/org/apache/giraph/comm/ConnectionTest.java
  1370086 
 
 Diff: https://reviews.apache.org/r/6008/diff/
 
 
 Testing
 ---
 
 Passed mvn verify.  Used for the Hadoop Summit presentation results.
 
 
 Thanks,
 
 Avery Ching
 




[jira] [Created] (GIRAPH-288) Bandwidth tracking - subset of GIRAPH-262

2012-08-07 Thread Avery Ching (JIRA)
Avery Ching created GIRAPH-288:
--

 Summary: Bandwidth tracking - subset of GIRAPH-262
 Key: GIRAPH-288
 URL: https://issues.apache.org/jira/browse/GIRAPH-288
 Project: Giraph
  Issue Type: New Feature
Reporter: Avery Ching
Assignee: Avery Ching




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (GIRAPH-288) Bandwidth tracking - subset of GIRAPH-262

2012-08-07 Thread Avery Ching (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Avery Ching updated GIRAPH-288:
---

Attachment: GIRAPH-288.patch

Split out the bandwidth tracking.

 Bandwidth tracking - subset of GIRAPH-262
 -

 Key: GIRAPH-288
 URL: https://issues.apache.org/jira/browse/GIRAPH-288
 Project: Giraph
  Issue Type: New Feature
Reporter: Avery Ching
Assignee: Avery Ching
 Attachments: GIRAPH-288.patch




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Created] (GIRAPH-289) Add channel pool to NettyClient and thread pool to NettyServer

2012-08-07 Thread Eugene Koontz (JIRA)
Eugene Koontz created GIRAPH-289:


 Summary: Add channel pool to NettyClient and thread pool to 
NettyServer
 Key: GIRAPH-289
 URL: https://issues.apache.org/jira/browse/GIRAPH-289
 Project: Giraph
  Issue Type: Improvement
Reporter: Eugene Koontz
Assignee: Eugene Koontz


Add thread and channel pooling on NettyClient and NettyServer.

Instead of a NettyClient's addressChannelMap being:

  address = Channel

it is:

  address = ChannelRotater

Originally part of GIRAPH-262, extracted here.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (GIRAPH-289) Add thread and channel pooling to NettyClient and NettyServer

2012-08-07 Thread Eugene Koontz (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-289?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Eugene Koontz updated GIRAPH-289:
-

Summary: Add thread and channel pooling to NettyClient and NettyServer  
(was: Add channel pool to NettyClient and thread pool to NettyServer)

 Add thread and channel pooling to NettyClient and NettyServer
 -

 Key: GIRAPH-289
 URL: https://issues.apache.org/jira/browse/GIRAPH-289
 Project: Giraph
  Issue Type: Improvement
Reporter: Eugene Koontz
Assignee: Eugene Koontz

 Add thread and channel pooling on NettyClient and NettyServer.
 Instead of a NettyClient's addressChannelMap being:
   address = Channel
 it is:
   address = ChannelRotater
 Originally part of GIRAPH-262, extracted here.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (GIRAPH-289) Add thread and channel pooling to NettyClient and NettyServer

2012-08-07 Thread Eugene Koontz (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-289?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Eugene Koontz updated GIRAPH-289:
-

Attachment: GIRAPH-289.patch

passes mvn clean verify.

 Add thread and channel pooling to NettyClient and NettyServer
 -

 Key: GIRAPH-289
 URL: https://issues.apache.org/jira/browse/GIRAPH-289
 Project: Giraph
  Issue Type: Improvement
Reporter: Eugene Koontz
Assignee: Eugene Koontz
 Attachments: GIRAPH-289.patch


 Add thread and channel pooling on NettyClient and NettyServer.
 Instead of a NettyClient's addressChannelMap being:
   address = Channel
 it is:
   address = ChannelRotater
 Originally part of GIRAPH-262, extracted here.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (GIRAPH-288) Bandwidth tracking - subset of GIRAPH-262

2012-08-07 Thread Avery Ching (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Avery Ching updated GIRAPH-288:
---

Attachment: (was: GIRAPH-288.patch)

 Bandwidth tracking - subset of GIRAPH-262
 -

 Key: GIRAPH-288
 URL: https://issues.apache.org/jira/browse/GIRAPH-288
 Project: Giraph
  Issue Type: New Feature
Reporter: Avery Ching
Assignee: Avery Ching
 Attachments: GIRAPH-288.patch




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-288) Bandwidth tracking - subset of GIRAPH-262

2012-08-07 Thread Eugene Koontz (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-288?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430538#comment-13430538
 ] 

Eugene Koontz commented on GIRAPH-288:
--

+1 

 Bandwidth tracking - subset of GIRAPH-262
 -

 Key: GIRAPH-288
 URL: https://issues.apache.org/jira/browse/GIRAPH-288
 Project: Giraph
  Issue Type: New Feature
Reporter: Avery Ching
Assignee: Avery Ching
 Attachments: GIRAPH-288.2.patch, GIRAPH-288.patch




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Resolved] (GIRAPH-288) Bandwidth tracking - subset of GIRAPH-262

2012-08-07 Thread Avery Ching (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Avery Ching resolved GIRAPH-288.


Resolution: Fixed

Thanks Eugene.  Committed.

 Bandwidth tracking - subset of GIRAPH-262
 -

 Key: GIRAPH-288
 URL: https://issues.apache.org/jira/browse/GIRAPH-288
 Project: Giraph
  Issue Type: New Feature
Reporter: Avery Ching
Assignee: Avery Ching
 Attachments: GIRAPH-288.2.patch, GIRAPH-288.patch




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-287) Add option to limit the number of open requests

2012-08-07 Thread Jakob Homan (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430558#comment-13430558
 ] 

Jakob Homan commented on GIRAPH-287:


How has this patch been tested?

 Add option to limit the number of open requests
 ---

 Key: GIRAPH-287
 URL: https://issues.apache.org/jira/browse/GIRAPH-287
 Project: Giraph
  Issue Type: Improvement
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo
 Attachments: GIRAPH-287.patch


 As I mentioned in the discussion on the mailing list, GIRAPH-45 patch itself 
 wasn't enough to run jobs with any amount of messages data. The thing which 
 can still happen is that we have too many send messages requests to which we 
 didn't get replies yet, so these requests use all our memory. Adding limit on 
 the number of open requests will fix that last case.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-287) Add option to limit the number of open requests

2012-08-07 Thread Maja Kabiljo (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430569#comment-13430569
 ] 

Maja Kabiljo commented on GIRAPH-287:
-

I tested it on a cluster, as I described on the mailing list, using 
RandomMessageBenchmark. I got a job which was running out of memory because of 
too many open requests to finish successfully.

 Add option to limit the number of open requests
 ---

 Key: GIRAPH-287
 URL: https://issues.apache.org/jira/browse/GIRAPH-287
 Project: Giraph
  Issue Type: Improvement
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo
 Attachments: GIRAPH-287.patch


 As I mentioned in the discussion on the mailing list, GIRAPH-45 patch itself 
 wasn't enough to run jobs with any amount of messages data. The thing which 
 can still happen is that we have too many send messages requests to which we 
 didn't get replies yet, so these requests use all our memory. Adding limit on 
 the number of open requests will fix that last case.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-288) Bandwidth tracking - subset of GIRAPH-262

2012-08-07 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-288?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430572#comment-13430572
 ] 

Hudson commented on GIRAPH-288:
---

Integrated in Giraph-trunk-Commit #161 (See 
[https://builds.apache.org/job/Giraph-trunk-Commit/161/])
GIRAPH-288: Bandwidth tracking - subset of GIRAPH-262. (aching) (Revision 
1370438)

 Result = SUCCESS
aching : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1370438
Files : 
* /giraph/trunk/CHANGELOG
* /giraph/trunk/src/main/java/org/apache/giraph/comm/ByteCounter.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyClient.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyServer.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/RequestDecoder.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/ResponseClientHandler.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/SendVertexRequest.java
* /giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java
* /giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
* /giraph/trunk/src/main/java/org/apache/giraph/utils/TimedLogger.java
* /giraph/trunk/src/test/java/org/apache/giraph/comm/ConnectionTest.java


 Bandwidth tracking - subset of GIRAPH-262
 -

 Key: GIRAPH-288
 URL: https://issues.apache.org/jira/browse/GIRAPH-288
 Project: Giraph
  Issue Type: New Feature
Reporter: Avery Ching
Assignee: Avery Ching
 Attachments: GIRAPH-288.2.patch, GIRAPH-288.patch




--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-262) Netty optimization to handle requests locally whenever possible

2012-08-07 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-262?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430570#comment-13430570
 ] 

Hudson commented on GIRAPH-262:
---

Integrated in Giraph-trunk-Commit #161 (See 
[https://builds.apache.org/job/Giraph-trunk-Commit/161/])
GIRAPH-288: Bandwidth tracking - subset of GIRAPH-262. (aching) (Revision 
1370438)

 Result = SUCCESS
aching : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1370438
Files : 
* /giraph/trunk/CHANGELOG
* /giraph/trunk/src/main/java/org/apache/giraph/comm/ByteCounter.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyClient.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyServer.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/RequestDecoder.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/ResponseClientHandler.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/SendVertexRequest.java
* /giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java
* /giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
* /giraph/trunk/src/main/java/org/apache/giraph/utils/TimedLogger.java
* /giraph/trunk/src/test/java/org/apache/giraph/comm/ConnectionTest.java


 Netty optimization to handle requests locally whenever possible
 ---

 Key: GIRAPH-262
 URL: https://issues.apache.org/jira/browse/GIRAPH-262
 Project: Giraph
  Issue Type: Improvement
Reporter: Avery Ching
Assignee: Avery Ching
 Attachments: GIRAPH-262.2.patch, GIRAPH-262.3.patch, 
 GIRAPH-262.4.patch, GIRAPH-262.5.patch, GIRAPH-262.patch


 * Makes netty the default instead of HadoopRPC
 * Added optimization to handle requests locally when possible rather than go 
 over the network
 * Added TimedLogger to print only within a given time period
 * Added optimization for using multiple channels between clients/servers when 
 bandwidth is limited per connection
 * Added ByteCounter to track bandwidth across Netty (can be later integrated 
 with GIRAPH-232).
 * Upgraded rat to 0.8 and excluded iml files (Intellij Idea)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-289) Add thread and channel pooling to NettyClient and NettyServer

2012-08-07 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430571#comment-13430571
 ] 

Hudson commented on GIRAPH-289:
---

Integrated in Giraph-trunk-Commit #161 (See 
[https://builds.apache.org/job/Giraph-trunk-Commit/161/])
GIRAPH-289: Add thread and channel pooling to NettyClient and
NettyServer. (ekoontz via aching) (Revision 1370430)

 Result = SUCCESS
aching : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1370430
Files : 
* /giraph/trunk/CHANGELOG
* /giraph/trunk/src/main/java/org/apache/giraph/comm/ChannelRotater.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyClient.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyServer.java
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyWorkerClient.java
* /giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphJob.java
* /giraph/trunk/src/test/java/org/apache/giraph/comm/ConnectionTest.java
* /giraph/trunk/src/test/java/org/apache/giraph/comm/RequestTest.java


 Add thread and channel pooling to NettyClient and NettyServer
 -

 Key: GIRAPH-289
 URL: https://issues.apache.org/jira/browse/GIRAPH-289
 Project: Giraph
  Issue Type: Improvement
Reporter: Eugene Koontz
Assignee: Eugene Koontz
 Attachments: GIRAPH-289.patch


 Add thread and channel pooling on NettyClient and NettyServer.
 Instead of a NettyClient's addressChannelMap being:
   address = Channel
 it is:
   address = ChannelRotater
 Originally part of GIRAPH-262, extracted here.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




Please welcome our newest committer, Alessandro!

2012-08-07 Thread Avery Ching
I'm excited to announce that the Giraph PMC has voted Alessandro Presta 
in as a committer and that he has accepted our offer.  Alessandro has 
been very active in the Giraph community, here is a list of some of his 
great contributions.


GIRAPH-276: Fix broken tests in pseudo-distributed mode.
(Alessandro Presta via jghoman)
GIRAPH-244: Vertex API redesign (apresta via aching).
GIRAPH-224: Netty server-side combiner (apresta via aching).
GIRAPH-242: HashMapVertex stores neighbor ids twice. (Alessandro Presta 
via hyunsik)

GIRAPH-239: IntIntNullIntVertex doesn't save halted state (apresta
via aching)
GIRAPH-238: BasicVertex should have default Writable implementation
(apresta via aching)
GIRAPH-221: Make iteration over edges more explicit (apresta via aching).
GIRAPH-220: Default implementation of BasicVertex#sendMsgToAllEdges().
(Alessandro Presta via jghoman)

In the spirit of having him commit his first piece of code, please take 
a look at https://issues.apache.org/jira/browse/GIRAPH-290 .


Welcome Alessandro and happy Giraphing!

Avery Ching


[jira] [Commented] (GIRAPH-262) Netty optimization to handle requests locally whenever possible

2012-08-07 Thread Eugene Koontz (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-262?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430577#comment-13430577
 ] 

Eugene Koontz commented on GIRAPH-262:
--

+1. Passes mvn clean verify with no -P (Hadoop 0.20.203.0) and with 
-Phadoop_trunk.



 Netty optimization to handle requests locally whenever possible
 ---

 Key: GIRAPH-262
 URL: https://issues.apache.org/jira/browse/GIRAPH-262
 Project: Giraph
  Issue Type: Improvement
Reporter: Avery Ching
Assignee: Avery Ching
 Attachments: GIRAPH-262.2.patch, GIRAPH-262.3.patch, 
 GIRAPH-262.4.patch, GIRAPH-262.5.patch, GIRAPH-262.patch


 * Makes netty the default instead of HadoopRPC
 * Added optimization to handle requests locally when possible rather than go 
 over the network
 * Added TimedLogger to print only within a given time period
 * Added optimization for using multiple channels between clients/servers when 
 bandwidth is limited per connection
 * Added ByteCounter to track bandwidth across Netty (can be later integrated 
 with GIRAPH-232).
 * Upgraded rat to 0.8 and excluded iml files (Intellij Idea)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-287) Add option to limit the number of open requests

2012-08-07 Thread Jakob Homan (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430588#comment-13430588
 ] 

Jakob Homan commented on GIRAPH-287:


OK. My only comment would be to move the definition of the constants to 
NettyClient.  It's best not to load GiraphJob up with definitions for other 
components.  Current patch passes mvn verify.  +1 once the constants are moved.

 Add option to limit the number of open requests
 ---

 Key: GIRAPH-287
 URL: https://issues.apache.org/jira/browse/GIRAPH-287
 Project: Giraph
  Issue Type: Improvement
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo
 Attachments: GIRAPH-287.patch


 As I mentioned in the discussion on the mailing list, GIRAPH-45 patch itself 
 wasn't enough to run jobs with any amount of messages data. The thing which 
 can still happen is that we have too many send messages requests to which we 
 didn't get replies yet, so these requests use all our memory. Adding limit on 
 the number of open requests will fix that last case.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-287) Add option to limit the number of open requests

2012-08-07 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-287?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430632#comment-13430632
 ] 

Hudson commented on GIRAPH-287:
---

Integrated in Giraph-trunk-Commit #162 (See 
[https://builds.apache.org/job/Giraph-trunk-Commit/162/])
GIRAPH-287. Add option to limit the number of open requests. Contributed by 
Maja Kabiljo. (Revision 1370522)

 Result = SUCCESS
jghoman : 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1370522
Files : 
* /giraph/trunk/CHANGELOG
* /giraph/trunk/src/main/java/org/apache/giraph/comm/NettyClient.java


 Add option to limit the number of open requests
 ---

 Key: GIRAPH-287
 URL: https://issues.apache.org/jira/browse/GIRAPH-287
 Project: Giraph
  Issue Type: Improvement
Affects Versions: 0.2.0
Reporter: Maja Kabiljo
Assignee: Maja Kabiljo
 Fix For: 0.2.0

 Attachments: GIRAPH-287-2.patch, GIRAPH-287.patch


 As I mentioned in the discussion on the mailing list, GIRAPH-45 patch itself 
 wasn't enough to run jobs with any amount of messages data. The thing which 
 can still happen is that we have too many send messages requests to which we 
 didn't get replies yet, so these requests use all our memory. Adding limit on 
 the number of open requests will fix that last case.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-246) Periodic worker calls to context.progress() will prevent timeout on some Hadoop clusters during barrier waits

2012-08-07 Thread Avery Ching (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430675#comment-13430675
 ] 

Avery Ching commented on GIRAPH-246:


I don't understand why you are removing Context from PredicateLock?  
waitForever() needs context to progress.

 Periodic worker calls to context.progress() will prevent timeout on some 
 Hadoop clusters during barrier waits
 -

 Key: GIRAPH-246
 URL: https://issues.apache.org/jira/browse/GIRAPH-246
 Project: Giraph
  Issue Type: Improvement
  Components: bsp
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
Priority: Minor
  Labels: hadoop, patch
 Fix For: 0.2.0

 Attachments: GIRAPH-246-1.patch, GIRAPH-246-2.patch, 
 GIRAPH-246-3.patch, GIRAPH-246-4.patch, GIRAPH-246-5.patch, GIRAPH-246-6.patch


 This simple change creates a command-line configurable option in GiraphJob to 
 control the time between calls to context().progress() that allows workers to 
 avoid timeouts during long data load-ins in which some works complete their 
 input split reads much faster than others, or finish a super step faster. I 
 found this allowed jobs that were large-scale but with low memory overhead to 
 complete even when they would previously time out during runs on a Hadoop 
 cluster. Timeout is still possible when the worker crashes or runs out of 
 memory or has other GC or RPC trouble that is legitimate, but prevents 
 unintentional crashes when the worker is actually still healthy.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Comment Edited] (GIRAPH-246) Periodic worker calls to context.progress() will prevent timeout on some Hadoop clusters during barrier waits

2012-08-07 Thread Avery Ching (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430688#comment-13430688
 ] 

Avery Ching edited comment on GIRAPH-246 at 8/7/12 10:29 PM:
-

I prefer the changes of GIRAPH-246-1.patch as I think that PredicateLock works 
nicely with Context for waitForever.  

Can you please explain why you are doing this?

{code}
-  ++vertexCount;
+  if (vertexCount++ % 100 == 0) {
+getContext().progress();
+  }
   edgeCount += readerVertex.getNumEdges();
-  getContext().progress();
{code}

Is getContext.progress() pretty expensive?

  was (Author: aching):
I prefer the changes of GIRAPH-246-1.patch as I think that PredicateLock 
works nicely with Context for waitForever.  

Can you please explain why you are doing this?

-  ++vertexCount;
+  if (vertexCount++ % 100 == 0) {
+getContext().progress();
+  }
   edgeCount += readerVertex.getNumEdges();
-  getContext().progress();

Is getContext.progress() pretty expensive?
  
 Periodic worker calls to context.progress() will prevent timeout on some 
 Hadoop clusters during barrier waits
 -

 Key: GIRAPH-246
 URL: https://issues.apache.org/jira/browse/GIRAPH-246
 Project: Giraph
  Issue Type: Improvement
  Components: bsp
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
Priority: Minor
  Labels: hadoop, patch
 Fix For: 0.2.0

 Attachments: GIRAPH-246-1.patch, GIRAPH-246-2.patch, 
 GIRAPH-246-3.patch, GIRAPH-246-4.patch, GIRAPH-246-5.patch, GIRAPH-246-6.patch


 This simple change creates a command-line configurable option in GiraphJob to 
 control the time between calls to context().progress() that allows workers to 
 avoid timeouts during long data load-ins in which some works complete their 
 input split reads much faster than others, or finish a super step faster. I 
 found this allowed jobs that were large-scale but with low memory overhead to 
 complete even when they would previously time out during runs on a Hadoop 
 cluster. Timeout is still possible when the worker crashes or runs out of 
 memory or has other GC or RPC trouble that is legitimate, but prevents 
 unintentional crashes when the worker is actually still healthy.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-246) Periodic worker calls to context.progress() will prevent timeout on some Hadoop clusters during barrier waits

2012-08-07 Thread Jakob Homan (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430690#comment-13430690
 ] 

Jakob Homan commented on GIRAPH-246:


The PredicateLock stuff is fine, but according to Jaeho, doesn't seem to be 
working.  Once the reason is nailed down, GIRAPH-274 can re-introduce it.

 Periodic worker calls to context.progress() will prevent timeout on some 
 Hadoop clusters during barrier waits
 -

 Key: GIRAPH-246
 URL: https://issues.apache.org/jira/browse/GIRAPH-246
 Project: Giraph
  Issue Type: Improvement
  Components: bsp
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
Priority: Minor
  Labels: hadoop, patch
 Fix For: 0.2.0

 Attachments: GIRAPH-246-1.patch, GIRAPH-246-2.patch, 
 GIRAPH-246-3.patch, GIRAPH-246-4.patch, GIRAPH-246-5.patch, GIRAPH-246-6.patch


 This simple change creates a command-line configurable option in GiraphJob to 
 control the time between calls to context().progress() that allows workers to 
 avoid timeouts during long data load-ins in which some works complete their 
 input split reads much faster than others, or finish a super step faster. I 
 found this allowed jobs that were large-scale but with low memory overhead to 
 complete even when they would previously time out during runs on a Hadoop 
 cluster. Timeout is still possible when the worker crashes or runs out of 
 memory or has other GC or RPC trouble that is legitimate, but prevents 
 unintentional crashes when the worker is actually still healthy.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-246) Periodic worker calls to context.progress() will prevent timeout on some Hadoop clusters during barrier waits

2012-08-07 Thread Eli Reisman (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430702#comment-13430702
 ] 

Eli Reisman commented on GIRAPH-246:


lets call it every time, I was not aware it was logging them and not calling it 
right away, the calls are cheap, I will change that back. My thought was I need 
this to work, since the other fix it has not, and once Jaeho has a better 
solution worked out and functional, he can repost on 274 as Jakob said. I just 
need this to work so I can continue running big jobs on Giraph without getting 
timed out.
I'll fix and repost right now.


 Periodic worker calls to context.progress() will prevent timeout on some 
 Hadoop clusters during barrier waits
 -

 Key: GIRAPH-246
 URL: https://issues.apache.org/jira/browse/GIRAPH-246
 Project: Giraph
  Issue Type: Improvement
  Components: bsp
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
Priority: Minor
  Labels: hadoop, patch
 Fix For: 0.2.0

 Attachments: GIRAPH-246-1.patch, GIRAPH-246-2.patch, 
 GIRAPH-246-3.patch, GIRAPH-246-4.patch, GIRAPH-246-5.patch, GIRAPH-246-6.patch


 This simple change creates a command-line configurable option in GiraphJob to 
 control the time between calls to context().progress() that allows workers to 
 avoid timeouts during long data load-ins in which some works complete their 
 input split reads much faster than others, or finish a super step faster. I 
 found this allowed jobs that were large-scale but with low memory overhead to 
 complete even when they would previously time out during runs on a Hadoop 
 cluster. Timeout is still possible when the worker crashes or runs out of 
 memory or has other GC or RPC trouble that is legitimate, but prevents 
 unintentional crashes when the worker is actually still healthy.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (GIRAPH-256) Partitioning outgoing graph data during INPUT_SUPERSTEP by # of vertices results in wide variance in RPC message sizes

2012-08-07 Thread Eli Reisman (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-256?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Eli Reisman updated GIRAPH-256:
---

Attachment: GIRAPH-256-6.patch

This refactors msg size regulator into a separate class, adds unit test etc. 
and is a lot cleaner in general. Passes mvn verify etc.

 Partitioning outgoing graph data during INPUT_SUPERSTEP by # of vertices 
 results in wide variance in RPC message sizes
 --

 Key: GIRAPH-256
 URL: https://issues.apache.org/jira/browse/GIRAPH-256
 Project: Giraph
  Issue Type: Improvement
  Components: bsp, graph
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
  Labels: patch
 Fix For: 0.2.0

 Attachments: GIRAPH-256-1.patch, GIRAPH-256-2.patch, 
 GIRAPH-256-3.patch, GIRAPH-256-4.patch, GIRAPH-256-5.patch, GIRAPH-256-6.patch


 This relates to GIRAPH-247. The unfortunately named 
 MAX_VERTICES_PER_PARTITION fooled me into thinking this value was 
 regulating the size of initial Partition objects as they were composed during 
 INPUT_SUPERSTEP from InputSplits each worker reads.
 In fact this configuration option only regulates the size of the outgoing RPC 
 messages, stored locally in Partition objects but decomposed into Collections 
 of BasicVertex for transfer to their eventual homes on another (or this) 
 worker. There they are combined into the actual Partitions they will exist in 
 for the job run.
 By partitioning these outgoing messages by # of vertices, metrics load tests 
 have shown the size of the average message is not well regulated and can 
 create overloads on either side of these transfers. This is important because:
 1. Throughput and memory are at a premium during INPUT_SUPERSTEP.
 2. Only one crashed worker in a Giraph job causes cascading job failure, even 
 in an otherwise healthy workflow.
 This JIRA renames the offending variables/config options and further 
 regulates outgoing graph data in INPUT_SUPERSTEP by the # of edges and THEN 
 the # of vertices in a candidate for transfer. This much more effectively 
 regulates message size for typical social graph data and has been show in 
 testing to greatly improve the amount of load-in data Giraph can handle 
 without failure given fixed memory and worker limits.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-256) Partitioning outgoing graph data during INPUT_SUPERSTEP by # of vertices results in wide variance in RPC message sizes

2012-08-07 Thread Jakob Homan (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-256?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430725#comment-13430725
 ] 

Jakob Homan commented on GIRAPH-256:


Looking good.  Can you move the new constants to the classes that use them, 
rather than GiraphJob? Also, there are still some spurious white space changes 
to be removed.  Thanks.

 Partitioning outgoing graph data during INPUT_SUPERSTEP by # of vertices 
 results in wide variance in RPC message sizes
 --

 Key: GIRAPH-256
 URL: https://issues.apache.org/jira/browse/GIRAPH-256
 Project: Giraph
  Issue Type: Improvement
  Components: bsp, graph
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
  Labels: patch
 Fix For: 0.2.0

 Attachments: GIRAPH-256-1.patch, GIRAPH-256-2.patch, 
 GIRAPH-256-3.patch, GIRAPH-256-4.patch, GIRAPH-256-5.patch, GIRAPH-256-6.patch


 This relates to GIRAPH-247. The unfortunately named 
 MAX_VERTICES_PER_PARTITION fooled me into thinking this value was 
 regulating the size of initial Partition objects as they were composed during 
 INPUT_SUPERSTEP from InputSplits each worker reads.
 In fact this configuration option only regulates the size of the outgoing RPC 
 messages, stored locally in Partition objects but decomposed into Collections 
 of BasicVertex for transfer to their eventual homes on another (or this) 
 worker. There they are combined into the actual Partitions they will exist in 
 for the job run.
 By partitioning these outgoing messages by # of vertices, metrics load tests 
 have shown the size of the average message is not well regulated and can 
 create overloads on either side of these transfers. This is important because:
 1. Throughput and memory are at a premium during INPUT_SUPERSTEP.
 2. Only one crashed worker in a Giraph job causes cascading job failure, even 
 in an otherwise healthy workflow.
 This JIRA renames the offending variables/config options and further 
 regulates outgoing graph data in INPUT_SUPERSTEP by the # of edges and THEN 
 the # of vertices in a candidate for transfer. This much more effectively 
 regulates message size for typical social graph data and has been show in 
 testing to greatly improve the amount of load-in data Giraph can handle 
 without failure given fixed memory and worker limits.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-246) Periodic worker calls to context.progress() will prevent timeout on some Hadoop clusters during barrier waits

2012-08-07 Thread Avery Ching (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430755#comment-13430755
 ] 

Avery Ching commented on GIRAPH-246:


The configurableness of the progress is nice to match the zookeeper 
configurableness of the timeouts.

 Periodic worker calls to context.progress() will prevent timeout on some 
 Hadoop clusters during barrier waits
 -

 Key: GIRAPH-246
 URL: https://issues.apache.org/jira/browse/GIRAPH-246
 Project: Giraph
  Issue Type: Improvement
  Components: bsp
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
Priority: Minor
  Labels: hadoop, patch
 Fix For: 0.2.0

 Attachments: GIRAPH-246-1.patch, GIRAPH-246-2.patch, 
 GIRAPH-246-3.patch, GIRAPH-246-4.patch, GIRAPH-246-5.patch, 
 GIRAPH-246-6.patch, GIRAPH-246-7.patch


 This simple change creates a command-line configurable option in GiraphJob to 
 control the time between calls to context().progress() that allows workers to 
 avoid timeouts during long data load-ins in which some works complete their 
 input split reads much faster than others, or finish a super step faster. I 
 found this allowed jobs that were large-scale but with low memory overhead to 
 complete even when they would previously time out during runs on a Hadoop 
 cluster. Timeout is still possible when the worker crashes or runs out of 
 memory or has other GC or RPC trouble that is legitimate, but prevents 
 unintentional crashes when the worker is actually still healthy.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-246) Periodic worker calls to context.progress() will prevent timeout on some Hadoop clusters during barrier waits

2012-08-07 Thread Avery Ching (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430802#comment-13430802
 ] 

Avery Ching commented on GIRAPH-246:


I appreciate your testing Eli, glad to hear that it's working well.

There are only 2 things that I would like to see:

1) The code block I wrote about above

{code}
@@ -286,6 +288,7 @@ public class BspServiceWorkerI extends WritableComparable,
   // an InputSplit has finished.
   getInputSplitsStateChangedEvent().waitMsecs(60 * 1000);
   getInputSplitsStateChangedEvent().reset();
+  getContext().progress();
{code}

2) Can't we just keep PredicateLock using Context, without reverting it?  It's 
not hurting anything and it's a very tiny piece of code that should support 
waitForever() correctly.  The old way of doing waitForever() is wrong without 
the context.progress() getting called internally.

I'm totally +1'ed on everything else and I'd be happy to help review this once 
we resolve the above.

 Periodic worker calls to context.progress() will prevent timeout on some 
 Hadoop clusters during barrier waits
 -

 Key: GIRAPH-246
 URL: https://issues.apache.org/jira/browse/GIRAPH-246
 Project: Giraph
  Issue Type: Improvement
  Components: bsp
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
Priority: Minor
  Labels: hadoop, patch
 Fix For: 0.2.0

 Attachments: GIRAPH-246-1.patch, GIRAPH-246-2.patch, 
 GIRAPH-246-3.patch, GIRAPH-246-4.patch, GIRAPH-246-5.patch, 
 GIRAPH-246-6.patch, GIRAPH-246-7.patch


 This simple change creates a command-line configurable option in GiraphJob to 
 control the time between calls to context().progress() that allows workers to 
 avoid timeouts during long data load-ins in which some works complete their 
 input split reads much faster than others, or finish a super step faster. I 
 found this allowed jobs that were large-scale but with low memory overhead to 
 complete even when they would previously time out during runs on a Hadoop 
 cluster. Timeout is still possible when the worker crashes or runs out of 
 memory or has other GC or RPC trouble that is legitimate, but prevents 
 unintentional crashes when the worker is actually still healthy.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (GIRAPH-246) Periodic worker calls to context.progress() will prevent timeout on some Hadoop clusters during barrier waits

2012-08-07 Thread Eli Reisman (JIRA)

 [ 
https://issues.apache.org/jira/browse/GIRAPH-246?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Eli Reisman updated GIRAPH-246:
---

Attachment: GIRAPH-246-8.patch

Jakob had a very useful insight about Progressable earlier, while looking at 
this I think I might have stumbled on the solution. I think everyone will be 
happy with this version. Passes mvn verify, test etc. Running it now.


 Periodic worker calls to context.progress() will prevent timeout on some 
 Hadoop clusters during barrier waits
 -

 Key: GIRAPH-246
 URL: https://issues.apache.org/jira/browse/GIRAPH-246
 Project: Giraph
  Issue Type: Improvement
  Components: bsp
Affects Versions: 0.2.0
Reporter: Eli Reisman
Assignee: Eli Reisman
Priority: Minor
  Labels: hadoop, patch
 Fix For: 0.2.0

 Attachments: GIRAPH-246-1.patch, GIRAPH-246-2.patch, 
 GIRAPH-246-3.patch, GIRAPH-246-4.patch, GIRAPH-246-5.patch, 
 GIRAPH-246-6.patch, GIRAPH-246-7.patch, GIRAPH-246-8.patch


 This simple change creates a command-line configurable option in GiraphJob to 
 control the time between calls to context().progress() that allows workers to 
 avoid timeouts during long data load-ins in which some works complete their 
 input split reads much faster than others, or finish a super step faster. I 
 found this allowed jobs that were large-scale but with low memory overhead to 
 complete even when they would previously time out during runs on a Hadoop 
 cluster. Timeout is still possible when the worker crashes or runs out of 
 memory or has other GC or RPC trouble that is legitimate, but prevents 
 unintentional crashes when the worker is actually still healthy.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (GIRAPH-274) Jobs still failing due to tasks timeout during INPUT_SUPERSTEP

2012-08-07 Thread Jaeho Shin (JIRA)

[ 
https://issues.apache.org/jira/browse/GIRAPH-274?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13430879#comment-13430879
 ] 

Jaeho Shin commented on GIRAPH-274:
---

As I just commented on GIRAPH-246, I would be happy to continue pursuing a 
better solution to Hadoop timeout here, but really wish we kept the 
PredicateLock fix from GIRAPH-267.  The other part of it using waitForever() 
wasn't a complete fix for the timeout problem, but the new PredicateLock does 
prevent us from falling into cases which certainly causes timeouts.  Besides, I 
believe it doesn't conflict with Eli's fix of using waitMsecs().

 Jobs still failing due to tasks timeout during INPUT_SUPERSTEP
 --

 Key: GIRAPH-274
 URL: https://issues.apache.org/jira/browse/GIRAPH-274
 Project: Giraph
  Issue Type: Bug
Affects Versions: 0.2.0
Reporter: Jaeho Shin
Assignee: Jaeho Shin
 Fix For: 0.2.0

 Attachments: GIRAPH-274-alt-1.patch, GIRAPH-274.patch


 Even after GIRAPH-267, jobs were failing during INPUT_SUPERSTEP when some 
 workers don't get to reserve an input split, while others were loading 
 vertices for a long time.  (related to GIRAPH-246 and GIRAPH-267)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira