[ 
https://issues.apache.org/jira/browse/BOOKKEEPER-1102?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sijie Guo resolved BOOKKEEPER-1102.
-----------------------------------
       Resolution: Fixed
    Fix Version/s: 4.5.0

Issue resolved by merging pull request 275
            [https://github.com/apache/bookkeeper/pull/275]

            {noformat}
            commit a5f8580f53464065243a9af038935f5893434166
Author:     Samuel Just <sj...@salesforce.com>
AuthorDate: Mon Jul 31 23:34:07 2017 -0700
Commit:     Sijie Guo <si...@apache.org>
CommitDate: Mon Jul 31 23:34:07 2017 -0700

    BOOKKEEPER-1102: Clarify BookieInfoReader and fix associated test flappers
    
    BookieInfoReader:
    
    The previous syncronization logic wasn't really correct, and the logic
    at the top of the method was far more complicated than it needed to be.
    Restrict bookies to be non-null.  Restructure the code to simply use
    the BookieInfoReader instance as a single lock.
    
    One significant behavioral change is that we scan every bookie not in
    the map, and we clear from the map bookies which returned an error.
    
    Also, explicitely cache the most recent bookie set reported by the
    BookieWatcher.  This eliminates the need to call into BookieWatcher
    from getReadWriteBookieInfo and the corresponding error path.  The
    periodic scan continues to explicitely check.
    
    Another departure is the addition of an explicit retry-on-error param to
    trigger retry if any of the requests failed
    (getBookieInfoRetryIntervalSeconds).  We'll only retry the ones that
    actually failed (along with any new additions since the last run).  This
    is useful because bookie startup triggers the addition of the bookie
    node to zk before the bookie actually becomes available for the bookie
    info request, so there can be rare races in the unit tests between
    BookieInfoReader requesting the info and the bookie actually being up.
    
    Also, add a method to allow tests to wait for updates to be reflected.
    
    PerChannelBookieClient: fix error handling for BookieInfo keys
    
    Passing a key corresponding to a GET_BOOKIE_INFO operation to
    errorOutReadKey results in a casting exception, clean up the invalid
    calls.
    
    BookKeeperClusterTestCase: add killBookieAndWaitForZK
    
    Should reduce the need for tests to wait for an arbitrary period to let
    the cluster "settle".
    
    BookKeeperDiskSpaceWeightedLedgerPlacementTest:
    
    This test was heavily time dependent, and the Thread.sleep values did
    not work universally.  Instead, eliminate the arbitrary Thread.sleep
    values and instead verify the free space changes on each change.
    
    Also, switch the delay on
    testDiskSpaceWeightedBookieSelectionWithPeriodicBookieInfoUpdate
    to simply use an atomic boolean to signal the value switch.
    
    Signed-off-by: Samuel Just <sjustsalesforce.com>
    
    Author: Samuel Just <sj...@salesforce.com>
    
    Reviewers: Enrico Olivelli <eolive...@gmail.com>
    
    This patch had conflicts when merged, resolved by
    Committer: Sijie Guo <si...@apache.org>
    
    This closes #275 from athanatos/forupstream/BOOKKEEPER-1102

            {noformat}
            

> org.apache.bookkeeper.client.BookKeeperDiskSpaceWeightedLedgerPlacementTest.testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded
>  is unreliable
> ---------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-1102
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-1102
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.5.0
>            Reporter: Samuel Just
>            Assignee: Samuel Just
>            Priority: Minor
>             Fix For: 4.5.0
>
>
> org.apache.bookkeeper.client.BookKeeperDiskSpaceWeightedLedgerPlacementTest.testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded
>  can intermittently fail depending on the timing of the client receiving the 
> info back from the bookies.
> Additionally, the synchronization in BookieInfoReader is more complicated 
> than necessary and not entirely correct.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to