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.io.hfile.HFile;
61  import org.apache.hadoop.hbase.master.HMaster;
62  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
63  import org.apache.hadoop.hbase.regionserver.FlushRequester;
64  import org.apache.hadoop.hbase.regionserver.HRegion;
65  import org.apache.hadoop.hbase.regionserver.HRegionServer;
66  import org.apache.hadoop.hbase.regionserver.RegionScanner;
67  import org.apache.hadoop.hbase.regionserver.RegionServerServices;
68  import org.apache.hadoop.hbase.regionserver.Store;
69  import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
70  import org.apache.hadoop.hbase.security.User;
71  import org.apache.hadoop.hbase.util.Bytes;
72  import org.apache.hadoop.hbase.util.EnvironmentEdge;
73  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
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 = "testReplayEditsWrittenViaHRegion";
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     Path f =  new Path(basedir, "hfile");
321     HFile.Writer writer =
322       HFile.getWriterFactoryNoCache(conf).withPath(fs, f).create();
323     byte [] family = htd.getFamilies().iterator().next().getName();
324     byte [] row = Bytes.toBytes(tableNameStr);
325     writer.append(new KeyValue(row, family, family, row));
326     writer.close();
327     List <Pair<byte[],String>>  hfs= new ArrayList<Pair<byte[],String>>(1);
328     hfs.add(Pair.newPair(family, f.toString()));
329     region.bulkLoadHFiles(hfs, true);
330     // Add an edit so something in the WAL
331     region.put((new Put(row)).add(family, family, family));
332     wal.sync();
333 
334     // Now 'crash' the region by stealing its wal
335     final Configuration newConf = HBaseConfiguration.create(this.conf);
336     User user = HBaseTestingUtility.getDifferentUser(newConf,
337         tableNameStr);
338     user.runAs(new PrivilegedExceptionAction() {
339       public Object run() throws Exception {
340         runWALSplit(newConf);
341         HLog wal2 = createWAL(newConf);
342         HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
343           newConf, hri, htd, null);
344         long seqid2 = region2.initialize();
345         assertTrue(seqid2 > -1);
346 
347         // I can't close wal1.  Its been appropriated when we split.
348         region2.close();
349         wal2.closeAndDelete();
350         return null;
351       }
352     });
353   }
354 
355   /**
356    * Test writing edits into an HRegion, closing it, splitting logs, opening
357    * Region again.  Verify seqids.
358    * @throws IOException
359    * @throws IllegalAccessException
360    * @throws NoSuchFieldException
361    * @throws IllegalArgumentException
362    * @throws SecurityException
363    */
364   @Test
365   public void testReplayEditsWrittenViaHRegion()
366   throws IOException, SecurityException, IllegalArgumentException,
367       NoSuchFieldException, IllegalAccessException, InterruptedException {
368     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
369     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
370     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
371     deleteDir(basedir);
372     final byte[] rowName = Bytes.toBytes(tableNameStr);
373     final int countPerFamily = 10;
374     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
375     HRegion region3 = HRegion.createHRegion(hri,
376             hbaseRootDir, this.conf, htd);
377     region3.close();
378     region3.getLog().closeAndDelete();
379     // Write countPerFamily edits into the three families.  Do a flush on one
380     // of the families during the load of edits so its seqid is not same as
381     // others to test we do right thing when different seqids.
382     HLog wal = createWAL(this.conf);
383     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
384     long seqid = region.initialize();
385     // HRegionServer usually does this. It knows the largest seqid across all regions.
386     wal.setSequenceNumber(seqid);
387     boolean first = true;
388     for (HColumnDescriptor hcd: htd.getFamilies()) {
389       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
390       if (first ) {
391         // If first, so we have at least one family w/ different seqid to rest.
392         region.flushcache();
393         first = false;
394       }
395     }
396     // Now assert edits made it in.
397     final Get g = new Get(rowName);
398     Result result = region.get(g, null);
399     assertEquals(countPerFamily * htd.getFamilies().size(),
400       result.size());
401     // Now close the region (without flush), split the log, reopen the region and assert that
402     // replay of log has the correct effect, that our seqids are calculated correctly so
403     // all edits in logs are seen as 'stale'/old.
404     region.close(true);
405     wal.close();
406     runWALSplit(this.conf);
407     HLog wal2 = createWAL(this.conf);
408     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
409     long seqid2 = region2.initialize();
410     // HRegionServer usually does this. It knows the largest seqid across all regions.
411     wal2.setSequenceNumber(seqid2);
412     assertTrue(seqid + result.size() < seqid2);
413     final Result result1b = region2.get(g, null);
414     assertEquals(result.size(), result1b.size());
415 
416     // Next test.  Add more edits, then 'crash' this region by stealing its wal
417     // out from under it and assert that replay of the log adds the edits back
418     // correctly when region is opened again.
419     for (HColumnDescriptor hcd: htd.getFamilies()) {
420       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
421     }
422     // Get count of edits.
423     final Result result2 = region2.get(g, null);
424     assertEquals(2 * result.size(), result2.size());
425     wal2.sync();
426     // Set down maximum recovery so we dfsclient doesn't linger retrying something
427     // long gone.
428     HBaseTestingUtility.setMaxRecoveryErrorCount(wal2.getOutputStream(), 1);
429     final Configuration newConf = HBaseConfiguration.create(this.conf);
430     User user = HBaseTestingUtility.getDifferentUser(newConf,
431       tableNameStr);
432     user.runAs(new PrivilegedExceptionAction() {
433       public Object run() throws Exception {
434         runWALSplit(newConf);
435         FileSystem newFS = FileSystem.get(newConf);
436         // Make a new wal for new region open.
437         HLog wal3 = createWAL(newConf);
438         final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
439         HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) {
440           @Override
441           protected boolean restoreEdit(Store s, KeyValue kv) {
442             boolean b = super.restoreEdit(s, kv);
443             countOfRestoredEdits.incrementAndGet();
444             return b;
445           }
446         };
447         long seqid3 = region3.initialize();
448         // HRegionServer usually does this. It knows the largest seqid across all regions.
449         wal3.setSequenceNumber(seqid3);
450         Result result3 = region3.get(g, null);
451         // Assert that count of cells is same as before crash.
452         assertEquals(result2.size(), result3.size());
453         assertEquals(htd.getFamilies().size() * countPerFamily,
454           countOfRestoredEdits.get());
455 
456         // I can't close wal1.  Its been appropriated when we split.
457         region3.close();
458         wal3.closeAndDelete();
459         return null;
460       }
461     });
462   }
463 
464   /**
465    * Test that we recover correctly when there is a failure in between the
466    * flushes. i.e. Some stores got flushed but others did not.
467    *
468    * Unfortunately, there is no easy hook to flush at a store level. The way
469    * we get around this is by flushing at the region level, and then deleting
470    * the recently flushed store file for one of the Stores. This would put us
471    * back in the situation where all but that store got flushed and the region
472    * died.
473    *
474    * We restart Region again, and verify that the edits were replayed.
475    *
476    * @throws IOException
477    * @throws IllegalAccessException
478    * @throws NoSuchFieldException
479    * @throws IllegalArgumentException
480    * @throws SecurityException
481    */
482   @Test
483   public void testReplayEditsAfterPartialFlush()
484   throws IOException, SecurityException, IllegalArgumentException,
485       NoSuchFieldException, IllegalAccessException, InterruptedException {
486     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
487     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
488     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
489     deleteDir(basedir);
490     final byte[] rowName = Bytes.toBytes(tableNameStr);
491     final int countPerFamily = 10;
492     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
493     HRegion region3 = HRegion.createHRegion(hri,
494             hbaseRootDir, this.conf, htd);
495     region3.close();
496     region3.getLog().closeAndDelete();
497     // Write countPerFamily edits into the three families.  Do a flush on one
498     // of the families during the load of edits so its seqid is not same as
499     // others to test we do right thing when different seqids.
500     HLog wal = createWAL(this.conf);
501     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
502     long seqid = region.initialize();
503     // HRegionServer usually does this. It knows the largest seqid across all regions.
504     wal.setSequenceNumber(seqid);
505     for (HColumnDescriptor hcd: htd.getFamilies()) {
506       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
507     }
508 
509     // Now assert edits made it in.
510     final Get g = new Get(rowName);
511     Result result = region.get(g, null);
512     assertEquals(countPerFamily * htd.getFamilies().size(),
513       result.size());
514 
515     // Let us flush the region
516     region.flushcache();
517     region.close(true);
518     wal.close();
519 
520     // delete the store files in the second column family to simulate a failure
521     // in between the flushcache();
522     // we have 3 families. killing the middle one ensures that taking the maximum
523     // will make us fail.
524     int cf_count = 0;
525     for (HColumnDescriptor hcd: htd.getFamilies()) {
526       cf_count++;
527       if (cf_count == 2) {
528         this.fs.delete(new Path(region.getRegionDir(), Bytes.toString(hcd.getName()))
529             , true);
530       }
531     }
532 
533 
534     // Let us try to split and recover
535     runWALSplit(this.conf);
536     HLog wal2 = createWAL(this.conf);
537     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
538     long seqid2 = region2.initialize();
539     // HRegionServer usually does this. It knows the largest seqid across all regions.
540     wal2.setSequenceNumber(seqid2);
541     assertTrue(seqid + result.size() < seqid2);
542 
543     final Result result1b = region2.get(g, null);
544     assertEquals(result.size(), result1b.size());
545   }
546 
547   /**
548    * Test that we could recover the data correctly after aborting flush. In the
549    * test, first we abort flush after writing some data, then writing more data
550    * and flush again, at last verify the data.
551    * @throws IOException
552    */
553   @Test
554   public void testReplayEditsAfterAbortingFlush() throws IOException {
555     final String tableNameStr = "testReplayEditsAfterAbortingFlush";
556     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
557     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
558     deleteDir(basedir);
559     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
560     HRegion region3 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
561     region3.close();
562     region3.getLog().closeAndDelete();
563     // Write countPerFamily edits into the three families. Do a flush on one
564     // of the families during the load of edits so its seqid is not same as
565     // others to test we do right thing when different seqids.
566     HLog wal = createWAL(this.conf);
567     final AtomicBoolean throwExceptionWhenFlushing = new AtomicBoolean(false);
568     RegionServerServices rsServices = Mockito.mock(RegionServerServices.class);
569     Mockito.doReturn(false).when(rsServices).isAborted();
570     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd,
571         rsServices) {
572       @Override
573       protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
574           throws IOException {
575         return new Store(tableDir, this, c, fs, conf) {
576           @Override
577           protected Path flushCache(final long logCacheFlushId,
578               SortedSet<KeyValue> snapshot,
579               TimeRangeTracker snapshotTimeRangeTracker,
580               AtomicLong flushedSize, MonitoredTask status) throws IOException {
581             if (throwExceptionWhenFlushing.get()) {
582               throw new IOException("Simulated exception by tests");
583             }
584             return super.flushCache(logCacheFlushId, snapshot,
585                 snapshotTimeRangeTracker, flushedSize, status);
586           }
587         };
588       }
589     };
590     long seqid = region.initialize();
591     // HRegionServer usually does this. It knows the largest seqid across all
592     // regions.
593     wal.setSequenceNumber(seqid);
594 
595     int writtenRowCount = 10;
596     List<HColumnDescriptor> families = new ArrayList<HColumnDescriptor>(
597         htd.getFamilies());
598     for (int i = 0; i < writtenRowCount; i++) {
599       Put put = new Put(Bytes.toBytes(tableNameStr + Integer.toString(i)));
600       put.add(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
601           Bytes.toBytes("val"));
602       region.put(put);
603     }
604 
605     // Now assert edits made it in.
606     RegionScanner scanner = region.getScanner(new Scan());
607     assertEquals(writtenRowCount, getScannedCount(scanner));
608 
609     // Let us flush the region
610     throwExceptionWhenFlushing.set(true);
611     try {
612       region.flushcache();
613       fail("Injected exception hasn't been thrown");
614     } catch (Throwable t) {
615       LOG.info("Expected simulated exception when flushing region,"
616           + t.getMessage());
617       // simulated to abort server
618       Mockito.doReturn(true).when(rsServices).isAborted();
619     }
620     // writing more data
621     int moreRow = 10;
622     for (int i = writtenRowCount; i < writtenRowCount + moreRow; i++) {
623       Put put = new Put(Bytes.toBytes(tableNameStr + Integer.toString(i)));
624       put.add(families.get(i % families.size()).getName(), Bytes.toBytes("q"),
625           Bytes.toBytes("val"));
626       region.put(put);
627     }
628     writtenRowCount += moreRow;
629     // call flush again
630     throwExceptionWhenFlushing.set(false);
631     try {
632       region.flushcache();
633     } catch (IOException t) {
634       LOG.info("Expected exception when flushing region because server is stopped,"
635           + t.getMessage());
636     }
637 
638     region.close(true);
639     wal.close();
640 
641     // Let us try to split and recover
642     runWALSplit(this.conf);
643     HLog wal2 = createWAL(this.conf);
644     Mockito.doReturn(false).when(rsServices).isAborted();
645     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd,
646         rsServices);
647     long seqid2 = region2.initialize();
648     // HRegionServer usually does this. It knows the largest seqid across all
649     // regions.
650     wal2.setSequenceNumber(seqid2);
651 
652     scanner = region2.getScanner(new Scan());
653     assertEquals(writtenRowCount, getScannedCount(scanner));
654   }
655 
656   private int getScannedCount(RegionScanner scanner) throws IOException {
657     int scannedCount = 0;
658     List<KeyValue> results = new ArrayList<KeyValue>();
659     while (true) {
660       boolean existMore = scanner.next(results);
661       if (!results.isEmpty())
662         scannedCount++;
663       if (!existMore)
664         break;
665       results.clear();
666     }
667     return scannedCount;
668   }
669 
670   /**
671    * Create an HRegion with the result of a HLog split and test we only see the
672    * good edits
673    * @throws Exception
674    */
675   @Test
676   public void testReplayEditsWrittenIntoWAL() throws Exception {
677     final String tableNameStr = "testReplayEditsWrittenIntoWAL";
678     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
679     final Path basedir = new Path(hbaseRootDir, tableNameStr);
680     deleteDir(basedir);
681     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
682     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
683     HRegion region2 = HRegion.createHRegion(hri,
684             hbaseRootDir, this.conf, htd);
685     region2.close();
686     region2.getLog().closeAndDelete();
687     final HLog wal = createWAL(this.conf);
688     final byte[] tableName = Bytes.toBytes(tableNameStr);
689     final byte[] rowName = tableName;
690     final byte[] regionName = hri.getEncodedNameAsBytes();
691 
692     // Add 1k to each family.
693     final int countPerFamily = 1000;
694     for (HColumnDescriptor hcd: htd.getFamilies()) {
695       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
696           ee, wal, htd);
697     }
698 
699     // Add a cache flush, shouldn't have any effect
700     long logSeqId = wal.startCacheFlush(regionName);
701     wal.completeCacheFlush(regionName, tableName, logSeqId, hri.isMetaRegion());
702 
703     // Add an edit to another family, should be skipped.
704     WALEdit edit = new WALEdit();
705     long now = ee.currentTimeMillis();
706     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
707       now, rowName));
708     wal.append(hri, tableName, edit, now, htd);
709 
710     // Delete the c family to verify deletes make it over.
711     edit = new WALEdit();
712     now = ee.currentTimeMillis();
713     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now,
714       KeyValue.Type.DeleteFamily));
715     wal.append(hri, tableName, edit, now, htd);
716 
717     // Sync.
718     wal.sync();
719     // Set down maximum recovery so we dfsclient doesn't linger retrying something
720     // long gone.
721     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
722     // Make a new conf and a new fs for the splitter to run on so we can take
723     // over old wal.
724     final Configuration newConf = HBaseConfiguration.create(this.conf);
725     User user = HBaseTestingUtility.getDifferentUser(newConf,
726       ".replay.wal.secondtime");
727     user.runAs(new PrivilegedExceptionAction() {
728       public Object run() throws Exception {
729         runWALSplit(newConf);
730         FileSystem newFS = FileSystem.get(newConf);
731         // 100k seems to make for about 4 flushes during HRegion#initialize.
732         newConf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 100);
733         // Make a new wal for new region.
734         HLog newWal = createWAL(newConf);
735         final AtomicInteger flushcount = new AtomicInteger(0);
736         try {
737           final HRegion region =
738               new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) {
739             protected boolean internalFlushcache(
740                 final HLog wal, final long myseqid, MonitoredTask status)
741             throws IOException {
742               LOG.info("InternalFlushCache Invoked");
743               boolean b = super.internalFlushcache(wal, myseqid,
744                   Mockito.mock(MonitoredTask.class));
745               flushcount.incrementAndGet();
746               return b;
747             };
748           };
749           long seqid = region.initialize();
750           // We flushed during init.
751           assertTrue("Flushcount=" + flushcount.get(), flushcount.get() > 0);
752           assertTrue(seqid > wal.getSequenceNumber());
753 
754           Get get = new Get(rowName);
755           Result result = region.get(get, -1);
756           // Make sure we only see the good edits
757           assertEquals(countPerFamily * (htd.getFamilies().size() - 1),
758             result.size());
759           region.close();
760         } finally {
761           newWal.closeAndDelete();
762         }
763         return null;
764       }
765     });
766   }
767 
768   @Test
769   public void testSequentialEditLogSeqNum() throws IOException {
770     final String tableNameStr = "testSequentialEditLogSeqNum";
771     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
772     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
773     deleteDir(basedir);
774     final byte[] rowName = Bytes.toBytes(tableNameStr);
775     final int countPerFamily = 10;
776     final HTableDescriptor htd = createBasic1FamilyHTD(tableNameStr);
777 
778     // Mock the HLog
779     MockHLog wal = createMockWAL(this.conf);
780     
781     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
782     long seqid = region.initialize();
783     // HRegionServer usually does this. It knows the largest seqid across all
784     // regions.
785     wal.setSequenceNumber(seqid);
786     for (HColumnDescriptor hcd : htd.getFamilies()) {
787       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
788     }
789     // get the seq no after first set of entries.
790     long sequenceNumber = wal.getSequenceNumber();
791 
792     // Let us flush the region
793     // But this time completeflushcache is not yet done
794     region.flushcache();
795     for (HColumnDescriptor hcd : htd.getFamilies()) {
796       addRegionEdits(rowName, hcd.getName(), 5, this.ee, region, "x");
797     }
798     long lastestSeqNumber = wal.getSequenceNumber();
799     // get the current seq no
800     wal.doCompleteCacheFlush = true;
801     // allow complete cache flush with the previous seq number got after first
802     // set of edits.
803     wal.completeCacheFlush(hri.getEncodedNameAsBytes(), hri.getTableName(), sequenceNumber, false);
804     wal.close();
805     FileStatus[] listStatus = this.fs.listStatus(wal.getDir());
806     HLogSplitter.splitLogFile(hbaseRootDir, listStatus[0], this.fs, this.conf,
807         null);
808     FileStatus[] listStatus1 = this.fs.listStatus(new Path(hbaseRootDir + "/"
809         + tableNameStr + "/" + hri.getEncodedName() + "/recovered.edits"));
810     int editCount = 0;
811     for (FileStatus fileStatus : listStatus1) {
812       editCount = Integer.parseInt(fileStatus.getPath().getName());
813     }
814     // The sequence number should be same 
815     assertEquals(
816         "The sequence number of the recoverd.edits and the current edit seq should be same",
817         lastestSeqNumber, editCount);
818   }
819   
820    static class MockHLog extends HLog {
821     boolean doCompleteCacheFlush = false;
822 
823     public MockHLog(FileSystem fs, Path dir, Path oldLogDir, Configuration conf) throws IOException {
824       super(fs, dir, oldLogDir, conf);
825     }
826 
827     @Override
828     public void completeCacheFlush(byte[] encodedRegionName, byte[] tableName, long logSeqId,
829         boolean isMetaRegion) throws IOException {
830       if (!doCompleteCacheFlush) {
831         return;
832       }
833       super.completeCacheFlush(encodedRegionName, tableName, logSeqId, isMetaRegion);
834     }
835   }
836 
837   private HTableDescriptor createBasic1FamilyHTD(final String tableName) {
838     HTableDescriptor htd = new HTableDescriptor(tableName);
839     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
840     htd.addFamily(a);
841     return htd;
842   }
843   
844   private MockHLog createMockWAL(Configuration conf) throws IOException {
845     MockHLog wal = new MockHLog(FileSystem.get(conf), logDir, oldLogDir, conf);
846     // Set down maximum recovery so we dfsclient doesn't linger retrying something
847     // long gone.
848     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
849     return wal;
850   }
851   
852     
853   // Flusher used in this test.  Keep count of how often we are called and
854   // actually run the flush inside here.
855   class TestFlusher implements FlushRequester {
856     private int count = 0;
857     private HRegion r;
858 
859     @Override
860     public void requestFlush(HRegion region) {
861       count++;
862       try {
863         r.flushcache();
864       } catch (IOException e) {
865         throw new RuntimeException("Exception flushing", e);
866       }
867     }
868 
869     @Override
870     public void requestDelayedFlush(HRegion region, long when) {
871       // TODO Auto-generated method stub
872       
873     }
874   }
875 
876   private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
877       final byte [] rowName, final byte [] family,
878       final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
879   throws IOException {
880     String familyStr = Bytes.toString(family);
881     for (int j = 0; j < count; j++) {
882       byte[] qualifierBytes = Bytes.toBytes(Integer.toString(j));
883       byte[] columnBytes = Bytes.toBytes(familyStr + ":" + Integer.toString(j));
884       WALEdit edit = new WALEdit();
885       edit.add(new KeyValue(rowName, family, qualifierBytes,
886         ee.currentTimeMillis(), columnBytes));
887       wal.append(hri, tableName, edit, ee.currentTimeMillis(), htd);
888     }
889   }
890 
891   private void addRegionEdits (final byte [] rowName, final byte [] family,
892       final int count, EnvironmentEdge ee, final HRegion r,
893       final String qualifierPrefix)
894   throws IOException {
895     for (int j = 0; j < count; j++) {
896       byte[] qualifier = Bytes.toBytes(qualifierPrefix + Integer.toString(j));
897       Put p = new Put(rowName);
898       p.add(family, qualifier, ee.currentTimeMillis(), rowName);
899       r.put(p);
900     }
901   }
902 
903   /*
904    * Creates an HRI around an HTD that has <code>tableName</code> and three
905    * column families named 'a','b', and 'c'.
906    * @param tableName Name of table to use when we create HTableDescriptor.
907    */
908    private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
909     return new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
910    }
911 
912   /*
913    * Run the split.  Verify only single split file made.
914    * @param c
915    * @return The single split file made
916    * @throws IOException
917    */
918   private Path runWALSplit(final Configuration c) throws IOException {
919     FileSystem fs = FileSystem.get(c);
920     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(c,
921         this.hbaseRootDir, this.logDir, this.oldLogDir, fs);
922     List<Path> splits = logSplitter.splitLog();
923     // Split should generate only 1 file since there's only 1 region
924     assertEquals("splits=" + splits, 1, splits.size());
925     // Make sure the file exists
926     assertTrue(fs.exists(splits.get(0)));
927     LOG.info("Split file=" + splits.get(0));
928     return splits.get(0);
929   }
930 
931   /*
932    * @param c
933    * @return WAL with retries set down from 5 to 1 only.
934    * @throws IOException
935    */
936   private HLog createWAL(final Configuration c) throws IOException {
937     HLog wal = new HLog(FileSystem.get(c), logDir, oldLogDir, c);
938     // Set down maximum recovery so we dfsclient doesn't linger retrying something
939     // long gone.
940     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
941     return wal;
942   }
943 
944   private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
945     HTableDescriptor htd = new HTableDescriptor(tableName);
946     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
947     htd.addFamily(a);
948     HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
949     htd.addFamily(b);
950     HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
951     htd.addFamily(c);
952     return htd;
953   }
954 
955   @org.junit.Rule
956   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
957     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
958 }
959