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.regionserver;
20
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertTrue;
23
24 import java.net.URLEncoder;
25 import java.util.ArrayList;
26 import java.util.Collection;
27 import java.util.List;
28 import java.util.SortedMap;
29 import java.util.SortedSet;
30 import java.util.TreeSet;
31 import java.util.UUID;
32 import java.util.concurrent.CountDownLatch;
33 import java.util.concurrent.atomic.AtomicLong;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.FileSystem;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.TableName;
41 import org.apache.hadoop.hbase.ClusterId;
42 import org.apache.hadoop.hbase.HBaseConfiguration;
43 import org.apache.hadoop.hbase.HBaseTestingUtility;
44 import org.apache.hadoop.hbase.HColumnDescriptor;
45 import org.apache.hadoop.hbase.HConstants;
46 import org.apache.hadoop.hbase.HRegionInfo;
47 import org.apache.hadoop.hbase.HTableDescriptor;
48 import org.apache.hadoop.hbase.KeyValue;
49 import org.apache.hadoop.hbase.testclassification.MediumTests;
50 import org.apache.hadoop.hbase.Server;
51 import org.apache.hadoop.hbase.ServerName;
52 import org.apache.hadoop.hbase.catalog.CatalogTracker;
53 import org.apache.hadoop.hbase.regionserver.wal.HLog;
54 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
55 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
56 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
57 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
58 import org.apache.hadoop.hbase.replication.ReplicationFactory;
59 import org.apache.hadoop.hbase.replication.ReplicationPeers;
60 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
61 import org.apache.hadoop.hbase.replication.ReplicationQueues;
62 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
63 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
64 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
65 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
66 import org.apache.hadoop.hbase.util.Bytes;
67 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
68 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
69 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
70 import org.junit.After;
71 import org.junit.AfterClass;
72 import org.junit.Before;
73 import org.junit.BeforeClass;
74 import org.junit.Test;
75 import org.junit.experimental.categories.Category;
76
77 import com.google.common.collect.Sets;
78
79 @Category(MediumTests.class)
80 public class TestReplicationSourceManager {
81
82 private static final Log LOG =
83 LogFactory.getLog(TestReplicationSourceManager.class);
84
85 private static Configuration conf;
86
87 private static HBaseTestingUtility utility;
88
89 private static Replication replication;
90
91 private static ReplicationSourceManager manager;
92
93 private static ZooKeeperWatcher zkw;
94
95 private static HTableDescriptor htd;
96
97 private static HRegionInfo hri;
98
99 private static final byte[] r1 = Bytes.toBytes("r1");
100
101 private static final byte[] r2 = Bytes.toBytes("r2");
102
103 private static final byte[] f1 = Bytes.toBytes("f1");
104
105 private static final TableName test =
106 TableName.valueOf("test");
107
108 private static final String slaveId = "1";
109
110 private static FileSystem fs;
111
112 private static String logName;
113
114 private static Path oldLogDir;
115
116 private static Path logDir;
117
118 private static CountDownLatch latch;
119
120 private static List<String> files = new ArrayList<String>();
121
122 @BeforeClass
123 public static void setUpBeforeClass() throws Exception {
124
125 conf = HBaseConfiguration.create();
126 conf.set("replication.replicationsource.implementation",
127 ReplicationSourceDummy.class.getCanonicalName());
128 conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY,
129 HConstants.REPLICATION_ENABLE_DEFAULT);
130 utility = new HBaseTestingUtility(conf);
131 utility.startMiniZKCluster();
132
133 zkw = new ZooKeeperWatcher(conf, "test", null);
134 ZKUtil.createWithParents(zkw, "/hbase/replication");
135 ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1");
136 ZKUtil.setData(zkw, "/hbase/replication/peers/1",
137 Bytes.toBytes(conf.get(HConstants.ZOOKEEPER_QUORUM) + ":"
138 + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":/1"));
139 ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state");
140 ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state",
141 ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
142 ZKUtil.createWithParents(zkw, "/hbase/replication/state");
143 ZKUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
144
145 ZKClusterId.setClusterId(zkw, new ClusterId());
146 fs = FileSystem.get(conf);
147 oldLogDir = new Path(utility.getDataTestDir(),
148 HConstants.HREGION_OLDLOGDIR_NAME);
149 logDir = new Path(utility.getDataTestDir(),
150 HConstants.HREGION_LOGDIR_NAME);
151 replication = new Replication(new DummyServer(), fs, logDir, oldLogDir);
152 manager = replication.getReplicationManager();
153
154 logName = HConstants.HREGION_LOGDIR_NAME;
155
156 manager.addSource(slaveId);
157
158 htd = new HTableDescriptor(test);
159 HColumnDescriptor col = new HColumnDescriptor("f1");
160 col.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
161 htd.addFamily(col);
162 col = new HColumnDescriptor("f2");
163 col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
164 htd.addFamily(col);
165
166 hri = new HRegionInfo(htd.getTableName(), r1, r2);
167 }
168
169 @AfterClass
170 public static void tearDownAfterClass() throws Exception {
171 manager.join();
172 utility.shutdownMiniCluster();
173 }
174
175 @Before
176 public void setUp() throws Exception {
177 fs.delete(logDir, true);
178 fs.delete(oldLogDir, true);
179 }
180
181 @After
182 public void tearDown() throws Exception {
183 setUp();
184 }
185
186 @Test
187 public void testLogRoll() throws Exception {
188 long seq = 0;
189 long baseline = 1000;
190 long time = baseline;
191 KeyValue kv = new KeyValue(r1, f1, r1);
192 WALEdit edit = new WALEdit();
193 edit.add(kv);
194
195 List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
196 listeners.add(replication);
197 HLog hlog = HLogFactory.createHLog(fs, utility.getDataTestDir(), logName,
198 conf, listeners, URLEncoder.encode("regionserver:60020", "UTF8"));
199 final AtomicLong sequenceId = new AtomicLong(1);
200 manager.init();
201 HTableDescriptor htd = new HTableDescriptor();
202 htd.addFamily(new HColumnDescriptor(f1));
203
204 for(long i = 1; i < 101; i++) {
205 if(i > 1 && i % 20 == 0) {
206 hlog.rollWriter();
207 }
208 LOG.info(i);
209 HLogKey key = new HLogKey(hri.getRegionName(), test, seq++,
210 System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
211 hlog.append(hri, test, edit, System.currentTimeMillis(), htd, sequenceId);
212 }
213
214
215
216 LOG.info(baseline + " and " + time);
217 baseline += 101;
218 time = baseline;
219 LOG.info(baseline + " and " + time);
220
221 for (int i = 0; i < 3; i++) {
222 hlog.append(hri, test, edit, System.currentTimeMillis(), htd, sequenceId);
223 }
224
225 assertEquals(6, manager.getHLogs().get(slaveId).size());
226
227 hlog.rollWriter();
228
229 manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
230 "1", 0, false, false);
231
232 hlog.append(hri, test, edit, System.currentTimeMillis(), htd, sequenceId);
233
234 assertEquals(1, manager.getHLogs().size());
235
236
237
238 }
239
240 @Test
241 public void testClaimQueues() throws Exception {
242 LOG.debug("testNodeFailoverWorkerCopyQueuesFromRSUsingMulti");
243 conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
244 final Server server = new DummyServer("hostname0.example.org");
245 ReplicationQueues rq =
246 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
247 server);
248 rq.init(server.getServerName().toString());
249
250 files.add("log1");
251 files.add("log2");
252 for (String file : files) {
253 rq.addLog("1", file);
254 }
255
256 Server s1 = new DummyServer("dummyserver1.example.org");
257 Server s2 = new DummyServer("dummyserver2.example.org");
258 Server s3 = new DummyServer("dummyserver3.example.org");
259
260
261 DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(
262 server.getServerName().getServerName(), s1);
263 DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(
264 server.getServerName().getServerName(), s2);
265 DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(
266 server.getServerName().getServerName(), s3);
267
268 latch = new CountDownLatch(3);
269
270 w1.start();
271 w2.start();
272 w3.start();
273
274 int populatedMap = 0;
275
276 latch.await();
277 populatedMap += w1.isLogZnodesMapPopulated() + w2.isLogZnodesMapPopulated()
278 + w3.isLogZnodesMapPopulated();
279 assertEquals(1, populatedMap);
280 server.abort("", null);
281 }
282
283 @Test
284 public void testCleanupFailoverQueues() throws Exception {
285 final Server server = new DummyServer("hostname1.example.org");
286 ReplicationQueues rq =
287 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
288 server);
289 rq.init(server.getServerName().toString());
290
291 SortedSet<String> files = new TreeSet<String>();
292 files.add("log1");
293 files.add("log2");
294 for (String file : files) {
295 rq.addLog("1", file);
296 }
297 Server s1 = new DummyServer("dummyserver1.example.org");
298 ReplicationQueues rq1 =
299 ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
300 rq1.init(s1.getServerName().toString());
301 ReplicationPeers rp1 =
302 ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1);
303 rp1.init();
304 NodeFailoverWorker w1 =
305 manager.new NodeFailoverWorker(server.getServerName().getServerName(), rq1, rp1, new UUID(
306 new Long(1), new Long(2)));
307 w1.start();
308 w1.join(5000);
309 assertEquals(1, manager.getHlogsByIdRecoveredQueues().size());
310 String id = "1-" + server.getServerName().getServerName();
311 assertEquals(files, manager.getHlogsByIdRecoveredQueues().get(id));
312 manager.cleanOldLogs("log2", id, true);
313
314 assertEquals(Sets.newHashSet("log2"), manager.getHlogsByIdRecoveredQueues().get(id));
315 }
316
317 @Test
318 public void testNodeFailoverDeadServerParsing() throws Exception {
319 LOG.debug("testNodeFailoverDeadServerParsing");
320 conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
321 final Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com");
322 ReplicationQueues repQueues =
323 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), conf, server);
324 repQueues.init(server.getServerName().toString());
325
326 files.add("log1");
327 files.add("log2");
328 for (String file : files) {
329 repQueues.addLog("1", file);
330 }
331
332
333 Server s1 = new DummyServer("ip-10-8-101-114.ec2.internal");
334 Server s2 = new DummyServer("ec2-107-20-52-47.compute-1.amazonaws.com");
335 Server s3 = new DummyServer("ec2-23-20-187-167.compute-1.amazonaws.com");
336
337
338 ReplicationQueues rq1 =
339 ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
340 rq1.init(s1.getServerName().toString());
341 SortedMap<String, SortedSet<String>> testMap =
342 rq1.claimQueues(server.getServerName().getServerName());
343 ReplicationQueues rq2 =
344 ReplicationFactory.getReplicationQueues(s2.getZooKeeper(), s2.getConfiguration(), s2);
345 rq2.init(s2.getServerName().toString());
346 testMap = rq2.claimQueues(s1.getServerName().getServerName());
347 ReplicationQueues rq3 =
348 ReplicationFactory.getReplicationQueues(s3.getZooKeeper(), s3.getConfiguration(), s3);
349 rq3.init(s3.getServerName().toString());
350 testMap = rq3.claimQueues(s2.getServerName().getServerName());
351
352 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(testMap.firstKey());
353 List<String> result = replicationQueueInfo.getDeadRegionServers();
354
355
356 assertTrue(result.contains(server.getServerName().getServerName()));
357 assertTrue(result.contains(s1.getServerName().getServerName()));
358 assertTrue(result.contains(s2.getServerName().getServerName()));
359
360 server.abort("", null);
361 }
362
363 @Test
364 public void testFailoverDeadServerCversionChange() throws Exception {
365 LOG.debug("testFailoverDeadServerCversionChange");
366
367 conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
368 final Server s0 = new DummyServer("cversion-change0.example.org");
369 ReplicationQueues repQueues =
370 ReplicationFactory.getReplicationQueues(s0.getZooKeeper(), conf, s0);
371 repQueues.init(s0.getServerName().toString());
372
373 files.add("log1");
374 files.add("log2");
375 for (String file : files) {
376 repQueues.addLog("1", file);
377 }
378
379 Server s1 = new DummyServer("cversion-change1.example.org");
380 ReplicationQueues rq1 =
381 ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
382 rq1.init(s1.getServerName().toString());
383
384 ReplicationQueuesClient client =
385 ReplicationFactory.getReplicationQueuesClient(s1.getZooKeeper(), s1.getConfiguration(), s1);
386
387 int v0 = client.getQueuesZNodeCversion();
388 rq1.claimQueues(s0.getServerName().getServerName());
389 int v1 = client.getQueuesZNodeCversion();
390
391 assertEquals(v0 + 1, v1);
392
393 s0.abort("", null);
394 }
395
396 static class DummyNodeFailoverWorker extends Thread {
397 private SortedMap<String, SortedSet<String>> logZnodesMap;
398 Server server;
399 private String deadRsZnode;
400 ReplicationQueues rq;
401
402 public DummyNodeFailoverWorker(String znode, Server s) throws Exception {
403 this.deadRsZnode = znode;
404 this.server = s;
405 this.rq =
406 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
407 server);
408 this.rq.init(this.server.getServerName().toString());
409 }
410
411 @Override
412 public void run() {
413 try {
414 logZnodesMap = rq.claimQueues(deadRsZnode);
415 server.abort("Done with testing", null);
416 } catch (Exception e) {
417 LOG.error("Got exception while running NodeFailoverWorker", e);
418 } finally {
419 latch.countDown();
420 }
421 }
422
423
424
425
426 private int isLogZnodesMapPopulated() {
427 Collection<SortedSet<String>> sets = logZnodesMap.values();
428 if (sets.size() > 1) {
429 throw new RuntimeException("unexpected size of logZnodesMap: " + sets.size());
430 }
431 if (sets.size() == 1) {
432 SortedSet<String> s = sets.iterator().next();
433 for (String file : files) {
434
435 if (!s.contains(file)) {
436 return 0;
437 }
438 }
439 return 1;
440 }
441 return 0;
442 }
443 }
444
445 static class DummyServer implements Server {
446 String hostname;
447
448 DummyServer() {
449 hostname = "hostname.example.org";
450 }
451
452 DummyServer(String hostname) {
453 this.hostname = hostname;
454 }
455
456 @Override
457 public Configuration getConfiguration() {
458 return conf;
459 }
460
461 @Override
462 public ZooKeeperWatcher getZooKeeper() {
463 return zkw;
464 }
465
466 @Override
467 public CatalogTracker getCatalogTracker() {
468 return null;
469 }
470
471 @Override
472 public ServerName getServerName() {
473 return ServerName.valueOf(hostname, 1234, 1L);
474 }
475
476 @Override
477 public void abort(String why, Throwable e) {
478
479 }
480
481 @Override
482 public boolean isAborted() {
483 return false;
484 }
485
486 @Override
487 public void stop(String why) {
488
489 }
490
491 @Override
492 public boolean isStopped() {
493 return false;
494 }
495 }
496
497 }
498