You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2019/05/14 21:39:31 UTC

[hbase] branch master updated: HBASE-21784 Dump replication queue should show list of wal files ordered chronologically

This is an automated email from the ASF dual-hosted git repository.

apurtell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new fab2e15  HBASE-21784 Dump replication queue should show list of wal files ordered chronologically
fab2e15 is described below

commit fab2e15ae43c08792f557d03a8a68f1e671bea9b
Author: Wellington Chevreuil <we...@ChevreuilWellington-MBP15.local>
AuthorDate: Mon May 13 20:30:01 2019 +0100

    HBASE-21784 Dump replication queue should show list of wal files ordered chronologically
    
    Change-Id: I18c372406290e2b1e2b5503e2c87adcb9bf6fe91
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../regionserver/DumpReplicationQueues.java        |  2 +
 .../regionserver/TestDumpReplicationQueues.java    | 98 ++++++++++++++++++++++
 .../org/apache/hadoop/hbase/zookeeper/ZKUtil.java  |  2 +
 3 files changed, 102 insertions(+)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index a960c31..432dbcd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -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));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDumpReplicationQueues.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDumpReplicationQueues.java
new file mode 100644
index 0000000..f49f9b9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDumpReplicationQueues.java
@@ -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);
+  }
+
+}
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 51401b0..5e91b90 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -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)) {