Skip to content

Commit dee333c

Browse files
committed
HBASE-27217 Revisit the DumpReplicationQueues tool
1 parent 77e8bcb commit dee333c

File tree

3 files changed

+66
-38
lines changed

3 files changed

+66
-38
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java

Lines changed: 41 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@
1919

2020
import java.io.FileNotFoundException;
2121
import java.io.IOException;
22+
import java.net.URLEncoder;
23+
import java.nio.charset.StandardCharsets;
2224
import java.util.ArrayList;
2325
import java.util.Arrays;
2426
import java.util.Collections;
@@ -44,16 +46,14 @@
4446
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
4547
import org.apache.hadoop.hbase.replication.ReplicationException;
4648
import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;
49+
import org.apache.hadoop.hbase.replication.ReplicationOffsetUtil;
4750
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
4851
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
4952
import org.apache.hadoop.hbase.replication.ReplicationQueueData;
5053
import org.apache.hadoop.hbase.replication.ReplicationQueueId;
5154
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
5255
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
53-
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
5456
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
55-
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
56-
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
5757
import org.apache.hadoop.util.Tool;
5858
import org.apache.hadoop.util.ToolRunner;
5959
import org.apache.yetus.audience.InterfaceAudience;
@@ -210,13 +210,7 @@ private int dumpReplicationQueues(DumpOptions opts) throws Exception {
210210
Connection connection = ConnectionFactory.createConnection(conf);
211211
Admin admin = connection.getAdmin();
212212

213-
ZKWatcher zkw =
214-
new ZKWatcher(conf, "DumpReplicationQueues" + EnvironmentEdgeManager.currentTime(),
215-
new WarnOnlyAbortable(), true);
216-
217213
try {
218-
// Our zk watcher
219-
LOG.info("Our Quorum: " + zkw.getQuorum());
220214
List<TableCFs> replicatedTableCFs = admin.listReplicatedTableCFs();
221215
if (replicatedTableCFs.isEmpty()) {
222216
LOG.info("No tables with a configured replication peer were found.");
@@ -238,7 +232,7 @@ private int dumpReplicationQueues(DumpOptions opts) throws Exception {
238232
LOG.info("Found [--distributed], will poll each RegionServer.");
239233
Set<String> peerIds =
240234
peers.stream().map((peer) -> peer.getPeerId()).collect(Collectors.toSet());
241-
System.out.println(dumpQueues(zkw, connection, peerIds, opts.isHdfs()));
235+
System.out.println(dumpQueues(connection, peerIds, opts.isHdfs()));
242236
System.out.println(dumpReplicationSummary());
243237
} else {
244238
// use replication table instead
@@ -253,7 +247,7 @@ private int dumpReplicationQueues(DumpOptions opts) throws Exception {
253247
}
254248
}
255249

256-
public String dumpReplicationViaTable(Connection connection) throws ReplicationException {
250+
private String dumpReplicationViaTable(Connection connection) throws ReplicationException {
257251
StringBuilder sb = new StringBuilder();
258252
ReplicationQueueStorage queueStorage =
259253
ReplicationStorageFactory.getReplicationQueueStorage(connection, getConf());
@@ -327,13 +321,14 @@ public String dumpPeersState(List<ReplicationPeerDescription> peers) throws Exce
327321
return sb.toString();
328322
}
329323

330-
public String dumpQueues(ZKWatcher zkw, Connection connection, Set<String> peerIds, boolean hdfs)
324+
public String dumpQueues(Connection connection, Set<String> peerIds, boolean hdfs)
331325
throws Exception {
332326
StringBuilder sb = new StringBuilder();
333327
ReplicationQueueStorage queueStorage =
334328
ReplicationStorageFactory.getReplicationQueueStorage(connection, getConf());
335-
Set<ServerName> liveRegionServers = ZKUtil.listChildrenNoWatch(zkw, zkw.getZNodePaths().rsZNode)
336-
.stream().map(ServerName::parseServerName).collect(Collectors.toSet());
329+
330+
Set<ServerName> liveRegionServers =
331+
connection.getAdmin().getClusterMetrics().getLiveServerMetrics().keySet();
337332

338333
List<ServerName> regionServers = queueStorage.listAllReplicators();
339334
if (regionServers == null || regionServers.isEmpty()) {
@@ -346,37 +341,51 @@ public String dumpQueues(ZKWatcher zkw, Connection connection, Set<String> peerI
346341
deadRegionServers.add(regionServer.getServerName());
347342
}
348343
for (ReplicationQueueId queueId : queueIds) {
349-
List<String> wals = null;
350-
if (queueId.isRecovered()) {
351-
wals = AbstractFSWALProvider
352-
.getArchivedWALFiles(connection.getConfiguration(), queueId.getSourceServerName().get(),
353-
queueId.getSourceServerName().get().toString())
354-
.stream().map(Path::toString).collect(Collectors.toList());
355-
} else {
356-
wals = AbstractFSWALProvider
357-
.getArchivedWALFiles(connection.getConfiguration(), queueId.getServerName(),
358-
queueId.getServerName().toString())
359-
.stream().map(Path::toString).collect(Collectors.toList());
344+
// wals
345+
List<String> tmpWals = AbstractFSWALProvider
346+
.getWALFiles(connection.getConfiguration(),
347+
URLEncoder.encode(queueId.getServerWALsBelongTo().toString(),
348+
StandardCharsets.UTF_8.name()))
349+
.stream().map(Path::toString).collect(Collectors.toList());
350+
351+
// old wals
352+
tmpWals.addAll(AbstractFSWALProvider
353+
.getArchivedWALFiles(connection.getConfiguration(), queueId.getServerWALsBelongTo(),
354+
URLEncoder.encode(queueId.getServerWALsBelongTo().toString(),
355+
StandardCharsets.UTF_8.name()))
356+
.stream().map(Path::toString).collect(Collectors.toList()));
357+
358+
Map<String, ReplicationGroupOffset> offsets = queueStorage.getOffsets(queueId);
359+
// filter out the wal files that should replicate
360+
List<String> wals = new ArrayList<>();
361+
for (Map.Entry<String, ReplicationGroupOffset> entry : offsets.entrySet()) {
362+
ReplicationGroupOffset offset = entry.getValue();
363+
for (String wal : tmpWals) {
364+
if (ReplicationOffsetUtil.shouldReplicate(offset, wal)) {
365+
wals.add(wal);
366+
}
367+
}
360368
}
369+
361370
Collections.sort(wals);
362371
if (!peerIds.contains(queueId.getPeerId())) {
363372
deletedQueues.add(regionServer + "/" + queueId);
364-
sb.append(formatQueue(regionServer, queueStorage, wals, queueId, true, hdfs));
373+
sb.append(formatQueue(regionServer, offsets, wals, queueId, true, hdfs));
365374
} else {
366-
sb.append(formatQueue(regionServer, queueStorage, wals, queueId, false, hdfs));
375+
sb.append(formatQueue(regionServer, offsets, wals, queueId, false, hdfs));
367376
}
368377
}
369378
}
370379
return sb.toString();
371380
}
372381

373-
private String formatQueue(ServerName regionServer, ReplicationQueueStorage queueStorage,
382+
private String formatQueue(ServerName regionServer, Map<String, ReplicationGroupOffset> offsets,
374383
List<String> wals, ReplicationQueueId queueId, boolean isDeleted, boolean hdfs)
375384
throws Exception {
376385
StringBuilder sb = new StringBuilder();
377386

378387
sb.append("Dumping replication queue info for RegionServer: [" + regionServer + "]" + "\n");
379-
sb.append(" Queue znode: " + queueId + "\n");
388+
sb.append(" Queue id: " + queueId + "\n");
380389
sb.append(" PeerID: " + queueId.getPeerId() + "\n");
381390
sb.append(" Recovered: " + queueId.isRecovered() + "\n");
382391
// In new version, we only record the first dead RegionServer in queueId.
@@ -389,19 +398,14 @@ private String formatQueue(ServerName regionServer, ReplicationQueueStorage queu
389398
sb.append(" Number of WALs in replication queue: " + wals.size() + "\n");
390399
peersQueueSize.addAndGet(queueId.getPeerId(), wals.size());
391400

392-
Set<Map.Entry<String, ReplicationGroupOffset>> offsets =
393-
queueStorage.getOffsets(queueId).entrySet();
394-
395-
for (Map.Entry<String, ReplicationGroupOffset> entry : offsets) {
396-
String walGroup = null;
397-
walGroup = entry.getKey();
401+
for (Map.Entry<String, ReplicationGroupOffset> entry : offsets.entrySet()) {
402+
String walGroup = entry.getKey();
403+
ReplicationGroupOffset offset = entry.getValue();
398404
for (String wal : wals) {
399-
ReplicationGroupOffset offset = entry.getValue();
400405
if (offset.getWal().equals(wal)) {
401406
long position = offset.getOffset();
402407
sb.append(
403408
" Replication position for " + (walGroup != null ? walGroup + "/" + wal : wal) + ": ");
404-
405409
// Position is -1, which means that the file has already been fully replicated,
406410
// the logic here is different from the previous version.
407411
if (position == -1) {

hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,9 @@
3535
import org.apache.hadoop.fs.FSDataInputStream;
3636
import org.apache.hadoop.fs.FileStatus;
3737
import org.apache.hadoop.fs.FileSystem;
38+
import org.apache.hadoop.fs.LocatedFileStatus;
3839
import org.apache.hadoop.fs.Path;
40+
import org.apache.hadoop.fs.RemoteIterator;
3941
import org.apache.hadoop.hbase.Abortable;
4042
import org.apache.hadoop.hbase.HConstants;
4143
import org.apache.hadoop.hbase.ServerName;
@@ -381,6 +383,28 @@ public static List<Path> getArchivedWALFiles(Configuration conf, ServerName serv
381383
return archivedWalFiles;
382384
}
383385

386+
/**
387+
* List all the wal files for a logPrefix.
388+
*/
389+
public static List<Path> getWALFiles(Configuration c, String logPrefix) throws IOException {
390+
Path walRoot = new Path(CommonFSUtils.getWALRootDir(c), HConstants.HREGION_LOGDIR_NAME);
391+
FileSystem fs = walRoot.getFileSystem(c);
392+
List<Path> walFiles = new ArrayList<>();
393+
try {
394+
RemoteIterator<LocatedFileStatus> it = fs.listFiles(walRoot, true);
395+
while (it.hasNext()) {
396+
LocatedFileStatus lfs = it.next();
397+
if (lfs.isFile() && lfs.getPath().getName().startsWith(logPrefix)) {
398+
walFiles.add(lfs.getPath());
399+
}
400+
}
401+
} catch (FileNotFoundException e) {
402+
LOG.info("WAL dir {} not exists", walRoot);
403+
return Collections.emptyList();
404+
}
405+
return walFiles;
406+
}
407+
384408
/**
385409
* Pulls a ServerName out of a Path generated according to our layout rules. In the below layouts,
386410
* this method ignores the format of the logfile component. Current format: [base directory for

hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDumpReplicationQueues.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@ public void testDumpReplicationReturnsWalSorted() throws Exception {
8383
Set<String> peerIds = new HashSet<>();
8484
peerIds.add("1");
8585
dumpQueues.setConf(config);
86-
String dump = dumpQueues.dumpQueues(zkWatcherMock, null, peerIds, false);
86+
String dump = dumpQueues.dumpQueues(null, peerIds, false);
8787
String[] parsedDump = dump.split("Replication position for");
8888
assertEquals("Parsed dump should have 4 parts.", 4, parsedDump.length);
8989
assertTrue(

0 commit comments

Comments
 (0)