-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
TestReplicationStateBasic.java
368 lines (317 loc) · 12.8 KB
/
TestReplicationStateBasic.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
/**
* 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;
import static org.junit.Assert.*;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.zookeeper.KeeperException;
import org.junit.Before;
import org.junit.Test;
/**
* White box testing for replication state interfaces. Implementations should extend this class, and
* initialize the interfaces properly.
*/
public abstract class TestReplicationStateBasic {
protected ReplicationQueues rq1;
protected ReplicationQueues rq2;
protected ReplicationQueues rq3;
protected ReplicationQueuesClient rqc;
protected String server1 = ServerName.valueOf("hostname1.example.org", 1234, -1L).toString();
protected String server2 = ServerName.valueOf("hostname2.example.org", 1234, -1L).toString();
protected String server3 = ServerName.valueOf("hostname3.example.org", 1234, -1L).toString();
protected ReplicationPeers rp;
protected static final String ID_ONE = "1";
protected static final String ID_TWO = "2";
protected static String KEY_ONE;
protected static String KEY_TWO;
// For testing when we try to replicate to ourself
protected String OUR_ID = "3";
protected String OUR_KEY;
protected static int zkTimeoutCount;
protected static final int ZK_MAX_COUNT = 300;
protected static final int ZK_SLEEP_INTERVAL = 100; // millis
private static final Log LOG = LogFactory.getLog(TestReplicationStateBasic.class);
@Before
public void setUp() {
zkTimeoutCount = 0;
}
@Test
public void testReplicationQueuesClient() throws ReplicationException, KeeperException {
rqc.init();
// Test methods with empty state
assertEquals(0, rqc.getListOfReplicators().size());
assertNull(rqc.getLogsInQueue(server1, "qId1"));
assertNull(rqc.getAllQueues(server1));
/*
* Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each --
* server2: zero queues
*/
rq1.init(server1);
rq2.init(server2);
rq1.addLog("qId1", "trash");
rq1.removeLog("qId1", "trash");
rq1.addLog("qId2", "filename1");
rq1.addLog("qId3", "filename2");
rq1.addLog("qId3", "filename3");
rq2.addLog("trash", "trash");
rq2.removeQueue("trash");
List<String> reps = rqc.getListOfReplicators();
assertEquals(2, reps.size());
assertTrue(server1, reps.contains(server1));
assertTrue(server2, reps.contains(server2));
assertNull(rqc.getLogsInQueue("bogus", "bogus"));
assertNull(rqc.getLogsInQueue(server1, "bogus"));
assertEquals(0, rqc.getLogsInQueue(server1, "qId1").size());
assertEquals(1, rqc.getLogsInQueue(server1, "qId2").size());
assertEquals("filename1", rqc.getLogsInQueue(server1, "qId2").get(0));
assertNull(rqc.getAllQueues("bogus"));
assertEquals(0, rqc.getAllQueues(server2).size());
List<String> list = rqc.getAllQueues(server1);
assertEquals(3, list.size());
assertTrue(list.contains("qId2"));
assertTrue(list.contains("qId3"));
}
@Test
public void testReplicationQueues() throws ReplicationException {
rq1.init(server1);
rq2.init(server2);
rq3.init(server3);
//Initialize ReplicationPeer so we can add peers (we don't transfer lone queues)
rp.init();
// 3 replicators should exist
assertEquals(3, rq1.getListOfReplicators().size());
rq1.removeQueue("bogus");
rq1.removeLog("bogus", "bogus");
rq1.removeAllQueues();
assertEquals(0, rq1.getAllQueues().size());
assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
assertNull(rq1.getLogsInQueue("bogus"));
assertEquals(0, rq1.claimQueues(ServerName.valueOf("bogus", 1234, -1L).toString()).size());
rq1.setLogPosition("bogus", "bogus", 5L);
populateQueues();
assertEquals(3, rq1.getListOfReplicators().size());
assertEquals(0, rq2.getLogsInQueue("qId1").size());
assertEquals(5, rq3.getLogsInQueue("qId5").size());
assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
rq3.setLogPosition("qId5", "filename4", 354L);
assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
assertEquals(5, rq3.getLogsInQueue("qId5").size());
assertEquals(0, rq2.getLogsInQueue("qId1").size());
assertEquals(0, rq1.getAllQueues().size());
assertEquals(1, rq2.getAllQueues().size());
assertEquals(5, rq3.getAllQueues().size());
assertEquals(0, rq3.claimQueues(server1).size());
assertEquals(2, rq3.getListOfReplicators().size());
Map<String, Set<String>> queues = rq2.claimQueues(server3);
assertEquals(5, queues.size());
assertEquals(1, rq2.getListOfReplicators().size());
// Try to claim our own queues
assertEquals(0, rq2.claimQueues(server2).size());
assertEquals(6, rq2.getAllQueues().size());
rq2.removeAllQueues();
assertEquals(0, rq2.getListOfReplicators().size());
}
@Test
public void testInvalidClusterKeys() throws ReplicationException, KeeperException {
rp.init();
try {
rp.addPeer(ID_ONE,
new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
fail("Should throw an IllegalArgumentException because "
+ "zookeeper.znode.parent is missing leading '/'.");
} catch (IllegalArgumentException e) {
// Expected.
}
try {
rp.addPeer(ID_ONE,
new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
} catch (IllegalArgumentException e) {
// Expected.
}
try {
rp.addPeer(ID_ONE,
new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
fail("Should throw an IllegalArgumentException because "
+ "hbase.zookeeper.property.clientPort is missing.");
} catch (IllegalArgumentException e) {
// Expected.
}
}
@Test
public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
rp.init();
rq1.init(server1);
rqc.init();
List<String> files1 = new ArrayList<String>(3);
files1.add("file_1");
files1.add("file_2");
files1.add("file_3");
assertNull(rqc.getReplicableHFiles(ID_ONE));
assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
rq1.addPeerToHFileRefs(ID_ONE);
rq1.addHFileRefs(ID_ONE, files1);
assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
List<String> files2 = new ArrayList<>(files1);
String removedString = files2.remove(0);
rq1.removeHFileRefs(ID_ONE, files2);
assertEquals(1, rqc.getReplicableHFiles(ID_ONE).size());
files2 = new ArrayList<>(1);
files2.add(removedString);
rq1.removeHFileRefs(ID_ONE, files2);
assertEquals(0, rqc.getReplicableHFiles(ID_ONE).size());
rp.removePeer(ID_ONE);
}
@Test
public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
rq1.init(server1);
rqc.init();
rp.init();
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
rq1.addPeerToHFileRefs(ID_ONE);
rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
rq1.addPeerToHFileRefs(ID_TWO);
List<String> files1 = new ArrayList<String>(3);
files1.add("file_1");
files1.add("file_2");
files1.add("file_3");
rq1.addHFileRefs(ID_ONE, files1);
rq1.addHFileRefs(ID_TWO, files1);
assertEquals(2, rqc.getAllPeersFromHFileRefsQueue().size());
assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
rp.removePeer(ID_ONE);
rq1.removePeerFromHFileRefs(ID_ONE);
assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
assertNull(rqc.getReplicableHFiles(ID_ONE));
assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
rp.removePeer(ID_TWO);
rq1.removePeerFromHFileRefs(ID_TWO);
assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
assertNull(rqc.getReplicableHFiles(ID_TWO));
}
@Test
public void testReplicationPeers() throws Exception {
rp.init();
// Test methods with non-existent peer ids
try {
rp.removePeer("bogus");
fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
} catch (IllegalArgumentException e) {
}
try {
rp.enablePeer("bogus");
fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
} catch (IllegalArgumentException e) {
}
try {
rp.disablePeer("bogus");
fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
} catch (IllegalArgumentException e) {
}
try {
rp.getStatusOfPeer("bogus");
fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
} catch (IllegalArgumentException e) {
}
assertFalse(rp.peerAdded("bogus"));
rp.peerRemoved("bogus");
assertNull(rp.getPeerConf("bogus"));
assertNumberOfPeers(0);
// Add some peers
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
assertNumberOfPeers(1);
rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
assertNumberOfPeers(2);
// Test methods with a peer that is added but not connected
try {
rp.getStatusOfPeer(ID_ONE);
fail("There are no connected peers, should have thrown an IllegalArgumentException");
} catch (IllegalArgumentException e) {
}
assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
rp.removePeer(ID_ONE);
rp.peerRemoved(ID_ONE);
assertNumberOfPeers(1);
// Add one peer
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
rp.peerAdded(ID_ONE);
assertNumberOfPeers(2);
assertTrue(rp.getStatusOfPeer(ID_ONE));
rp.disablePeer(ID_ONE);
assertConnectedPeerStatus(false, ID_ONE);
rp.enablePeer(ID_ONE);
assertConnectedPeerStatus(true, ID_ONE);
// Disconnect peer
rp.peerRemoved(ID_ONE);
assertNumberOfPeers(2);
try {
rp.getStatusOfPeer(ID_ONE);
fail("There are no connected peers, should have thrown an IllegalArgumentException");
} catch (IllegalArgumentException e) {
}
}
protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
// we can first check if the value was changed in the store, if it wasn't then fail right away
if (status != rp.getStatusOfPeerFromBackingStore(peerId)) {
fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
}
while (true) {
if (status == rp.getStatusOfPeer(peerId)) {
return;
}
if (zkTimeoutCount < ZK_MAX_COUNT) {
LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status
+ ", sleeping and trying again.");
Thread.sleep(ZK_SLEEP_INTERVAL);
} else {
fail("Timed out waiting for ConnectedPeerStatus to be " + status);
}
}
}
protected void assertNumberOfPeers(int total) {
assertEquals(total, rp.getAllPeerConfigs().size());
assertEquals(total, rp.getAllPeerIds().size());
assertEquals(total, rp.getAllPeerIds().size());
}
/*
* three replicators: rq1 has 0 queues, rq2 has 1 queue with no logs, rq3 has 5 queues with 1, 2,
* 3, 4, 5 log files respectively
*/
protected void populateQueues() throws ReplicationException {
rq1.addLog("trash", "trash");
rq1.removeQueue("trash");
rq2.addLog("qId1", "trash");
rq2.removeLog("qId1", "trash");
for (int i = 1; i < 6; i++) {
for (int j = 0; j < i; j++) {
rq3.addLog("qId" + i, "filename" + j);
}
//Add peers for the corresponding queues so they are not orphans
rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
}
}
}