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