Skip to content

Commit df27820

Browse files
committed
HBASE-21658 Should get the meta replica number from zk instead of config at client side
1 parent 3641e7a commit df27820

File tree

5 files changed

+67
-10
lines changed

5 files changed

+67
-10
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,9 @@
2626
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
2727

2828
import java.io.IOException;
29+
import java.util.List;
2930
import java.util.concurrent.CompletableFuture;
31+
import java.util.stream.Collectors;
3032
import org.apache.commons.lang3.mutable.MutableInt;
3133
import org.apache.hadoop.conf.Configuration;
3234
import org.apache.hadoop.hbase.ClusterId;
@@ -134,12 +136,13 @@ private Pair<RegionState.State, ServerName> getStateAndServerName(
134136
ServerName.valueOf(snProto.getHostName(), snProto.getPort(), snProto.getStartCode()));
135137
}
136138

137-
@Override
138-
public CompletableFuture<RegionLocations> getMetaRegionLocation() {
139-
CompletableFuture<RegionLocations> future = new CompletableFuture<>();
140-
HRegionLocation[] locs = new HRegionLocation[znodePaths.metaReplicaZNodes.size()];
139+
private void getMetaRegionLocation(CompletableFuture<RegionLocations> future,
140+
List<String> metaReplicaZNodes) {
141+
HRegionLocation[] locs = new HRegionLocation[metaReplicaZNodes.size()];
141142
MutableInt remaining = new MutableInt(locs.length);
142-
znodePaths.metaReplicaZNodes.forEach((replicaId, path) -> {
143+
for (String metaReplicaZNode : metaReplicaZNodes) {
144+
int replicaId = znodePaths.getMetaReplicaIdFromZnode(metaReplicaZNode);
145+
String path = ZNodePaths.joinZNode(znodePaths.baseZNode, metaReplicaZNode);
143146
if (replicaId == DEFAULT_REPLICA_ID) {
144147
addListener(getAndConvert(path, ZKAsyncRegistry::getMetaProto), (proto, error) -> {
145148
if (error != null) {
@@ -186,7 +189,23 @@ public CompletableFuture<RegionLocations> getMetaRegionLocation() {
186189
tryComplete(remaining, locs, future);
187190
});
188191
}
189-
});
192+
}
193+
}
194+
195+
@Override
196+
public CompletableFuture<RegionLocations> getMetaRegionLocation() {
197+
CompletableFuture<RegionLocations> future = new CompletableFuture<>();
198+
addListener(
199+
zk.list(znodePaths.baseZNode)
200+
.thenApply(children -> children.stream()
201+
.filter(c -> c.startsWith(znodePaths.metaZNodePrefix)).collect(Collectors.toList())),
202+
(metaReplicaZNodes, error) -> {
203+
if (error != null) {
204+
future.completeExceptionally(error);
205+
return;
206+
}
207+
getMetaRegionLocation(future, metaReplicaZNodes);
208+
});
190209
return future;
191210
}
192211

hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import java.io.IOException;
2525
import java.util.Arrays;
2626
import java.util.EnumSet;
27+
import java.util.List;
2728
import java.util.concurrent.CompletableFuture;
2829
import java.util.concurrent.DelayQueue;
2930
import java.util.concurrent.Delayed;
@@ -284,6 +285,22 @@ protected void doExec(ZooKeeper zk) {
284285
return future;
285286
}
286287

288+
public CompletableFuture<List<String>> list(String path) {
289+
if (closed.get()) {
290+
return FutureUtils.failedFuture(new DoNotRetryIOException("Client already closed"));
291+
}
292+
CompletableFuture<List<String>> future = new CompletableFuture<>();
293+
tasks.add(new ZKTask<List<String>>(path, future, "list") {
294+
295+
@Override
296+
protected void doExec(ZooKeeper zk) {
297+
zk.getChildren(path, false, (rc, path, ctx, children) -> onComplete(zk, rc, children, true),
298+
null);
299+
}
300+
});
301+
return future;
302+
}
303+
287304
private void closeZk() {
288305
if (zookeeper != null) {
289306
try {

hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -166,7 +166,7 @@ public String getZNodeForReplica(int replicaId) {
166166

167167
/**
168168
* Parse the meta replicaId from the passed znode
169-
* @param znode
169+
* @param znode the name of the znode, does not include baseZNode
170170
* @return replicaId
171171
*/
172172
public int getMetaReplicaIdFromZnode(String znode) {
@@ -178,7 +178,7 @@ public int getMetaReplicaIdFromZnode(String znode) {
178178

179179
/**
180180
* Is it the default meta replica's znode
181-
* @param znode
181+
* @param znode the name of the znode, does not include baseZNode
182182
* @return true or false
183183
*/
184184
public boolean isDefaultMetaReplicaZnode(String znode) {

hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,11 @@ public class TestZKAsyncRegistry {
6161
public static void setUp() throws Exception {
6262
TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
6363
TEST_UTIL.startMiniCluster(3);
64-
REGISTRY = new ZKAsyncRegistry(TEST_UTIL.getConfiguration());
64+
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
65+
// make sure that we do not depend on this config when getting locations for meta replicas, see
66+
// HBASE-21658.
67+
conf.setInt(META_REPLICAS_NUM, 1);
68+
REGISTRY = new ZKAsyncRegistry(conf);
6569
}
6670

6771
@AfterClass

hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java

Lines changed: 18 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,8 @@
3838
import static org.mockito.Mockito.when;
3939

4040
import java.io.IOException;
41+
import java.util.Collections;
42+
import java.util.List;
4143
import java.util.concurrent.CompletableFuture;
4244
import java.util.concurrent.Exchanger;
4345
import java.util.concurrent.ExecutionException;
@@ -126,9 +128,15 @@ public String explainFailure() throws Exception {
126128
}
127129

128130
@Test
129-
public void testGetAndExists() throws Exception {
131+
public void testRead() throws Exception {
130132
assertArrayEquals(DATA, RO_ZK.get(PATH).get());
131133
assertEquals(CHILDREN, RO_ZK.exists(PATH).get().getNumChildren());
134+
List<String> children = RO_ZK.list(PATH).get();
135+
assertEquals(CHILDREN, children.size());
136+
Collections.sort(children);
137+
for (int i = 0; i < CHILDREN; i++) {
138+
assertEquals("c" + i, children.get(i));
139+
}
132140
assertNotNull(RO_ZK.zookeeper);
133141
waitForIdleConnectionClosed();
134142
}
@@ -145,6 +153,15 @@ public void testNoNode() throws InterruptedException, ExecutionException {
145153
assertEquals(Code.NONODE, ke.code());
146154
assertEquals(pathNotExists, ke.getPath());
147155
}
156+
try {
157+
RO_ZK.list(pathNotExists).get();
158+
fail("should fail because of " + pathNotExists + " does not exist");
159+
} catch (ExecutionException e) {
160+
assertThat(e.getCause(), instanceOf(KeeperException.class));
161+
KeeperException ke = (KeeperException) e.getCause();
162+
assertEquals(Code.NONODE, ke.code());
163+
assertEquals(pathNotExists, ke.getPath());
164+
}
148165
// exists will not throw exception.
149166
assertNull(RO_ZK.exists(pathNotExists).get());
150167
}

0 commit comments

Comments
 (0)