This is an automated email from the ASF dual-hosted git repository.
reddycharan 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 d35aa22 Move common placementpolicy components to
TopologyAwareEnsemblePlacementPolicy.
d35aa22 is described below
commit d35aa22ade87969a4f8e932d925c5d134feb764b
Author: Charan Reddy Guttapalem <[email protected]>
AuthorDate: Mon May 13 16:06:41 2019 -0700
Move common placementpolicy components to
TopologyAwareEnsemblePlacementPolicy.
Descriptions of the changes in this PR:
- Moving components/methods which are common in nature with respect to
placementpolicy from RackawareEnsemblePlacementPolicyImpl to
TopologyAwareEnsemblePlacementPolicy, so that any new placementpolicy
implementation can extend TopologyAwareEnsemblePlacementPolicy and reuse
those common components/generic.
- Change signature of methods of RackChangeNotifier interface.
- Delete duplicate methods in RackawareEnsemblePlacementPolicyImpl and
TopologyAwareEnsemblePlacementPolicy (missed removing them in #2089)
- Created separate class for BookieNode
- This change has no functionality change, it is just reorganizing code.
Reviewers: Sijie Guo <[email protected]>, Venkateswararao Jujjuri (JV) <None>
This closes #2092 from reddycharan/placementpolicyrefactoring
---
.../ITopologyAwareEnsemblePlacementPolicy.java | 9 +-
.../bookkeeper/client/RackChangeNotifier.java | 8 +-
.../client/RackawareEnsemblePlacementPolicy.java | 3 +-
.../RackawareEnsemblePlacementPolicyImpl.java | 135 +--------------------
.../client/RegionAwareEnsemblePlacementPolicy.java | 1 +
.../TopologyAwareEnsemblePlacementPolicy.java | 69 ++++++-----
.../java/org/apache/bookkeeper/net/BookieNode.java | 53 ++++++++
.../TestRackawareEnsemblePlacementPolicy.java | 2 +-
.../apache/bookkeeper/util/StaticDNSResolver.java | 8 +-
9 files changed, 110 insertions(+), 178 deletions(-)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java
index b196236..d6197c0 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ITopologyAwareEnsemblePlacementPolicy.java
@@ -23,9 +23,9 @@ import java.util.Set;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import
org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble;
import
org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate;
-import
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode;
import org.apache.bookkeeper.common.annotation.InterfaceAudience;
import org.apache.bookkeeper.common.annotation.InterfaceStability;
+import org.apache.bookkeeper.net.BookieNode;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.net.Node;
@@ -185,4 +185,11 @@ public interface ITopologyAwareEnsemblePlacementPolicy<T
extends Node> extends E
* bookies that joined.
*/
void handleBookiesThatJoined(Set<BookieSocketAddress> joinedBookies);
+
+ /**
+ * Handle rack change for the bookies.
+ *
+ * @param bookieAddressList
+ */
+ void onBookieRackChange(List<BookieSocketAddress> bookieAddressList);
}
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java
index c66bcaf..e9fc3a6 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackChangeNotifier.java
@@ -18,8 +18,11 @@
package org.apache.bookkeeper.client;
+import org.apache.bookkeeper.net.BookieNode;
+
/**
- * Notifier used by the RackawareEnsemblePlacementPolicy to get notified if a
rack changes for a bookie.
+ * Notifier used by the RackawareEnsemblePlacementPolicy to get notified if a
+ * rack changes for a bookie.
*/
public interface RackChangeNotifier {
@@ -28,5 +31,6 @@ public interface RackChangeNotifier {
*
* @param rackawarePolicy
*/
- void registerRackChangeListener(RackawareEnsemblePlacementPolicyImpl
rackawarePolicy);
+ void registerRackChangeListener(
+ ITopologyAwareEnsemblePlacementPolicy<BookieNode> rackawarePolicy);
}
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java
index 49e9df5..380f556 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java
@@ -23,6 +23,7 @@ import java.util.Map;
import java.util.Set;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
+import org.apache.bookkeeper.net.BookieNode;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.apache.bookkeeper.net.Node;
@@ -34,7 +35,7 @@ import org.apache.bookkeeper.stats.StatsLogger;
* @see EnsemblePlacementPolicy
*/
public class RackawareEnsemblePlacementPolicy extends
RackawareEnsemblePlacementPolicyImpl
- implements
ITopologyAwareEnsemblePlacementPolicy<TopologyAwareEnsemblePlacementPolicy.BookieNode>
{
+ implements ITopologyAwareEnsemblePlacementPolicy<BookieNode> {
RackawareEnsemblePlacementPolicyImpl slave = null;
public RackawareEnsemblePlacementPolicy() {
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 be06e13..2b1090f 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
@@ -29,15 +29,12 @@ import static
org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.UN
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
import io.netty.util.HashedWheelTimer;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -55,9 +52,9 @@ import org.apache.bookkeeper.common.util.ReflectionUtils;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.conf.Configurable;
import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.net.BookieNode;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.bookkeeper.net.NetUtils;
import org.apache.bookkeeper.net.NetworkTopology;
import org.apache.bookkeeper.net.NetworkTopologyImpl;
import org.apache.bookkeeper.net.Node;
@@ -145,10 +142,6 @@ public class RackawareEnsemblePlacementPolicyImpl extends
TopologyAwareEnsembleP
topology = new NetworkTopologyImpl();
}
- protected BookieNode createBookieNode(BookieSocketAddress addr) {
- return new BookieNode(addr, resolveNetworkLocation(addr));
- }
-
/**
* Initialize the policy.
*
@@ -310,132 +303,6 @@ public class RackawareEnsemblePlacementPolicyImpl extends
TopologyAwareEnsembleP
// do nothing
}
- protected String resolveNetworkLocation(BookieSocketAddress addr) {
- return NetUtils.resolveNetworkLocation(dnsResolver,
addr.getSocketAddress());
- }
-
- public void onBookieRackChange(List<BookieSocketAddress>
bookieAddressList) {
- rwLock.writeLock().lock();
- try {
- for (BookieSocketAddress bookieAddress : bookieAddressList) {
- BookieNode node = knownBookies.get(bookieAddress);
- if (node != null) {
- // refresh the rack info if its a known bookie
- topology.remove(node);
- BookieNode newNode = createBookieNode(bookieAddress);
- topology.add(newNode);
- knownBookies.put(bookieAddress, newNode);
- }
- }
- } finally {
- rwLock.writeLock().unlock();
- }
- }
-
- @Override
- public Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress>
writableBookies,
- Set<BookieSocketAddress> readOnlyBookies) {
- rwLock.writeLock().lock();
- try {
- ImmutableSet<BookieSocketAddress> joinedBookies, leftBookies,
deadBookies;
- Set<BookieSocketAddress> oldBookieSet = knownBookies.keySet();
- // left bookies : bookies in known bookies, but not in new
writable bookie cluster.
- leftBookies = Sets.difference(oldBookieSet,
writableBookies).immutableCopy();
- // joined bookies : bookies in new writable bookie cluster, but
not in known bookies
- joinedBookies = Sets.difference(writableBookies,
oldBookieSet).immutableCopy();
- // dead bookies.
- deadBookies = Sets.difference(leftBookies,
readOnlyBookies).immutableCopy();
- LOG.debug("Cluster changed : left bookies are {}, joined bookies
are {}, while dead bookies are {}.",
- leftBookies, joinedBookies, deadBookies);
- handleBookiesThatLeft(leftBookies);
- handleBookiesThatJoined(joinedBookies);
- if (this.isWeighted && (leftBookies.size() > 0 ||
joinedBookies.size() > 0)) {
- this.weightedSelection.updateMap(this.bookieInfoMap);
- }
- if (!readOnlyBookies.isEmpty()) {
- this.readOnlyBookies = ImmutableSet.copyOf(readOnlyBookies);
- }
-
- return deadBookies;
- } finally {
- rwLock.writeLock().unlock();
- }
- }
-
- /*
- * this method should be called in writelock scope of 'rwLock'
- */
- @Override
- public void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies) {
- for (BookieSocketAddress addr : leftBookies) {
- try {
- BookieNode node = knownBookies.remove(addr);
- if (null != node) {
- topology.remove(node);
- if (this.isWeighted) {
- this.bookieInfoMap.remove(node);
- }
-
- bookiesLeftCounter.registerSuccessfulValue(1L);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cluster changed : bookie {} left from
cluster.", addr);
- }
- }
- } catch (Throwable t) {
- LOG.error("Unexpected exception while handling leaving bookie
{}", addr, t);
- if (bookiesLeftCounter != null) {
- bookiesLeftCounter.registerFailedValue(1L);
- }
- // no need to re-throw; we want to process the rest of the
bookies
- // exception anyways will be caught/logged/suppressed in the
ZK's event handler
- }
- }
- }
-
- /*
- * this method should be called in writelock scope of 'rwLock'
- */
- @Override
- public void handleBookiesThatJoined(Set<BookieSocketAddress>
joinedBookies) {
- // node joined
- for (BookieSocketAddress addr : joinedBookies) {
- try {
- BookieNode node = createBookieNode(addr);
- topology.add(node);
- knownBookies.put(addr, node);
- if (this.isWeighted) {
- this.bookieInfoMap.putIfAbsent(node, new BookieInfo());
- }
-
- bookiesJoinedCounter.registerSuccessfulValue(1L);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cluster changed : bookie {} joined the
cluster.", addr);
- }
- } catch (Throwable t) {
- // topology.add() throws unchecked exception
- LOG.error("Unexpected exception while handling joining bookie
{}", addr, t);
-
- bookiesJoinedCounter.registerFailedValue(1L);
- // no need to re-throw; we want to process the rest of the
bookies
- // exception anyways will be caught/logged/suppressed in the
ZK's event handler
- }
- }
- }
-
- protected Set<Node> convertBookiesToNodes(Collection<BookieSocketAddress>
excludeBookies) {
- Set<Node> nodes = new HashSet<Node>();
- for (BookieSocketAddress addr : excludeBookies) {
- BookieNode bn = knownBookies.get(addr);
- if (null == bn) {
- bn = createBookieNode(addr);
- }
- nodes.add(bn);
- }
- return nodes;
- }
-
/*
* this method should be called in readlock scope of 'rwLock'
*/
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
index 399e8aa..5ed8dca 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
@@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentMap;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.feature.Feature;
import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.net.BookieNode;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.apache.bookkeeper.net.NetworkTopology;
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
index 355632b..27b077d 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -40,6 +41,7 @@ import java.util.function.Supplier;
import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject;
+import org.apache.bookkeeper.net.BookieNode;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.apache.bookkeeper.net.NetUtils;
@@ -53,7 +55,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
abstract class TopologyAwareEnsemblePlacementPolicy implements
-
ITopologyAwareEnsemblePlacementPolicy<TopologyAwareEnsemblePlacementPolicy.BookieNode>
{
+ ITopologyAwareEnsemblePlacementPolicy<BookieNode> {
static final Logger LOG =
LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class);
protected final Map<BookieSocketAddress, BookieNode> knownBookies = new
HashMap<BookieSocketAddress, BookieNode>();
@@ -115,38 +117,6 @@ abstract class TopologyAwareEnsemblePlacementPolicy
implements
}
- protected static class BookieNode extends NodeBase {
- private final BookieSocketAddress addr; // identifier of a bookie node.
-
- BookieNode(BookieSocketAddress addr, String networkLoc) {
- super(addr.toString(), networkLoc);
- this.addr = addr;
- }
-
- public BookieSocketAddress getAddr() {
- return addr;
- }
-
- @Override
- public int hashCode() {
- return name.hashCode();
- }
-
- @Override
- public boolean equals(Object obj) {
- if (!(obj instanceof BookieNode)) {
- return false;
- }
- BookieNode other = (BookieNode) obj;
- return getName().equals(other.getName());
- }
-
- @Override
- public String toString() {
- return String.format("<Bookie:%s>", name);
- }
- }
-
/**
* A predicate checking the rack coverage for write quorum in {@link
RoundRobinDistributionSchedule},
* which ensures that a write quorum should be covered by at least two
racks.
@@ -330,8 +300,6 @@ abstract class TopologyAwareEnsemblePlacementPolicy
implements
}
}
-
-
final int distanceFromLeaves;
final int ensembleSize;
final int writeQuorumSize;
@@ -759,6 +727,25 @@ abstract class TopologyAwareEnsemblePlacementPolicy
implements
}
}
+ @Override
+ public void onBookieRackChange(List<BookieSocketAddress>
bookieAddressList) {
+ rwLock.writeLock().lock();
+ try {
+ for (BookieSocketAddress bookieAddress : bookieAddressList) {
+ BookieNode node = knownBookies.get(bookieAddress);
+ if (node != null) {
+ // refresh the rack info if its a known bookie
+ topology.remove(node);
+ BookieNode newNode = createBookieNode(bookieAddress);
+ topology.add(newNode);
+ knownBookies.put(bookieAddress, newNode);
+ }
+ }
+ } finally {
+ rwLock.writeLock().unlock();
+ }
+ }
+
protected BookieNode createBookieNode(BookieSocketAddress addr) {
return new BookieNode(addr, resolveNetworkLocation(addr));
}
@@ -766,4 +753,16 @@ abstract class TopologyAwareEnsemblePlacementPolicy
implements
protected String resolveNetworkLocation(BookieSocketAddress addr) {
return NetUtils.resolveNetworkLocation(dnsResolver,
addr.getSocketAddress());
}
+
+ protected Set<Node> convertBookiesToNodes(Collection<BookieSocketAddress>
excludeBookies) {
+ Set<Node> nodes = new HashSet<Node>();
+ for (BookieSocketAddress addr : excludeBookies) {
+ BookieNode bn = knownBookies.get(addr);
+ if (null == bn) {
+ bn = createBookieNode(addr);
+ }
+ nodes.add(bn);
+ }
+ return nodes;
+ }
}
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java
new file mode 100644
index 0000000..e7d0ef5
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieNode.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.bookkeeper.net;
+
+/**
+ * Bookie Node implementation.
+ */
+public class BookieNode extends NodeBase {
+ private final BookieSocketAddress addr; // identifier of a bookie node.
+
+ public BookieNode(BookieSocketAddress addr, String networkLoc) {
+ super(addr.toString(), networkLoc);
+ this.addr = addr;
+ }
+
+ public BookieSocketAddress getAddr() {
+ return addr;
+ }
+
+ @Override
+ public int hashCode() {
+ return name.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof BookieNode)) {
+ return false;
+ }
+ BookieNode other = (BookieNode) obj;
+ return getName().equals(other.getName());
+ }
+
+ @Override
+ public String toString() {
+ return String.format("<Bookie:%s>", name);
+ }
+}
\ No newline at end of file
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 26fa23f..4a93d4f 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
@@ -43,10 +43,10 @@ import junit.framework.TestCase;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
import
org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble;
-import
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode;
import
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints;
import
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate;
import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.net.BookieNode;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.apache.bookkeeper.net.NetworkTopology;
diff --git
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java
index d5cb067..b21d8aa 100644
---
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java
+++
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/StaticDNSResolver.java
@@ -22,9 +22,10 @@ import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy;
import org.apache.bookkeeper.client.RackChangeNotifier;
-import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl;
import org.apache.bookkeeper.net.AbstractDNSToSwitchMapping;
+import org.apache.bookkeeper.net.BookieNode;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.apache.bookkeeper.net.NetworkTopology;
@@ -87,10 +88,10 @@ public class StaticDNSResolver extends
AbstractDNSToSwitchMapping implements Rac
// nop
}
- private static RackawareEnsemblePlacementPolicyImpl rackawarePolicy = null;
+ private static ITopologyAwareEnsemblePlacementPolicy<BookieNode>
rackawarePolicy = null;
@Override
- public void
registerRackChangeListener(RackawareEnsemblePlacementPolicyImpl
rackawareEnsemblePolicy) {
+ public void
registerRackChangeListener(ITopologyAwareEnsemblePlacementPolicy<BookieNode>
rackawareEnsemblePolicy) {
rackawarePolicy = rackawareEnsemblePolicy;
}
@@ -101,5 +102,4 @@ public class StaticDNSResolver extends
AbstractDNSToSwitchMapping implements Rac
}
rackawarePolicy.onBookieRackChange(bookieAddressList);
}
-
}