1
2
3
4
5
6
7
8
9
10
11 package org.apache.hadoop.hbase.master;
12
13 import static org.junit.Assert.assertEquals;
14
15 import java.io.IOException;
16 import java.util.HashMap;
17 import java.util.List;
18
19 import org.apache.hadoop.conf.Configuration;
20 import org.apache.hadoop.hbase.CoordinatedStateManager;
21 import org.apache.hadoop.hbase.HBaseTestingUtility;
22 import org.apache.hadoop.hbase.HConstants;
23 import org.apache.hadoop.hbase.MiniHBaseCluster;
24 import org.apache.hadoop.hbase.ServerName;
25 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
26 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
27 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
28 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
29 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
30 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
31 import org.apache.hadoop.hbase.testclassification.MasterTests;
32 import org.apache.hadoop.hbase.testclassification.MediumTests;
33 import org.apache.hadoop.hbase.util.Pair;
34 import org.apache.zookeeper.KeeperException;
35 import org.junit.AfterClass;
36 import org.junit.BeforeClass;
37 import org.junit.Test;
38 import org.junit.experimental.categories.Category;
39 import org.slf4j.Logger;
40 import org.slf4j.LoggerFactory;
41
42 @Category({ MasterTests.class, MediumTests.class })
43 public class TestGetReplicationLoad {
44 private static final Logger LOG = LoggerFactory.getLogger(TestGetReplicationLoad.class);
45
46 private static HBaseTestingUtility TEST_UTIL;
47 private static MiniHBaseCluster cluster;
48 private static HMaster master;
49 private static ReplicationAdmin admin;
50
51 private static final String ID_1 = "1";
52 private static final String ID_2 = "2";
53 private static final String KEY_1 = "127.0.0.1:2181:/hbase";
54 private static final String KEY_2 = "127.0.0.1:2181:/hbase2";
55
56 public static class MyMaster extends HMaster {
57 public MyMaster(Configuration conf, CoordinatedStateManager csm)
58 throws IOException, KeeperException, InterruptedException {
59 super(conf, csm);
60 }
61
62 @Override
63 protected void tryRegionServerReport(long reportStartTime, long reportEndTime) {
64
65 }
66 }
67
68 @BeforeClass
69 public static void startCluster() throws Exception {
70 LOG.info("Starting cluster");
71 TEST_UTIL = new HBaseTestingUtility();
72 Configuration conf = TEST_UTIL.getConfiguration();
73 conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT);
74 TEST_UTIL.startMiniCluster(1, 1, 1, null, TestMasterMetrics.MyMaster.class, null);
75 cluster = TEST_UTIL.getHBaseCluster();
76 LOG.info("Waiting for active/ready master");
77 cluster.waitForActiveAndReadyMaster();
78 master = cluster.getMaster();
79 admin = new ReplicationAdmin(conf);
80 }
81
82 @AfterClass
83 public static void after() throws Exception {
84 if (admin != null) {
85 admin.close();
86 }
87 if (TEST_UTIL != null) {
88 TEST_UTIL.shutdownMiniCluster();
89 }
90 }
91
92 @Test
93 public void testGetReplicationMetrics() throws Exception {
94 String peer1 = "test1", peer2 = "test2";
95 long ageOfLastShippedOp = 2, replicationLag = 3, timeStampOfLastShippedOp = 4;
96 int sizeOfLogQueue = 5;
97 RegionServerStatusProtos.RegionServerReportRequest.Builder request =
98 RegionServerStatusProtos.RegionServerReportRequest.newBuilder();
99 ServerName serverName = cluster.getMaster(0).getServerName();
100 request.setServer(ProtobufUtil.toServerName(serverName));
101 ClusterStatusProtos.ReplicationLoadSource rload1 = ClusterStatusProtos.ReplicationLoadSource
102 .newBuilder().setPeerID(peer1).setAgeOfLastShippedOp(ageOfLastShippedOp)
103 .setReplicationLag(replicationLag).setTimeStampOfLastShippedOp(timeStampOfLastShippedOp)
104 .setSizeOfLogQueue(sizeOfLogQueue).build();
105 ClusterStatusProtos.ReplicationLoadSource rload2 =
106 ClusterStatusProtos.ReplicationLoadSource.newBuilder().setPeerID(peer2)
107 .setAgeOfLastShippedOp(ageOfLastShippedOp + 1).setReplicationLag(replicationLag + 1)
108 .setTimeStampOfLastShippedOp(timeStampOfLastShippedOp + 1)
109 .setSizeOfLogQueue(sizeOfLogQueue + 1).build();
110 ClusterStatusProtos.ServerLoad sl = ClusterStatusProtos.ServerLoad.newBuilder()
111 .addReplLoadSource(rload1).addReplLoadSource(rload2).build();
112 request.setLoad(sl);
113
114 ReplicationPeerConfig peerConfig_1 = new ReplicationPeerConfig();
115 peerConfig_1.setClusterKey(KEY_1);
116 ReplicationPeerConfig peerConfig_2 = new ReplicationPeerConfig();
117 peerConfig_2.setClusterKey(KEY_2);
118 admin.addPeer(ID_1, peerConfig_1);
119 admin.addPeer(ID_2, peerConfig_2);
120
121 master.getMasterRpcServices().regionServerReport(null, request.build());
122 HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>> replicationLoad =
123 master.getReplicationLoad(new ServerName[] { serverName });
124 assertEquals("peer size ", 2, replicationLoad.size());
125 assertEquals("load size ", 1, replicationLoad.get(peer1).size());
126 assertEquals("log queue size of peer1", sizeOfLogQueue,
127 replicationLoad.get(peer1).get(0).getSecond().getSizeOfLogQueue());
128 assertEquals("replication lag of peer2", replicationLag + 1,
129 replicationLoad.get(peer2).get(0).getSecond().getReplicationLag());
130
131 master.stopMaster();
132 }
133 }