> On Dec. 9, 2014, 12:08 p.m., Ivan Kelly wrote:
> > bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java,
> >  line 1199
> > <https://reviews.apache.org/r/28835/diff/1/?file=786288#file786288line1199>
> >
> >     what does 0 mean with regards to printing? should be (default false).
> 
> Rakesh R wrote:
>     Here I am taking option value as 'printMsgCount', using this I'm 
> calculate the printing cycle. For example the value 100 means. After every 
> 100, 200, 300... ledgers given for updation will print the message.

This isn't clear to the user. Perhaps it would also be better to print every 5 
seconds or so, rather than every X updates. Really what we want to show to the 
user is that the process is still working.


> On Dec. 9, 2014, 12:08 p.m., Ivan Kelly wrote:
> > bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java,
> >  line 140
> > <https://reviews.apache.org/r/28835/diff/1/?file=786287#file786287line140>
> >
> >     There's a lot of modification from afar going here around 
> > ledgersWaitForCompletion. Why do you need to countDown() the latch and to 
> > remove() it from the list? Really you only want to track how many requests 
> > have been started, and completed.
> >     
> >     It would be cleaner to allocate the tracker object in main #run thread, 
> > and pass it in at that point. That way, when we wait on it, its clear that 
> > we're waiting on something we have passed into the request. In fact, why 
> > not make ReadLedgerMetadataCb extends AbstractFuture<LedgerMetadata>? Then 
> > you can build a list of ReadLedgerMetadataCb objects in the run loop, and 
> > wait on all futures at the end. This will also handle the erroring, as you 
> > can make the future throw an error to tell you to stop.
> 
> Rakesh R wrote:
>     The idea for the 'ledgersWaitForCompletion' is after iterating over all 
> the ledgers, will wait for the items which are yet to be completed. For 
> example, assume there are 100,000 ledgers. It will be iterate and call the 
> ledgers based on the ratelimitting factor. During the iteration time most of 
> the ledgers might be completed their updations. Now, at the end of the 
> iteration it will be have only few ledgers which are waiting for the updates. 
> So this thread needs to wait for only those ledgers.
>     
>     
>     Say, if there are many ledgers in the system, adding ReadLedgerMetadataCb 
> objects into the list will create problem, sin't it? I'm afraid of keeping 
> all the ledgers callback objects in memory. Thats the reason I thought of 
> removing immedidately on success updation so that the list will shrink.

I meant something like 
https://github.com/ivankelly/bookkeeper/commit/39cbacc582e661561fd19c978e38ac9bf4ac7b84

I haven't tried to compile that, it's just to give an idea, but it would allow 
UpdateLedgerMetadataCb to be removed completely, and all the flow logic fits 
into one screen.


> On Dec. 9, 2014, 12:08 p.m., Ivan Kelly wrote:
> > bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java,
> >  line 83
> > <https://reviews.apache.org/r/28835/diff/1/?file=786287#file786287line83>
> >
> >     updateCb would be better than metaCb. there's a lot of metadata ops 
> > happening.
> >     
> >     Also, create it inside #run(). It doesn't need to be this far up in the 
> > scope.
> 
> Rakesh R wrote:
>     ok, i will rename it. I think I need to keep it outside #run() as it will 
> wait for the completion like:
>     
>     long ledgerUpdatedCount = updateCb.get();
>     
>     isn't it?

No, it the Future returned from submit that you have to all #get() on.


- Ivan


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


On Dec. 9, 2014, 4:05 a.m., Rakesh R wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/28835/
> -----------------------------------------------------------
> 
> (Updated Dec. 9, 2014, 4:05 a.m.)
> 
> 
> Review request for bookkeeper, fpj, Ivan Kelly, and Sijie Guo.
> 
> 
> Bugs: BOKKEEPER-634
>     https://issues.apache.org/jira/browse/BOKKEEPER-634
> 
> 
> Repository: bookkeeper-git
> 
> 
> Description
> -------
> 
> admin tool for changing the bookie's identifier to IP/hostname
> 
> 
> Diffs
> -----
> 
>   
> bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java
>  PRE-CREATION 
>   
> bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java 
> a1e4639 
>   
> bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java 
> 49d8e59 
>   
> bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerOpTest.java
>  PRE-CREATION 
> 
> Diff: https://reviews.apache.org/r/28835/diff/
> 
> 
> Testing
> -------
> 
> Added few tests
> 
> 
> Thanks,
> 
> Rakesh R
> 
>

Reply via email to