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