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