[
https://issues.apache.org/jira/browse/BOOKKEEPER-248?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13438597#comment-13438597
]
Ivan Kelly commented on BOOKKEEPER-248:
---------------------------------------
@Uma I had a look. Shape is good. Comments are as follows.
The new BookKeeperAdmin constructor should only take a BookKeeper. The
zookeeper client and bookies available path can be extracted from this.
In ReplicationWorker#run, don't catch a Throwable. This will catch exceptions
which you really don't want to catch, specifically, RuntimeExceptions like
NullPointerException or ArithmeticException which are a result of programming
error.
In ReplicationWorker#run, when you do catch an exception, you should return
from the run method. You don't want the while loop to run again. (you should
also call #stop() to cleanup).
ReplicationWorker#stop() should wait for #run() to have finished before
cleaning up the bk client. Otherwise you're asking for null pointer exceptions
to happen. Have a look guava Service [1]. It may be worth using.
A quick note on #doReplicateFragments would be good to explain what the boolean
return value is.
The flow is a bit strange between #run() and #doReplicateFragments(). You're
interacting with underreplicationManager in both, depending on booleans etc. I
think it would be cleaer to put all the interaction with
underreplicationManager in run(). The boolean return from doReplicateFragments
seems to be designed especially for this.
Im not sure if I like the fact that you only replicate to a single bookie from
an individual replication worker. I can see why you did it (so you'll only
replicate to local), but it seems to me as if it could cause problems later,
though I can't pin any specific reason right now.
Why not implement a sync openLedgerNoRecovery on BookKeeperAdmin rather than in
#getLedgerHandle?
Also, this needs more tests. Such as multiple running workers and multiple
bookie failures and ledger failures etc.
[1]
http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/Service.html
> Rereplicating of under replicated data
> --------------------------------------
>
> Key: BOOKKEEPER-248
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-248
> Project: Bookkeeper
> Issue Type: Sub-task
> Components: bookkeeper-client, bookkeeper-server
> Reporter: Ivan Kelly
> Assignee: Uma Maheswara Rao G
> Fix For: 4.2.0
>
> Attachments: BOOKKEEPER-248.patch
>
>
> This subtask discusses how we will rereplicate underreplicated entries.
--
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