1   /**
2    * Copyright 2010 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.regionserver.wal;
21  
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertTrue;
24  import static org.junit.Assert.fail;
25  
26  import java.io.IOException;
27  import java.security.PrivilegedExceptionAction;
28  import java.util.ArrayList;
29  import java.util.List;
30  import java.util.SortedSet;
31  import java.util.concurrent.atomic.AtomicBoolean;
32  import java.util.concurrent.atomic.AtomicInteger;
33  import java.util.concurrent.atomic.AtomicLong;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FileStatus;
39  import org.apache.hadoop.fs.FileSystem;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.hbase.HBaseConfiguration;
42  import org.apache.hadoop.hbase.HBaseTestingUtility;
43  import org.apache.hadoop.hbase.HColumnDescriptor;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HTableDescriptor;
47  import org.apache.hadoop.hbase.KeyValue;
48  import org.apache.hadoop.hbase.MasterNotRunningException;
49  import org.apache.hadoop.hbase.MediumTests;
50  import org.apache.hadoop.hbase.MiniHBaseCluster;
51  import org.apache.hadoop.hbase.ServerName;
52  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
53  import org.apache.hadoop.hbase.client.Delete;
54  import org.apache.hadoop.hbase.client.Get;
55  import org.apache.hadoop.hbase.client.HTable;
56  import org.apache.hadoop.hbase.client.Put;
57  import org.apache.hadoop.hbase.client.Result;
58  import org.apache.hadoop.hbase.client.ResultScanner;
59  import org.apache.hadoop.hbase.client.Scan;
60  import org.apache.hadoop.hbase.master.HMaster;
61  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
62  import org.apache.hadoop.hbase.regionserver.FlushRequester;
63  import org.apache.hadoop.hbase.regionserver.HRegion;
64  import org.apache.hadoop.hbase.regionserver.HRegionServer;
65  import org.apache.hadoop.hbase.regionserver.RegionScanner;
66  import org.apache.hadoop.hbase.regionserver.RegionServerServices;
67  import org.apache.hadoop.hbase.regionserver.Store;
68  import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
69  import org.apache.hadoop.hbase.security.User;
70  import org.apache.hadoop.hbase.util.Bytes;
71  import org.apache.hadoop.hbase.util.EnvironmentEdge;
72  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
73  import org.apache.hadoop.hbase.util.HFileTestUtil;
74  import org.apache.hadoop.hbase.util.Pair;
75  import org.junit.After;
76  import org.junit.AfterClass;
77  import org.junit.Before;
78  import org.junit.BeforeClass;
79  import org.junit.Test;
80  import org.junit.experimental.categories.Category;
81  import org.mockito.Mockito;
82  
83  /**
84   * Test replay of edits out of a WAL split.
85   */
86  @Category(MediumTests.class)
87  public class TestWALReplay {
88    public static final Log LOG = LogFactory.getLog(TestWALReplay.class);
89    static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
90    private final EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
91    private Path hbaseRootDir = null;
92    private Path oldLogDir;
93    private Path logDir;
94    private FileSystem fs;
95    private Configuration conf;
96  
97    @BeforeClass
98    public static void setUpBeforeClass() throws Exception {
99      Configuration conf = TEST_UTIL.getConfiguration();
100     conf.setBoolean("dfs.support.append", true);
101     // The below config supported by 0.20-append and CDH3b2
102     conf.setInt("dfs.client.block.recovery.retries", 2);
103     TEST_UTIL.startMiniCluster(3);
104     Path hbaseRootDir =
105       TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
106     LOG.info("hbase.rootdir=" + hbaseRootDir);
107     conf.set(HConstants.HBASE_DIR, hbaseRootDir.toString());
108   }
109 
110   @AfterClass
111   public static void tearDownAfterClass() throws Exception {
112     TEST_UTIL.shutdownMiniCluster();
113   }
114 
115   @Before
116   public void setUp() throws Exception {
117     this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
118     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
119     this.hbaseRootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
120     this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
121     this.logDir = new Path(this.hbaseRootDir, HConstants.HREGION_LOGDIR_NAME);
122     if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
123       TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
124     }
125   }
126 
127   @After
128   public void tearDown() throws Exception {
129     TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
130   }
131 
132   /*
133    * @param p Directory to cleanup
134    */
135   private void deleteDir(final Path p) throws IOException {
136     if (this.fs.exists(p)) {
137       if (!this.fs.delete(p, true)) {
138         throw new IOException("Failed remove of " + p);
139       }
140     }
141   }
142 
143   /**
144    * 
145    * @throws Exception
146    */
147   @Test
148   public void testReplayEditsAfterRegionMovedWithMultiCF() throws Exception {
149     final byte[] tableName = Bytes
150         .toBytes("testReplayEditsAfterRegionMovedWithMultiCF");
151     byte[] family1 = Bytes.toBytes("cf1");
152     byte[] family2 = Bytes.toBytes("cf2");
153     byte[] qualifier = Bytes.toBytes("q");
154     byte[] value = Bytes.toBytes("testV");
155     byte[][] familys = { family1, family2 };
156     TEST_UTIL.createTable(tableName, familys);
157     HTable htable = new HTable(TEST_UTIL.getConfiguration(), tableName);
158     Put put = new Put(Bytes.toBytes("r1"));
159     put.add(family1, qualifier, value);
160     htable.put(put);
161     ResultScanner resultScanner = htable.getScanner(new Scan());
162     int count = 0;
163     while (resultScanner.next() != null) {
164       count++;
165     }
166     resultScanner.close();
167     assertEquals(1, count);
168 
169     MiniHBaseCluster hbaseCluster = TEST_UTIL.getMiniHBaseCluster();
170     List<HRegion> regions = hbaseCluster.getRegions(tableName);
171     assertEquals(1, regions.size());
172 
173     // move region to another regionserver
174     HRegion destRegion = regions.get(0);
175     int originServerNum = hbaseCluster
176         .getServerWith(destRegion.getRegionName());
177     assertTrue("Please start more than 1 regionserver", hbaseCluster
178         .getRegionServerThreads().size() > 1);
179     int destServerNum = 0;
180     while (destServerNum == originServerNum) {
181       destServerNum++;
182     }
183     HRegionServer originServer = hbaseCluster.getRegionServer(originServerNum);
184     HRegionServer destServer = hbaseCluster.getRegionServer(destServerNum);
185     // move region to destination regionserver
186     moveRegionAndWait(destRegion, destServer);
187 
188     // delete the row
189     Delete del = new Delete(Bytes.toBytes("r1"));
190     htable.delete(del);
191     resultScanner = htable.getScanner(new Scan());
192     count = 0;
193     while (resultScanner.next() != null) {
194       count++;
195     }
196     resultScanner.close();
197     assertEquals(0, count);
198 
199     // flush region and make major compaction
200     destServer.getOnlineRegion(destRegion.getRegionName()).flushcache();
201     // wait to complete major compaction
202     for (Store store : destServer.getOnlineRegion(destRegion.getRegionName())
203         .getStores().values()) {
204       store.triggerMajorCompaction();
205     }
206     destServer.getOnlineRegion(destRegion.getRegionName()).compactStores();
207 
208     // move region to origin regionserver
209     moveRegionAndWait(destRegion, originServer);
210     // abort the origin regionserver
211     originServer.abort("testing");
212 
213     // see what we get
214     Result result = htable.get(new Get(Bytes.toBytes("r1")));
215     if (result != null) {
216       assertTrue("Row is deleted, but we get" + result.toString(),
217           (result == null) || result.isEmpty());
218     }
219     resultScanner.close();
220   }
221 
222   private void moveRegionAndWait(HRegion destRegion, HRegionServer destServer)
223       throws InterruptedException, MasterNotRunningException,
224       ZooKeeperConnectionException, IOException {
225     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
226     TEST_UTIL.getHBaseAdmin().move(
227         destRegion.getRegionInfo().getEncodedNameAsBytes(),
228         Bytes.toBytes(destServer.getServerName().getServerName()));
229     while (true) {
230       ServerName serverName = master.getAssignmentManager()
231           .getRegionServerOfRegion(destRegion.getRegionInfo());
232       if (serverName != null && serverName.equals(destServer.getServerName())) break;
233       Thread.sleep(10);
234     }
235   }
236 
237   /**
238    * Tests for hbase-2727.
239    * @throws Exception
240    * @see https://issues.apache.org/jira/browse/HBASE-2727
241    */
242   @Test
243   public void test2727() throws Exception {
244     // Test being able to have > 1 set of edits in the recovered.edits directory.
245     // Ensure edits are replayed properly.
246     final String tableNameStr = "test2727";
247     HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
248     Path basedir = new Path(hbaseRootDir, tableNameStr);
249     deleteDir(basedir);
250     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
251 
252     HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
253     HRegion region2 = HRegion.createHRegion(hri,
254         hbaseRootDir, this.conf, htd);
255     region2.close();
256     region2.getLog().closeAndDelete();
257     final byte [] tableName = Bytes.toBytes(tableNameStr);
258     final byte [] rowName = tableName;
259 
260     HLog wal1 = createWAL(this.conf);
261     // Add 1k to each family.
262     final int countPerFamily = 1000;
263     for (HColumnDescriptor hcd: htd.getFamilies()) {
264       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee,
265           wal1, htd);
266     }
267     wal1.close();
268     runWALSplit(this.conf);
269 
270     HLog wal2 = createWAL(this.conf);
271     // Up the sequenceid so that these edits are after the ones added above.
272     wal2.setSequenceNumber(wal1.getSequenceNumber());
273     // Add 1k to each family.
274     for (HColumnDescriptor hcd: htd.getFamilies()) {
275       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
276           ee, wal2, htd);
277     }
278     wal2.close();
279     runWALSplit(this.conf);
280 
281     HLog wal3 = createWAL(this.conf);
282     wal3.setSequenceNumber(wal2.getSequenceNumber());
283     try {
284       final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri,
285         htd, null);
286       long seqid = region.initialize();
287       assertTrue(seqid > wal3.getSequenceNumber());
288 
289       // TODO: Scan all.
290       region.close();
291     } finally {
292       wal3.closeAndDelete();
293     }
294   }
295 
296   /**
297    * Test case of HRegion that is only made out of bulk loaded files.  Assert
298    * that we don't 'crash'.
299    * @throws IOException
300    * @throws IllegalAccessException
301    * @throws NoSuchFieldException
302    * @throws IllegalArgumentException
303    * @throws SecurityException
304    */
305   @Test
306   public void testRegionMadeOfBulkLoadedFilesOnly()
307   throws IOException, SecurityException, IllegalArgumentException,
308       NoSuchFieldException, IllegalAccessException, InterruptedException {
309     final String tableNameStr = "testRegionMadeOfBulkLoadedFilesOnly";
310     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
311     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
312     deleteDir(basedir);
313     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
314     HRegion region2 = HRegion.createHRegion(hri,
315         hbaseRootDir, this.conf, htd);
316     region2.close();
317     region2.getLog().closeAndDelete();
318     HLog wal = createWAL(this.conf);
319     HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
320 
321     byte [] family = htd.getFamilies().iterator().next().getName();
322     Path f =  new Path(basedir, "hfile");
323     HFileTestUtil.createHFile(this.conf, fs, f, family, family, Bytes.toBytes(""),
324         Bytes.toBytes("z"), 10);
325     List <Pair<byte[],String>>  hfs= new ArrayList<Pair<byte[],String>>(1);
326     hfs.add(Pair.newPair(family, f.toString()));
327     region.bulkLoadHFiles(hfs, true);
328 
329     // Add an edit so something in the WAL
330     byte [] row = Bytes.toBytes(tableNameStr);
331     region.put((new Put(row)).add(family, family, family));
332     wal.sync();
333     final int rowsInsertedCount = 11;
334 
335     assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan())));
336 
337     // Now 'crash' the region by stealing its wal
338     final Configuration newConf = HBaseConfiguration.create(this.conf);
339     User user = HBaseTestingUtility.getDifferentUser(newConf,
340         tableNameStr);
341     user.runAs(new PrivilegedExceptionAction() {
342       public Object run() throws Exception {
343         runWALSplit(newConf);
344         HLog wal2 = createWAL(newConf);
345         HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
346           newConf, hri, htd, null);
347         long seqid2 = region2.initialize();
348         assertTrue(seqid2 > -1);
349         assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
350 
351         // I can't close wal1.  Its been appropriated when we split.
352         region2.close();
353         wal2.closeAndDelete();
354         return null;
355       }
356     });
357   }
358 
359   /**
360    * HRegion test case that is made of a major compacted HFile (created with three bulk loaded
361    * files) and an edit in the memstore.
362    * This is for HBASE-10958 "[dataloss] Bulk loading with seqids can prevent some log entries
363    * from being replayed"
364    * @throws IOException
365    * @throws IllegalAccessException
366    * @throws NoSuchFieldException
367    * @throws IllegalArgumentException
368    * @throws SecurityException
369    */
370   @Test
371   public void testCompactedBulkLoadedFiles()
372       throws IOException, SecurityException, IllegalArgumentException,
373       NoSuchFieldException, IllegalAccessException, InterruptedException {
374     final String tableNameStr = "testCompactedBulkLoadedFiles";
375     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
376     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
377     deleteDir(basedir);
378     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
379     HRegion region2 = HRegion.createHRegion(hri,
380         hbaseRootDir, this.conf, htd);
381     region2.close();
382     region2.getLog().closeAndDelete();
383     HLog wal = createWAL(this.conf);
384     HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
385 
386     // Add an edit so something in the WAL
387     byte [] row = Bytes.toBytes(tableNameStr);
388     byte [] family = htd.getFamilies().iterator().next().getName();
389     region.put((new Put(row)).add(family, family, family));
390     wal.sync();
391 
392     List <Pair<byte[],String>>  hfs= new ArrayList<Pair<byte[],String>>(1);
393     for (int i = 0; i < 3; i++) {
394       Path f = new Path(basedir, "hfile"+i);
395       HFileTestUtil.createHFile(this.conf, fs, f, family, family, Bytes.toBytes(i + "00"),
396           Bytes.toBytes(i + "50"), 10);
397       hfs.add(Pair.newPair(family, f.toString()));
398     }
399     region.bulkLoadHFiles(hfs, true);
400     final int rowsInsertedCount = 31;
401     assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan())));
402 
403     // major compact to turn all the bulk loaded files into one normal file
404     region.compactStores(true);
405     assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan())));
406 
407     // Now 'crash' the region by stealing its wal
408     final Configuration newConf = HBaseConfiguration.create(this.conf);
409     User user = HBaseTestingUtility.getDifferentUser(newConf,
410         tableNameStr);
411     user.runAs(new PrivilegedExceptionAction() {
412       public Object run() throws Exception {
413         runWALSplit(newConf);
414         HLog wal2 = createWAL(newConf);
415         HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
416             newConf, hri, htd, null);
417         long seqid2 = region2.initialize();
418         assertTrue(seqid2 > -1);
419         assertEquals(rowsInsertedCount, getScannedCount(region2.getScanner(new Scan())));
420 
421         // I can't close wal1.  Its been appropriated when we split.
422         region2.close();
423         wal2.closeAndDelete();
424         return null;
425       }
426     });
427   }
428 
429   /**
430    * Test writing edits into an HRegion, closing it, splitting logs, opening
431    * Region again.  Verify seqids.
432    * @throws IOException
433    * @throws IllegalAccessException
434    * @throws NoSuchFieldException
435    * @throws IllegalArgumentException
436    * @throws SecurityException
437    */
438   @Test
439   public void testReplayEditsWrittenViaHRegion()
440   throws IOException, SecurityException, IllegalArgumentException,
441       NoSuchFieldException, IllegalAccessException, InterruptedException {
442     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
443     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
444     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
445     deleteDir(basedir);
446     final byte[] rowName = Bytes.toBytes(tableNameStr);
447     final int countPerFamily = 10;
448     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
449     HRegion region3 = HRegion.createHRegion(hri,
450             hbaseRootDir, this.conf, htd);
451     region3.close();
452     region3.getLog().closeAndDelete();
453     // Write countPerFamily edits into the three families.  Do a flush on one
454     // of the families during the load of edits so its seqid is not same as
455     // others to test we do right thing when different seqids.
456     HLog wal = createWAL(this.conf);
457     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
458     long seqid = region.initialize();
459     // HRegionServer usually does this. It knows the largest seqid across all regions.
460     wal.setSequenceNumber(seqid);
461     boolean first = true;
462     for (HColumnDescriptor hcd: htd.getFamilies()) {
463       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
464       if (first ) {
465         // If first, so we have at least one family w/ different seqid to rest.
466         region.flushcache();
467         first = false;
468       }
469     }
470     // Now assert edits made it in.
471     final Get g = new Get(rowName);
472     Result result = region.get(g, null);
473     assertEquals(countPerFamily * htd.getFamilies().size(),
474       result.size());
475     // Now close the region (without flush), split the log, reopen the region and assert that
476     // replay of log has the correct effect, that our seqids are calculated correctly so
477     // all edits in logs are seen as 'stale'/old.
478     region.close(true);
479     wal.close();
480     runWALSplit(this.conf);
481     HLog wal2 = createWAL(this.conf);
482     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
483     long seqid2 = region2.initialize();
484     // HRegionServer usually does this. It knows the largest seqid across all regions.
485     wal2.setSequenceNumber(seqid2);
486     assertTrue(seqid + result.size() < seqid2);
487     final Result result1b = region2.get(g, null);
488     assertEquals(result.size(), result1b.size());
489 
490     // Next test.  Add more edits, then 'crash' this region by stealing its wal
491     // out from under it and assert that replay of the log adds the edits back
492     // correctly when region is opened again.
493     for (HColumnDescriptor hcd: htd.getFamilies()) {
494       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
495     }
496     // Get count of edits.
497     final Result result2 = region2.get(g, null);
498     assertEquals(2 * result.size(), result2.size());
499     wal2.sync();
500     // Set down maximum recovery so we dfsclient doesn't linger retrying something
501     // long gone.
502     HBaseTestingUtility.setMaxRecoveryErrorCount(wal2.getOutputStream(), 1);
503     final Configuration newConf = HBaseConfiguration.create(this.conf);
504     User user = HBaseTestingUtility.getDifferentUser(newConf,
505       tableNameStr);
506     user.runAs(new PrivilegedExceptionAction() {
507       public Object run() throws Exception {
508         runWALSplit(newConf);
509         FileSystem newFS = FileSystem.get(newConf);
510         // Make a new wal for new region open.
511         HLog wal3 = createWAL(newConf);
512         final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
513         HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) {
514           @Override
515           protected boolean restoreEdit(Store s, KeyValue kv) {
516             boolean b = super.restoreEdit(s, kv);
517             countOfRestoredEdits.incrementAndGet();
518             return b;
519           }
520         };
521         long seqid3 = region3.initialize();
522         // HRegionServer usually does this. It knows the largest seqid across all regions.
523         wal3.setSequenceNumber(seqid3);
524         Result result3 = region3.get(g, null);
525         // Assert that count of cells is same as before crash.
526         assertEquals(result2.size(), result3.size());
527         assertEquals(htd.getFamilies().size() * countPerFamily,
528           countOfRestoredEdits.get());
529 
530         // I can't close wal1.  Its been appropriated when we split.
531         region3.close();
532         wal3.closeAndDelete();
533         return null;
534       }
535     });
536   }
537 
538   /**
539    * Test that we recover correctly when there is a failure in between the
540    * flushes. i.e. Some stores got flushed but others did not.
541    *
542    * Unfortunately, there is no easy hook to flush at a store level. The way
543    * we get around this is by flushing at the region level, and then deleting
544    * the recently flushed store file for one of the Stores. This would put us
545    * back in the situation where all but that store got flushed and the region
546    * died.
547    *
548    * We restart Region again, and verify that the edits were replayed.
549    *
550    * @throws IOException
551    * @throws IllegalAccessException
552    * @throws NoSuchFieldException
553    * @throws IllegalArgumentException
554    * @throws SecurityException
555    */
556   @Test
557   public void testReplayEditsAfterPartialFlush()
558   throws IOException, SecurityException, IllegalArgumentException,
559       NoSuchFieldException, IllegalAccessException, InterruptedException {
560     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
561     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
562     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
563     deleteDir(basedir);
564     final byte[] rowName = Bytes.toBytes(tableNameStr);
565     final int countPerFamily = 10;
566     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
567     HRegion region3 = HRegion.createHRegion(hri,
568             hbaseRootDir, this.conf, htd);
569     region3.close();
570     region3.getLog().closeAndDelete();
571     // Write countPerFamily edits into the three families.  Do a flush on one
572     // of the families during the load of edits so its seqid is not same as
573     // others to test we do right thing when different seqids.
574     HLog wal = createWAL(this.conf);
575     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
576     long seqid = region.initialize();
577     // HRegionServer usually does this. It knows the largest seqid across all regions.
578     wal.setSequenceNumber(seqid);
579     for (HColumnDescriptor hcd: htd.getFamilies()) {
580       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
581     }
582 
583     // Now assert edits made it in.
584     final Get g = new Get(rowName);
585     Result result = region.get(g, null);
586     assertEquals(countPerFamily * htd.getFamilies().size(),
587       result.size());
588 
589     // Let us flush the region
590     region.flushcache();
591     region.close(true);
592     wal.close();
593 
594     // delete the store files in the second column family to simulate a failure
595     // in between the flushcache();
596     // we have 3 families. killing the middle one ensures that taking the maximum
597     // will make us fail.
598     int cf_count = 0;
599     for (HColumnDescriptor hcd: htd.getFamilies()) {
600       cf_count++;
601       if (cf_count == 2) {
602         this.fs.delete(new Path(region.getRegionDir(), Bytes.toString(hcd.getName()))
603             , true);
604       }
605     }
606 
607 
608     // Let us try to split and recover
609     runWALSplit(this.conf);
610     HLog wal2 = createWAL(this.conf);
611     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
612     long seqid2 = region2.initialize();
613     // HRegionServer usually does this. It knows the largest seqid across all regions.
614     wal2.setSequenceNumber(seqid2);
615     assertTrue(seqid + result.size() < seqid2);
616 
617     final Result result1b = region2.get(g, null);
618     assertEquals(result.size(), result1b.size());
619   }
620 
621   /**
622    * Test that we could recover the data correctly after aborting flush. In the
623    * test, first we abort flush after writing some data, then writing more data
624    * and flush again, at last verify the data.
625    * @throws IOException
626    */
627   @Test
628   public void testReplayEditsAfterAbortingFlush() throws IOException {
629     final String tableNameStr = "testReplayEditsAfterAbortingFlush";
630     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
631     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
632     deleteDir(basedir);
633     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
634     HRegion region3 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
635     region3.close();
636     region3.getLog().closeAndDelete();
637     // Write countPerFamily edits into the three families. Do a flush on one
638     // of the families during the load of edits so its seqid is not same as
639     // others to test we do right thing when different seqids.
640     HLog wal = createWAL(this.conf);
641     final AtomicBoolean throwExceptionWhenFlushing = new AtomicBoolean(false);
642     RegionServerServices rsServices = Mockito.mock(RegionServerServices.class);
643     Mockito.doReturn(false).when(rsServices).isAborted();
644     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd,
645         rsServices) {
646       @Override
647       protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
648           throws IOException {
649         return new Store(tableDir, this, c, fs, conf) {
650           @Override
651           protected Path flushCache(final long logCacheFlushId,
652               SortedSet<KeyValue> snapshot,
653               TimeRangeTracker snapshotTimeRangeTracker,
654               AtomicLong flushedSize, MonitoredTask status) throws IOException {
655             if (throwExceptionWhenFlushing.get()) {
656               throw new IOException("Simulated exception by tests");
657             }
658             return super.flushCache(logCacheFlushId, snapshot,
659                 snapshotTimeRangeTracker, flushedSize, status);
660           }
661         };
662       }
663     };
664     long seqid = region.initialize();
665     // HRegionServer usually does this. It knows the largest seqid across all
666     // regions.
667     wal.setSequenceNumber(seqid);
668 
669     int writtenRowCount = 10;
670     List<HColumnDescriptor> families = new ArrayList<HColumnDescriptor>(
671         htd.getFamilies());
672     for (int i = 0; i < writtenRowCount; i++) {
673       Put put = new Put(Bytes.toBytes(tableNameStr + Integer.toString(i)));
674       put.add(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
675           Bytes.toBytes("val"));
676       region.put(put);
677     }
678 
679     // Now assert edits made it in.
680     RegionScanner scanner = region.getScanner(new Scan());
681     assertEquals(writtenRowCount, getScannedCount(scanner));
682 
683     // Let us flush the region
684     throwExceptionWhenFlushing.set(true);
685     try {
686       region.flushcache();
687       fail("Injected exception hasn't been thrown");
688     } catch (Throwable t) {
689       LOG.info("Expected simulated exception when flushing region,"
690           + t.getMessage());
691       // simulated to abort server
692       Mockito.doReturn(true).when(rsServices).isAborted();
693     }
694     // writing more data
695     int moreRow = 10;
696     for (int i = writtenRowCount; i < writtenRowCount + moreRow; i++) {
697       Put put = new Put(Bytes.toBytes(tableNameStr + Integer.toString(i)));
698       put.add(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
699           Bytes.toBytes("val"));
700       region.put(put);
701     }
702     writtenRowCount += moreRow;
703     // call flush again
704     throwExceptionWhenFlushing.set(false);
705     try {
706       region.flushcache();
707     } catch (IOException t) {
708       LOG.info("Expected exception when flushing region because server is stopped,"
709           + t.getMessage());
710     }
711 
712     region.close(true);
713     wal.close();
714 
715     // Let us try to split and recover
716     runWALSplit(this.conf);
717     HLog wal2 = createWAL(this.conf);
718     Mockito.doReturn(false).when(rsServices).isAborted();
719     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd,
720         rsServices);
721     long seqid2 = region2.initialize();
722     // HRegionServer usually does this. It knows the largest seqid across all
723     // regions.
724     wal2.setSequenceNumber(seqid2);
725 
726     scanner = region2.getScanner(new Scan());
727     assertEquals(writtenRowCount, getScannedCount(scanner));
728   }
729 
730   private int getScannedCount(RegionScanner scanner) throws IOException {
731     int scannedCount = 0;
732     List<KeyValue> results = new ArrayList<KeyValue>();
733     while (true) {
734       boolean existMore = scanner.next(results);
735       if (!results.isEmpty())
736         scannedCount++;
737       if (!existMore)
738         break;
739       results.clear();
740     }
741     return scannedCount;
742   }
743 
744   /**
745    * Create an HRegion with the result of a HLog split and test we only see the
746    * good edits
747    * @throws Exception
748    */
749   @Test
750   public void testReplayEditsWrittenIntoWAL() throws Exception {
751     final String tableNameStr = "testReplayEditsWrittenIntoWAL";
752     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
753     final Path basedir = new Path(hbaseRootDir, tableNameStr);
754     deleteDir(basedir);
755     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
756     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
757     HRegion region2 = HRegion.createHRegion(hri,
758             hbaseRootDir, this.conf, htd);
759     region2.close();
760     region2.getLog().closeAndDelete();
761     final HLog wal = createWAL(this.conf);
762     final byte[] tableName = Bytes.toBytes(tableNameStr);
763     final byte[] rowName = tableName;
764     final byte[] regionName = hri.getEncodedNameAsBytes();
765 
766     // Add 1k to each family.
767     final int countPerFamily = 1000;
768     for (HColumnDescriptor hcd: htd.getFamilies()) {
769       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
770           ee, wal, htd);
771     }
772 
773     // Add a cache flush, shouldn't have any effect
774     long logSeqId = wal.startCacheFlush(regionName);
775     wal.completeCacheFlush(regionName, tableName, logSeqId, hri.isMetaRegion());
776 
777     // Add an edit to another family, should be skipped.
778     WALEdit edit = new WALEdit();
779     long now = ee.currentTimeMillis();
780     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
781       now, rowName));
782     wal.append(hri, tableName, edit, now, htd);
783 
784     // Delete the c family to verify deletes make it over.
785     edit = new WALEdit();
786     now = ee.currentTimeMillis();
787     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now,
788       KeyValue.Type.DeleteFamily));
789     wal.append(hri, tableName, edit, now, htd);
790 
791     // Sync.
792     wal.sync();
793     // Set down maximum recovery so we dfsclient doesn't linger retrying something
794     // long gone.
795     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
796     // Make a new conf and a new fs for the splitter to run on so we can take
797     // over old wal.
798     final Configuration newConf = HBaseConfiguration.create(this.conf);
799     User user = HBaseTestingUtility.getDifferentUser(newConf,
800       ".replay.wal.secondtime");
801     user.runAs(new PrivilegedExceptionAction() {
802       public Object run() throws Exception {
803         runWALSplit(newConf);
804         FileSystem newFS = FileSystem.get(newConf);
805         // 100k seems to make for about 4 flushes during HRegion#initialize.
806         newConf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 100);
807         // Make a new wal for new region.
808         HLog newWal = createWAL(newConf);
809         final AtomicInteger flushcount = new AtomicInteger(0);
810         try {
811           final HRegion region =
812               new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) {
813             protected FlushResult internalFlushcache(
814                 final HLog wal, final long myseqid, MonitoredTask status)
815             throws IOException {
816               LOG.info("InternalFlushCache Invoked");
817               FlushResult fs = super.internalFlushcache(wal, myseqid,
818                   Mockito.mock(MonitoredTask.class));
819               flushcount.incrementAndGet();
820               return fs;
821             };
822           };
823           long seqid = region.initialize();
824           // We flushed during init.
825           assertTrue("Flushcount=" + flushcount.get(), flushcount.get() > 0);
826           assertTrue(seqid > wal.getSequenceNumber());
827 
828           Get get = new Get(rowName);
829           Result result = region.get(get, -1);
830           // Make sure we only see the good edits
831           assertEquals(countPerFamily * (htd.getFamilies().size() - 1),
832             result.size());
833           region.close();
834         } finally {
835           newWal.closeAndDelete();
836         }
837         return null;
838       }
839     });
840   }
841 
842   @Test
843   public void testSequentialEditLogSeqNum() throws IOException {
844     final String tableNameStr = "testSequentialEditLogSeqNum";
845     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
846     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
847     deleteDir(basedir);
848     final byte[] rowName = Bytes.toBytes(tableNameStr);
849     final int countPerFamily = 10;
850     final HTableDescriptor htd = createBasic1FamilyHTD(tableNameStr);
851 
852     // Mock the HLog
853     MockHLog wal = createMockWAL(this.conf);
854     
855     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
856     long seqid = region.initialize();
857     // HRegionServer usually does this. It knows the largest seqid across all
858     // regions.
859     wal.setSequenceNumber(seqid);
860     for (HColumnDescriptor hcd : htd.getFamilies()) {
861       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
862     }
863     // get the seq no after first set of entries.
864     long sequenceNumber = wal.getSequenceNumber();
865 
866     // Let us flush the region
867     // But this time completeflushcache is not yet done
868     region.flushcache();
869     for (HColumnDescriptor hcd : htd.getFamilies()) {
870       addRegionEdits(rowName, hcd.getName(), 5, this.ee, region, "x");
871     }
872     long lastestSeqNumber = wal.getSequenceNumber();
873     // get the current seq no
874     wal.doCompleteCacheFlush = true;
875     // allow complete cache flush with the previous seq number got after first
876     // set of edits.
877     wal.completeCacheFlush(hri.getEncodedNameAsBytes(), hri.getTableName(), sequenceNumber, false);
878     wal.close();
879     FileStatus[] listStatus = this.fs.listStatus(wal.getDir());
880     HLogSplitter.splitLogFile(hbaseRootDir, listStatus[0], this.fs, this.conf,
881         null);
882     FileStatus[] listStatus1 = this.fs.listStatus(new Path(hbaseRootDir + "/"
883         + tableNameStr + "/" + hri.getEncodedName() + "/recovered.edits"));
884     int editCount = 0;
885     for (FileStatus fileStatus : listStatus1) {
886       editCount = Integer.parseInt(fileStatus.getPath().getName());
887     }
888     // The sequence number should be same 
889     assertEquals(
890         "The sequence number of the recoverd.edits and the current edit seq should be same",
891         lastestSeqNumber, editCount);
892   }
893   
894    static class MockHLog extends HLog {
895     boolean doCompleteCacheFlush = false;
896 
897     public MockHLog(FileSystem fs, Path dir, Path oldLogDir, Configuration conf) throws IOException {
898       super(fs, dir, oldLogDir, conf);
899     }
900 
901     @Override
902     public void completeCacheFlush(byte[] encodedRegionName, byte[] tableName, long logSeqId,
903         boolean isMetaRegion) throws IOException {
904       if (!doCompleteCacheFlush) {
905         return;
906       }
907       super.completeCacheFlush(encodedRegionName, tableName, logSeqId, isMetaRegion);
908     }
909   }
910 
911   private HTableDescriptor createBasic1FamilyHTD(final String tableName) {
912     HTableDescriptor htd = new HTableDescriptor(tableName);
913     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
914     htd.addFamily(a);
915     return htd;
916   }
917   
918   private MockHLog createMockWAL(Configuration conf) throws IOException {
919     MockHLog wal = new MockHLog(FileSystem.get(conf), logDir, oldLogDir, conf);
920     // Set down maximum recovery so we dfsclient doesn't linger retrying something
921     // long gone.
922     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
923     return wal;
924   }
925   
926     
927   // Flusher used in this test.  Keep count of how often we are called and
928   // actually run the flush inside here.
929   class TestFlusher implements FlushRequester {
930     private int count = 0;
931     private HRegion r;
932 
933     @Override
934     public void requestFlush(HRegion region) {
935       count++;
936       try {
937         r.flushcache();
938       } catch (IOException e) {
939         throw new RuntimeException("Exception flushing", e);
940       }
941     }
942 
943     @Override
944     public void requestDelayedFlush(HRegion region, long when) {
945       // TODO Auto-generated method stub
946       
947     }
948   }
949 
950   private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
951       final byte [] rowName, final byte [] family,
952       final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
953   throws IOException {
954     String familyStr = Bytes.toString(family);
955     for (int j = 0; j < count; j++) {
956       byte[] qualifierBytes = Bytes.toBytes(Integer.toString(j));
957       byte[] columnBytes = Bytes.toBytes(familyStr + ":" + Integer.toString(j));
958       WALEdit edit = new WALEdit();
959       edit.add(new KeyValue(rowName, family, qualifierBytes,
960         ee.currentTimeMillis(), columnBytes));
961       wal.append(hri, tableName, edit, ee.currentTimeMillis(), htd);
962     }
963   }
964 
965   private void addRegionEdits (final byte [] rowName, final byte [] family,
966       final int count, EnvironmentEdge ee, final HRegion r,
967       final String qualifierPrefix)
968   throws IOException {
969     for (int j = 0; j < count; j++) {
970       byte[] qualifier = Bytes.toBytes(qualifierPrefix + Integer.toString(j));
971       Put p = new Put(rowName);
972       p.add(family, qualifier, ee.currentTimeMillis(), rowName);
973       r.put(p);
974     }
975   }
976 
977   /*
978    * Creates an HRI around an HTD that has <code>tableName</code> and three
979    * column families named 'a','b', and 'c'.
980    * @param tableName Name of table to use when we create HTableDescriptor.
981    */
982    private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
983     return new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
984    }
985 
986   /*
987    * Run the split.  Verify only single split file made.
988    * @param c
989    * @return The single split file made
990    * @throws IOException
991    */
992   private Path runWALSplit(final Configuration c) throws IOException {
993     FileSystem fs = FileSystem.get(c);
994     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(c,
995         this.hbaseRootDir, this.logDir, this.oldLogDir, fs);
996     List<Path> splits = logSplitter.splitLog();
997     // Split should generate only 1 file since there's only 1 region
998     assertEquals("splits=" + splits, 1, splits.size());
999     // Make sure the file exists
1000     assertTrue(fs.exists(splits.get(0)));
1001     LOG.info("Split file=" + splits.get(0));
1002     return splits.get(0);
1003   }
1004 
1005   /*
1006    * @param c
1007    * @return WAL with retries set down from 5 to 1 only.
1008    * @throws IOException
1009    */
1010   private HLog createWAL(final Configuration c) throws IOException {
1011     HLog wal = new HLog(FileSystem.get(c), logDir, oldLogDir, c);
1012     // Set down maximum recovery so we dfsclient doesn't linger retrying something
1013     // long gone.
1014     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
1015     return wal;
1016   }
1017 
1018   private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
1019     HTableDescriptor htd = new HTableDescriptor(tableName);
1020     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
1021     htd.addFamily(a);
1022     HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
1023     htd.addFamily(b);
1024     HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
1025     htd.addFamily(c);
1026     return htd;
1027   }
1028 
1029   @org.junit.Rule
1030   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
1031     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
1032 }
1033