HBASE-21784 Dump replication queue should show list of wal files ordered chronologically

Change-Id: I18c372406290e2b1e2b5503e2c87adcb9bf6fe91

Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
Wellington Chevreuil 2019-05-13 20:30:01 +01:00 committed by Andrew Purtell
parent 81953d3e00
commit fab2e15ae4
No known key found for this signature in database
GPG Key ID: 8597754DD5365CCD
3 changed files with 102 additions and 0 deletions

View File

@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
@ -325,6 +326,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
for (String queueId : queueIds) {
ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
List<String> wals = queueStorage.getWALsInQueue(regionserver, queueId);
Collections.sort(wals);
if (!peerIds.contains(queueInfo.getPeerId())) {
deletedQueues.add(regionserver + "/" + queueId);
sb.append(formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, true, hdfs));

View File

@ -0,0 +1,98 @@
/**
* 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.hadoop.hbase.replication.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests for DumpReplicationQueues tool
*/
@Category({ ReplicationTests.class, SmallTests.class})
public class TestDumpReplicationQueues {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestDumpReplicationQueues.class);
/**
* Makes sure dumpQueues returns wals znodes ordered chronologically.
* @throws Exception if dumpqueues finds any error while handling list of znodes.
*/
@Test
public void testDumpReplicationReturnsWalSorted() throws Exception {
Configuration config = HBaseConfiguration.create();
ZKWatcher zkWatcherMock = mock(ZKWatcher.class);
ZNodePaths zNodePath = new ZNodePaths(config);
RecoverableZooKeeper recoverableZooKeeperMock = mock(RecoverableZooKeeper.class);
when(zkWatcherMock.getRecoverableZooKeeper()).thenReturn(recoverableZooKeeperMock);
when(zkWatcherMock.getZNodePaths()).thenReturn(zNodePath);
List<String> nodes = new ArrayList<>();
String server = "rs1,60030,"+System.currentTimeMillis();
nodes.add(server);
when(recoverableZooKeeperMock.getChildren("/hbase/rs", null)).thenReturn(nodes);
when(recoverableZooKeeperMock.getChildren("/hbase/replication/rs", null)).
thenReturn(nodes);
List<String> queuesIds = new ArrayList<>();
queuesIds.add("1");
when(recoverableZooKeeperMock.getChildren("/hbase/replication/rs/"+server, null)).
thenReturn(queuesIds);
List<String> wals = new ArrayList<>();
wals.add("rs1%2C60964%2C1549394085556.1549394101427");
wals.add("rs1%2C60964%2C1549394085556.1549394101426");
wals.add("rs1%2C60964%2C1549394085556.1549394101428");
when(recoverableZooKeeperMock.getChildren("/hbase/replication/rs/"+server+"/1",
null)).thenReturn(wals);
DumpReplicationQueues dumpQueues = new DumpReplicationQueues();
Set<String> peerIds = new HashSet<>();
peerIds.add("1");
dumpQueues.setConf(config);
String dump = dumpQueues.dumpQueues(zkWatcherMock, peerIds, false);
String[] parsedDump = dump.split("Replication position for");
assertEquals("Parsed dump should have 4 parts.", 4, parsedDump.length);
assertTrue("First wal should be rs1%2C60964%2C1549394085556.1549394101426, but got: "
+ parsedDump[1],
parsedDump[1].indexOf("rs1%2C60964%2C1549394085556.1549394101426")>=0);
assertTrue("Second wal should be rs1%2C60964%2C1549394085556.1549394101427, but got: "
+ parsedDump[2],
parsedDump[2].indexOf("rs1%2C60964%2C1549394085556.1549394101427")>=0);
assertTrue("Third wal should be rs1%2C60964%2C1549394085556.1549394101428, but got: "
+ parsedDump[3],
parsedDump[3].indexOf("rs1%2C60964%2C1549394085556.1549394101428")>=0);
}
}

View File

@ -29,6 +29,7 @@ import java.net.Socket;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Deque;
import java.util.HashMap;
import java.util.Iterator;
@ -1859,6 +1860,7 @@ public final class ZKUtil {
// do a ls -r on this znode
sb.append("\n").append(replicationZnode).append(": ");
List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
Collections.sort(children);
for (String child : children) {
String znode = ZNodePaths.joinZNode(replicationZnode, child);
if (znode.equals(zkw.getZNodePaths().peersZNode)) {