This is an automated email from the ASF dual-hosted git repository. sijie pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push: new e2b93f2 [MERGE YAHOO REPO] Only reorder read sequence if any bookie is not available e2b93f2 is described below commit e2b93f28813556c1971a04c1ffe4b65cbe40c427 Author: Jia Zhai <zhai...@apache.org> AuthorDate: Sat Feb 10 11:03:28 2018 +0800 [MERGE YAHOO REPO] Only reorder read sequence if any bookie is not available Descriptions of the changes in this PR: This is cherry-pick from yahoo repo of branch yahoo-4.3. original change is: https://github.com/yahoo/bookkeeper/commit/1ddd14a1 Only reorder read sequence if any bookie is not available Author: Jia Zhai <zhai...@apache.org> Author: Matteo Merli <mme...@apache.org> Reviewers: Enrico Olivelli <eolive...@gmail.com>, Sijie Guo <si...@apache.org> This closes #1120 from jiazhai/cherry_picks/i_161 --- .../RackawareEnsemblePlacementPolicyImpl.java | 22 ++++++++++++++++++++++ .../TestRackawareEnsemblePlacementPolicy.java | 10 +++++++--- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java index 7dcd728..9a58764 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java @@ -927,6 +927,28 @@ class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsemblePlacemen boolean useRegionAware = regionAware && (!myRegion.equals(UNKNOWN_REGION)); int ensembleSize = ensemble.size(); + // For rack aware, If all the bookies in the write set are available, simply return the original write set, + // to avoid creating more lists + boolean isAnyBookieUnavailable = false; + + if (useRegionAware || reorderReadsRandom) { + isAnyBookieUnavailable = true; + } else { + for (int i = 0; i < ensemble.size(); i++) { + BookieSocketAddress bookieAddr = ensemble.get(i); + if ((!knownBookies.containsKey(bookieAddr) && !readOnlyBookies.contains(bookieAddr)) + || slowBookies.getIfPresent(bookieAddr) != null) { + // Found at least one bookie not available in the ensemble, or in slowBookies + isAnyBookieUnavailable = true; + break; + } + } + } + + if (!isAnyBookieUnavailable) { + return writeSet; + } + for (int i = 0; i < writeSet.size(); i++) { int idx = writeSet.get(i); BookieSocketAddress address = ensemble.get(idx); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java index 01a6617..bb7c692 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java @@ -182,10 +182,8 @@ public class TestRackawareEnsemblePlacementPolicy extends TestCase { DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( ensemble, getBookiesHealthInfo(), writeSet); - DistributionSchedule.WriteSet expectedSet = writeSetFromValues(1, 2, 3, 0); LOG.info("reorder set : {}", reorderSet); - assertFalse(reorderSet.equals(origWriteSet)); - assertEquals(expectedSet, reorderSet); + assertEquals(reorderSet, origWriteSet); } @Test @@ -914,6 +912,12 @@ public class TestRackawareEnsemblePlacementPolicy extends TestCase { bookieFailures.put(addr1, 20L); bookieFailures.put(addr2, 22L); + // remove failure bookies: addr1 and addr2 + addrs = new HashSet<BookieSocketAddress>(); + addrs.add(addr3); + addrs.add(addr4); + repp.onClusterChanged(addrs, new HashSet<BookieSocketAddress>()); + DistributionSchedule.WriteSet reoderSet = repp.reorderReadSequence( ensemble, getBookiesHealthInfo(bookieFailures, new HashMap<>()), writeSet); LOG.info("reorder set : {}", reoderSet); -- To stop receiving notification emails like this one, please contact si...@apache.org.