[ https://issues.apache.org/jira/browse/BOOKKEEPER-336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13419056#comment-13419056 ]
Rakesh R commented on BOOKKEEPER-336: ------------------------------------- @Flavio bq. Could you post it on the review board, please, Rakesh? There are few more refactoring to be done as part of the comments in the JIRA, Is that ok to post for review after the changes? Also it would be great to see your thoughts on the following refactoring. @Sijie bq.as Flavio's suggestion, seems that it could be done in a separated distribution schedule. Yeah, I agree to make more generic to the LedgerHandle level rather than for each LedgerEntry. I feel we should have few more details of the health/proirity of bookies in the DistributionSchedule interface. ---- Following are just my thoughts to refactor the scheduling algo, please have a look and would like to know your opinon? *Proposal:* I'm just thinking to call this as different scheduling strategies: {code} org.apache.bookkeeper.client.DistributionSchedule.java ScheduleStrategy getScheduleStrategy(); public enum ScheduleStrategy { ROUNDROBIN, ORDEREDROUNDROBIN; } {code} Also, introducing new interface(thought of not adding ordering to the existing DistributionSchedule algo). New interface is an extended version of DistributionSchedule, which has the api getBookieIndexes(SchedulerContext) as follows: {code} org.apache.bookkeeper.client.OrderedDistributionSchedule.java public interface OrderedDistributionSchedule extends DistributionSchedule { SchedulerContext getSchedulerContext(); /** * Get the bookieIndexes of the given context. This bookie index will be ordered * according to the priority of bookies and high priority bookies will be first * in the list. */ public Set<Integer> getBookieIndexes(SchedulerContext schedulerContext); public interface SchedulerContext { void setEntryId(); void setQuorumSize(); void addCurrentEnsemble(ArrayList<InetSocketAddress> ensemble); void addLowPriorityBookie(InetSocketAddress bookieAddr); void removeLowPriorityBookie(InetSocketAddress bookieAddr); } } {code} ---- How the strategies will be used? {code} Will expose the strategy as a configuration item in bk_server.conf like: schedulingStrategy=RoundRobin (by default) schedulingStrategy=OrderedRoundRobin {code} When initializing the LedgerHandle, based on the strategy will create schedulers. Other modules will use like: {code} if(lh.distributionSchedulegetScheduleStrategy() == ScheduleStrategy.ORDEREDROUNDROBIN) // casting to specifc strategy algo and do necessary operations (OrderedDistributionSchedule)lh.distributionSchedule {code} > bookie readEntries is taking more time if the ensemble has failed bookie(s) > --------------------------------------------------------------------------- > > Key: BOOKKEEPER-336 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-336 > Project: Bookkeeper > Issue Type: Bug > Affects Versions: 4.1.0 > Reporter: Brahma Reddy Battula > Assignee: Rakesh R > Attachments: BOOKKEEPER-336.patch > > > Scenario: > 1) Start three bookies. Create ledger with ensemblesize=3, quorumsize=2 > 2) Add 100 entries to this ledger > 3) Make first bookie down and read the entries from 0-99 > Output: Each entry is going to fetch from the failed bookie and is waiting > for the bookie connection timeout, only after failure going to next bookie. > This is affecting the read entry performance. > Impact: Namenode switching time will be affected by adding this failed bookie > readTimeOut also. -- 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