This is an automated email from the ASF dual-hosted git repository.
chenhang 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 fb7138329f Correct RackawareEnsemblePlacementPolicyImpl defaultRack
when the bookie is not available. (#4439)
fb7138329f is described below
commit fb7138329ff2faaa3e10fb4f2028e2aaac28843e
Author: Yan Zhao <[email protected]>
AuthorDate: Mon Jul 8 17:17:08 2024 +0800
Correct RackawareEnsemblePlacementPolicyImpl defaultRack when the bookie is
not available. (#4439)
When the bookie is not available, the RackawareEnsemblePlacementPolicyImpl
default rack will be `/default-region/default-rack`, it should be
`/default-rack` for RackawareEnsemblePlacementPolicyImpl.
There are some logs.
```
2024-06-17T05:22:46,591+0000 [ReplicationWorker] ERROR
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Cannot
resolve bookieId `test-bk-3:3181` to a network address, resolving as
/default-region/default-rack
org.apache.bookkeeper.proto.BookieAddressResolver$BookieIdNotResolvedException:
Cannot resolve bookieId test-bk-3:3181, bookie does not exist or it is not
running
at
org.apache.bookkeeper.client.DefaultBookieAddressResolver.resolve(DefaultBookieAddressResolver.java:66)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.resolveNetworkLocation(TopologyAwareEnsemblePlacementPolicy.java:821)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.createBookieNode(TopologyAwareEnsemblePlacementPolicy.java:811)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.convertBookieToNode(TopologyAwareEnsemblePlacementPolicy.java:845)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.convertBookiesToNodes(TopologyAwareEnsemblePlacementPolicy.java:837)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.replaceBookie(RackawareEnsemblePlacementPolicyImpl.java:474)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.replaceBookie(RackawareEnsemblePlacementPolicy.java:119)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.BookKeeperAdmin.getReplacementBookiesByIndexes(BookKeeperAdmin.java:993)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.BookKeeperAdmin.replicateLedgerFragment(BookKeeperAdmin.java:1025)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.replication.ReplicationWorker.rereplicate(ReplicationWorker.java:473)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.replication.ReplicationWorker.rereplicate(ReplicationWorker.java:301)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.replication.ReplicationWorker.run(ReplicationWorker.java:249)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
~[io.netty-netty-common-4.1.108.Final.jar:4.1.108.Final]
at java.lang.Thread.run(Thread.java:840) ~[?:?]
Caused by:
org.apache.bookkeeper.client.BKException$BKBookieHandleNotAvailableException:
Bookie handle is not available
at
org.apache.bookkeeper.discover.ZKRegistrationClient.getBookieServiceInfo(ZKRegistrationClient.java:226)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
at
org.apache.bookkeeper.client.DefaultBookieAddressResolver.resolve(DefaultBookieAddressResolver.java:45)
~[io.streamnative-bookkeeper-server-4.16.5.2.jar:4.16.5.2]
... 13 more
```
---
.../RackawareEnsemblePlacementPolicyImpl.java | 1 +
.../TopologyAwareEnsemblePlacementPolicy.java | 9 ++-
.../TestRackawareEnsemblePlacementPolicy.java | 64 ++++++++++++++++++++++
3 files changed, 72 insertions(+), 2 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 dba9828e85..6258ede3c3 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
@@ -281,6 +281,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends
TopologyAwareEnsembleP
return this;
}
+ @Override
public String getDefaultRack() {
return defaultRack;
}
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 4976f96e8c..9eae79438f 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
@@ -824,12 +824,17 @@ abstract class TopologyAwareEnsemblePlacementPolicy
implements
if (null != historyBookie) {
return historyBookie.getNetworkLocation();
}
+ String defaultRack = getDefaultRack();
LOG.error("Cannot resolve bookieId {} to a network address,
resolving as {}. {}", addr,
- NetworkTopology.DEFAULT_REGION_AND_RACK,
err.getMessage());
- return NetworkTopology.DEFAULT_REGION_AND_RACK;
+ defaultRack, err.getMessage());
+ return defaultRack;
}
}
+ protected String getDefaultRack() {
+ return NetworkTopology.DEFAULT_REGION_AND_RACK;
+ }
+
protected Set<Node> convertBookiesToNodes(Collection<BookieId> bookies) {
Set<Node> nodes = new HashSet<Node>();
for (BookieId addr : bookies) {
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 ddd58d2cba..d1e2cd5323 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
@@ -29,6 +29,7 @@ import static org.junit.Assert.assertThat;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import io.netty.util.HashedWheelTimer;
import java.net.InetAddress;
+import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -40,6 +41,7 @@ import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import junit.framework.TestCase;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
@@ -2354,6 +2356,68 @@ public class TestRackawareEnsemblePlacementPolicy
extends TestCase {
assertEquals(ensemble.get(reorderSet.get(3)), addr2.toBookieId());
assertEquals(ensemble.get(reorderSet.get(0)), addr3.toBookieId());
assertEquals(ensemble.get(reorderSet.get(1)), addr4.toBookieId());
+ StaticDNSResolver.reset();
+ }
+
+ @Test
+ public void testReplaceNotAvailableBookieWithDefaultRack() throws
Exception {
+ repp.uninitalize();
+ repp.withDefaultRack(NetworkTopology.DEFAULT_RACK);
+ AtomicInteger counter = new AtomicInteger();
+ BookieAddressResolver mockResolver = new BookieAddressResolver() {
+ @Override
+ public BookieSocketAddress resolve(BookieId bookieId) throws
BookieIdNotResolvedException {
+ if (bookieId.equals(addr1.toBookieId()) &&
counter.getAndIncrement() >= 1) {
+ throw new BookieIdNotResolvedException(bookieId,
+ new RuntimeException(addr1.toBookieId() + "
shutdown"));
+ }
+ try {
+ return new BookieSocketAddress(bookieId.toString());
+ } catch (UnknownHostException err) {
+ throw new BookieIdNotResolvedException(bookieId, err);
+ }
+ }
+ };
+
+ repp.initialize(conf, Optional.<DNSToSwitchMapping>empty(), timer,
DISABLE_ALL, NullStatsLogger.INSTANCE,
+ mockResolver);
+ BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181);
+ BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181);
+ BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181);
+ BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181);
+ // update dns mapping
+ StaticDNSResolver.addNodeToRack(addr1.getHostName(),
NetworkTopology.DEFAULT_RACK);
+ StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/r1");
+ StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/r1");
+ StaticDNSResolver.addNodeToRack(addr4.getHostName(),
NetworkTopology.DEFAULT_RACK);
+
+ // Update cluster
+ Set<BookieId> addrs = new HashSet<BookieId>();
+ addrs.add(addr1.toBookieId());
+ addrs.add(addr2.toBookieId());
+ addrs.add(addr3.toBookieId());
+ addrs.add(addr4.toBookieId());
+ repp.onClusterChanged(addrs, new HashSet<BookieId>());
+
+ // replace node under r1
+ EnsemblePlacementPolicy.PlacementResult<BookieId>
replaceBookieResponse =
+ repp.replaceBookie(1, 1, 1, null, new ArrayList<>(),
addr1.toBookieId(), new HashSet<>());
+ BookieId replacedBookie = replaceBookieResponse.getResult();
+ assertEquals(addr4.toBookieId(), replacedBookie);
+
+ //clear history bookies and make addr1 shutdown.
+ repp = new RackawareEnsemblePlacementPolicy();
+ repp.initialize(conf, Optional.<DNSToSwitchMapping>empty(), timer,
DISABLE_ALL, NullStatsLogger.INSTANCE,
+ mockResolver);
+
+ addrs.remove(addr1.toBookieId());
+ repp.onClusterChanged(addrs, new HashSet<BookieId>());
+
+ // replace node under r1 again
+ replaceBookieResponse =
+ repp.replaceBookie(1, 1, 1, null, new ArrayList<>(),
addr1.toBookieId(), new HashSet<>());
+ replacedBookie = replaceBookieResponse.getResult();
+ assertEquals(addr4.toBookieId(), replacedBookie);
}
@Test