1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver.wal;
21  
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertNull;
24  import static org.junit.Assert.assertTrue;
25  
26  import java.io.FileNotFoundException;
27  import java.io.IOException;
28  import java.util.ArrayList;
29  import java.util.Collections;
30  import java.util.List;
31  import java.util.concurrent.atomic.AtomicBoolean;
32  import java.util.concurrent.atomic.AtomicLong;
33  
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.fs.FSDataInputStream;
36  import org.apache.hadoop.fs.FSDataOutputStream;
37  import org.apache.hadoop.fs.FileStatus;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.HBaseTestingUtility;
41  import org.apache.hadoop.hbase.HRegionInfo;
42  import org.apache.hadoop.hbase.HTableDescriptor;
43  import org.apache.hadoop.hbase.KeyValue;
44  import org.apache.hadoop.hbase.regionserver.HRegion;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.Threads;
47  import org.apache.hadoop.ipc.RemoteException;
48  import org.junit.After;
49  import org.junit.AfterClass;
50  import org.junit.Before;
51  import org.junit.BeforeClass;
52  import org.junit.Test;
53  
54  /**
55   * Testing {@link HLog} splitting code.
56   */
57  public class TestHLogSplit {
58  
59    private Configuration conf;
60    private FileSystem fs;
61  
62    private final static HBaseTestingUtility
63            TEST_UTIL = new HBaseTestingUtility();
64  
65  
66    private static final Path hbaseDir = new Path("/hbase");
67    private static final Path hlogDir = new Path(hbaseDir, "hlog");
68    private static final Path oldLogDir = new Path(hbaseDir, "hlog.old");
69    private static final Path corruptDir = new Path(hbaseDir, ".corrupt");
70  
71    private static final int NUM_WRITERS = 10;
72    private static final int ENTRIES = 10; // entries per writer per region
73  
74    private HLog.Writer[] writer = new HLog.Writer[NUM_WRITERS];
75    private long seq = 0;
76    private static final byte[] TABLE_NAME = "t1".getBytes();
77    private static final byte[] FAMILY = "f1".getBytes();
78    private static final byte[] QUALIFIER = "q1".getBytes();
79    private static final byte[] VALUE = "v1".getBytes();
80    private static final String HLOG_FILE_PREFIX = "hlog.dat.";
81    private static List<String> regions;
82    private static final String HBASE_SKIP_ERRORS = "hbase.hlog.split.skip.errors";
83  
84  
85    static enum Corruptions {
86      INSERT_GARBAGE_ON_FIRST_LINE,
87      INSERT_GARBAGE_IN_THE_MIDDLE,
88      APPEND_GARBAGE,
89      TRUNCATE,
90    }
91  
92    @BeforeClass
93    public static void setUpBeforeClass() throws Exception {
94      TEST_UTIL.getConfiguration().
95              setInt("hbase.regionserver.flushlogentries", 1);
96      TEST_UTIL.getConfiguration().
97              setBoolean("dfs.support.append", true);
98      TEST_UTIL.getConfiguration().
99              setStrings("hbase.rootdir", hbaseDir.toString());
100     TEST_UTIL.getConfiguration().
101             setClass("hbase.regionserver.hlog.writer.impl",
102                 InstrumentedSequenceFileLogWriter.class, HLog.Writer.class);
103 
104     TEST_UTIL.startMiniDFSCluster(2);
105   }
106 
107   @AfterClass
108   public static void tearDownAfterClass() throws Exception {
109     TEST_UTIL.shutdownMiniDFSCluster();
110   }
111 
112   @Before
113   public void setUp() throws Exception {
114     conf = TEST_UTIL.getConfiguration();
115     fs = TEST_UTIL.getDFSCluster().getFileSystem();
116     FileStatus[] entries = fs.listStatus(new Path("/"));
117     for (FileStatus dir : entries){
118       fs.delete(dir.getPath(), true);
119     }
120     seq = 0;
121     regions = new ArrayList<String>();
122     Collections.addAll(regions, "bbb", "ccc");
123     InstrumentedSequenceFileLogWriter.activateFailure = false;
124     // Set the soft lease for hdfs to be down from default of 5 minutes or so.
125     TEST_UTIL.setNameNodeNameSystemLeasePeriod(100, 50000);
126   }
127 
128   @After
129   public void tearDown() throws Exception {
130   }
131 
132   @Test(expected = IOException.class)
133   public void testSplitFailsIfNewHLogGetsCreatedAfterSplitStarted()
134   throws IOException {
135     AtomicBoolean stop = new AtomicBoolean(false);
136     generateHLogs(-1);
137     fs.initialize(fs.getUri(), conf);
138     try {
139     (new ZombieNewLogWriterRegionServer(stop)).start();
140     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
141     } finally {
142       stop.set(true);
143     }
144   }
145 
146 
147   @Test
148   public void testSplitPreservesEdits() throws IOException{
149     final String REGION = "region__1";
150     regions.removeAll(regions);
151     regions.add(REGION);
152 
153     generateHLogs(1, 10, -1);
154     fs.initialize(fs.getUri(), conf);
155     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
156 
157     Path originalLog = (fs.listStatus(oldLogDir))[0].getPath();
158     Path splitLog = getLogForRegion(hbaseDir, TABLE_NAME, REGION);
159 
160     assertEquals("edits differ after split", true, logsAreEqual(originalLog, splitLog));
161   }
162 
163 
164   @Test
165   public void testEmptyLogFiles() throws IOException {
166 
167     injectEmptyFile(".empty", true);
168     generateHLogs(Integer.MAX_VALUE);
169     injectEmptyFile("empty", true);
170 
171     // make fs act as a different client now
172     // initialize will create a new DFSClient with a new client ID
173     fs.initialize(fs.getUri(), conf);
174 
175     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
176 
177 
178     for (String region : regions) {
179       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
180       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
181     }
182 
183   }
184 
185 
186   @Test
187   public void testEmptyOpenLogFiles() throws IOException {
188     injectEmptyFile(".empty", false);
189     generateHLogs(Integer.MAX_VALUE);
190     injectEmptyFile("empty", false);
191 
192     // make fs act as a different client now
193     // initialize will create a new DFSClient with a new client ID
194     fs.initialize(fs.getUri(), conf);
195 
196     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
197 
198     for (String region : regions) {
199       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
200       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
201     }
202   }
203 
204   @Test
205   public void testOpenZeroLengthReportedFileButWithDataGetsSplit() throws IOException {
206     // generate logs but leave hlog.dat.5 open.
207     generateHLogs(5);
208 
209     fs.initialize(fs.getUri(), conf);
210 
211     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
212 
213     for (String region : regions) {
214       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
215       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
216     }
217 
218 
219   }
220 
221 
222   @Test
223   public void testTralingGarbageCorruptionFileSkipErrorsPasses() throws IOException {
224     conf.setBoolean(HBASE_SKIP_ERRORS, true);
225     generateHLogs(Integer.MAX_VALUE);
226     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
227             Corruptions.APPEND_GARBAGE, true, fs);
228     fs.initialize(fs.getUri(), conf);
229     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
230 
231     for (String region : regions) {
232       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
233       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
234     }
235 
236 
237   }
238 
239   @Test
240   public void testFirstLineCorruptionLogFileSkipErrorsPasses() throws IOException {
241     conf.setBoolean(HBASE_SKIP_ERRORS, true);
242     generateHLogs(Integer.MAX_VALUE);
243     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
244             Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true, fs);
245     fs.initialize(fs.getUri(), conf);
246     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
247 
248     for (String region : regions) {
249       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
250       assertEquals((NUM_WRITERS - 1) * ENTRIES, countHLog(logfile, fs, conf));
251     }
252 
253 
254   }
255 
256 
257   @Test
258   public void testMiddleGarbageCorruptionSkipErrorsReadsHalfOfFile() throws IOException {
259     conf.setBoolean(HBASE_SKIP_ERRORS, true);
260     generateHLogs(Integer.MAX_VALUE);
261     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
262             Corruptions.INSERT_GARBAGE_IN_THE_MIDDLE, false, fs);
263     fs.initialize(fs.getUri(), conf);
264     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
265 
266     for (String region : regions) {
267       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
268       // the entries in the original logs are alternating regions
269       // considering the sequence file header, the middle corruption should
270       // affect at least half of the entries
271       int goodEntries = (NUM_WRITERS - 1) * ENTRIES;
272       int firstHalfEntries = (int) Math.ceil(ENTRIES / 2) - 1;
273       assertTrue("The file up to the corrupted area hasn't been parsed",
274               goodEntries + firstHalfEntries <= countHLog(logfile, fs, conf));
275     }
276   }
277 
278   // TODO: fix this test (HBASE-2935)
279   //@Test
280   public void testCorruptedFileGetsArchivedIfSkipErrors() throws IOException {
281     conf.setBoolean(HBASE_SKIP_ERRORS, true);
282 
283     Path c1 = new Path(hlogDir, HLOG_FILE_PREFIX + "0");
284     Path c2 = new Path(hlogDir, HLOG_FILE_PREFIX + "5");
285     Path c3 = new Path(hlogDir, HLOG_FILE_PREFIX + (NUM_WRITERS - 1));
286     generateHLogs(-1);
287     corruptHLog(c1, Corruptions.INSERT_GARBAGE_IN_THE_MIDDLE, false, fs);
288     corruptHLog(c2, Corruptions.APPEND_GARBAGE, true, fs);
289     corruptHLog(c3, Corruptions.INSERT_GARBAGE_ON_FIRST_LINE, true, fs);
290 
291     fs.initialize(fs.getUri(), conf);
292     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
293 
294     FileStatus[] archivedLogs = fs.listStatus(corruptDir);
295 
296     assertEquals("expected a different file", c1.getName(), archivedLogs[0].getPath().getName());
297     assertEquals("expected a different file", c2.getName(), archivedLogs[1].getPath().getName());
298     assertEquals("expected a different file", c3.getName(), archivedLogs[2].getPath().getName());
299     assertEquals(archivedLogs.length, 3);
300 
301   }
302 
303   @Test
304   public void testEOFisIgnored() throws IOException {
305     conf.setBoolean(HBASE_SKIP_ERRORS, false);
306 
307     final String REGION = "region__1";
308     regions.removeAll(regions);
309     regions.add(REGION);
310 
311     int entryCount = 10;
312     Path c1 = new Path(hlogDir, HLOG_FILE_PREFIX + "0");
313     generateHLogs(1, entryCount, -1);
314     corruptHLog(c1, Corruptions.TRUNCATE, true, fs);
315 
316     fs.initialize(fs.getUri(), conf);
317     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
318 
319     Path originalLog = (fs.listStatus(oldLogDir))[0].getPath();
320     Path splitLog = getLogForRegion(hbaseDir, TABLE_NAME, REGION);
321 
322     int actualCount = 0;
323     HLog.Reader in = HLog.getReader(fs, splitLog, conf);
324     HLog.Entry entry;
325     while ((entry = in.next()) != null) ++actualCount;
326     assertEquals(entryCount-1, actualCount);
327     
328     // should not have stored the EOF files as corrupt
329     FileStatus[] archivedLogs = fs.listStatus(corruptDir);
330     assertEquals(archivedLogs.length, 0);
331   }
332   
333   @Test
334   public void testLogsGetArchivedAfterSplit() throws IOException {
335     conf.setBoolean(HBASE_SKIP_ERRORS, false);
336 
337     generateHLogs(-1);
338 
339     fs.initialize(fs.getUri(), conf);
340     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
341 
342     FileStatus[] archivedLogs = fs.listStatus(oldLogDir);
343 
344     assertEquals("wrong number of files in the archive log", NUM_WRITERS, archivedLogs.length);
345   }
346 
347 
348 
349   // TODO: fix this test (HBASE-2935)
350   //@Test(expected = IOException.class)
351   public void testTrailingGarbageCorruptionLogFileSkipErrorsFalseThrows() throws IOException {
352     conf.setBoolean(HBASE_SKIP_ERRORS, false);
353     generateHLogs(Integer.MAX_VALUE);
354     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
355             Corruptions.APPEND_GARBAGE, true, fs);
356 
357     fs.initialize(fs.getUri(), conf);
358     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
359   }
360 
361   // TODO: fix this test (HBASE-2935)
362   //@Test
363   public void testCorruptedLogFilesSkipErrorsFalseDoesNotTouchLogs() throws IOException {
364     conf.setBoolean(HBASE_SKIP_ERRORS, false);
365     generateHLogs(-1);
366     corruptHLog(new Path(hlogDir, HLOG_FILE_PREFIX + "5"),
367             Corruptions.APPEND_GARBAGE, true, fs);
368     fs.initialize(fs.getUri(), conf);
369     try {
370       HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
371     } catch (IOException e) {/* expected */}
372 
373     assertEquals("if skip.errors is false all files should remain in place",
374             NUM_WRITERS, fs.listStatus(hlogDir).length);
375   }
376 
377 
378   @Test
379   public void testSplit() throws IOException {
380     generateHLogs(-1);
381     fs.initialize(fs.getUri(), conf);
382     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
383 
384 
385 
386     for (String region : regions) {
387       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, region);
388       assertEquals(NUM_WRITERS * ENTRIES, countHLog(logfile, fs, conf));
389 
390     }
391   }
392 
393   @Test
394   public void testLogDirectoryShouldBeDeletedAfterSuccessfulSplit()
395   throws IOException {
396     generateHLogs(-1);
397     fs.initialize(fs.getUri(), conf);
398     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
399     FileStatus [] statuses = null;
400     try {
401       statuses = fs.listStatus(hlogDir);
402       assertNull(statuses);
403     } catch (FileNotFoundException e) {
404       // hadoop 0.21 throws FNFE whereas hadoop 0.20 returns null
405     }
406   }
407 /* DISABLED for now.  TODO: HBASE-2645 
408   @Test
409   public void testLogCannotBeWrittenOnceParsed() throws IOException {
410     AtomicLong counter = new AtomicLong(0);
411     AtomicBoolean stop = new AtomicBoolean(false);
412     generateHLogs(9);
413     fs.initialize(fs.getUri(), conf);
414 
415     Thread zombie = new ZombieLastLogWriterRegionServer(writer[9], counter, stop);
416 
417 
418 
419     try {
420       zombie.start();
421 
422       HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
423 
424       Path logfile = getLogForRegion(hbaseDir, TABLE_NAME, "juliet");
425 
426       // It's possible that the writer got an error while appending and didn't count it
427       // however the entry will in fact be written to file and split with the rest
428       long numberOfEditsInRegion = countHLog(logfile, fs, conf);
429       assertTrue("The log file could have at most 1 extra log entry, but " +
430               "can't have less. Zombie could write "+counter.get() +" and logfile had only"+ numberOfEditsInRegion+" "  + logfile, counter.get() == numberOfEditsInRegion ||
431                       counter.get() + 1 == numberOfEditsInRegion);
432     } finally {
433       stop.set(true);
434     }
435   }
436 */
437 
438   @Test
439   public void testSplitWillNotTouchLogsIfNewHLogGetsCreatedAfterSplitStarted()
440   throws IOException {
441     AtomicBoolean stop = new AtomicBoolean(false);
442     generateHLogs(-1);
443     fs.initialize(fs.getUri(), conf);
444     Thread zombie = new ZombieNewLogWriterRegionServer(stop);
445     
446     try {
447       zombie.start();
448       try {
449         HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
450       } catch (IOException ex) {/* expected */}
451       int logFilesNumber = fs.listStatus(hlogDir).length;
452 
453       assertEquals("Log files should not be archived if there's an extra file after split",
454               NUM_WRITERS + 1, logFilesNumber);
455     } finally {
456       stop.set(true);
457     }
458 
459   }
460 
461 
462 
463   @Test(expected = IOException.class)
464   public void testSplitWillFailIfWritingToRegionFails() throws Exception {
465     //leave 5th log open so we could append the "trap"
466     generateHLogs(4);
467 
468     fs.initialize(fs.getUri(), conf);
469 
470     InstrumentedSequenceFileLogWriter.activateFailure = false;
471     appendEntry(writer[4], TABLE_NAME, Bytes.toBytes("break"), ("r" + 999).getBytes(), FAMILY, QUALIFIER, VALUE, 0);
472     writer[4].close();
473 
474 
475     try {
476       InstrumentedSequenceFileLogWriter.activateFailure = true;
477       HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
478 
479     } catch (IOException e) {
480       assertEquals("java.io.IOException: This exception is instrumented and should only be thrown for testing", e.getMessage());
481       throw e;
482     } finally {
483       InstrumentedSequenceFileLogWriter.activateFailure = false;
484     }
485   }
486 
487 
488 //  @Test
489   public void testSplittingLargeNumberOfRegionsConsistency() throws IOException {
490 
491     regions.removeAll(regions);
492     for (int i=0; i<100; i++) {
493       regions.add("region__"+i);
494     }
495 
496     generateHLogs(1, 100, -1);
497     fs.initialize(fs.getUri(), conf);
498 
499     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
500     fs.rename(oldLogDir, hlogDir);
501     Path firstSplitPath = new Path(hbaseDir, Bytes.toString(TABLE_NAME) + ".first");
502     Path splitPath = new Path(hbaseDir, Bytes.toString(TABLE_NAME));
503     fs.rename(splitPath,
504             firstSplitPath);
505 
506 
507     fs.initialize(fs.getUri(), conf);
508     HLog.splitLog(hbaseDir, hlogDir, oldLogDir, fs, conf);
509 
510     assertEquals(0, compareHLogSplitDirs(firstSplitPath, splitPath));
511   }
512 
513   /**
514    * This thread will keep writing to the file after the split process has started
515    * It simulates a region server that was considered dead but woke up and wrote
516    * some more to he last log entry
517    */
518   class ZombieLastLogWriterRegionServer extends Thread {
519     AtomicLong editsCount;
520     AtomicBoolean stop;
521     Path log;
522     HLog.Writer lastLogWriter;
523     public ZombieLastLogWriterRegionServer(HLog.Writer writer, AtomicLong counter, AtomicBoolean stop) {
524       this.stop = stop;
525       this.editsCount = counter;
526       this.lastLogWriter = writer;
527     }
528 
529     @Override
530     public void run() {
531       if (stop.get()){
532         return;
533       }
534       flushToConsole("starting");
535       while (true) {
536         try {
537 
538           appendEntry(lastLogWriter, TABLE_NAME, "juliet".getBytes(),
539                   ("r" + editsCount).getBytes(), FAMILY, QUALIFIER, VALUE, 0);
540           lastLogWriter.sync();
541           editsCount.incrementAndGet();
542           try {
543             Thread.sleep(1);
544           } catch (InterruptedException e) {
545             //
546           }
547 
548 
549         } catch (IOException ex) {
550           if (ex instanceof RemoteException) {
551             flushToConsole("Juliet: got RemoteException " +
552                     ex.getMessage() + " while writing " + (editsCount.get() + 1));
553             break;
554           } else {
555             assertTrue("Failed to write " + editsCount.get(), false);
556           }
557 
558         }
559       }
560 
561 
562     }
563   }
564 
565   /**
566    * This thread will keep adding new log files
567    * It simulates a region server that was considered dead but woke up and wrote
568    * some more to a new hlog
569    */
570   class ZombieNewLogWriterRegionServer extends Thread {
571     AtomicBoolean stop;
572     public ZombieNewLogWriterRegionServer(AtomicBoolean stop) {
573       super("ZombieNewLogWriterRegionServer");
574       this.stop = stop;
575     }
576 
577     @Override
578     public void run() {
579       if (stop.get()) {
580         return;
581       }
582       boolean splitStarted = false;
583       Path p = new Path(hbaseDir, new String(TABLE_NAME));
584       while (!splitStarted) {
585         try {
586           FileStatus [] statuses = fs.listStatus(p);
587           // In 0.20, listStatus comes back with a null if file doesn't exit.
588           // In 0.21, it throws FNFE.
589           if (statuses != null && statuses.length > 0) {
590             // Done.
591             break;
592           }
593         } catch (FileNotFoundException e) {
594           // Expected in hadoop 0.21
595         } catch (IOException e1) {
596           assertTrue("Failed to list status ", false);
597         }
598         flushToConsole("Juliet: split not started, sleeping a bit...");
599         Threads.sleep(100);
600       }
601 
602       Path julietLog = new Path(hlogDir, HLOG_FILE_PREFIX + ".juliet");
603       try {
604         HLog.Writer writer = HLog.createWriter(fs,
605                 julietLog, conf);
606         appendEntry(writer, "juliet".getBytes(), ("juliet").getBytes(),
607                 ("r").getBytes(), FAMILY, QUALIFIER, VALUE, 0);
608         writer.close();
609         flushToConsole("Juliet file creator: created file " + julietLog);
610       } catch (IOException e1) {
611         assertTrue("Failed to create file " + julietLog, false);
612       }
613     }
614   }
615 
616   private void flushToConsole(String s) {
617     System.out.println(s);
618     System.out.flush();
619   }
620 
621 
622   private void generateHLogs(int leaveOpen) throws IOException {
623     generateHLogs(NUM_WRITERS, ENTRIES, leaveOpen);
624   }
625 
626   private void generateHLogs(int writers, int entries, int leaveOpen) throws IOException {
627     for (int i = 0; i < writers; i++) {
628       writer[i] = HLog.createWriter(fs, new Path(hlogDir, HLOG_FILE_PREFIX + i), conf);
629       for (int j = 0; j < entries; j++) {
630         int prefix = 0;
631         for (String region : regions) {
632           String row_key = region + prefix++ + i + j;
633           appendEntry(writer[i], TABLE_NAME, region.getBytes(),
634                   row_key.getBytes(), FAMILY, QUALIFIER, VALUE, seq);
635         }
636       }
637       if (i != leaveOpen) {
638         writer[i].close();
639         flushToConsole("Closing writer " + i);
640       }
641     }
642   }
643 
644   private Path getLogForRegion(Path rootdir, byte[] table, String region)
645   throws IOException {
646     Path tdir = HTableDescriptor.getTableDir(rootdir, table);
647     Path editsdir = HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
648       HRegionInfo.encodeRegionName(region.getBytes())));
649     FileStatus [] files = this.fs.listStatus(editsdir);
650     assertEquals(1, files.length);
651     return files[0].getPath();
652   }
653 
654   private void corruptHLog(Path path, Corruptions corruption, boolean close,
655                            FileSystem fs) throws IOException {
656 
657     FSDataOutputStream out;
658     int fileSize = (int) fs.listStatus(path)[0].getLen();
659 
660     FSDataInputStream in = fs.open(path);
661     byte[] corrupted_bytes = new byte[fileSize];
662     in.readFully(0, corrupted_bytes, 0, fileSize);
663     in.close();
664 
665     switch (corruption) {
666       case APPEND_GARBAGE:
667         out = fs.append(path);
668         out.write("-----".getBytes());
669         closeOrFlush(close, out);
670         break;
671 
672       case INSERT_GARBAGE_ON_FIRST_LINE:
673         fs.delete(path, false);
674         out = fs.create(path);
675         out.write(0);
676         out.write(corrupted_bytes);
677         closeOrFlush(close, out);
678         break;
679 
680       case INSERT_GARBAGE_IN_THE_MIDDLE:
681         fs.delete(path, false);
682         out = fs.create(path);
683         int middle = (int) Math.floor(corrupted_bytes.length / 2);
684         out.write(corrupted_bytes, 0, middle);
685         out.write(0);
686         out.write(corrupted_bytes, middle, corrupted_bytes.length - middle);
687         closeOrFlush(close, out);
688         break;
689         
690       case TRUNCATE:
691         fs.delete(path, false);
692         out = fs.create(path);
693         out.write(corrupted_bytes, 0, fileSize-32);
694         closeOrFlush(close, out);
695         
696         break;
697     }
698 
699 
700   }
701 
702   private void closeOrFlush(boolean close, FSDataOutputStream out)
703   throws IOException {
704     if (close) {
705       out.close();
706     } else {
707       out.sync();
708       // Not in 0out.hflush();
709     }
710   }
711 
712   @SuppressWarnings("unused")
713   private void dumpHLog(Path log, FileSystem fs, Configuration conf) throws IOException {
714     HLog.Entry entry;
715     HLog.Reader in = HLog.getReader(fs, log, conf);
716     while ((entry = in.next()) != null) {
717       System.out.println(entry);
718     }
719   }
720 
721   private int countHLog(Path log, FileSystem fs, Configuration conf) throws IOException {
722     int count = 0;
723     HLog.Reader in = HLog.getReader(fs, log, conf);
724     while (in.next() != null) {
725       count++;
726     }
727     return count;
728   }
729 
730 
731   public long appendEntry(HLog.Writer writer, byte[] table, byte[] region,
732                           byte[] row, byte[] family, byte[] qualifier,
733                           byte[] value, long seq)
734           throws IOException {
735 
736     long time = System.nanoTime();
737     WALEdit edit = new WALEdit();
738     seq++;
739     edit.add(new KeyValue(row, family, qualifier, time, KeyValue.Type.Put, value));
740     writer.append(new HLog.Entry(new HLogKey(region, table, seq, time), edit));
741     writer.sync();
742     return seq;
743 
744   }
745 
746 
747   private void injectEmptyFile(String suffix, boolean closeFile)
748           throws IOException {
749     HLog.Writer writer = HLog.createWriter(
750             fs, new Path(hlogDir, HLOG_FILE_PREFIX + suffix), conf);
751     if (closeFile) writer.close();
752   }
753 
754   @SuppressWarnings("unused")
755   private void listLogs(FileSystem fs, Path dir) throws IOException {
756     for (FileStatus file : fs.listStatus(dir)) {
757       System.out.println(file.getPath());
758     }
759 
760   }
761 
762   private int compareHLogSplitDirs(Path p1, Path p2) throws IOException {
763     FileStatus[] f1 = fs.listStatus(p1);
764     FileStatus[] f2 = fs.listStatus(p2);
765 
766     for (int i=0; i<f1.length; i++) {
767       // Regions now have a directory named RECOVERED_EDITS_DIR and in here
768       // are split edit files.  In below presume only 1.
769       Path rd1 = HLog.getRegionDirRecoveredEditsDir(f1[i].getPath());
770       FileStatus [] rd1fs = fs.listStatus(rd1);
771       assertEquals(1, rd1fs.length);
772       Path rd2 = HLog.getRegionDirRecoveredEditsDir(f2[i].getPath());
773       FileStatus [] rd2fs = fs.listStatus(rd2);
774       assertEquals(1, rd2fs.length);
775       if (!logsAreEqual(rd1fs[0].getPath(), rd2fs[0].getPath())) {
776         return -1;
777       }
778     }
779     return 0;
780   }
781 
782   private boolean logsAreEqual(Path p1, Path p2) throws IOException {
783     HLog.Reader in1, in2;
784     in1 = HLog.getReader(fs, p1, conf);
785     in2 = HLog.getReader(fs, p2, conf);
786     HLog.Entry entry1;
787     HLog.Entry entry2;
788     while ((entry1 = in1.next()) != null) {
789       entry2 = in2.next();
790       if ((entry1.getKey().compareTo(entry2.getKey()) != 0) ||
791               (!entry1.getEdit().toString().equals(entry2.getEdit().toString()))) {
792         return false;
793       }
794     }
795     return true;
796   }
797 }