1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication;
20
21 import static org.junit.Assert.*;
22
23 import java.io.IOException;
24 import java.util.List;
25 import java.util.SortedMap;
26 import java.util.SortedSet;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.ServerName;
31 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
32 import org.apache.zookeeper.KeeperException;
33 import org.junit.Before;
34 import org.junit.Test;
35
36
37
38
39
40 public abstract class TestReplicationStateBasic {
41
42 protected ReplicationQueues rq1;
43 protected ReplicationQueues rq2;
44 protected ReplicationQueues rq3;
45 protected ReplicationQueuesClient rqc;
46 protected String server1 = new ServerName("hostname1.example.org", 1234, -1L).toString();
47 protected String server2 = new ServerName("hostname2.example.org", 1234, -1L).toString();
48 protected String server3 = new ServerName("hostname3.example.org", 1234, -1L).toString();
49 protected ReplicationPeers rp;
50 protected static final String ID_ONE = "1";
51 protected static final String ID_TWO = "2";
52 protected static String KEY_ONE;
53 protected static String KEY_TWO;
54
55
56 protected String OUR_ID = "3";
57 protected String OUR_KEY;
58
59 protected static int zkTimeoutCount;
60 protected static final int ZK_MAX_COUNT = 300;
61 protected static final int ZK_SLEEP_INTERVAL = 100;
62
63 private static final Log LOG = LogFactory.getLog(TestReplicationStateBasic.class);
64
65 @Before
66 public void setUp() {
67 zkTimeoutCount = 0;
68 }
69
70 @Test
71 public void testReplicationQueuesClient() throws KeeperException {
72 rqc.init();
73
74 assertEquals(0, rqc.getListOfReplicators().size());
75 assertNull(rqc.getLogsInQueue(server1, "qId1"));
76 assertNull(rqc.getAllQueues(server1));
77
78
79
80
81
82 rq1.init(server1);
83 rq2.init(server2);
84 rq1.addLog("qId1", "trash");
85 rq1.removeLog("qId1", "trash");
86 rq1.addLog("qId2", "filename1");
87 rq1.addLog("qId3", "filename2");
88 rq1.addLog("qId3", "filename3");
89 rq2.addLog("trash", "trash");
90 rq2.removeQueue("trash");
91
92 List<String> reps = rqc.getListOfReplicators();
93 assertEquals(2, reps.size());
94 assertTrue(server1, reps.contains(server1));
95 assertTrue(server2, reps.contains(server2));
96
97 assertNull(rqc.getLogsInQueue("bogus", "bogus"));
98 assertNull(rqc.getLogsInQueue(server1, "bogus"));
99 assertEquals(0, rqc.getLogsInQueue(server1, "qId1").size());
100 assertEquals(1, rqc.getLogsInQueue(server1, "qId2").size());
101 assertEquals("filename1", rqc.getLogsInQueue(server1, "qId2").get(0));
102
103 assertNull(rqc.getAllQueues("bogus"));
104 assertEquals(0, rqc.getAllQueues(server2).size());
105 List<String> list = rqc.getAllQueues(server1);
106 assertEquals(3, list.size());
107 assertTrue(list.contains("qId2"));
108 assertTrue(list.contains("qId3"));
109 }
110
111 @Test
112 public void testReplicationQueues() throws KeeperException, IOException {
113 rq1.init(server1);
114 rq2.init(server2);
115 rq3.init(server3);
116
117 rp.init();
118
119
120 assertEquals(3, rq1.getListOfReplicators().size());
121 rq1.removeQueue("bogus");
122 rq1.removeLog("bogus", "bogus");
123 rq1.removeAllQueues();
124 assertNull(rq1.getAllQueues());
125 assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
126 assertNull(rq1.getLogsInQueue("bogus"));
127 assertEquals(0, rq1.claimQueues(new ServerName("bogus", 1234, -1L).toString()).size());
128
129 rq1.setLogPosition("bogus", "bogus", 5L);
130
131 populateQueues();
132
133 assertEquals(3, rq1.getListOfReplicators().size());
134 assertEquals(0, rq2.getLogsInQueue("qId1").size());
135 assertEquals(5, rq3.getLogsInQueue("qId5").size());
136 assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
137 rq3.setLogPosition("qId5", "filename4", 354L);
138 assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
139
140 assertEquals(5, rq3.getLogsInQueue("qId5").size());
141 assertEquals(0, rq2.getLogsInQueue("qId1").size());
142 assertEquals(0, rq1.getAllQueues().size());
143 assertEquals(1, rq2.getAllQueues().size());
144 assertEquals(5, rq3.getAllQueues().size());
145
146 assertEquals(0, rq3.claimQueues(server1).size());
147 assertEquals(2, rq3.getListOfReplicators().size());
148
149 SortedMap<String, SortedSet<String>> queues = rq2.claimQueues(server3);
150 assertEquals(5, queues.size());
151 assertEquals(1, rq2.getListOfReplicators().size());
152
153
154 assertEquals(0, rq2.claimQueues(server2).size());
155
156 assertEquals(6, rq2.getAllQueues().size());
157
158 rq2.removeAllQueues();
159
160 assertEquals(0, rq2.getListOfReplicators().size());
161 }
162
163 @Test
164 public void testReplicationPeers() throws Exception {
165 rp.init();
166
167
168 try {
169 rp.removePeer("bogus");
170 fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
171 } catch (IllegalArgumentException e) {
172 }
173 try {
174 rp.enablePeer("bogus");
175 fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
176 } catch (IllegalArgumentException e) {
177 }
178 try {
179 rp.disablePeer("bogus");
180 fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
181 } catch (IllegalArgumentException e) {
182 }
183 try {
184 rp.getStatusOfConnectedPeer("bogus");
185 fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
186 } catch (IllegalArgumentException e) {
187 }
188 assertFalse(rp.connectToPeer("bogus"));
189 rp.disconnectFromPeer("bogus");
190 assertEquals(0, rp.getRegionServersOfConnectedPeer("bogus").size());
191 assertNull(rp.getPeerUUID("bogus"));
192 assertNull(rp.getPeerConf("bogus"));
193 assertNumberOfPeers(0, 0);
194
195
196 rp.addPeer(ID_ONE, KEY_ONE);
197 assertNumberOfPeers(0, 1);
198 rp.addPeer(ID_TWO, KEY_TWO);
199 assertNumberOfPeers(0, 2);
200
201
202 try {
203 rp.getStatusOfConnectedPeer(ID_ONE);
204 fail("There are no connected peers, should have thrown an IllegalArgumentException");
205 } catch (IllegalArgumentException e) {
206 }
207 assertNull(rp.getPeerUUID(ID_ONE));
208 assertEquals(KEY_ONE, ZKUtil.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE)));
209 rp.disconnectFromPeer(ID_ONE);
210 assertEquals(0, rp.getRegionServersOfConnectedPeer(ID_ONE).size());
211
212
213 rp.connectToPeer(ID_ONE);
214 assertNumberOfPeers(1, 2);
215 assertTrue(rp.getStatusOfConnectedPeer(ID_ONE));
216 rp.disablePeer(ID_ONE);
217 assertConnectedPeerStatus(false, ID_ONE);
218 rp.enablePeer(ID_ONE);
219 assertConnectedPeerStatus(true, ID_ONE);
220 assertEquals(1, rp.getRegionServersOfConnectedPeer(ID_ONE).size());
221 assertNotNull(rp.getPeerUUID(ID_ONE).toString());
222
223
224 rp.disconnectFromPeer(ID_ONE);
225 assertNumberOfPeers(0, 2);
226 try {
227 rp.getStatusOfConnectedPeer(ID_ONE);
228 fail("There are no connected peers, should have thrown an IllegalArgumentException");
229 } catch (IllegalArgumentException e) {
230 }
231 }
232
233 protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
234
235 if (status != rp.getStatusOfPeerFromBackingStore(peerId)) {
236 fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
237 }
238 while (true) {
239 if (status == rp.getStatusOfConnectedPeer(peerId)) {
240 return;
241 }
242 if (zkTimeoutCount < ZK_MAX_COUNT) {
243 LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status
244 + ", sleeping and trying again.");
245 Thread.sleep(ZK_SLEEP_INTERVAL);
246 } else {
247 fail("Timed out waiting for ConnectedPeerStatus to be " + status);
248 }
249 }
250 }
251
252 protected void assertNumberOfPeers(int connected, int total) {
253 assertEquals(total, rp.getAllPeerClusterKeys().size());
254 assertEquals(connected, rp.getConnectedPeers().size());
255 assertEquals(total, rp.getAllPeerIds().size());
256 }
257
258
259
260
261
262 protected void populateQueues() throws KeeperException, IOException {
263 rq1.addLog("trash", "trash");
264 rq1.removeQueue("trash");
265
266 rq2.addLog("qId1", "trash");
267 rq2.removeLog("qId1", "trash");
268
269 for (int i = 1; i < 6; i++) {
270 for (int j = 0; j < i; j++) {
271 rq3.addLog("qId" + i, "filename" + j);
272 }
273
274 rp.addPeer("qId" + i, "bogus" + i);
275 }
276 }
277 }
278