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  
25  import java.io.IOException;
26  import java.security.PrivilegedExceptionAction;
27  import java.util.ArrayList;
28  import java.util.List;
29  import java.util.concurrent.atomic.AtomicInteger;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.HBaseConfiguration;
37  import org.apache.hadoop.hbase.HBaseTestingUtility;
38  import org.apache.hadoop.hbase.HColumnDescriptor;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.HRegionInfo;
41  import org.apache.hadoop.hbase.HTableDescriptor;
42  import org.apache.hadoop.hbase.KeyValue;
43  import org.apache.hadoop.hbase.client.Get;
44  import org.apache.hadoop.hbase.client.Put;
45  import org.apache.hadoop.hbase.client.Result;
46  import org.apache.hadoop.hbase.io.hfile.HFile;
47  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
48  import org.apache.hadoop.hbase.regionserver.FlushRequester;
49  import org.apache.hadoop.hbase.regionserver.HRegion;
50  import org.apache.hadoop.hbase.regionserver.Store;
51  import org.apache.hadoop.hbase.security.User;
52  import org.apache.hadoop.hbase.util.Bytes;
53  import org.apache.hadoop.hbase.util.EnvironmentEdge;
54  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
55  import org.apache.hadoop.hbase.util.Pair;
56  import org.junit.After;
57  import org.junit.AfterClass;
58  import org.junit.Before;
59  import org.junit.BeforeClass;
60  import org.junit.Test;
61  import org.mockito.Mockito;
62  
63  /**
64   * Test replay of edits out of a WAL split.
65   */
66  public class TestWALReplay {
67    public static final Log LOG = LogFactory.getLog(TestWALReplay.class);
68    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
69    private final EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
70    private Path hbaseRootDir = null;
71    private Path oldLogDir;
72    private Path logDir;
73    private FileSystem fs;
74    private Configuration conf;
75  
76    @BeforeClass
77    public static void setUpBeforeClass() throws Exception {
78      Configuration conf = TEST_UTIL.getConfiguration();
79      conf.setBoolean("dfs.support.append", true);
80      // The below config supported by 0.20-append and CDH3b2
81      conf.setInt("dfs.client.block.recovery.retries", 2);
82      TEST_UTIL.startMiniDFSCluster(3);
83      Path hbaseRootDir =
84        TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
85      LOG.info("hbase.rootdir=" + hbaseRootDir);
86      conf.set(HConstants.HBASE_DIR, hbaseRootDir.toString());
87    }
88  
89    @AfterClass
90    public static void tearDownAfterClass() throws Exception {
91      TEST_UTIL.shutdownMiniDFSCluster();
92    }
93  
94    @Before
95    public void setUp() throws Exception {
96      this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
97      this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
98      this.hbaseRootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
99      this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
100     this.logDir = new Path(this.hbaseRootDir, HConstants.HREGION_LOGDIR_NAME);
101     if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
102       TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
103     }
104   }
105 
106   @After
107   public void tearDown() throws Exception {
108     TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
109   }
110 
111   /*
112    * @param p Directory to cleanup
113    */
114   private void deleteDir(final Path p) throws IOException {
115     if (this.fs.exists(p)) {
116       if (!this.fs.delete(p, true)) {
117         throw new IOException("Failed remove of " + p);
118       }
119     }
120   }
121 
122   /**
123    * Tests for hbase-2727.
124    * @throws Exception
125    * @see https://issues.apache.org/jira/browse/HBASE-2727
126    */
127   @Test
128   public void test2727() throws Exception {
129     // Test being able to have > 1 set of edits in the recovered.edits directory.
130     // Ensure edits are replayed properly.
131     final String tableNameStr = "test2727";
132     HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
133     Path basedir = new Path(hbaseRootDir, tableNameStr);
134     deleteDir(basedir);
135     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
136 
137     HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
138     HRegion region2 = HRegion.createHRegion(hri,
139         hbaseRootDir, this.conf, htd);
140 
141     final byte [] tableName = Bytes.toBytes(tableNameStr);
142     final byte [] rowName = tableName;
143 
144     HLog wal1 = createWAL(this.conf);
145     // Add 1k to each family.
146     final int countPerFamily = 1000;
147     for (HColumnDescriptor hcd: htd.getFamilies()) {
148       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee,
149           wal1, htd);
150     }
151     wal1.close();
152     runWALSplit(this.conf);
153 
154     HLog wal2 = createWAL(this.conf);
155     // Up the sequenceid so that these edits are after the ones added above.
156     wal2.setSequenceNumber(wal1.getSequenceNumber());
157     // Add 1k to each family.
158     for (HColumnDescriptor hcd: htd.getFamilies()) {
159       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
160           ee, wal2, htd);
161     }
162     wal2.close();
163     runWALSplit(this.conf);
164 
165     HLog wal3 = createWAL(this.conf);
166     wal3.setSequenceNumber(wal2.getSequenceNumber());
167     try {
168       final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri,
169         htd, null);
170       long seqid = region.initialize();
171       assertTrue(seqid > wal3.getSequenceNumber());
172 
173       // TODO: Scan all.
174       region.close();
175     } finally {
176       wal3.closeAndDelete();
177     }
178   }
179 
180   /**
181    * Test case of HRegion that is only made out of bulk loaded files.  Assert
182    * that we don't 'crash'.
183    * @throws IOException
184    * @throws IllegalAccessException
185    * @throws NoSuchFieldException
186    * @throws IllegalArgumentException
187    * @throws SecurityException
188    */
189   @Test
190   public void testRegionMadeOfBulkLoadedFilesOnly()
191   throws IOException, SecurityException, IllegalArgumentException,
192       NoSuchFieldException, IllegalAccessException, InterruptedException {
193     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
194     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
195     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
196     deleteDir(basedir);
197     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
198     HRegion region2 = HRegion.createHRegion(hri,
199         hbaseRootDir, this.conf, htd);
200     HLog wal = createWAL(this.conf);
201     HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
202     Path f =  new Path(basedir, "hfile");
203     HFile.Writer writer =
204       HFile.getWriterFactory(conf).createWriter(this.fs, f);
205     byte [] family = htd.getFamilies().iterator().next().getName();
206     byte [] row = Bytes.toBytes(tableNameStr);
207     writer.append(new KeyValue(row, family, family, row));
208     writer.close();
209     List <Pair<byte[],String>>  hfs= new ArrayList<Pair<byte[],String>>(1);
210     hfs.add(Pair.newPair(family, f.toString()));
211     region.bulkLoadHFiles(hfs);
212     // Add an edit so something in the WAL
213     region.put((new Put(row)).add(family, family, family));
214     wal.sync();
215 
216     // Now 'crash' the region by stealing its wal
217     final Configuration newConf = HBaseConfiguration.create(this.conf);
218     User user = HBaseTestingUtility.getDifferentUser(newConf,
219         tableNameStr);
220     user.runAs(new PrivilegedExceptionAction() {
221       public Object run() throws Exception {
222         runWALSplit(newConf);
223         HLog wal2 = createWAL(newConf);
224         HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
225           newConf, hri, htd, null);
226         long seqid2 = region2.initialize();
227         assertTrue(seqid2 > -1);
228 
229         // I can't close wal1.  Its been appropriated when we split.
230         region2.close();
231         wal2.closeAndDelete();
232         return null;
233       }
234     });
235   }
236 
237   /**
238    * Test writing edits into an HRegion, closing it, splitting logs, opening
239    * Region again.  Verify seqids.
240    * @throws IOException
241    * @throws IllegalAccessException
242    * @throws NoSuchFieldException
243    * @throws IllegalArgumentException
244    * @throws SecurityException
245    */
246   @Test
247   public void testReplayEditsWrittenViaHRegion()
248   throws IOException, SecurityException, IllegalArgumentException,
249       NoSuchFieldException, IllegalAccessException, InterruptedException {
250     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
251     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
252     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
253     deleteDir(basedir);
254     final byte[] rowName = Bytes.toBytes(tableNameStr);
255     final int countPerFamily = 10;
256     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
257     HRegion region3 = HRegion.createHRegion(hri,
258             hbaseRootDir, this.conf, htd);
259 
260     // Write countPerFamily edits into the three families.  Do a flush on one
261     // of the families during the load of edits so its seqid is not same as
262     // others to test we do right thing when different seqids.
263     HLog wal = createWAL(this.conf);
264     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
265     long seqid = region.initialize();
266     // HRegionServer usually does this. It knows the largest seqid across all regions.
267     wal.setSequenceNumber(seqid);
268     boolean first = true;
269     for (HColumnDescriptor hcd: htd.getFamilies()) {
270       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
271       if (first ) {
272         // If first, so we have at least one family w/ different seqid to rest.
273         region.flushcache();
274         first = false;
275       }
276     }
277     // Now assert edits made it in.
278     final Get g = new Get(rowName);
279     Result result = region.get(g, null);
280     assertEquals(countPerFamily * htd.getFamilies().size(),
281       result.size());
282     // Now close the region (without flush), split the log, reopen the region and assert that
283     // replay of log has the correct effect, that our seqids are calculated correctly so
284     // all edits in logs are seen as 'stale'/old.
285     region.close(true);
286     wal.close();
287     runWALSplit(this.conf);
288     HLog wal2 = createWAL(this.conf);
289     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
290     long seqid2 = region2.initialize();
291     // HRegionServer usually does this. It knows the largest seqid across all regions.
292     wal2.setSequenceNumber(seqid2);
293     assertTrue(seqid + result.size() < seqid2);
294     final Result result1b = region2.get(g, null);
295     assertEquals(result.size(), result1b.size());
296 
297     // Next test.  Add more edits, then 'crash' this region by stealing its wal
298     // out from under it and assert that replay of the log adds the edits back
299     // correctly when region is opened again.
300     for (HColumnDescriptor hcd: htd.getFamilies()) {
301       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
302     }
303     // Get count of edits.
304     final Result result2 = region2.get(g, null);
305     assertEquals(2 * result.size(), result2.size());
306     wal2.sync();
307     // Set down maximum recovery so we dfsclient doesn't linger retrying something
308     // long gone.
309     HBaseTestingUtility.setMaxRecoveryErrorCount(wal2.getOutputStream(), 1);
310     final Configuration newConf = HBaseConfiguration.create(this.conf);
311     User user = HBaseTestingUtility.getDifferentUser(newConf,
312       tableNameStr);
313     user.runAs(new PrivilegedExceptionAction() {
314       public Object run() throws Exception {
315         runWALSplit(newConf);
316         FileSystem newFS = FileSystem.get(newConf);
317         // Make a new wal for new region open.
318         HLog wal3 = createWAL(newConf);
319         final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
320         HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) {
321           @Override
322           protected boolean restoreEdit(Store s, KeyValue kv) {
323             boolean b = super.restoreEdit(s, kv);
324             countOfRestoredEdits.incrementAndGet();
325             return b;
326           }
327         };
328         long seqid3 = region3.initialize();
329         // HRegionServer usually does this. It knows the largest seqid across all regions.
330         wal3.setSequenceNumber(seqid3);
331         Result result3 = region3.get(g, null);
332         // Assert that count of cells is same as before crash.
333         assertEquals(result2.size(), result3.size());
334         assertEquals(htd.getFamilies().size() * countPerFamily,
335           countOfRestoredEdits.get());
336 
337         // I can't close wal1.  Its been appropriated when we split.
338         region3.close();
339         wal3.closeAndDelete();
340         return null;
341       }
342     });
343   }
344 
345   /**
346    * Test that we recover correctly when there is a failure in between the
347    * flushes. i.e. Some stores got flushed but others did not.
348    *
349    * Unfortunately, there is no easy hook to flush at a store level. The way
350    * we get around this is by flushing at the region level, and then deleting
351    * the recently flushed store file for one of the Stores. This would put us
352    * back in the situation where all but that store got flushed and the region
353    * died.
354    *
355    * We restart Region again, and verify that the edits were replayed.
356    *
357    * @throws IOException
358    * @throws IllegalAccessException
359    * @throws NoSuchFieldException
360    * @throws IllegalArgumentException
361    * @throws SecurityException
362    */
363   @Test
364   public void testReplayEditsAfterPartialFlush()
365   throws IOException, SecurityException, IllegalArgumentException,
366       NoSuchFieldException, IllegalAccessException, InterruptedException {
367     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
368     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
369     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
370     deleteDir(basedir);
371     final byte[] rowName = Bytes.toBytes(tableNameStr);
372     final int countPerFamily = 10;
373     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
374     HRegion region3 = HRegion.createHRegion(hri,
375             hbaseRootDir, this.conf, htd);
376 
377     // Write countPerFamily edits into the three families.  Do a flush on one
378     // of the families during the load of edits so its seqid is not same as
379     // others to test we do right thing when different seqids.
380     HLog wal = createWAL(this.conf);
381     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
382     long seqid = region.initialize();
383     // HRegionServer usually does this. It knows the largest seqid across all regions.
384     wal.setSequenceNumber(seqid);
385     for (HColumnDescriptor hcd: htd.getFamilies()) {
386       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
387     }
388 
389     // Now assert edits made it in.
390     final Get g = new Get(rowName);
391     Result result = region.get(g, null);
392     assertEquals(countPerFamily * htd.getFamilies().size(),
393       result.size());
394 
395     // Let us flush the region
396     region.flushcache();
397     region.close(true);
398     wal.close();
399 
400     // delete the store files in the second column family to simulate a failure
401     // in between the flushcache();
402     // we have 3 families. killing the middle one ensures that taking the maximum
403     // will make us fail.
404     int cf_count = 0;
405     for (HColumnDescriptor hcd: htd.getFamilies()) {
406       cf_count++;
407       if (cf_count == 2) {
408         this.fs.delete(new Path(region.getRegionDir(), Bytes.toString(hcd.getName()))
409             , true);
410       }
411     }
412 
413 
414     // Let us try to split and recover
415     runWALSplit(this.conf);
416     HLog wal2 = createWAL(this.conf);
417     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
418     long seqid2 = region2.initialize();
419     // HRegionServer usually does this. It knows the largest seqid across all regions.
420     wal2.setSequenceNumber(seqid2);
421     assertTrue(seqid + result.size() < seqid2);
422 
423     final Result result1b = region2.get(g, null);
424     assertEquals(result.size(), result1b.size());
425   }
426 
427   /**
428    * Create an HRegion with the result of a HLog split and test we only see the
429    * good edits
430    * @throws Exception
431    */
432   @Test
433   public void testReplayEditsWrittenIntoWAL() throws Exception {
434     final String tableNameStr = "testReplayEditsWrittenIntoWAL";
435     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
436     final Path basedir = new Path(hbaseRootDir, tableNameStr);
437     deleteDir(basedir);
438     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
439     final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
440     HRegion region2 = HRegion.createHRegion(hri,
441             hbaseRootDir, this.conf, htd);
442 
443     final HLog wal = createWAL(this.conf);
444     final byte[] tableName = Bytes.toBytes(tableNameStr);
445     final byte[] rowName = tableName;
446     final byte[] regionName = hri.getEncodedNameAsBytes();
447 
448     // Add 1k to each family.
449     final int countPerFamily = 1000;
450     for (HColumnDescriptor hcd: htd.getFamilies()) {
451       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily,
452           ee, wal, htd);
453     }
454 
455     // Add a cache flush, shouldn't have any effect
456     long logSeqId = wal.startCacheFlush(regionName);
457     wal.completeCacheFlush(regionName, tableName, logSeqId, hri.isMetaRegion());
458 
459     // Add an edit to another family, should be skipped.
460     WALEdit edit = new WALEdit();
461     long now = ee.currentTimeMillis();
462     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
463       now, rowName));
464     wal.append(hri, tableName, edit, now, htd);
465 
466     // Delete the c family to verify deletes make it over.
467     edit = new WALEdit();
468     now = ee.currentTimeMillis();
469     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now,
470       KeyValue.Type.DeleteFamily));
471     wal.append(hri, tableName, edit, now, htd);
472 
473     // Sync.
474     wal.sync();
475     // Set down maximum recovery so we dfsclient doesn't linger retrying something
476     // long gone.
477     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
478     // Make a new conf and a new fs for the splitter to run on so we can take
479     // over old wal.
480     final Configuration newConf = HBaseConfiguration.create(this.conf);
481     User user = HBaseTestingUtility.getDifferentUser(newConf,
482       ".replay.wal.secondtime");
483     user.runAs(new PrivilegedExceptionAction() {
484       public Object run() throws Exception {
485         runWALSplit(newConf);
486         FileSystem newFS = FileSystem.get(newConf);
487         // 100k seems to make for about 4 flushes during HRegion#initialize.
488         newConf.setInt("hbase.hregion.memstore.flush.size", 1024 * 100);
489         // Make a new wal for new region.
490         HLog newWal = createWAL(newConf);
491         final AtomicInteger flushcount = new AtomicInteger(0);
492         try {
493           final HRegion region =
494               new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) {
495             protected boolean internalFlushcache(
496                 final HLog wal, final long myseqid, MonitoredTask status)
497             throws IOException {
498               LOG.info("InternalFlushCache Invoked");
499               boolean b = super.internalFlushcache(wal, myseqid,
500                   Mockito.mock(MonitoredTask.class));
501               flushcount.incrementAndGet();
502               return b;
503             };
504           };
505           long seqid = region.initialize();
506           // We flushed during init.
507           assertTrue("Flushcount=" + flushcount.get(), flushcount.get() > 0);
508           assertTrue(seqid > wal.getSequenceNumber());
509 
510           Get get = new Get(rowName);
511           Result result = region.get(get, -1);
512           // Make sure we only see the good edits
513           assertEquals(countPerFamily * (htd.getFamilies().size() - 1),
514             result.size());
515           region.close();
516         } finally {
517           newWal.closeAndDelete();
518         }
519         return null;
520       }
521     });
522   }
523 
524   // Flusher used in this test.  Keep count of how often we are called and
525   // actually run the flush inside here.
526   class TestFlusher implements FlushRequester {
527     private int count = 0;
528     private HRegion r;
529 
530     @Override
531     public void requestFlush(HRegion region) {
532       count++;
533       try {
534         r.flushcache();
535       } catch (IOException e) {
536         throw new RuntimeException("Exception flushing", e);
537       }
538     }
539   }
540 
541   private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
542       final byte [] rowName, final byte [] family,
543       final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
544   throws IOException {
545     String familyStr = Bytes.toString(family);
546     for (int j = 0; j < count; j++) {
547       byte[] qualifierBytes = Bytes.toBytes(Integer.toString(j));
548       byte[] columnBytes = Bytes.toBytes(familyStr + ":" + Integer.toString(j));
549       WALEdit edit = new WALEdit();
550       edit.add(new KeyValue(rowName, family, qualifierBytes,
551         ee.currentTimeMillis(), columnBytes));
552       wal.append(hri, tableName, edit, ee.currentTimeMillis(), htd);
553     }
554   }
555 
556   private void addRegionEdits (final byte [] rowName, final byte [] family,
557       final int count, EnvironmentEdge ee, final HRegion r,
558       final String qualifierPrefix)
559   throws IOException {
560     for (int j = 0; j < count; j++) {
561       byte[] qualifier = Bytes.toBytes(qualifierPrefix + Integer.toString(j));
562       Put p = new Put(rowName);
563       p.add(family, qualifier, ee.currentTimeMillis(), rowName);
564       r.put(p);
565     }
566   }
567 
568   /*
569    * Creates an HRI around an HTD that has <code>tableName</code> and three
570    * column families named 'a','b', and 'c'.
571    * @param tableName Name of table to use when we create HTableDescriptor.
572    */
573    private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
574     return new HRegionInfo(Bytes.toBytes(tableName), null, null, false);
575    }
576 
577   /*
578    * Run the split.  Verify only single split file made.
579    * @param c
580    * @return The single split file made
581    * @throws IOException
582    */
583   private Path runWALSplit(final Configuration c) throws IOException {
584     FileSystem fs = FileSystem.get(c);
585     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(c,
586         this.hbaseRootDir, this.logDir, this.oldLogDir, fs);
587     List<Path> splits = logSplitter.splitLog();
588     // Split should generate only 1 file since there's only 1 region
589     assertEquals("splits=" + splits, 1, splits.size());
590     // Make sure the file exists
591     assertTrue(fs.exists(splits.get(0)));
592     LOG.info("Split file=" + splits.get(0));
593     return splits.get(0);
594   }
595 
596   /*
597    * @param c
598    * @return WAL with retries set down from 5 to 1 only.
599    * @throws IOException
600    */
601   private HLog createWAL(final Configuration c) throws IOException {
602     HLog wal = new HLog(FileSystem.get(c), logDir, oldLogDir, c);
603     // Set down maximum recovery so we dfsclient doesn't linger retrying something
604     // long gone.
605     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
606     return wal;
607   }
608 
609   private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
610     HTableDescriptor htd = new HTableDescriptor(tableName);
611     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
612     htd.addFamily(a);
613     HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
614     htd.addFamily(b);
615     HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
616     htd.addFamily(c);
617     return htd;
618   }
619 }