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