----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28835/#review64360 -----------------------------------------------------------
bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107012> trailing whitespace bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107011> trailing whitespace bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107016> Passing in stop as a variable makes the flow of operation unclear. In the while loop that its used in, its unclear as to what 'stop' relates to. Who sets it? It would be better to have a hasErrorOccurred() method in UpdateLedgerMetadataCallback and call that instead of !stop.get(). bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107013> Latch is not needed. The executor will return a future. just call .get() on it to get the same behaviour as the latch. bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107017> 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. bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107010> You can use guava thread factory builder for this. bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107019> There's a race here. What if nothing is in ledgersWaitForCompletion by the time we get to this line? But then something gets added after? Also, these 4 lines should be wrapped into a method of UpdateLedgerMetadataCb. bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107014> It's a little ugly to mix business logic and presentation. I think it would be better to pass in a notifier/callback type like. interface BlahNotifier { void progress(long updated, long issued); } And then have BookieShell pass in an implementaiton of this. bookkeeper-server/src/main/java/org/apache/bookkeeper/admin/UpdateLedgerOp.java <https://reviews.apache.org/r/28835/#comment107021> 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. bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java <https://reviews.apache.org/r/28835/#comment107024> long option should be updatesPerSec. Bandwidth is confusing in this context. bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java <https://reviews.apache.org/r/28835/#comment107025> 'no:' should be 'no.' bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java <https://reviews.apache.org/r/28835/#comment107022> updation -> update In fact I would reword. Maximum number of ledgers to update (default: no limit). bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java <https://reviews.apache.org/r/28835/#comment107023> what does 0 mean with regards to printing? should be (default false). bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java <https://reviews.apache.org/r/28835/#comment107026> Why make this public rather than moving UpdateLedgerOp into client package? - Ivan Kelly 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 > >
