1   /**
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_final_transistion_failed;
23  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
24  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_err;
25  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_resigned;
26  import static org.junit.Assert.assertEquals;
27  import static org.junit.Assert.assertTrue;
28  
29  import java.io.IOException;
30  import java.util.Iterator;
31  import java.util.List;
32  import java.util.NavigableSet;
33  import java.util.TreeSet;
34  import java.util.concurrent.atomic.AtomicLong;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.fs.FileStatus;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.HBaseConfiguration;
43  import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MiniHBaseCluster;
48  import org.apache.hadoop.hbase.client.HTable;
49  import org.apache.hadoop.hbase.client.Put;
50  import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
51  import org.apache.hadoop.hbase.regionserver.HRegion;
52  import org.apache.hadoop.hbase.regionserver.HRegionServer;
53  import org.apache.hadoop.hbase.regionserver.wal.HLog;
54  import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException;
55  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
56  import org.apache.hadoop.hbase.util.Bytes;
57  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
58  import org.apache.hadoop.hbase.util.FSUtils;
59  import org.apache.hadoop.hbase.util.Threads;
60  import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
61  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
62  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
63  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
64  import org.apache.log4j.Level;
65  import org.apache.log4j.Logger;
66  import org.apache.zookeeper.KeeperException;
67  import org.junit.After;
68  import org.junit.Before;
69  import org.junit.Test;
70  
71  public class TestDistributedLogSplitting {
72    private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
73    static {
74      Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
75    }
76  
77    // Start a cluster with 2 masters and 3 regionservers
78    final int NUM_MASTERS = 2;
79    final int NUM_RS = 6;
80  
81    MiniHBaseCluster cluster;
82    HMaster master;
83    Configuration conf;
84    HBaseTestingUtility TEST_UTIL;
85  
86    private void startCluster(int num_rs) throws Exception{
87      ZKSplitLog.Counters.resetCounters();
88      LOG.info("Starting cluster");
89      conf = HBaseConfiguration.create();
90      conf.getLong("hbase.splitlog.max.resubmit", 0);
91      TEST_UTIL = new HBaseTestingUtility(conf);
92      TEST_UTIL.startMiniCluster(NUM_MASTERS, num_rs);
93      cluster = TEST_UTIL.getHBaseCluster();
94      LOG.info("Waiting for active/ready master");
95      cluster.waitForActiveAndReadyMaster();
96      master = cluster.getMaster();
97      while (cluster.getLiveRegionServerThreads().size() < num_rs) {
98        Threads.sleep(1);
99      }
100   }
101 
102   @After
103   public void after() throws Exception {
104     TEST_UTIL.shutdownMiniCluster();
105   }
106 
107   @Test (timeout=300000)
108   public void testThreeRSAbort() throws Exception {
109     LOG.info("testThreeRSAbort");
110     final int NUM_REGIONS_TO_CREATE = 40;
111     final int NUM_ROWS_PER_REGION = 100;
112 
113     startCluster(NUM_RS); // NUM_RS=6.
114 
115     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
116         "distributed log splitting test", null);
117 
118     HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
119     populateDataInTable(NUM_ROWS_PER_REGION, "family");
120 
121 
122     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
123     assertEquals(NUM_RS, rsts.size());
124     rsts.get(0).getRegionServer().abort("testing");
125     rsts.get(1).getRegionServer().abort("testing");
126     rsts.get(2).getRegionServer().abort("testing");
127 
128     long start = EnvironmentEdgeManager.currentTimeMillis();
129     while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
130       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
131         assertTrue(false);
132       }
133       Thread.sleep(200);
134     }
135 
136     start = EnvironmentEdgeManager.currentTimeMillis();
137     while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
138       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
139         assertTrue(false);
140       }
141       Thread.sleep(200);
142     }
143 
144     assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
145         TEST_UTIL.countRows(ht));
146   }
147 
148   @Test(expected=OrphanHLogAfterSplitException.class, timeout=300000)
149   public void testOrphanLogCreation() throws Exception {
150     LOG.info("testOrphanLogCreation");
151     startCluster(NUM_RS);
152     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
153     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
154 
155     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
156     HRegionServer hrs = rsts.get(0).getRegionServer();
157     Path rootdir = FSUtils.getRootDir(conf);
158     final Path logDir = new Path(rootdir,
159         HLog.getHLogDirectoryName(hrs.getServerName().toString()));
160 
161     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
162         "table", "family", 40);
163 
164     makeHLog(hrs.getWAL(), hrs.getOnlineRegions(), "table",
165         1000, 100);
166 
167     new Thread() {
168       public void run() {
169         while (true) {
170           int i = 0;
171           try {
172             while(ZKSplitLog.Counters.tot_mgr_log_split_batch_start.get() ==
173               0) {
174               Thread.yield();
175             }
176             fs.createNewFile(new Path(logDir, "foo" + i++));
177           } catch (Exception e) {
178             LOG.debug("file creation failed", e);
179             return;
180           }
181         }
182       }
183     }.start();
184     slm.splitLogDistributed(logDir);
185     FileStatus[] files = fs.listStatus(logDir);
186     if (files != null) {
187       for (FileStatus file : files) {
188         LOG.debug("file still there " + file.getPath());
189       }
190     }
191   }
192 
193   @Test (timeout=300000)
194   public void testRecoveredEdits() throws Exception {
195     LOG.info("testRecoveredEdits");
196     startCluster(NUM_RS);
197     final int NUM_LOG_LINES = 1000;
198     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
199     // turn off load balancing to prevent regions from moving around otherwise
200     // they will consume recovered.edits
201     master.balanceSwitch(false);
202     FileSystem fs = master.getMasterFileSystem().getFileSystem();
203 
204     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
205     
206     Path rootdir = FSUtils.getRootDir(conf);
207 
208     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
209         "table", "family", 40);
210     byte[] table = Bytes.toBytes("table");
211     List<HRegionInfo> regions = null;
212     HRegionServer hrs = null;
213     for (int i = 0; i < NUM_RS; i++) {
214       hrs = rsts.get(i).getRegionServer();
215       regions = hrs.getOnlineRegions();
216       if (regions.size() != 0) break;
217     }
218     final Path logDir = new Path(rootdir, HLog.getHLogDirectoryName(hrs
219         .getServerName().toString()));
220     
221     LOG.info("#regions = " + regions.size());
222     Iterator<HRegionInfo> it = regions.iterator();
223     while (it.hasNext()) {
224       HRegionInfo region = it.next();
225       if (region.isMetaTable()) {
226         it.remove();
227       }
228     }
229     makeHLog(hrs.getWAL(), regions, "table",
230         NUM_LOG_LINES, 100);
231 
232     slm.splitLogDistributed(logDir);
233 
234     int count = 0;
235     for (HRegionInfo hri : regions) {
236 
237       Path tdir = HTableDescriptor.getTableDir(rootdir, table);
238       Path editsdir =
239         HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
240         hri.getEncodedName()));
241       LOG.debug("checking edits dir " + editsdir);
242       FileStatus[] files = fs.listStatus(editsdir);
243       assertEquals(1, files.length);
244       int c = countHLog(files[0].getPath(), fs, conf);
245       count += c;
246       LOG.info(c + " edits in " + files[0].getPath());
247     }
248     assertEquals(NUM_LOG_LINES, count);
249   }
250 
251   @Test (timeout=300000)
252   public void testWorkerAbort() throws Exception {
253     LOG.info("testWorkerAbort");
254     startCluster(1);
255     final int NUM_LOG_LINES = 10000;
256     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
257     FileSystem fs = master.getMasterFileSystem().getFileSystem();
258 
259     final List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
260     HRegionServer hrs = rsts.get(0).getRegionServer();
261     Path rootdir = FSUtils.getRootDir(conf);
262     final Path logDir = new Path(rootdir,
263         HLog.getHLogDirectoryName(hrs.getServerName().toString()));
264 
265     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
266         "table", "family", 40);
267     makeHLog(hrs.getWAL(), hrs.getOnlineRegions(), "table",
268         NUM_LOG_LINES, 100);
269 
270     new Thread() {
271       public void run() {
272         waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
273         for (RegionServerThread rst : rsts) {
274           rst.getRegionServer().abort("testing");
275         }
276       }
277     }.start();
278     // slm.splitLogDistributed(logDir);
279     FileStatus[] logfiles = fs.listStatus(logDir);
280     TaskBatch batch = new TaskBatch();
281     slm.installTask(logfiles[0].getPath().toString(), batch);
282     //waitForCounter but for one of the 2 counters
283     long curt = System.currentTimeMillis();
284     long endt = curt + 30000;
285     while (curt < endt) {
286       if ((tot_wkr_task_resigned.get() + tot_wkr_task_err.get() + 
287           tot_wkr_final_transistion_failed.get()) == 0) {
288         Thread.yield();
289         curt = System.currentTimeMillis();
290       } else {
291         assertEquals(1, (tot_wkr_task_resigned.get() + tot_wkr_task_err.get() +
292             tot_wkr_final_transistion_failed.get()));
293         return;
294       }
295     }
296     assertEquals(1, batch.done);
297     // fail("region server completed the split before aborting");
298     return;
299   }
300 
301   HTable installTable(ZooKeeperWatcher zkw, String tname, String fname,
302       int nrs ) throws Exception {
303     // Create a table with regions
304     byte [] table = Bytes.toBytes(tname);
305     byte [] family = Bytes.toBytes(fname);
306     LOG.info("Creating table with " + nrs + " regions");
307     HTable ht = TEST_UTIL.createTable(table, family);
308     int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family, nrs);
309     assertEquals(nrs, numRegions);
310       LOG.info("Waiting for no more RIT\n");
311     blockUntilNoRIT(zkw, master);
312     // disable-enable cycle to get rid of table's dead regions left behind
313     // by createMultiRegions
314     LOG.debug("Disabling table\n");
315     TEST_UTIL.getHBaseAdmin().disableTable(table);
316     LOG.debug("Waiting for no more RIT\n");
317     blockUntilNoRIT(zkw, master);
318     NavigableSet<String> regions = getAllOnlineRegions(cluster);
319     LOG.debug("Verifying only catalog regions are assigned\n");
320     if (regions.size() != 2) {
321       for (String oregion : regions)
322         LOG.debug("Region still online: " + oregion);
323     }
324     assertEquals(2, regions.size());
325     LOG.debug("Enabling table\n");
326     TEST_UTIL.getHBaseAdmin().enableTable(table);
327     LOG.debug("Waiting for no more RIT\n");
328     blockUntilNoRIT(zkw, master);
329     LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
330     regions = getAllOnlineRegions(cluster);
331     assertEquals(numRegions + 2, regions.size());
332     return ht;
333   }
334 
335   void populateDataInTable(int nrows, String fname) throws Exception {
336     byte [] family = Bytes.toBytes(fname);
337 
338     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
339     assertEquals(NUM_RS, rsts.size());
340 
341     for (RegionServerThread rst : rsts) {
342       HRegionServer hrs = rst.getRegionServer();
343       List<HRegionInfo> hris = hrs.getOnlineRegions();
344       for (HRegionInfo hri : hris) {
345         if (hri.isMetaTable()) {
346           continue;
347         }
348         LOG.debug("adding data to rs = " + rst.getName() +
349             " region = "+ hri.getRegionNameAsString());
350         HRegion region = hrs.getOnlineRegion(hri.getRegionName());
351         assertTrue(region != null);
352         putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
353       }
354     }
355   }
356 
357   public void makeHLog(HLog log,
358       List<HRegionInfo> hris, String tname,
359       int num_edits, int edit_size) throws IOException {
360 
361     byte[] table = Bytes.toBytes(tname);
362     HTableDescriptor htd = new HTableDescriptor(tname);
363     byte[] value = new byte[edit_size];
364     for (int i = 0; i < edit_size; i++) {
365       value[i] = (byte)('a' + (i % 26));
366     }
367     int n = hris.size();
368     int[] counts = new int[n];
369     int j = 0;
370     if (n > 0) {
371       for (int i = 0; i < num_edits; i += 1) {
372         WALEdit e = new WALEdit();
373         byte [] row = Bytes.toBytes("r" + Integer.toString(i));
374         byte [] family = Bytes.toBytes("f");
375         byte [] qualifier = Bytes.toBytes("c" + Integer.toString(i));
376         e.add(new KeyValue(row, family, qualifier,
377             System.currentTimeMillis(), value));
378         // LOG.info("Region " + i + ": " + e);
379         j++;
380         log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd);
381         counts[j % n] += 1;
382         // if ((i % 8096) == 0) {
383         // log.sync();
384         //  }
385       }
386     }
387     log.sync();
388     log.close();
389     for (int i = 0; i < n; i++) {
390       LOG.info("region " + hris.get(i).getRegionNameAsString() +
391           " has " + counts[i] + " edits");
392     }
393     return;
394   }
395 
396   private int countHLog(Path log, FileSystem fs, Configuration conf)
397   throws IOException {
398     int count = 0;
399     HLog.Reader in = HLog.getReader(fs, log, conf);
400     while (in.next() != null) {
401       count++;
402     }
403     return count;
404   }
405 
406   private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
407   throws KeeperException, InterruptedException {
408     ZKAssign.blockUntilNoRIT(zkw);
409     master.assignmentManager.waitUntilNoRegionsInTransition(60000);
410   }
411 
412   private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,
413       byte [] ...families)
414   throws IOException {
415     for(int i = 0; i < numRows; i++) {
416       Put put = new Put(Bytes.add(startRow, Bytes.toBytes(i)));
417       for(byte [] family : families) {
418         put.add(family, qf, null);
419       }
420       region.put(put);
421     }
422   }
423 
424   private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
425       throws IOException {
426     NavigableSet<String> online = new TreeSet<String>();
427     for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
428       for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
429         online.add(region.getRegionNameAsString());
430       }
431     }
432     return online;
433   }
434 
435   private void waitForCounter(AtomicLong ctr, long oldval, long newval,
436       long timems) {
437     long curt = System.currentTimeMillis();
438     long endt = curt + timems;
439     while (curt < endt) {
440       if (ctr.get() == oldval) {
441         Thread.yield();
442         curt = System.currentTimeMillis();
443       } else {
444         assertEquals(newval, ctr.get());
445         return;
446       }
447     }
448     assertTrue(false);
449   }
450 }