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.*;
23  import static org.junit.Assert.assertEquals;
24  import static org.junit.Assert.assertTrue;
25  import static org.junit.Assert.fail;
26  
27  import java.io.IOException;
28  import java.util.Iterator;
29  import java.util.List;
30  import java.util.NavigableSet;
31  import java.util.TreeSet;
32  import java.util.concurrent.atomic.AtomicLong;
33  import java.util.concurrent.Executors;
34  import java.util.concurrent.ExecutorService;
35  import java.util.concurrent.Future;
36  import java.util.concurrent.TimeoutException;
37  import java.util.concurrent.TimeUnit;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.fs.FSDataOutputStream;
43  import org.apache.hadoop.fs.FileStatus;
44  import org.apache.hadoop.fs.FileSystem;
45  import org.apache.hadoop.fs.Path;
46  import org.apache.hadoop.hbase.*;
47  import org.apache.hadoop.hbase.client.HTable;
48  import org.apache.hadoop.hbase.client.Put;
49  import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
50  import org.apache.hadoop.hbase.regionserver.HRegion;
51  import org.apache.hadoop.hbase.regionserver.HRegionServer;
52  import org.apache.hadoop.hbase.regionserver.wal.HLog;
53  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
56  import org.apache.hadoop.hbase.util.FSUtils;
57  import org.apache.hadoop.hbase.util.Threads;
58  import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
59  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
60  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
61  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
62  import org.apache.log4j.Level;
63  import org.apache.log4j.Logger;
64  import org.apache.zookeeper.KeeperException;
65  import org.junit.After;
66  import org.junit.Test;
67  import org.junit.experimental.categories.Category;
68  
69  @Category(LargeTests.class)
70  public class TestDistributedLogSplitting {
71    private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
72    static {
73      Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
74    }
75  
76    // Start a cluster with 2 masters and 3 regionservers
77    final int NUM_MASTERS = 2;
78    final int NUM_RS = 6;
79  
80    MiniHBaseCluster cluster;
81    HMaster master;
82    Configuration conf;
83    HBaseTestingUtility TEST_UTIL;
84  
85    private void startCluster(int num_rs) throws Exception{
86      ZKSplitLog.Counters.resetCounters();
87      LOG.info("Starting cluster");
88      conf = HBaseConfiguration.create();
89      conf.getLong("hbase.splitlog.max.resubmit", 0);
90      // Make the failure test faster
91      conf.setInt("zookeeper.recovery.retry", 0);
92      TEST_UTIL = new HBaseTestingUtility(conf);
93      TEST_UTIL.startMiniCluster(NUM_MASTERS, num_rs);
94      cluster = TEST_UTIL.getHBaseCluster();
95      LOG.info("Waiting for active/ready master");
96      cluster.waitForActiveAndReadyMaster();
97      master = cluster.getMaster();
98      while (cluster.getLiveRegionServerThreads().size() < num_rs) {
99        Threads.sleep(1);
100     }
101   }
102 
103   @After
104   public void after() throws Exception {
105     TEST_UTIL.shutdownMiniCluster();
106   }
107 
108   @Test (timeout=300000)
109   public void testThreeRSAbort() throws Exception {
110     LOG.info("testThreeRSAbort");
111     final int NUM_REGIONS_TO_CREATE = 40;
112     final int NUM_ROWS_PER_REGION = 100;
113 
114     startCluster(NUM_RS); // NUM_RS=6.
115 
116     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
117         "distributed log splitting test", null);
118 
119     HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
120     populateDataInTable(NUM_ROWS_PER_REGION, "family");
121 
122 
123     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
124     assertEquals(NUM_RS, rsts.size());
125     rsts.get(0).getRegionServer().abort("testing");
126     rsts.get(1).getRegionServer().abort("testing");
127     rsts.get(2).getRegionServer().abort("testing");
128 
129     long start = EnvironmentEdgeManager.currentTimeMillis();
130     while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
131       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
132         assertTrue(false);
133       }
134       Thread.sleep(200);
135     }
136 
137     start = EnvironmentEdgeManager.currentTimeMillis();
138     while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
139       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
140         assertTrue(false);
141       }
142       Thread.sleep(200);
143     }
144 
145     assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
146         TEST_UTIL.countRows(ht));
147     ht.close();
148   }
149 
150   @Test (timeout=300000)
151   public void testRecoveredEdits() throws Exception {
152     LOG.info("testRecoveredEdits");
153     startCluster(NUM_RS);
154     final int NUM_LOG_LINES = 1000;
155     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
156     // turn off load balancing to prevent regions from moving around otherwise
157     // they will consume recovered.edits
158     master.balanceSwitch(false);
159     FileSystem fs = master.getMasterFileSystem().getFileSystem();
160 
161     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
162     
163     Path rootdir = FSUtils.getRootDir(conf);
164 
165     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
166         "table", "family", 40);
167     byte[] table = Bytes.toBytes("table");
168     List<HRegionInfo> regions = null;
169     HRegionServer hrs = null;
170     for (int i = 0; i < NUM_RS; i++) {
171       hrs = rsts.get(i).getRegionServer();
172       regions = hrs.getOnlineRegions();
173       if (regions.size() != 0) break;
174     }
175     final Path logDir = new Path(rootdir, HLog.getHLogDirectoryName(hrs
176         .getServerName().toString()));
177     
178     LOG.info("#regions = " + regions.size());
179     Iterator<HRegionInfo> it = regions.iterator();
180     while (it.hasNext()) {
181       HRegionInfo region = it.next();
182       if (region.isMetaTable()) {
183         it.remove();
184       }
185     }
186     makeHLog(hrs.getWAL(), regions, "table",
187         NUM_LOG_LINES, 100);
188 
189     slm.splitLogDistributed(logDir);
190 
191     int count = 0;
192     for (HRegionInfo hri : regions) {
193 
194       Path tdir = HTableDescriptor.getTableDir(rootdir, table);
195       Path editsdir =
196         HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
197         hri.getEncodedName()));
198       LOG.debug("checking edits dir " + editsdir);
199       FileStatus[] files = fs.listStatus(editsdir);
200       assertEquals(1, files.length);
201       int c = countHLog(files[0].getPath(), fs, conf);
202       count += c;
203       LOG.info(c + " edits in " + files[0].getPath());
204     }
205     assertEquals(NUM_LOG_LINES, count);
206   }
207 
208   /**
209    * The original intention of this test was to force an abort of a region
210    * server and to make sure that the failure path in the region servers is
211    * properly evaluated. But it is difficult to ensure that the region server
212    * doesn't finish the log splitting before it aborts. Also now, there is
213    * this code path where the master will preempt the region server when master
214    * detects that the region server has aborted.
215    * @throws Exception
216    */
217   @Test (timeout=300000)
218   public void testWorkerAbort() throws Exception {
219     LOG.info("testWorkerAbort");
220     startCluster(1);
221     final int NUM_LOG_LINES = 10000;
222     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
223     FileSystem fs = master.getMasterFileSystem().getFileSystem();
224 
225     final List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
226     HRegionServer hrs = rsts.get(0).getRegionServer();
227     Path rootdir = FSUtils.getRootDir(conf);
228     final Path logDir = new Path(rootdir,
229         HLog.getHLogDirectoryName(hrs.getServerName().toString()));
230 
231     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
232         "table", "family", 40);
233     makeHLog(hrs.getWAL(), hrs.getOnlineRegions(), "table",
234         NUM_LOG_LINES, 100);
235 
236     new Thread() {
237       public void run() {
238         waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
239         for (RegionServerThread rst : rsts) {
240           rst.getRegionServer().abort("testing");
241         }
242       }
243     }.start();
244     // slm.splitLogDistributed(logDir);
245     FileStatus[] logfiles = fs.listStatus(logDir);
246     TaskBatch batch = new TaskBatch();
247     slm.enqueueSplitTask(logfiles[0].getPath().toString(), batch);
248     //waitForCounter but for one of the 2 counters
249     long curt = System.currentTimeMillis();
250     long waitTime = 80000;
251     long endt = curt + waitTime;
252     while (curt < endt) {
253       if ((tot_wkr_task_resigned.get() + tot_wkr_task_err.get() + 
254           tot_wkr_final_transistion_failed.get() + tot_wkr_task_done.get() +
255           tot_wkr_preempt_task.get()) == 0) {
256         Thread.yield();
257         curt = System.currentTimeMillis();
258       } else {
259         assertEquals(1, (tot_wkr_task_resigned.get() + tot_wkr_task_err.get() + 
260             tot_wkr_final_transistion_failed.get() + tot_wkr_task_done.get() +
261             tot_wkr_preempt_task.get()));
262         return;
263       }
264     }
265     fail("none of the following counters went up in " + waitTime + 
266         " milliseconds - " +
267         "tot_wkr_task_resigned, tot_wkr_task_err, " +
268         "tot_wkr_final_transistion_failed, tot_wkr_task_done, " +
269         "tot_wkr_preempt_task");
270   }
271 
272   @Test(timeout=30000)
273   public void testDelayedDeleteOnFailure() throws Exception {
274     LOG.info("testDelayedDeleteOnFailure");
275     startCluster(1);
276     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
277     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
278     final Path logDir = new Path(FSUtils.getRootDir(conf), "x");
279     fs.mkdirs(logDir);
280     ExecutorService executor = null;
281     try {
282       final Path corruptedLogFile = new Path(logDir, "x");
283       FSDataOutputStream out;
284       out = fs.create(corruptedLogFile);
285       out.write(0);
286       out.write(Bytes.toBytes("corrupted bytes"));
287       out.close();
288       slm.ignoreZKDeleteForTesting = true;
289       executor = Executors.newSingleThreadExecutor();
290       Runnable runnable = new Runnable() {
291        @Override
292        public void run() {
293           try {
294             // since the logDir is a fake, corrupted one, so the split log worker
295             // will finish it quickly with error, and this call will fail and throw
296             // an IOException.
297             slm.splitLogDistributed(logDir);
298           } catch (IOException ioe) {
299             try {
300               assertTrue(fs.exists(corruptedLogFile));
301               // this call will block waiting for the task to be removed from the
302               // tasks map which is not going to happen since ignoreZKDeleteForTesting
303               // is set to true, until it is interrupted.
304               slm.splitLogDistributed(logDir);
305             } catch (IOException e) {
306               assertTrue(Thread.currentThread().isInterrupted());
307               return;
308             }
309             fail("did not get the expected IOException from the 2nd call");
310           }
311           fail("did not get the expected IOException from the 1st call");
312         }
313       };
314       Future<?> result = executor.submit(runnable);
315       try {
316         result.get(2000, TimeUnit.MILLISECONDS);
317       } catch (TimeoutException te) {
318         // it is ok, expected.
319       }
320       waitForCounter(tot_mgr_wait_for_zk_delete, 0, 1, 10000);
321       executor.shutdownNow();
322       executor = null;
323 
324       // make sure the runnable is finished with no exception thrown.
325       result.get();
326     } finally {
327       if (executor != null) {
328         // interrupt the thread in case the test fails in the middle.
329         // it has no effect if the thread is already terminated.
330         executor.shutdownNow();
331       }
332       fs.delete(logDir, true);
333     }
334   }
335 
336   HTable installTable(ZooKeeperWatcher zkw, String tname, String fname,
337       int nrs ) throws Exception {
338     // Create a table with regions
339     byte [] table = Bytes.toBytes(tname);
340     byte [] family = Bytes.toBytes(fname);
341     LOG.info("Creating table with " + nrs + " regions");
342     HTable ht = TEST_UTIL.createTable(table, family);
343     int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family, nrs);
344     assertEquals(nrs, numRegions);
345       LOG.info("Waiting for no more RIT\n");
346     blockUntilNoRIT(zkw, master);
347     // disable-enable cycle to get rid of table's dead regions left behind
348     // by createMultiRegions
349     LOG.debug("Disabling table\n");
350     TEST_UTIL.getHBaseAdmin().disableTable(table);
351     LOG.debug("Waiting for no more RIT\n");
352     blockUntilNoRIT(zkw, master);
353     NavigableSet<String> regions = getAllOnlineRegions(cluster);
354     LOG.debug("Verifying only catalog regions are assigned\n");
355     if (regions.size() != 2) {
356       for (String oregion : regions)
357         LOG.debug("Region still online: " + oregion);
358     }
359     assertEquals(2, regions.size());
360     LOG.debug("Enabling table\n");
361     TEST_UTIL.getHBaseAdmin().enableTable(table);
362     LOG.debug("Waiting for no more RIT\n");
363     blockUntilNoRIT(zkw, master);
364     LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
365     regions = getAllOnlineRegions(cluster);
366     assertEquals(numRegions + 2, regions.size());
367     return ht;
368   }
369 
370   void populateDataInTable(int nrows, String fname) throws Exception {
371     byte [] family = Bytes.toBytes(fname);
372 
373     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
374     assertEquals(NUM_RS, rsts.size());
375 
376     for (RegionServerThread rst : rsts) {
377       HRegionServer hrs = rst.getRegionServer();
378       List<HRegionInfo> hris = hrs.getOnlineRegions();
379       for (HRegionInfo hri : hris) {
380         if (hri.isMetaTable()) {
381           continue;
382         }
383         LOG.debug("adding data to rs = " + rst.getName() +
384             " region = "+ hri.getRegionNameAsString());
385         HRegion region = hrs.getOnlineRegion(hri.getRegionName());
386         assertTrue(region != null);
387         putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
388       }
389     }
390   }
391 
392   public void makeHLog(HLog log,
393       List<HRegionInfo> hris, String tname,
394       int num_edits, int edit_size) throws IOException {
395 
396     byte[] table = Bytes.toBytes(tname);
397     HTableDescriptor htd = new HTableDescriptor(tname);
398     byte[] value = new byte[edit_size];
399     for (int i = 0; i < edit_size; i++) {
400       value[i] = (byte)('a' + (i % 26));
401     }
402     int n = hris.size();
403     int[] counts = new int[n];
404     int j = 0;
405     if (n > 0) {
406       for (int i = 0; i < num_edits; i += 1) {
407         WALEdit e = new WALEdit();
408         byte [] row = Bytes.toBytes("r" + Integer.toString(i));
409         byte [] family = Bytes.toBytes("f");
410         byte [] qualifier = Bytes.toBytes("c" + Integer.toString(i));
411         e.add(new KeyValue(row, family, qualifier,
412             System.currentTimeMillis(), value));
413         j++;
414         log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd);
415         counts[j % n] += 1;
416       }
417     }
418     log.sync();
419     log.close();
420     for (int i = 0; i < n; i++) {
421       LOG.info("region " + hris.get(i).getRegionNameAsString() +
422           " has " + counts[i] + " edits");
423     }
424     return;
425   }
426 
427   private int countHLog(Path log, FileSystem fs, Configuration conf)
428   throws IOException {
429     int count = 0;
430     HLog.Reader in = HLog.getReader(fs, log, conf);
431     while (in.next() != null) {
432       count++;
433     }
434     return count;
435   }
436 
437   private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
438   throws KeeperException, InterruptedException {
439     ZKAssign.blockUntilNoRIT(zkw);
440     master.assignmentManager.waitUntilNoRegionsInTransition(60000);
441   }
442 
443   private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,
444       byte [] ...families)
445   throws IOException {
446     for(int i = 0; i < numRows; i++) {
447       Put put = new Put(Bytes.add(startRow, Bytes.toBytes(i)));
448       for(byte [] family : families) {
449         put.add(family, qf, null);
450       }
451       region.put(put);
452     }
453   }
454 
455   private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
456       throws IOException {
457     NavigableSet<String> online = new TreeSet<String>();
458     for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
459       for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
460         online.add(region.getRegionNameAsString());
461       }
462     }
463     return online;
464   }
465 
466   private void waitForCounter(AtomicLong ctr, long oldval, long newval,
467       long timems) {
468     long curt = System.currentTimeMillis();
469     long endt = curt + timems;
470     while (curt < endt) {
471       if (ctr.get() == oldval) {
472         Thread.yield();
473         curt = System.currentTimeMillis();
474       } else {
475         assertEquals(newval, ctr.get());
476         return;
477       }
478     }
479     assertTrue(false);
480   }
481 
482   @org.junit.Rule
483   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
484     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
485 }
486