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