[ 
https://issues.apache.org/jira/browse/BOOKKEEPER-610?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13656158#comment-13656158
 ] 

Sijie Guo edited comment on BOOKKEEPER-610 at 5/13/13 5:50 PM:
---------------------------------------------------------------

I don't see any strong benefits using an executor. One downside that I could 
see is that using an executor, you could not actually stop scheduling the 
checkpointing tasks when a bookie encountered critical exceptions, you silenced 
the exceptions.

I prefer the original implementation, since it makes the flow clearer than 
using an executor.

{code}
+        executor.scheduleAtFixedRate(new Runnable() {
+                public void run() {
+                    try {
+                        synchronized (suspensionLock) {
+                            while (suspended) {
+                                try {
+                                    suspensionLock.wait();
+                                } catch (InterruptedException e) {
+                                    Thread.currentThread().interrupt();
+                                    continue;
+                                }
+                            }
+                        }
+                        checkpoint(checkpointSource.newCheckpoint());
+                    } catch (Throwable t) {
+                        LOG.error("Exception in SyncThread", t);
+                        dirsListener.fatalError();
+                    }
+                }
+            }, flushInterval, flushInterval, TimeUnit.MILLISECONDS);
{code}

Another problem in your exception handling is a bit difference from the 
original implementation and these exceptions handling are also hidden from the 
ledger dirs listener. it was a bit difficult to find an issue, especially for 
the cases, e.g. shutting down.

{code}
+        } catch (IOException e) {
+            LOG.error("Exception flushing ledgers", e);
{code}

why you just log exceptions when caught IOException.


                
      was (Author: hustlmsp):
    I don't see any strong benefits using an executor. One downside that I 
could see is that using an executor, you could not actually stop scheduling the 
checkpointing tasks when a bookie encountered critical exceptions, you silenced 
the exceptions.

I prefer the original implementation, since it makes the flow clearer than 
using an executor.

{code}
+        executor.scheduleAtFixedRate(new Runnable() {
+                public void run() {
+                    try {
+                        synchronized (suspensionLock) {
+                            while (suspended) {
+                                try {
+                                    suspensionLock.wait();
+                                } catch (InterruptedException e) {
+                                    Thread.currentThread().interrupt();
+                                    continue;
+                                }
+                            }
+                        }
+                        checkpoint(checkpointSource.newCheckpoint());
+                    } catch (Throwable t) {
+                        LOG.error("Exception in SyncThread", t);
+                        dirsListener.fatalError();
+                    }
+                }
+            }, flushInterval, flushInterval, TimeUnit.MILLISECONDS);
{code}

Another problem in your exception handling is a bit from the original 
implementation.

{code}
+        } catch (IOException e) {
+            LOG.error("Exception flushing ledgers", e);
{code}

why you just log exceptions when caught IOException.


                  
> Make SyncThread use an executor
> -------------------------------
>
>                 Key: BOOKKEEPER-610
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-610
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Ivan Kelly
>            Assignee: Ivan Kelly
>             Fix For: 4.3.0
>
>         Attachments: 
> 0001-BOOKKEEPER-610-Make-SyncThread-use-an-executor.patch, 
> 0001-BOOKKEEPER-610-Make-SyncThread-use-an-executor.patch
>
>
> Currently we have a bunch of boolean variables to control the lifecycle of 
> the SyncThread. We're effectively replicating what an Executor does, so we 
> should just use an executor.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to