1   /**
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver.wal;
21  
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertFalse;
24  import static org.junit.Assert.assertNotNull;
25  import static org.junit.Assert.assertTrue;
26  import static org.junit.Assert.fail;
27  
28  import java.io.FileNotFoundException;
29  import java.io.IOException;
30  import java.lang.reflect.Method;
31  import java.util.ArrayList;
32  import java.util.Collections;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.NavigableSet;
36  import java.util.concurrent.atomic.AtomicBoolean;
37  import java.util.concurrent.atomic.AtomicLong;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.fs.FSDataInputStream;
43  import org.apache.hadoop.fs.FSDataOutputStream;
44  import org.apache.hadoop.fs.FileStatus;
45  import org.apache.hadoop.fs.FileSystem;
46  import org.apache.hadoop.fs.FileUtil;
47  import org.apache.hadoop.fs.Path;
48  import org.apache.hadoop.hbase.HBaseTestingUtility;
49  import org.apache.hadoop.hbase.HColumnDescriptor;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.HTableDescriptor;
53  import org.apache.hadoop.hbase.KeyValue;
54  import org.apache.hadoop.hbase.LargeTests;
55  import org.apache.hadoop.hbase.regionserver.HRegion;
56  import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
57  import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
58  import org.apache.hadoop.hbase.util.Bytes;
59  import org.apache.hadoop.hbase.util.CancelableProgressable;
60  import org.apache.hadoop.hbase.util.Threads;
61  import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
62  import org.apache.hadoop.ipc.RemoteException;
63  import org.junit.After;
64  import org.junit.AfterClass;
65  import org.junit.Assert;
66  import org.junit.Before;
67  import org.junit.BeforeClass;
68  import org.junit.Ignore;
69  import org.junit.Test;
70  import org.junit.experimental.categories.Category;
71  import org.mockito.Mockito;
72  import org.mockito.invocation.InvocationOnMock;
73  import org.mockito.stubbing.Answer;
74  
75  import com.google.common.base.Joiner;
76  import com.google.common.collect.ImmutableList;
77  
78  /**
79   * Testing {@link HLog} splitting code.
80   */
81  @Category(LargeTests.class)
82  public class TestHLogSplit {
83  
84    private final static Log LOG = LogFactory.getLog(TestHLogSplit.class);
85  
86    private Configuration conf;
87    private FileSystem fs;
88  
89    protected final static HBaseTestingUtility
90            TEST_UTIL = new HBaseTestingUtility();
91  
92  
93    private static final Path hbaseDir = new Path("/hbase");
94    private static final Path hlogDir = new Path(hbaseDir, "hlog");
95    private static final Path oldLogDir = new Path(hbaseDir, "hlog.old");
96    private static final Path corruptDir = new Path(hbaseDir, ".corrupt");
97  
98    private static final int NUM_WRITERS = 10;
99    private static final int ENTRIES = 10; // entries per writer per region
100 
101   private HLog.Writer[] writer = new HLog.Writer[NUM_WRITERS];
102   private long seq = 0;
103   private static final byte[] TABLE_NAME = "t1".getBytes();
104   private static final byte[] FAMILY = "f1".getBytes();
105   private static final byte[] QUALIFIER = "q1".getBytes();
106   private static final byte[] VALUE = "v1".getBytes();
107   private static final String HLOG_FILE_PREFIX = "hlog.dat.";
108   private static List<String> regions;
109   private static final String HBASE_SKIP_ERRORS = "hbase.hlog.split.skip.errors";
110   private static final Path tabledir =
111       new Path(hbaseDir, Bytes.toString(TABLE_NAME));
112 
113   static enum Corruptions {
114     INSERT_GARBAGE_ON_FIRST_LINE,
115     INSERT_GARBAGE_IN_THE_MIDDLE,
116     APPEND_GARBAGE,
117     TRUNCATE,
118   }
119 
120   @BeforeClass
121   public static void setUpBeforeClass() throws Exception {
122     TEST_UTIL.getConfiguration().
123             setStrings("hbase.rootdir", hbaseDir.toString());
124     TEST_UTIL.getConfiguration().
125             setClass("hbase.regionserver.hlog.writer.impl",
126                 InstrumentedSequenceFileLogWriter.class, HLog.Writer.class);
127 
128     TEST_UTIL.startMiniDFSCluster(2);
129   }
130 
131   @AfterClass
132   public static void tearDownAfterClass() throws Exception {
133     TEST_UTIL.shutdownMiniDFSCluster();
134   }
135 
136   @Before
137   public void setUp() throws Exception {
138     flushToConsole("Cleaning up cluster for new test\n"
139         + "--------------------------");
140     conf = TEST_UTIL.getConfiguration();
141     fs = TEST_UTIL.getDFSCluster().getFileSystem();
142     FileStatus[] entries = fs.listStatus(new Path("/"));
143     flushToConsole("Num entries in /:" + entries.length);
144     for (FileStatus dir : entries){
145       assertTrue("Deleting " + dir.getPath(),
146           fs.delete(dir.getPath(), true));
147     }
148     // create the HLog directory because recursive log creates are not allowed
149     fs.mkdirs(hlogDir);
150     seq = 0;
151     regions = new ArrayList<String>();
152     Collections.addAll(regions, "bbb", "ccc");
153     InstrumentedSequenceFileLogWriter.activateFailure = false;
154   }
155 
156   @After
157   public void tearDown() throws Exception {
158   }
159 
160   /**
161    * @throws IOException
162    * @see https://issues.apache.org/jira/browse/HBASE-3020
163    */
164   @Test 
165   public void testRecoveredEditsPathForMeta() throws IOException {
166     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
167     byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
168     Path tdir = new Path(hbaseDir, Bytes.toString(HConstants.META_TABLE_NAME));
169     Path regiondir = new Path(tdir,
170         HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
171     fs.mkdirs(regiondir);
172     long now = System.currentTimeMillis();
173     HLog.Entry entry =
174         new HLog.Entry(new HLogKey(encoded,
175             HConstants.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
176       new WALEdit());
177     Path p = HLogSplitter.getRegionSplitEditsPath(fs, entry, hbaseDir, true);
178     String parentOfParent = p.getParent().getParent().getName();
179     assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
180   }
181 
182   @Test(expected = OrphanHLogAfterSplitException.class)
183   public void testSplitFailsIfNewHLogGetsCreatedAfterSplitStarted()
184   throws IOException {
185     AtomicBoolean stop = new AtomicBoolean(false);
186 
187     assertFalse("Previous test should clean up table dir",
188       fs.exists(new Path("/hbase/t1")));
189 
190     generateHLogs(-1);
191 
192     try {
193     (new ZombieNewLogWriterRegionServer(stop)).start();
194     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
195         hbaseDir, hlogDir, oldLogDir, fs);
196     logSplitter.splitLog();
197     } finally {
198       stop.set(true);
199     }
200   }
201 
202   @Test
203   public void testSplitPreservesEdits() throws IOException{
204     final String REGION = "region__1";
205     regions.removeAll(regions);
206     regions.add(REGION);
207 
208     generateHLogs(1, 10, -1);
209     fs.initialize(fs.getUri(), conf);
210     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
211       hbaseDir, hlogDir, oldLogDir, fs);
212     logSplitter.splitLog();
213 
214     Path originalLog = (fs.listStatus(oldLogDir))[0].getPath();
215     Path splitLog = getLogForRegion(hbaseDir, TABLE_NAME, REGION);
216 
217     assertEquals("edits differ after split", true, logsAreEqual(originalLog, splitLog));
218   }
219 
220 
221   @Test
222   public void testEmptyLogFiles() throws IOException {
223 
224     injectEmptyFile(".empty", true);
225     generateHLogs(Integer.MAX_VALUE);
226     injectEmptyFile("empty", true);
227 
228     // make fs act as a different client now
229     // initialize will create a new DFSClient with a new client ID
230     fs.initialize(fs.getUri(), conf);
231 
232     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
233         hbaseDir, hlogDir, oldLogDir, fs);
234     logSplitter.splitLog();
235 
236 
237     for (String region : regions) {
238       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
239       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
240     }
241 
242   }
243 
244 
245   @Test
246   public void testEmptyOpenLogFiles() throws IOException {
247     injectEmptyFile(".empty", false);
248     generateHLogs(Integer.MAX_VALUE);
249     injectEmptyFile("empty", false);
250 
251     // make fs act as a different client now
252     // initialize will create a new DFSClient with a new client ID
253     fs.initialize(fs.getUri(), conf);
254 
255     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
256         hbaseDir, hlogDir, oldLogDir, fs);
257     logSplitter.splitLog();
258 
259     for (String region : regions) {
260       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
261       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
262     }
263   }
264 
265   @Test
266   public void testOpenZeroLengthReportedFileButWithDataGetsSplit() throws IOException {
267     // generate logs but leave hlog.dat.5 open.
268     generateHLogs(5);
269 
270     fs.initialize(fs.getUri(), conf);
271 
272     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
273         hbaseDir, hlogDir, oldLogDir, fs);
274     logSplitter.splitLog();
275 
276     for (String region : regions) {
277       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
278       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
279     }
280 
281 
282   }
283 
284 
285   @Test
286   public void testTralingGarbageCorruptionFileSkipErrorsPasses() throws IOException {
287     conf.setBoolean(HBASE_SKIP_ERRORS, true);
288     generateHLogs(Integer.MAX_VALUE);
289     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
290             Corruptions.APPEND_GARBAGE, true, fs);
291     fs.initialize(fs.getUri(), conf);
292 
293     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
294         hbaseDir, hlogDir, oldLogDir, fs);
295     logSplitter.splitLog();
296     for (String region : regions) {
297       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
298       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
299     }
300 
301 
302   }
303 
304   @Test
305   public void testFirstLineCorruptionLogFileSkipErrorsPasses() throws IOException {
306     conf.setBoolean(HBASE_SKIP_ERRORS, true);
307     generateHLogs(Integer.MAX_VALUE);
308     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
309             Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true, fs);
310     fs.initialize(fs.getUri(), conf);
311 
312     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
313         hbaseDir, hlogDir, oldLogDir, fs);
314     logSplitter.splitLog();
315     for (String region : regions) {
316       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
317       assertEquals((NUM_WRITERS - 1) * ENTRIES, countHLog(logfile, fs, conf));
318     }
319 
320 
321   }
322 
323 
324   @Test
325   public void testMiddleGarbageCorruptionSkipErrorsReadsHalfOfFile() throws IOException {
326     conf.setBoolean(HBASE_SKIP_ERRORS, true);
327     generateHLogs(Integer.MAX_VALUE);
328     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
329             Corruptions.INSERT_GARBAGE_IN_THE_MIDDLE, false, fs);
330     fs.initialize(fs.getUri(), conf);
331     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
332         hbaseDir, hlogDir, oldLogDir, fs);
333     logSplitter.splitLog();
334 
335     for (String region : regions) {
336       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
337       // the entries in the original logs are alternating regions
338       // considering the sequence file header, the middle corruption should
339       // affect at least half of the entries
340       int goodEntries = (NUM_WRITERS - 1) * ENTRIES;
341       int firstHalfEntries = (int) Math.ceil(ENTRIES / 2) - 1;
342       assertTrue("The file up to the corrupted area hasn't been parsed",
343               goodEntries + firstHalfEntries <= countHLog(logfile, fs, conf));
344     }
345   }
346 
347   @Test
348   public void testCorruptedFileGetsArchivedIfSkipErrors() throws IOException {
349     conf.setBoolean(HBASE_SKIP_ERRORS, true);
350     Class<?> backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
351         Reader.class);
352     InstrumentedSequenceFileLogWriter.activateFailure = false;
353     HLog.resetLogReaderClass();
354 
355     try {
356     Path c1 = new Path(hlogDir, HLOG_FILE_PREFIX + "0");
357       conf.setClass("hbase.regionserver.hlog.reader.impl",
358           FaultySequenceFileLogReader.class, HLog.Reader.class);
359       for (FaultySequenceFileLogReader.FailureType  failureType : FaultySequenceFileLogReader.FailureType.values()) {
360         conf.set("faultysequencefilelogreader.failuretype", failureType.name());
361         generateHLogs(1, ENTRIES, -1);
362         fs.initialize(fs.getUri(), conf);
363         HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
364             hbaseDir, hlogDir, oldLogDir, fs);
365         logSplitter.splitLog();
366         FileStatus[] archivedLogs = fs.listStatus(corruptDir);
367         assertEquals("expected a different file", c1.getName(), archivedLogs[0]
368             .getPath().getName());
369         assertEquals(archivedLogs.length, 1);
370         fs.delete(new Path(oldLogDir, HLOG_FILE_PREFIX + "0"), false);
371       }
372     } finally {
373       conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass,
374           Reader.class);
375       HLog.resetLogReaderClass();
376     }
377   }
378 
379   @Test(expected = IOException.class)
380   public void testTrailingGarbageCorruptionLogFileSkipErrorsFalseThrows()
381       throws IOException {
382     conf.setBoolean(HBASE_SKIP_ERRORS, false);
383     Class<?> backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
384         Reader.class);
385     InstrumentedSequenceFileLogWriter.activateFailure = false;
386     HLog.resetLogReaderClass();
387 
388     try {
389       conf.setClass("hbase.regionserver.hlog.reader.impl",
390           FaultySequenceFileLogReader.class, HLog.Reader.class);
391       conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name());
392       generateHLogs(Integer.MAX_VALUE);
393     fs.initialize(fs.getUri(), conf);
394     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
395         hbaseDir, hlogDir, oldLogDir, fs);
396     logSplitter.splitLog();
397     } finally {
398       conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass,
399           Reader.class);
400       HLog.resetLogReaderClass();
401     }
402 
403   }
404 
405   @Test
406   public void testCorruptedLogFilesSkipErrorsFalseDoesNotTouchLogs()
407       throws IOException {
408     conf.setBoolean(HBASE_SKIP_ERRORS, false);
409     Class<?> backupClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
410         Reader.class);
411     InstrumentedSequenceFileLogWriter.activateFailure = false;
412     HLog.resetLogReaderClass();
413 
414     try {
415       conf.setClass("hbase.regionserver.hlog.reader.impl",
416           FaultySequenceFileLogReader.class, HLog.Reader.class);
417       conf.set("faultysequencefilelogreader.failuretype", FaultySequenceFileLogReader.FailureType.BEGINNING.name());
418       generateHLogs(-1);
419       fs.initialize(fs.getUri(), conf);
420       HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
421           hbaseDir, hlogDir, oldLogDir, fs);
422       try {
423         logSplitter.splitLog();
424       } catch (IOException e) {
425         assertEquals(
426             "if skip.errors is false all files should remain in place",
427             NUM_WRITERS, fs.listStatus(hlogDir).length);
428       }
429     } finally {
430       conf.setClass("hbase.regionserver.hlog.reader.impl", backupClass,
431           Reader.class);
432       HLog.resetLogReaderClass();
433     }
434 
435   }
436 
437   @Test
438   public void testEOFisIgnored() throws IOException {
439     conf.setBoolean(HBASE_SKIP_ERRORS, false);
440 
441     final String REGION = "region__1";
442     regions.removeAll(regions);
443     regions.add(REGION);
444 
445     int entryCount = 10;
446     Path c1 = new Path(hlogDir, HLOG_FILE_PREFIX + "0");
447     generateHLogs(1, entryCount, -1);
448     corruptHLog(c1, Corruptions.TRUNCATE, true, fs);
449 
450     fs.initialize(fs.getUri(), conf);
451     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
452         hbaseDir, hlogDir, oldLogDir, fs);
453     logSplitter.splitLog();
454 
455     Path originalLog = (fs.listStatus(oldLogDir))[0].getPath();
456     Path splitLog = getLogForRegion(hbaseDir, TABLE_NAME, REGION);
457 
458     int actualCount = 0;
459     HLog.Reader in = HLog.getReader(fs, splitLog, conf);
460     HLog.Entry entry;
461     while ((entry = in.next()) != null) ++actualCount;
462     assertEquals(entryCount-1, actualCount);
463 
464     // should not have stored the EOF files as corrupt
465     FileStatus[] archivedLogs = fs.listStatus(corruptDir);
466     assertEquals(archivedLogs.length, 0);
467   }
468 
469   @Test
470   public void testLogsGetArchivedAfterSplit() throws IOException {
471     conf.setBoolean(HBASE_SKIP_ERRORS, false);
472 
473     generateHLogs(-1);
474 
475     fs.initialize(fs.getUri(), conf);
476     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
477         hbaseDir, hlogDir, oldLogDir, fs);
478     logSplitter.splitLog();
479 
480     FileStatus[] archivedLogs = fs.listStatus(oldLogDir);
481 
482     assertEquals("wrong number of files in the archive log", NUM_WRITERS, archivedLogs.length);
483   }
484 
485   @Test
486   public void testSplit() throws IOException {
487     generateHLogs(-1);
488     fs.initialize(fs.getUri(), conf);
489     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
490         hbaseDir, hlogDir, oldLogDir, fs);
491     logSplitter.splitLog();
492 
493     for (String region : regions) {
494       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
495       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
496 
497     }
498   }
499 
500   @Test
501   public void testLogDirectoryShouldBeDeletedAfterSuccessfulSplit()
502   throws IOException {
503     generateHLogs(-1);
504     fs.initialize(fs.getUri(), conf);
505     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
506         hbaseDir, hlogDir, oldLogDir, fs);
507     logSplitter.splitLog();
508     FileStatus [] statuses = null;
509     try {
510       statuses = fs.listStatus(hlogDir);
511       if (statuses != null) {
512         Assert.fail("Files left in log dir: " +
513             Joiner.on(",").join(FileUtil.stat2Paths(statuses)));
514       }
515     } catch (FileNotFoundException e) {
516       // hadoop 0.21 throws FNFE whereas hadoop 0.20 returns null
517     }
518   }
519 /* DISABLED for now.  TODO: HBASE-2645
520   @Test
521   public void testLogCannotBeWrittenOnceParsed() throws IOException {
522     AtomicLong counter = new AtomicLong(0);
523     AtomicBoolean stop = new AtomicBoolean(false);
524     generateHLogs(9);
525     fs.initialize(fs.getUri(), conf);
526 
527     Thread zombie = new ZombieLastLogWriterRegionServer(writer[9], counter, stop);
528 
529 
530 
531     try {
532       zombie.start();
533 
534       HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
535 
536       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, "juliet");
537 
538       // It's possible that the writer got an error while appending and didn't count it
539       // however the entry will in fact be written to file and split with the rest
540       long numberOfEditsInRegion = countHLog(logfile, fs, conf);
541       assertTrue("The log file could have at most 1 extra log entry, but " +
542               "can't have less. Zombie could write "+counter.get() +" and logfile had only"+ numberOfEditsInRegion+" "  + logfile, counter.get() == numberOfEditsInRegion ||
543                       counter.get() + 1 == numberOfEditsInRegion);
544     } finally {
545       stop.set(true);
546     }
547   }
548 */
549 
550   @Test
551   public void testSplitWillNotTouchLogsIfNewHLogGetsCreatedAfterSplitStarted()
552   throws IOException {
553     AtomicBoolean stop = new AtomicBoolean(false);
554     generateHLogs(-1);
555     fs.initialize(fs.getUri(), conf);
556     Thread zombie = new ZombieNewLogWriterRegionServer(stop);
557 
558     try {
559       zombie.start();
560       try {
561         HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
562             hbaseDir, hlogDir, oldLogDir, fs);
563         logSplitter.splitLog();
564       } catch (IOException ex) {/* expected */}
565       int logFilesNumber = fs.listStatus(hlogDir).length;
566 
567       assertEquals("Log files should not be archived if there's an extra file after split",
568               NUM_WRITERS + 1, logFilesNumber);
569     } finally {
570       stop.set(true);
571     }
572 
573   }
574 
575 
576 
577   @Test(expected = IOException.class)
578   public void testSplitWillFailIfWritingToRegionFails() throws Exception {
579     //leave 5th log open so we could append the "trap"
580     generateHLogs(4);
581 
582     fs.initialize(fs.getUri(), conf);
583 
584     String region = "break";
585     Path regiondir = new Path(tabledir, region);
586     fs.mkdirs(regiondir);
587 
588     InstrumentedSequenceFileLogWriter.activateFailure = false;
589     appendEntry(writer[4], TABLE_NAME, Bytes.toBytes(region),
590         ("r" + 999).getBytes(), FAMILY, QUALIFIER, VALUE, 0);
591     writer[4].close();
592 
593     try {
594       InstrumentedSequenceFileLogWriter.activateFailure = true;
595       HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
596           hbaseDir, hlogDir, oldLogDir, fs);
597       logSplitter.splitLog();
598 
599     } catch (IOException e) {
600       assertEquals("This exception is instrumented and should only be thrown for testing", e.getMessage());
601       throw e;
602     } finally {
603       InstrumentedSequenceFileLogWriter.activateFailure = false;
604     }
605   }
606 
607 
608   // @Test TODO this test has been disabled since it was created!
609   // It currently fails because the second split doesn't output anything
610   // -- because there are no region dirs after we move aside the first
611   // split result
612   public void testSplittingLargeNumberOfRegionsConsistency() throws IOException {
613 
614     regions.removeAll(regions);
615     for (int i=0; i<100; i++) {
616       regions.add("region__"+i);
617     }
618 
619     generateHLogs(1, 100, -1);
620     fs.initialize(fs.getUri(), conf);
621 
622     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
623         hbaseDir, hlogDir, oldLogDir, fs);
624     logSplitter.splitLog();
625     fs.rename(oldLogDir, hlogDir);
626     Path firstSplitPath = new Path(hbaseDir, Bytes.toString(TABLE_NAME) + ".first");
627     Path splitPath = new Path(hbaseDir, Bytes.toString(TABLE_NAME));
628     fs.rename(splitPath,
629             firstSplitPath);
630 
631 
632     fs.initialize(fs.getUri(), conf);
633     logSplitter = HLogSplitter.createLogSplitter(conf,
634         hbaseDir, hlogDir, oldLogDir, fs);
635     logSplitter.splitLog();
636 
637     assertEquals(0, compareHLogSplitDirs(firstSplitPath, splitPath));
638   }
639 
640   @Test
641   public void testSplitDeletedRegion() throws IOException {
642     regions.removeAll(regions);
643     String region = "region_that_splits";
644     regions.add(region);
645 
646     generateHLogs(1);
647 
648     fs.initialize(fs.getUri(), conf);
649 
650     Path regiondir = new Path(tabledir, region);
651     fs.delete(regiondir, true);
652 
653     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
654         hbaseDir, hlogDir, oldLogDir, fs);
655     logSplitter.splitLog();
656 
657     assertFalse(fs.exists(regiondir));
658   }
659 
660   @Test
661   public void testIOEOnOutputThread() throws Exception {
662     conf.setBoolean(HBASE_SKIP_ERRORS, false);
663 
664     generateHLogs(-1);
665 
666     fs.initialize(fs.getUri(), conf);
667     // Set up a splitter that will throw an IOE on the output side
668     HLogSplitter logSplitter = new HLogSplitter(
669         conf, hbaseDir, hlogDir, oldLogDir, fs) {
670       protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
671       throws IOException {
672         HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class);
673         Mockito.doThrow(new IOException("Injected")).when(mockWriter).append(Mockito.<HLog.Entry>any());
674         return mockWriter;
675 
676       }
677     };
678     try {
679       logSplitter.splitLog();
680       fail("Didn't throw!");
681     } catch (IOException ioe) {
682       assertTrue(ioe.toString().contains("Injected"));
683     }
684   }
685 
686   // Test for HBASE-3412
687   @Test
688   public void testMovedHLogDuringRecovery() throws Exception {
689     generateHLogs(-1);
690 
691     fs.initialize(fs.getUri(), conf);
692 
693     // This partial mock will throw LEE for every file simulating
694     // files that were moved
695     FileSystem spiedFs = Mockito.spy(fs);
696     // The "File does not exist" part is very important,
697     // that's how it comes out of HDFS
698     Mockito.doThrow(new LeaseExpiredException("Injected: File does not exist")).
699         when(spiedFs).append(Mockito.<Path>any());
700 
701     HLogSplitter logSplitter = new HLogSplitter(
702         conf, hbaseDir, hlogDir, oldLogDir, spiedFs);
703 
704     try {
705       logSplitter.splitLog();
706       assertEquals(NUM_WRITERS, fs.listStatus(oldLogDir).length);
707       assertFalse(fs.exists(hlogDir));
708     } catch (IOException e) {
709       fail("There shouldn't be any exception but: " + e.toString());
710     }
711   }
712 
713   /**
714    * Test log split process with fake data and lots of edits to trigger threading
715    * issues.
716    */
717   @Test
718   public void testThreading() throws Exception {
719     doTestThreading(20000, 128*1024*1024, 0);
720   }
721 
722   /**
723    * Test blocking behavior of the log split process if writers are writing slower
724    * than the reader is reading.
725    */
726   @Test
727   public void testThreadingSlowWriterSmallBuffer() throws Exception {
728     doTestThreading(200, 1024, 50);
729   }
730 
731   /**
732    * Sets up a log splitter with a mock reader and writer. The mock reader generates
733    * a specified number of edits spread across 5 regions. The mock writer optionally
734    * sleeps for each edit it is fed.
735    * *
736    * After the split is complete, verifies that the statistics show the correct number
737    * of edits output into each region.
738    *
739    * @param numFakeEdits number of fake edits to push through pipeline
740    * @param bufferSize size of in-memory buffer
741    * @param writerSlowness writer threads will sleep this many ms per edit
742    */
743   private void doTestThreading(final int numFakeEdits,
744       final int bufferSize,
745       final int writerSlowness) throws Exception {
746 
747     Configuration localConf = new Configuration(conf);
748     localConf.setInt("hbase.regionserver.hlog.splitlog.buffersize", bufferSize);
749 
750     // Create a fake log file (we'll override the reader to produce a stream of edits)
751     FSDataOutputStream out = fs.create(new Path(hlogDir, HLOG_FILE_PREFIX + ".fake"));
752     out.close();
753 
754     // Make region dirs for our destination regions so the output doesn't get skipped
755     final List<String> regions = ImmutableList.of("r0", "r1", "r2", "r3", "r4");
756     makeRegionDirs(fs, regions);
757 
758     // Create a splitter that reads and writes the data without touching disk
759     HLogSplitter logSplitter = new HLogSplitter(
760         localConf, hbaseDir, hlogDir, oldLogDir, fs) {
761 
762       /* Produce a mock writer that doesn't write anywhere */
763       protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
764       throws IOException {
765         HLog.Writer mockWriter = Mockito.mock(HLog.Writer.class);
766         Mockito.doAnswer(new Answer<Void>() {
767           int expectedIndex = 0;
768 
769           @Override
770           public Void answer(InvocationOnMock invocation) {
771             if (writerSlowness > 0) {
772               try {
773                 Thread.sleep(writerSlowness);
774               } catch (InterruptedException ie) {
775                 Thread.currentThread().interrupt();
776               }
777             }
778             HLog.Entry entry = (Entry) invocation.getArguments()[0];
779             WALEdit edit = entry.getEdit();
780             List<KeyValue> keyValues = edit.getKeyValues();
781             assertEquals(1, keyValues.size());
782             KeyValue kv = keyValues.get(0);
783 
784             // Check that the edits come in the right order.
785             assertEquals(expectedIndex, Bytes.toInt(kv.getRow()));
786             expectedIndex++;
787             return null;
788           }
789         }).when(mockWriter).append(Mockito.<HLog.Entry>any());
790         return mockWriter;
791       }
792 
793 
794       /* Produce a mock reader that generates fake entries */
795       protected Reader getReader(FileSystem fs, Path curLogFile, Configuration conf)
796       throws IOException {
797         Reader mockReader = Mockito.mock(Reader.class);
798         Mockito.doAnswer(new Answer<HLog.Entry>() {
799           int index = 0;
800 
801           @Override
802           public HLog.Entry answer(InvocationOnMock invocation) throws Throwable {
803             if (index >= numFakeEdits) return null;
804 
805             // Generate r0 through r4 in round robin fashion
806             int regionIdx = index % regions.size();
807             byte region[] = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
808 
809             HLog.Entry ret = createTestEntry(TABLE_NAME, region,
810                 Bytes.toBytes((int)(index / regions.size())),
811                 FAMILY, QUALIFIER, VALUE, index);
812             index++;
813             return ret;
814           }
815         }).when(mockReader).next();
816         return mockReader;
817       }
818     };
819 
820     logSplitter.splitLog();
821 
822     // Verify number of written edits per region
823 
824     Map<byte[], Long> outputCounts = logSplitter.getOutputCounts();
825     for (Map.Entry<byte[], Long> entry : outputCounts.entrySet()) {
826       LOG.info("Got " + entry.getValue() + " output edits for region " +
827           Bytes.toString(entry.getKey()));
828 
829       assertEquals((long)entry.getValue(), numFakeEdits / regions.size());
830     }
831     assertEquals(regions.size(), outputCounts.size());
832   }
833 
834   // HBASE-2312: tests the case where a RegionServer enters a GC pause,
835   // comes back online after the master declared it dead and started to split.
836   // Want log rolling after a master split to fail
837   @Test
838   @Ignore("Need HADOOP-6886, HADOOP-6840, & HDFS-617 for this. HDFS 0.20.205.1+ should have this")
839   public void testLogRollAfterSplitStart() throws IOException {
840     // set flush interval to a large number so it doesn't interrupt us
841     final String F_INTERVAL = "hbase.regionserver.optionallogflushinterval";
842     long oldFlushInterval = conf.getLong(F_INTERVAL, 1000);
843     conf.setLong(F_INTERVAL, 1000*1000*100);
844     HLog log = null;
845     Path thisTestsDir = new Path(hbaseDir, "testLogRollAfterSplitStart");
846 
847     try {
848       // put some entries in an HLog
849       byte [] tableName = Bytes.toBytes(this.getClass().getName());
850       HRegionInfo regioninfo = new HRegionInfo(tableName,
851           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
852       log = new HLog(fs, thisTestsDir, oldLogDir, conf);
853       final int total = 20;
854       for (int i = 0; i < total; i++) {
855         WALEdit kvs = new WALEdit();
856         kvs.add(new KeyValue(Bytes.toBytes(i), tableName, tableName));
857         HTableDescriptor htd = new HTableDescriptor(tableName);
858         htd.addFamily(new HColumnDescriptor("column"));
859         log.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd);
860       }
861       // Send the data to HDFS datanodes and close the HDFS writer
862       log.sync();
863       log.cleanupCurrentWriter(log.getFilenum());
864 
865       /* code taken from ProcessServerShutdown.process()
866        * handles RS shutdowns (as observed by the Master)
867        */
868       // rename the directory so a rogue RS doesn't create more HLogs
869       Path rsSplitDir = new Path(thisTestsDir.getParent(),
870                                  thisTestsDir.getName() + "-splitting");
871       fs.rename(thisTestsDir, rsSplitDir);
872       LOG.debug("Renamed region directory: " + rsSplitDir);
873 
874       // Process the old log files
875       HLogSplitter splitter = HLogSplitter.createLogSplitter(conf,
876         hbaseDir, rsSplitDir, oldLogDir, fs);
877       splitter.splitLog();
878 
879       // Now, try to roll the HLog and verify failure
880       try {
881         log.rollWriter();
882         Assert.fail("rollWriter() did not throw any exception.");
883       } catch (IOException ioe) {
884         if (ioe.getCause().getMessage().contains("FileNotFound")) {
885           LOG.info("Got the expected exception: ", ioe.getCause());
886         } else {
887           Assert.fail("Unexpected exception: " + ioe);
888         }
889       }
890     } finally {
891       conf.setLong(F_INTERVAL, oldFlushInterval);
892       if (log != null) {
893         log.close();
894       }
895       if (fs.exists(thisTestsDir)) {
896         fs.delete(thisTestsDir, true);
897       }
898     }
899   }
900 
901   /**
902    * This thread will keep writing to the file after the split process has started
903    * It simulates a region server that was considered dead but woke up and wrote
904    * some more to he last log entry
905    */
906   class ZombieLastLogWriterRegionServer extends Thread {
907     AtomicLong editsCount;
908     AtomicBoolean stop;
909     Path log;
910     HLog.Writer lastLogWriter;
911     public ZombieLastLogWriterRegionServer(HLog.Writer writer, AtomicLong counter, AtomicBoolean stop) {
912       this.stop = stop;
913       this.editsCount = counter;
914       this.lastLogWriter = writer;
915     }
916 
917     @Override
918     public void run() {
919       if (stop.get()){
920         return;
921       }
922       flushToConsole("starting");
923       while (true) {
924         try {
925           String region = "juliet";
926 
927           fs.mkdirs(new Path(new Path(hbaseDir, region), region));
928           appendEntry(lastLogWriter, TABLE_NAME, region.getBytes(),
929                   ("r" + editsCount).getBytes(), FAMILY, QUALIFIER, VALUE, 0);
930           lastLogWriter.sync();
931           editsCount.incrementAndGet();
932           try {
933             Thread.sleep(1);
934           } catch (InterruptedException e) {
935             //
936           }
937 
938 
939         } catch (IOException ex) {
940           if (ex instanceof RemoteException) {
941             flushToConsole("Juliet: got RemoteException " +
942                     ex.getMessage() + " while writing " + (editsCount.get() + 1));
943             break;
944           } else {
945             assertTrue("Failed to write " + editsCount.get(), false);
946           }
947 
948         }
949       }
950 
951 
952     }
953   }
954 
955   /**
956    * This thread will keep adding new log files
957    * It simulates a region server that was considered dead but woke up and wrote
958    * some more to a new hlog
959    */
960   class ZombieNewLogWriterRegionServer extends Thread {
961     AtomicBoolean stop;
962     public ZombieNewLogWriterRegionServer(AtomicBoolean stop) {
963       super("ZombieNewLogWriterRegionServer");
964       this.stop = stop;
965     }
966 
967     @Override
968     public void run() {
969       if (stop.get()) {
970         return;
971       }
972       Path tableDir = new Path(hbaseDir, new String(TABLE_NAME));
973       Path regionDir = new Path(tableDir, regions.get(0));
974       Path recoveredEdits = new Path(regionDir, HLogSplitter.RECOVERED_EDITS);
975       String region = "juliet";
976       Path julietLog = new Path(hlogDir, HLOG_FILE_PREFIX + ".juliet");
977       try {
978 
979         while (!fs.exists(recoveredEdits) && !stop.get()) {
980           flushToConsole("Juliet: split not started, sleeping a bit...");
981           Threads.sleep(10);
982         }
983  
984         fs.mkdirs(new Path(tableDir, region));
985         HLog.Writer writer = HLog.createWriter(fs,
986             julietLog, conf);
987         appendEntry(writer, "juliet".getBytes(), ("juliet").getBytes(),
988             ("r").getBytes(), FAMILY, QUALIFIER, VALUE, 0);
989         writer.close();
990         flushToConsole("Juliet file creator: created file " + julietLog);
991       } catch (IOException e1) {
992         assertTrue("Failed to create file " + julietLog, false);
993       }
994     }
995   }
996 
997   private CancelableProgressable reporter = new CancelableProgressable() {
998     int count = 0;
999 
1000     @Override
1001     public boolean progress() {
1002       count++;
1003       LOG.debug("progress = " + count);
1004       return true;
1005     }
1006   };
1007 
1008   @Test
1009   public void testSplitLogFileWithOneRegion() throws IOException {
1010     LOG.info("testSplitLogFileWithOneRegion");
1011     final String REGION = "region__1";
1012     regions.removeAll(regions);
1013     regions.add(REGION);
1014 
1015 
1016     generateHLogs(1, 10, -1);
1017     FileStatus logfile = fs.listStatus(hlogDir)[0];
1018     fs.initialize(fs.getUri(), conf);
1019     HLogSplitter.splitLogFile(hbaseDir, logfile, fs, conf, reporter);
1020     HLogSplitter.finishSplitLogFile(hbaseDir, oldLogDir, logfile.getPath()
1021         .toString(), conf);
1022 
1023 
1024     Path originalLog = (fs.listStatus(oldLogDir))[0].getPath();
1025     Path splitLog = getLogForRegion(hbaseDir, TABLE_NAME, REGION);
1026 
1027 
1028     assertEquals(true, logsAreEqual(originalLog, splitLog));
1029   }
1030   
1031   @Test
1032   public void testSplitLogFileDeletedRegionDir()
1033   throws IOException {
1034 	LOG.info("testSplitLogFileDeletedRegionDir");
1035 	final String REGION = "region__1";
1036     regions.removeAll(regions);
1037     regions.add(REGION);
1038 
1039 
1040     generateHLogs(1, 10, -1);
1041     FileStatus logfile = fs.listStatus(hlogDir)[0];
1042     fs.initialize(fs.getUri(), conf);
1043     
1044     Path regiondir = new Path(tabledir, REGION);
1045     LOG.info("Region directory is" + regiondir);
1046     fs.delete(regiondir, true);
1047     
1048     HLogSplitter.splitLogFile(hbaseDir, logfile, fs, conf, reporter);
1049     HLogSplitter.finishSplitLogFile(hbaseDir, oldLogDir, logfile.getPath()
1050         .toString(), conf);
1051     
1052     assertTrue(!fs.exists(regiondir));
1053     assertTrue(true);
1054   }
1055 
1056   
1057   
1058   @Test
1059   public void testSplitLogFileEmpty() throws IOException {
1060     LOG.info("testSplitLogFileEmpty");
1061     injectEmptyFile(".empty", true);
1062     FileStatus logfile = fs.listStatus(hlogDir)[0];
1063 
1064     fs.initialize(fs.getUri(), conf);
1065 
1066     HLogSplitter.splitLogFile(hbaseDir, logfile, fs, conf, reporter);
1067     HLogSplitter.finishSplitLogFile(hbaseDir, oldLogDir, logfile.getPath()
1068         .toString(), conf);
1069     Path tdir = HTableDescriptor.getTableDir(hbaseDir, TABLE_NAME);
1070     assertFalse(fs.exists(tdir));
1071 
1072     assertEquals(0, countHLog(fs.listStatus(oldLogDir)[0].getPath(), fs, conf));
1073   }
1074 
1075   @Test
1076   public void testSplitLogFileMultipleRegions() throws IOException {
1077     LOG.info("testSplitLogFileMultipleRegions");
1078     generateHLogs(1, 10, -1);
1079     FileStatus logfile = fs.listStatus(hlogDir)[0];
1080     fs.initialize(fs.getUri(), conf);
1081 
1082     HLogSplitter.splitLogFile(hbaseDir, logfile, fs, conf, reporter);
1083     HLogSplitter.finishSplitLogFile(hbaseDir, oldLogDir, logfile.getPath()
1084         .toString(), conf);
1085     for (String region : regions) {
1086       Path recovered = getLogForRegion(hbaseDir, TABLE_NAME, region);
1087       assertEquals(10, countHLog(recovered, fs, conf));
1088     }
1089   }
1090 
1091   @Test
1092   public void testSplitLogFileFirstLineCorruptionLog()
1093   throws IOException {
1094     conf.setBoolean(HBASE_SKIP_ERRORS, true);
1095     generateHLogs(1, 10, -1);
1096     FileStatus logfile = fs.listStatus(hlogDir)[0];
1097 
1098     corruptHLog(logfile.getPath(),
1099         Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true, fs);
1100 
1101     fs.initialize(fs.getUri(), conf);
1102     HLogSplitter.splitLogFile(hbaseDir, logfile, fs, conf, reporter);
1103     HLogSplitter.finishSplitLogFile(hbaseDir, oldLogDir, logfile.getPath()
1104         .toString(), conf);
1105 
1106     final Path corruptDir = new Path(conf.get(HConstants.HBASE_DIR), conf.get(
1107         "hbase.regionserver.hlog.splitlog.corrupt.dir", ".corrupt"));
1108     assertEquals(1, fs.listStatus(corruptDir).length);
1109   }
1110 
1111   /**
1112    * @throws IOException
1113    * @see https://issues.apache.org/jira/browse/HBASE-4862
1114    */
1115   @Test
1116   public void testConcurrentSplitLogAndReplayRecoverEdit() throws IOException {
1117     LOG.info("testConcurrentSplitLogAndReplayRecoverEdit");
1118     // Generate hlogs for our destination region
1119     String regionName = "r0";
1120     final Path regiondir = new Path(tabledir, regionName);
1121     regions = new ArrayList<String>();
1122     regions.add(regionName);
1123     generateHLogs(-1);
1124 
1125     HLogSplitter logSplitter = new HLogSplitter(
1126         conf, hbaseDir, hlogDir, oldLogDir, fs) {
1127       protected HLog.Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
1128       throws IOException {
1129         HLog.Writer writer = HLog.createWriter(fs, logfile, conf);
1130         // After creating writer, simulate region's
1131         // replayRecoveredEditsIfAny() which gets SplitEditFiles of this
1132         // region and delete them, excluding files with '.temp' suffix.
1133         NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs,
1134             regiondir);
1135         if (files != null && !files.isEmpty()) {
1136           for (Path file : files) {
1137             if (!this.fs.delete(file, false)) {
1138               LOG.error("Failed delete of " + file);
1139             } else {
1140               LOG.debug("Deleted recovered.edits file=" + file);
1141             }
1142           }
1143         }
1144         return writer;
1145       }
1146     };
1147     try{
1148       logSplitter.splitLog();
1149     } catch (IOException e) {
1150       LOG.info(e);
1151       Assert.fail("Throws IOException when spliting "
1152           + "log, it is most likely because writing file does not "
1153           + "exist which is caused by concurrent replayRecoveredEditsIfAny()");
1154     }
1155     if (fs.exists(corruptDir)) {
1156       if (fs.listStatus(corruptDir).length > 0) {
1157         Assert.fail("There are some corrupt logs, "
1158                 + "it is most likely caused by concurrent replayRecoveredEditsIfAny()");
1159       }
1160     }
1161   }
1162 
1163   private void flushToConsole(String s) {
1164     System.out.println(s);
1165     System.out.flush();
1166   }
1167 
1168 
1169   private void generateHLogs(int leaveOpen) throws IOException {
1170     generateHLogs(NUM_WRITERS, ENTRIES, leaveOpen);
1171   }
1172 
1173   private void makeRegionDirs(FileSystem fs, List<String> regions) throws IOException {
1174     for (String region : regions) {
1175       flushToConsole("Creating dir for region " + region);
1176       fs.mkdirs(new Path(tabledir, region));
1177     }
1178   }
1179 
1180   private void generateHLogs(int writers, int entries, int leaveOpen) throws IOException {
1181     makeRegionDirs(fs, regions);
1182     fs.mkdirs(hlogDir);
1183     for (int i = 0; i < writers; i++) {
1184       writer[i] = HLog.createWriter(fs, new Path(hlogDir, HLOG_FILE_PREFIX + i), conf);
1185       for (int j = 0; j < entries; j++) {
1186         int prefix = 0;
1187         for (String region : regions) {
1188           String row_key = region + prefix++ + i + j;
1189           appendEntry(writer[i], TABLE_NAME, region.getBytes(),
1190                   row_key.getBytes(), FAMILY, QUALIFIER, VALUE, seq);
1191         }
1192       }
1193       if (i != leaveOpen) {
1194         writer[i].close();
1195         flushToConsole("Closing writer " + i);
1196       }
1197     }
1198   }
1199 
1200   private Path getLogForRegion(Path rootdir, byte[] table, String region)
1201   throws IOException {
1202     Path tdir = HTableDescriptor.getTableDir(rootdir, table);
1203     Path editsdir = HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
1204       Bytes.toString(region.getBytes())));
1205     FileStatus [] files = this.fs.listStatus(editsdir);
1206     assertEquals(1, files.length);
1207     return files[0].getPath();
1208   }
1209 
1210   private void corruptHLog(Path path, Corruptions corruption, boolean close,
1211                            FileSystem fs) throws IOException {
1212 
1213     FSDataOutputStream out;
1214     int fileSize = (int) fs.listStatus(path)[0].getLen();
1215 
1216     FSDataInputStream in = fs.open(path);
1217     byte[] corrupted_bytes = new byte[fileSize];
1218     in.readFully(0, corrupted_bytes, 0, fileSize);
1219     in.close();
1220 
1221     switch (corruption) {
1222       case APPEND_GARBAGE:
1223         fs.delete(path, false);
1224         out = fs.create(path);
1225         out.write(corrupted_bytes);
1226         out.write("-----".getBytes());
1227         closeOrFlush(close, out);
1228         break;
1229 
1230       case INSERT_GARBAGE_ON_FIRST_LINE:
1231         fs.delete(path, false);
1232         out = fs.create(path);
1233         out.write(0);
1234         out.write(corrupted_bytes);
1235         closeOrFlush(close, out);
1236         break;
1237 
1238       case INSERT_GARBAGE_IN_THE_MIDDLE:
1239         fs.delete(path, false);
1240         out = fs.create(path);
1241         int middle = (int) Math.floor(corrupted_bytes.length / 2);
1242         out.write(corrupted_bytes, 0, middle);
1243         out.write(0);
1244         out.write(corrupted_bytes, middle, corrupted_bytes.length - middle);
1245         closeOrFlush(close, out);
1246         break;
1247 
1248       case TRUNCATE:
1249         fs.delete(path, false);
1250         out = fs.create(path);
1251         out.write(corrupted_bytes, 0, fileSize-32);
1252         closeOrFlush(close, out);
1253 
1254         break;
1255     }
1256 
1257 
1258   }
1259 
1260   private void closeOrFlush(boolean close, FSDataOutputStream out)
1261   throws IOException {
1262     if (close) {
1263       out.close();
1264     } else {
1265       Method syncMethod = null;
1266       try {
1267         syncMethod = out.getClass().getMethod("hflush", new Class<?> []{});
1268       } catch (NoSuchMethodException e) {
1269         try {
1270           syncMethod = out.getClass().getMethod("sync", new Class<?> []{});
1271         } catch (NoSuchMethodException ex) {
1272           throw new IOException("This version of Hadoop supports " +
1273               "neither Syncable.sync() nor Syncable.hflush().");
1274         }
1275       }
1276       try {
1277         syncMethod.invoke(out, new Object[]{});
1278       } catch (Exception e) {
1279         throw new IOException(e);
1280       }
1281       // Not in 0out.hflush();
1282     }
1283   }
1284 
1285   @SuppressWarnings("unused")
1286   private void dumpHLog(Path log, FileSystem fs, Configuration conf) throws IOException {
1287     HLog.Entry entry;
1288     HLog.Reader in = HLog.getReader(fs, log, conf);
1289     while ((entry = in.next()) != null) {
1290       System.out.println(entry);
1291     }
1292   }
1293 
1294   private int countHLog(Path log, FileSystem fs, Configuration conf) throws IOException {
1295     int count = 0;
1296     HLog.Reader in = HLog.getReader(fs, log, conf);
1297     while (in.next() != null) {
1298       count++;
1299     }
1300     return count;
1301   }
1302 
1303 
1304   public long appendEntry(HLog.Writer writer, byte[] table, byte[] region,
1305                           byte[] row, byte[] family, byte[] qualifier,
1306                           byte[] value, long seq)
1307           throws IOException {
1308 
1309     writer.append(createTestEntry(table, region, row, family, qualifier, value, seq));
1310     writer.sync();
1311     return seq;
1312   }
1313 
1314   private HLog.Entry createTestEntry(
1315       byte[] table, byte[] region,
1316       byte[] row, byte[] family, byte[] qualifier,
1317       byte[] value, long seq) {
1318     long time = System.nanoTime();
1319     WALEdit edit = new WALEdit();
1320     seq++;
1321     edit.add(new KeyValue(row, family, qualifier, time, KeyValue.Type.Put, value));
1322     return new HLog.Entry(new HLogKey(region, table, seq, time,
1323         HConstants.DEFAULT_CLUSTER_ID), edit);
1324   }
1325 
1326 
1327   private void injectEmptyFile(String suffix, boolean closeFile)
1328           throws IOException {
1329     HLog.Writer writer = HLog.createWriter(
1330             fs, new Path(hlogDir, HLOG_FILE_PREFIX + suffix), conf);
1331     if (closeFile) writer.close();
1332   }
1333 
1334   @SuppressWarnings("unused")
1335   private void listLogs(FileSystem fs, Path dir) throws IOException {
1336     for (FileStatus file : fs.listStatus(dir)) {
1337       System.out.println(file.getPath());
1338     }
1339 
1340   }
1341 
1342   private int compareHLogSplitDirs(Path p1, Path p2) throws IOException {
1343     FileStatus[] f1 = fs.listStatus(p1);
1344     FileStatus[] f2 = fs.listStatus(p2);
1345     assertNotNull("Path " + p1 + " doesn't exist", f1);
1346     assertNotNull("Path " + p2 + " doesn't exist", f2);
1347 
1348     System.out.println("Files in " + p1 + ": " +
1349         Joiner.on(",").join(FileUtil.stat2Paths(f1)));
1350     System.out.println("Files in " + p2 + ": " +
1351         Joiner.on(",").join(FileUtil.stat2Paths(f2)));
1352     assertEquals(f1.length, f2.length);
1353 
1354     for (int i = 0; i < f1.length; i++) {
1355       // Regions now have a directory named RECOVERED_EDITS_DIR and in here
1356       // are split edit files. In below presume only 1.
1357       Path rd1 = HLog.getRegionDirRecoveredEditsDir(f1[i].getPath());
1358       FileStatus[] rd1fs = fs.listStatus(rd1);
1359       assertEquals(1, rd1fs.length);
1360       Path rd2 = HLog.getRegionDirRecoveredEditsDir(f2[i].getPath());
1361       FileStatus[] rd2fs = fs.listStatus(rd2);
1362       assertEquals(1, rd2fs.length);
1363       if (!logsAreEqual(rd1fs[0].getPath(), rd2fs[0].getPath())) {
1364         return -1;
1365       }
1366     }
1367     return 0;
1368   }
1369 
1370   private boolean logsAreEqual(Path p1, Path p2) throws IOException {
1371     HLog.Reader in1, in2;
1372     in1 = HLog.getReader(fs, p1, conf);
1373     in2 = HLog.getReader(fs, p2, conf);
1374     HLog.Entry entry1;
1375     HLog.Entry entry2;
1376     while ((entry1 = in1.next()) != null) {
1377       entry2 = in2.next();
1378       if ((entry1.getKey().compareTo(entry2.getKey()) != 0) ||
1379               (!entry1.getEdit().toString().equals(entry2.getEdit().toString()))) {
1380         return false;
1381       }
1382     }
1383     return true;
1384   }
1385 
1386   @org.junit.Rule
1387   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
1388     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
1389 }
1390