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