[ https://issues.apache.org/jira/browse/BOOKKEEPER-336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13429286#comment-13429286 ]
Ivan Kelly commented on BOOKKEEPER-336: --------------------------------------- I'd prefer to only have one implementation of the reading schedule, which covers all usecases, and that it is completely encapsulated in PendingReadOp. Flavio and I discussed this last week and came up with something like the following (it has changed a little since I've thought about it more). For each bookie, record the response time of the last read request. When making a read request, if the last response time for the replica is greater than a configured threshold, make the read request for the next replica also. To do this, we maintain a array of int in PendingReadOp, int[] responseTimes = new int[ensembleSize]; The response time can be UNKNOWN, PENDING, or a positive integer. Initially all will be initialized to UNKNOWN. When a read request is made to a bookie, we check the response time. If it is UNKNOWN, we make the request to the bookie, and to the next bookie also. If is PENDING, we do not send to the bookie, but we do send to the next bookie. If it is a positive int, we check it against the threshold and send to the next bookie also if it exceeds the threshold. This does mean quite a large change to PendingReadOp, as at the moment it tries to read all entries in parallel. > 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.1.patch, 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