> On Dec. 18, 2014, 5:54 a.m., Sijie Guo wrote: > > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java, > > line 183 > > <https://reviews.apache.org/r/28835/diff/2/?file=792808#file792808line183> > > > > this is totally wrong. > > > > #addListener is "to Registers a listener to be run on the given > > executor. The listener will run when the Future's computation is complete > > or, if the computation is already complete, immediately." > > > > in your implementation, you submit the runnable immediately, without > > checking if the future's compuation is complete or not. > > > > > > you should use SettableFuture<Void> > > > > private final static class ReadLedgerMetadataCb .. { > > > > final SettableFuture<Void> resultFuture = ... > > > > public addListener(...) { > > resultFuture.addListener(...); > > } > > > > public void operationComplete(...) { > > resultFuture.set(...) > > } > > > > } > > Rakesh R wrote: > Its new learning for me. I tried using the SettableFuture in latest > patch. Please let me know the usage is proper. thanks! > > I met the following exception as the listener is a kind of async > notification. What I'm thinking is, readCb.get() will comeout immedaitely > after future.set() is called but before invoking the registered listener. Now > during the execution there could be chances of finshing the main thread and > pExecutor shutdown. I feel the below exception is expected and is OK. > > 2014-12-21 22:54:07,268 - INFO - > [UpdateLedgerThread:UpdateLedgerOp$1@142] - Total number of ledgers updated = > 1 > Dec 21, 2014 10:54:07 PM > com.google.common.util.concurrent.ExecutionList$RunnableExecutorPair execute > SEVERE: RuntimeException while executing runnable > org.apache.bookkeeper.client.UpdateLedgerOp$1$1@7da1f182 with executor > java.util.concurrent.Executors$FinalizableDelegatedExecutorService@34741c9d > java.util.concurrent.RejectedExecutionException: Task > org.apache.bookkeeper.client.UpdateLedgerOp$1$1@7da1f182 rejected from > java.util.concurrent.ThreadPoolExecutor@f64cb82[Terminated, pool size = 0, > active threads = 0, queued tasks = 0, completed tasks = 0] > at > java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(Unknown > Source) > at java.util.concurrent.ThreadPoolExecutor.reject(Unknown Source) > at java.util.concurrent.ThreadPoolExecutor.execute(Unknown Source) > at > java.util.concurrent.Executors$DelegatedExecutorService.execute(Unknown > Source) > at > com.google.common.util.concurrent.ExecutionList$RunnableExecutorPair.execute(ExecutionList.java:149) > at > com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:134) > at > com.google.common.util.concurrent.AbstractFuture.set(AbstractFuture.java:175) > at > com.google.common.util.concurrent.SettableFuture.set(SettableFuture.java:53) > at > org.apache.bookkeeper.client.UpdateLedgerOp$ReadLedgerMetadataCb$1.operationComplete(UpdateLedgerOp.java:241) > at > org.apache.bookkeeper.client.UpdateLedgerOp$ReadLedgerMetadataCb$1.operationComplete(UpdateLedgerOp.java:1) > at > org.apache.bookkeeper.meta.CleanupLedgerManager$CleanupGenericCallback.operationComplete(CleanupLedgerManager.java:51) > at > org.apache.bookkeeper.meta.AbstractZkLedgerManager$3.processResult(AbstractZkLedgerManager.java:360) > at > org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:545) > at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:497) > > Sijie Guo wrote: > you shutdown the executor before all futures are satisfied.
Latest patch doesn't requires this executor and I've removed this executor usage. - Rakesh ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28835/#review65458 ----------------------------------------------------------- On Jan. 11, 2015, 3:41 p.m., Rakesh R wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/28835/ > ----------------------------------------------------------- > > (Updated Jan. 11, 2015, 3:41 p.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/bookie/BookieShell.java > a1e4639 > > bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java > PRE-CREATION > > bookkeeper-server/src/test/java/org/apache/bookkeeper/client/UpdateLedgerCmdTest.java > PRE-CREATION > > 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 > >
