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