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