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