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.assertTrue;
24  
25  import java.io.IOException;
26  import java.security.PrivilegedExceptionAction;
27  import java.util.List;
28  import java.util.concurrent.atomic.AtomicInteger;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.fs.FileSystem;
34  import org.apache.hadoop.fs.Path;
35  import org.apache.hadoop.hbase.HBaseConfiguration;
36  import org.apache.hadoop.hbase.HBaseTestingUtility;
37  import org.apache.hadoop.hbase.HColumnDescriptor;
38  import org.apache.hadoop.hbase.HConstants;
39  import org.apache.hadoop.hbase.HRegionInfo;
40  import org.apache.hadoop.hbase.HTableDescriptor;
41  import org.apache.hadoop.hbase.KeyValue;
42  import org.apache.hadoop.hbase.client.Get;
43  import org.apache.hadoop.hbase.client.Put;
44  import org.apache.hadoop.hbase.client.Result;
45  import org.apache.hadoop.hbase.io.hfile.HFile;
46  import org.apache.hadoop.hbase.regionserver.HRegion;
47  import org.apache.hadoop.hbase.regionserver.Store;
48  import org.apache.hadoop.hbase.util.Bytes;
49  import org.apache.hadoop.hbase.util.EnvironmentEdge;
50  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
51  import org.apache.hadoop.security.UserGroupInformation;
52  import org.junit.After;
53  import org.junit.AfterClass;
54  import org.junit.Before;
55  import org.junit.BeforeClass;
56  import org.junit.Test;
57  
58  /**
59   * Test replay of edits out of a WAL split.
60   */
61  public class TestWALReplay {
62    public static final Log LOG = LogFactory.getLog(TestWALReplay.class);
63    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
64    private final EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
65    private Path hbaseRootDir = null;
66    private Path oldLogDir;
67    private Path logDir;
68    private FileSystem fs;
69    private Configuration conf;
70  
71    @BeforeClass
72    public static void setUpBeforeClass() throws Exception {
73      Configuration conf = TEST_UTIL.getConfiguration();
74      conf.setBoolean("dfs.support.append", true);
75      // The below config supported by 0.20-append and CDH3b2
76      conf.setInt("dfs.client.block.recovery.retries", 2);
77      conf.setInt("hbase.regionserver.flushlogentries", 1);
78      TEST_UTIL.startMiniDFSCluster(3);
79      TEST_UTIL.setNameNodeNameSystemLeasePeriod(100, 10000);
80      Path hbaseRootDir =
81        TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
82      LOG.info("hbase.rootdir=" + hbaseRootDir);
83      conf.set(HConstants.HBASE_DIR, hbaseRootDir.toString());
84    }
85  
86    @AfterClass
87    public static void tearDownAfterClass() throws Exception {
88      TEST_UTIL.shutdownMiniDFSCluster();
89    }
90  
91    @Before
92    public void setUp() throws Exception {
93      this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
94      this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
95      this.hbaseRootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
96      this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
97      this.logDir = new Path(this.hbaseRootDir, HConstants.HREGION_LOGDIR_NAME);
98      if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
99        TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
100     }
101   }
102 
103   @After
104   public void tearDown() throws Exception {
105     TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
106   }
107 
108   /*
109    * @param p Directory to cleanup
110    */
111   private void deleteDir(final Path p) throws IOException {
112     if (this.fs.exists(p)) {
113       if (!this.fs.delete(p, true)) {
114         throw new IOException("Failed remove of " + p);
115       }
116     }
117   }
118 
119   /**
120    * Tests for hbase-2727.
121    * @throws Exception
122    * @see https://issues.apache.org/jira/browse/HBASE-2727
123    */
124   @Test
125   public void test2727() throws Exception {
126     // Test being able to have > 1 set of edits in the recovered.edits directory.
127     // Ensure edits are replayed properly.
128     final String tableNameStr = "test2727";
129     HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
130     Path basedir = new Path(hbaseRootDir, tableNameStr);
131     deleteDir(basedir);
132 
133     final byte [] tableName = Bytes.toBytes(tableNameStr);
134     final byte [] rowName = tableName;
135 
136     HLog wal1 = createWAL(this.conf);
137     // Add 1k to each family.
138     final int countPerFamily = 1000;
139     for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
140       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal1);
141     }
142     wal1.close();
143     runWALSplit(this.conf);
144 
145     HLog wal2 = createWAL(this.conf);
146     // Up the sequenceid so that these edits are after the ones added above.
147     wal2.setSequenceNumber(wal1.getSequenceNumber());
148     // Add 1k to each family.
149     for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
150       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal2);
151     }
152     wal2.close();
153     runWALSplit(this.conf);
154 
155     HLog wal3 = createWAL(this.conf);
156     wal3.setSequenceNumber(wal2.getSequenceNumber());
157     try {
158       final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri,
159           null);
160       long seqid = region.initialize();
161       assertTrue(seqid > wal3.getSequenceNumber());
162 
163       // TODO: Scan all.
164       region.close();
165     } finally {
166       wal3.closeAndDelete();
167     }
168   }
169 
170   /**
171    * Test case of HRegion that is only made out of bulk loaded files.  Assert
172    * that we don't 'crash'.
173    * @throws IOException
174    * @throws IllegalAccessException 
175    * @throws NoSuchFieldException 
176    * @throws IllegalArgumentException 
177    * @throws SecurityException 
178    */
179   @Test
180   public void testRegionMadeOfBulkLoadedFilesOnly()
181   throws IOException, SecurityException, IllegalArgumentException,
182       NoSuchFieldException, IllegalAccessException, InterruptedException {
183     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
184     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
185     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
186     deleteDir(basedir);
187     HLog wal = createWAL(this.conf);
188     HRegion region = HRegion.openHRegion(hri, basedir, wal, this.conf);
189     Path f =  new Path(basedir, "hfile");
190     HFile.Writer writer = new HFile.Writer(this.fs, f);
191     byte [] family = hri.getTableDesc().getFamilies().iterator().next().getName();
192     byte [] row = Bytes.toBytes(tableNameStr);
193     writer.append(new KeyValue(row, family, family, row));
194     writer.close();
195     region.bulkLoadHFile(f.toString(), family);
196     // Add an edit so something in the WAL
197     region.put((new Put(row)).add(family, family, family));
198     wal.sync();
199 
200     // Now 'crash' the region by stealing its wal
201     UserGroupInformation newUGI = HBaseTestingUtility.getDifferentUser(this.conf,
202         tableNameStr);
203     newUGI.doAs(new PrivilegedExceptionAction<Object>() {
204       public Object run() throws Exception {
205         runWALSplit(conf);
206         HLog wal2 = createWAL(conf);
207         HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(conf),
208           conf, hri, null);
209         long seqid2 = region2.initialize();
210         assertTrue(seqid2 > -1);
211 
212         // I can't close wal1.  Its been appropriated when we split.
213         region2.close();
214         wal2.closeAndDelete();
215 
216         return null;
217       }
218     });
219   }
220 
221   /**
222    * Test writing edits into an HRegion, closing it, splitting logs, opening
223    * Region again.  Verify seqids.
224    * @throws IOException
225    * @throws IllegalAccessException 
226    * @throws NoSuchFieldException 
227    * @throws IllegalArgumentException 
228    * @throws SecurityException 
229    */
230   @Test
231   public void testReplayEditsWrittenViaHRegion()
232   throws IOException, SecurityException, IllegalArgumentException,
233       NoSuchFieldException, IllegalAccessException, InterruptedException {
234     final String tableNameStr = "testReplayEditsWrittenViaHRegion";
235     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
236     final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
237     deleteDir(basedir);
238     final byte[] rowName = Bytes.toBytes(tableNameStr);
239     final int countPerFamily = 10;
240 
241     // Write countPerFamily edits into the three families.  Do a flush on one
242     // of the families during the load of edits so its seqid is not same as
243     // others to test we do right thing when different seqids.
244     HLog wal = createWAL(this.conf);
245     HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, null);
246     long seqid = region.initialize();
247     // HRegionServer usually does this. It knows the largest seqid across all regions.
248     wal.setSequenceNumber(seqid);
249     boolean first = true;
250     for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
251       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x");
252       if (first ) {
253         // If first, so we have at least one family w/ different seqid to rest.
254         region.flushcache();
255         first = false;
256       }
257     }
258     // Now assert edits made it in.
259     final Get g = new Get(rowName);
260     Result result = region.get(g, null);
261     assertEquals(countPerFamily * hri.getTableDesc().getFamilies().size(),
262       result.size());
263     // Now close the region, split the log, reopen the region and assert that
264     // replay of log has no effect, that our seqids are calculated correctly so
265     // all edits in logs are seen as 'stale'/old.
266     region.close();
267     wal.close();
268     runWALSplit(this.conf);
269     HLog wal2 = createWAL(this.conf);
270     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, null) {
271       @Override
272       protected boolean restoreEdit(Store s, KeyValue kv) {
273         super.restoreEdit(s, kv);
274         throw new RuntimeException("Called when it should not have been!");
275       }
276     };
277     long seqid2 = region2.initialize();
278     // HRegionServer usually does this. It knows the largest seqid across all regions.
279     wal2.setSequenceNumber(seqid2);
280     assertTrue(seqid + result.size() < seqid2);
281 
282     // Next test.  Add more edits, then 'crash' this region by stealing its wal
283     // out from under it and assert that replay of the log adds the edits back
284     // correctly when region is opened again.
285     for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
286       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
287     }
288     // Get count of edits.
289     final Result result2 = region2.get(g, null);
290     assertEquals(2 * result.size(), result2.size());
291     wal2.sync();
292     // Set down maximum recovery so we dfsclient doesn't linger retrying something
293     // long gone.
294     HBaseTestingUtility.setMaxRecoveryErrorCount(wal2.getOutputStream(), 1);
295     UserGroupInformation newUGI = HBaseTestingUtility.getDifferentUser(this.conf,
296       tableNameStr);
297     newUGI.doAs(new PrivilegedExceptionAction<Object>() {
298       public Object run() throws Exception {
299         runWALSplit(conf);
300         FileSystem newFS = FileSystem.get(conf);
301         // Make a new wal for new region open.
302         HLog wal3 = createWAL(conf);
303         final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
304         HRegion region3 = new HRegion(basedir, wal3, newFS, conf, hri, null) {
305           @Override
306           protected boolean restoreEdit(Store s, KeyValue kv) {
307             boolean b = super.restoreEdit(s, kv);
308             countOfRestoredEdits.incrementAndGet();
309             return b;
310           }
311         };
312         long seqid3 = region3.initialize();
313         // HRegionServer usually does this. It knows the largest seqid across all regions.
314         wal3.setSequenceNumber(seqid3);
315         Result result3 = region3.get(g, null);
316         // Assert that count of cells is same as before crash.
317         assertEquals(result2.size(), result3.size());
318         assertEquals(hri.getTableDesc().getFamilies().size() * countPerFamily,
319           countOfRestoredEdits.get());
320 
321         // I can't close wal1.  Its been appropriated when we split.
322         region3.close();
323         wal3.closeAndDelete();
324         return null;
325       }
326     });
327   }
328 
329   /**
330    * Create an HRegion with the result of a HLog split and test we only see the
331    * good edits
332    * @throws Exception
333    */
334   @Test
335   public void testReplayEditsWrittenIntoWAL() throws Exception {
336     final String tableNameStr = "testReplayEditsWrittenIntoWAL";
337     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
338     final Path basedir = new Path(hbaseRootDir, tableNameStr);
339     deleteDir(basedir);
340     final HLog wal = createWAL(this.conf);
341     final byte[] tableName = Bytes.toBytes(tableNameStr);
342     final byte[] rowName = tableName;
343     final byte[] regionName = hri.getRegionName();
344 
345     // Add 1k to each family.
346     final int countPerFamily = 1000;
347     for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
348       addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal);
349     }
350 
351     // Add a cache flush, shouldn't have any effect
352     long logSeqId = wal.startCacheFlush();
353     wal.completeCacheFlush(regionName, tableName, logSeqId, hri.isMetaRegion());
354 
355     // Add an edit to another family, should be skipped.
356     WALEdit edit = new WALEdit();
357     long now = ee.currentTimeMillis();
358     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
359       now, rowName));
360     wal.append(hri, tableName, edit, now);
361 
362     // Delete the c family to verify deletes make it over.
363     edit = new WALEdit();
364     now = ee.currentTimeMillis();
365     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now,
366       KeyValue.Type.DeleteFamily));
367     wal.append(hri, tableName, edit, now);
368 
369     // Sync.
370     wal.sync();
371     // Set down maximum recovery so we dfsclient doesn't linger retrying something
372     // long gone.
373     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
374 
375     // Make a new fs for the splitter and run as a new user so we can take
376     // over old wal.
377     UserGroupInformation newUGI = HBaseTestingUtility.getDifferentUser(this.conf,
378       ".replay.wal.secondtime");
379     final Configuration newConf = new Configuration(conf);
380     newUGI.doAs(new PrivilegedExceptionAction<Object>() {
381       public Object run() throws Exception {
382         runWALSplit(newConf);
383         FileSystem newFS = FileSystem.get(newConf);
384         // 100k seems to make for about 4 flushes during HRegion#initialize.
385         newConf.setInt("hbase.hregion.memstore.flush.size", 1024 * 100);
386         // Make a new wal for new region.
387         HLog newWal = createWAL(newConf);
388         final AtomicInteger flushcount = new AtomicInteger(0);
389         try {
390           final HRegion region = new HRegion(basedir, newWal, newFS, newConf, hri,
391               null) {
392             protected boolean internalFlushcache(HLog wal, long myseqid)
393             throws IOException {
394               boolean b = super.internalFlushcache(wal, myseqid);
395               flushcount.incrementAndGet();
396               return b;
397             };
398           };
399           long seqid = region.initialize();
400           // We flushed during init.
401           assertTrue(flushcount.get() > 0);
402           assertTrue(seqid > wal.getSequenceNumber());
403 
404           Get get = new Get(rowName);
405           Result result = region.get(get, -1);
406           // Make sure we only see the good edits
407           assertEquals(countPerFamily * (hri.getTableDesc().getFamilies().size() - 1),
408             result.size());
409           region.close();
410         } finally {
411           newWal.closeAndDelete();
412         }
413 
414         return null;
415       }
416     });
417   }
418 
419   private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
420       final byte [] rowName, final byte [] family, 
421       final int count, EnvironmentEdge ee, final HLog wal)
422   throws IOException {
423     String familyStr = Bytes.toString(family);
424     for (int j = 0; j < count; j++) {
425       byte[] qualifierBytes = Bytes.toBytes(Integer.toString(j));
426       byte[] columnBytes = Bytes.toBytes(familyStr + ":" + Integer.toString(j));
427       WALEdit edit = new WALEdit();
428       edit.add(new KeyValue(rowName, family, qualifierBytes,
429         ee.currentTimeMillis(), columnBytes));
430       wal.append(hri, tableName, edit, ee.currentTimeMillis());
431     }
432   }
433 
434   private void addRegionEdits (final byte [] rowName, final byte [] family, 
435       final int count, EnvironmentEdge ee, final HRegion r,
436       final String qualifierPrefix)
437   throws IOException {
438     for (int j = 0; j < count; j++) {
439       byte[] qualifier = Bytes.toBytes(qualifierPrefix + Integer.toString(j));
440       Put p = new Put(rowName);
441       p.add(family, qualifier, ee.currentTimeMillis(), rowName);
442       r.put(p);
443     }
444   }
445 
446   /*
447    * Creates an HRI around an HTD that has <code>tableName</code> and three
448    * column families named 'a','b', and 'c'.
449    * @param tableName Name of table to use when we create HTableDescriptor.
450    */
451   private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
452     HTableDescriptor htd = new HTableDescriptor(tableName);
453     HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
454     htd.addFamily(a);
455     HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
456     htd.addFamily(b);
457     HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
458     htd.addFamily(c);
459     return new HRegionInfo(htd, null, null, false);
460   }
461 
462 
463   /*
464    * Run the split.  Verify only single split file made.
465    * @param c
466    * @return The single split file made
467    * @throws IOException
468    */
469   private Path runWALSplit(final Configuration c) throws IOException {
470     FileSystem fs = FileSystem.get(c);
471     List<Path> splits = HLog.splitLog(this.hbaseRootDir, this.logDir,
472       this.oldLogDir, fs, c);
473     // Split should generate only 1 file since there's only 1 region
474     assertEquals(1, splits.size());
475     // Make sure the file exists
476     assertTrue(fs.exists(splits.get(0)));
477     LOG.info("Split file=" + splits.get(0));
478     return splits.get(0);
479   }
480 
481   /*
482    * @param c
483    * @return WAL with retries set down from 5 to 1 only.
484    * @throws IOException
485    */
486   private HLog createWAL(final Configuration c) throws IOException {
487     HLog wal = new HLog(FileSystem.get(c), logDir, oldLogDir, c, null);
488     // Set down maximum recovery so we dfsclient doesn't linger retrying something
489     // long gone.
490     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
491     return wal;
492   }
493 }