View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver.wal;
20  
21  import static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertNotNull;
24  import static org.junit.Assert.assertTrue;
25  import static org.junit.Assert.fail;
26  
27  import java.io.IOException;
28  import java.lang.reflect.Field;
29  import java.util.ArrayList;
30  import java.util.Comparator;
31  import java.util.HashMap;
32  import java.util.List;
33  import java.util.Map;
34  import java.util.Set;
35  import java.util.UUID;
36  import java.util.concurrent.atomic.AtomicLong;
37  
38  import org.apache.commons.lang.mutable.MutableBoolean;
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.fs.FileStatus;
43  import org.apache.hadoop.fs.FileSystem;
44  import org.apache.hadoop.fs.Path;
45  import org.apache.hadoop.hbase.CellScanner;
46  import org.apache.hadoop.hbase.Coprocessor;
47  import org.apache.hadoop.hbase.HBaseConfiguration;
48  import org.apache.hadoop.hbase.HBaseTestingUtility;
49  import org.apache.hadoop.hbase.HColumnDescriptor;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.HTableDescriptor;
53  import org.apache.hadoop.hbase.KeyValue;
54  import org.apache.hadoop.hbase.testclassification.MediumTests;
55  import org.apache.hadoop.hbase.TableName;
56  import org.apache.hadoop.hbase.client.Get;
57  import org.apache.hadoop.hbase.client.Put;
58  import org.apache.hadoop.hbase.client.Result;
59  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
60  import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
61  import org.apache.hadoop.hbase.regionserver.HRegion;
62  import org.apache.hadoop.hbase.util.Bytes;
63  import org.apache.hadoop.hbase.util.EnvironmentEdge;
64  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
65  import org.apache.hadoop.hbase.util.FSUtils;
66  import org.apache.hadoop.hbase.util.Threads;
67  import org.apache.hadoop.hbase.wal.DefaultWALProvider;
68  import org.apache.hadoop.hbase.wal.WAL;
69  import org.apache.hadoop.hbase.wal.WALKey;
70  import org.junit.After;
71  import org.junit.AfterClass;
72  import org.junit.Before;
73  import org.junit.BeforeClass;
74  import org.junit.Rule;
75  import org.junit.Test;
76  import org.junit.experimental.categories.Category;
77  import org.junit.rules.TestName;
78  
79  /**
80   * Provides FSHLog test cases.
81   */
82  @Category(MediumTests.class)
83  public class TestFSHLog {
84    protected static final Log LOG = LogFactory.getLog(TestFSHLog.class);
85  
86    protected static Configuration conf;
87    protected static FileSystem fs;
88    protected static Path dir;
89    protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
90  
91    @Rule
92    public final TestName currentTest = new TestName();
93  
94    @Before
95    public void setUp() throws Exception {
96      FileStatus[] entries = fs.listStatus(new Path("/"));
97      for (FileStatus dir : entries) {
98        fs.delete(dir.getPath(), true);
99      }
100     final Path hbaseDir = TEST_UTIL.createRootDir();
101     dir = new Path(hbaseDir, currentTest.getMethodName());
102   }
103 
104   @After
105   public void tearDown() throws Exception {
106   }
107 
108   @BeforeClass
109   public static void setUpBeforeClass() throws Exception {
110     // Make block sizes small.
111     TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
112     // quicker heartbeat interval for faster DN death notification
113     TEST_UTIL.getConfiguration().setInt("dfs.namenode.heartbeat.recheck-interval", 5000);
114     TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
115     TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
116 
117     // faster failover with cluster.shutdown();fs.close() idiom
118     TEST_UTIL.getConfiguration()
119         .setInt("hbase.ipc.client.connect.max.retries", 1);
120     TEST_UTIL.getConfiguration().setInt(
121         "dfs.client.block.recovery.retries", 1);
122     TEST_UTIL.getConfiguration().setInt(
123       "hbase.ipc.client.connection.maxidletime", 500);
124     TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
125         SampleRegionWALObserver.class.getName());
126     TEST_UTIL.startMiniDFSCluster(3);
127 
128     conf = TEST_UTIL.getConfiguration();
129     fs = TEST_UTIL.getDFSCluster().getFileSystem();
130   }
131 
132   @AfterClass
133   public static void tearDownAfterClass() throws Exception {
134     TEST_UTIL.shutdownMiniCluster();
135   }
136 
137   /**
138    * A loaded WAL coprocessor won't break existing WAL test cases.
139    */
140   @Test
141   public void testWALCoprocessorLoaded() throws Exception {
142     // test to see whether the coprocessor is loaded or not.
143     FSHLog log = null;
144     try {
145       log = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
146           HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
147       WALCoprocessorHost host = log.getCoprocessorHost();
148       Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class.getName());
149       assertNotNull(c);
150     } finally {
151       if (log != null) {
152         log.close();
153       }
154     }
155   }
156 
157   protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd, int times,
158       AtomicLong sequenceId) throws IOException {
159     final byte[] row = Bytes.toBytes("row");
160     for (int i = 0; i < times; i++) {
161       long timestamp = System.currentTimeMillis();
162       WALEdit cols = new WALEdit();
163       cols.add(new KeyValue(row, row, row, timestamp, row));
164       log.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp),
165         cols, sequenceId, true, null);
166     }
167     log.sync();
168   }
169 
170   /**
171    * helper method to simulate region flush for a WAL.
172    * @param wal
173    * @param regionEncodedName
174    */
175   protected void flushRegion(WAL wal, byte[] regionEncodedName, Set<byte[]> flushedFamilyNames) {
176     wal.startCacheFlush(regionEncodedName, flushedFamilyNames);
177     wal.completeCacheFlush(regionEncodedName);
178   }
179 
180   /**
181    * tests the log comparator. Ensure that we are not mixing meta logs with non-meta logs (throws
182    * exception if we do). Comparison is based on the timestamp present in the wal name.
183    * @throws Exception
184    */
185   @Test 
186   public void testWALComparator() throws Exception {
187     FSHLog wal1 = null;
188     FSHLog walMeta = null;
189     try {
190       wal1 = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
191           HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
192       LOG.debug("Log obtained is: " + wal1);
193       Comparator<Path> comp = wal1.LOG_NAME_COMPARATOR;
194       Path p1 = wal1.computeFilename(11);
195       Path p2 = wal1.computeFilename(12);
196       // comparing with itself returns 0
197       assertTrue(comp.compare(p1, p1) == 0);
198       // comparing with different filenum.
199       assertTrue(comp.compare(p1, p2) < 0);
200       walMeta = new FSHLog(fs, FSUtils.getRootDir(conf), dir.toString(),
201           HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null,
202           DefaultWALProvider.META_WAL_PROVIDER_ID);
203       Comparator<Path> compMeta = walMeta.LOG_NAME_COMPARATOR;
204 
205       Path p1WithMeta = walMeta.computeFilename(11);
206       Path p2WithMeta = walMeta.computeFilename(12);
207       assertTrue(compMeta.compare(p1WithMeta, p1WithMeta) == 0);
208       assertTrue(compMeta.compare(p1WithMeta, p2WithMeta) < 0);
209       // mixing meta and non-meta logs gives error
210       boolean ex = false;
211       try {
212         comp.compare(p1WithMeta, p2);
213       } catch (IllegalArgumentException e) {
214         ex = true;
215       }
216       assertTrue("Comparator doesn't complain while checking meta log files", ex);
217       boolean exMeta = false;
218       try {
219         compMeta.compare(p1WithMeta, p2);
220       } catch (IllegalArgumentException e) {
221         exMeta = true;
222       }
223       assertTrue("Meta comparator doesn't complain while checking log files", exMeta);
224     } finally {
225       if (wal1 != null) {
226         wal1.close();
227       }
228       if (walMeta != null) {
229         walMeta.close();
230       }
231     }
232   }
233 
234   /**
235    * On rolling a wal after reaching the threshold, {@link WAL#rollWriter()} returns the
236    * list of regions which should be flushed in order to archive the oldest wal file.
237    * <p>
238    * This method tests this behavior by inserting edits and rolling the wal enough times to reach
239    * the max number of logs threshold. It checks whether we get the "right regions" for flush on
240    * rolling the wal.
241    * @throws Exception
242    */
243   @Test 
244   public void testFindMemStoresEligibleForFlush() throws Exception {
245     LOG.debug("testFindMemStoresEligibleForFlush");
246     Configuration conf1 = HBaseConfiguration.create(conf);
247     conf1.setInt("hbase.regionserver.maxlogs", 1);
248     FSHLog wal = new FSHLog(fs, FSUtils.getRootDir(conf1), dir.toString(),
249         HConstants.HREGION_OLDLOGDIR_NAME, conf1, null, true, null, null);
250     HTableDescriptor t1 =
251         new HTableDescriptor(TableName.valueOf("t1")).addFamily(new HColumnDescriptor("row"));
252     HTableDescriptor t2 =
253         new HTableDescriptor(TableName.valueOf("t2")).addFamily(new HColumnDescriptor("row"));
254     HRegionInfo hri1 =
255         new HRegionInfo(t1.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
256     HRegionInfo hri2 =
257         new HRegionInfo(t2.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
258     // variables to mock region sequenceIds
259     final AtomicLong sequenceId1 = new AtomicLong(1);
260     final AtomicLong sequenceId2 = new AtomicLong(1);
261     // add edits and roll the wal
262     try {
263       addEdits(wal, hri1, t1, 2, sequenceId1);
264       wal.rollWriter();
265       // add some more edits and roll the wal. This would reach the log number threshold
266       addEdits(wal, hri1, t1, 2, sequenceId1);
267       wal.rollWriter();
268       // with above rollWriter call, the max logs limit is reached.
269       assertTrue(wal.getNumRolledLogFiles() == 2);
270 
271       // get the regions to flush; since there is only one region in the oldest wal, it should
272       // return only one region.
273       byte[][] regionsToFlush = wal.findRegionsToForceFlush();
274       assertEquals(1, regionsToFlush.length);
275       assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]);
276       // insert edits in second region
277       addEdits(wal, hri2, t2, 2, sequenceId2);
278       // get the regions to flush, it should still read region1.
279       regionsToFlush = wal.findRegionsToForceFlush();
280       assertEquals(regionsToFlush.length, 1);
281       assertEquals(hri1.getEncodedNameAsBytes(), regionsToFlush[0]);
282       // flush region 1, and roll the wal file. Only last wal which has entries for region1 should
283       // remain.
284       flushRegion(wal, hri1.getEncodedNameAsBytes(), t1.getFamiliesKeys());
285       wal.rollWriter();
286       // only one wal should remain now (that is for the second region).
287       assertEquals(1, wal.getNumRolledLogFiles());
288       // flush the second region
289       flushRegion(wal, hri2.getEncodedNameAsBytes(), t2.getFamiliesKeys());
290       wal.rollWriter(true);
291       // no wal should remain now.
292       assertEquals(0, wal.getNumRolledLogFiles());
293       // add edits both to region 1 and region 2, and roll.
294       addEdits(wal, hri1, t1, 2, sequenceId1);
295       addEdits(wal, hri2, t2, 2, sequenceId2);
296       wal.rollWriter();
297       // add edits and roll the writer, to reach the max logs limit.
298       assertEquals(1, wal.getNumRolledLogFiles());
299       addEdits(wal, hri1, t1, 2, sequenceId1);
300       wal.rollWriter();
301       // it should return two regions to flush, as the oldest wal file has entries
302       // for both regions.
303       regionsToFlush = wal.findRegionsToForceFlush();
304       assertEquals(2, regionsToFlush.length);
305       // flush both regions
306       flushRegion(wal, hri1.getEncodedNameAsBytes(), t1.getFamiliesKeys());
307       flushRegion(wal, hri2.getEncodedNameAsBytes(), t2.getFamiliesKeys());
308       wal.rollWriter(true);
309       assertEquals(0, wal.getNumRolledLogFiles());
310       // Add an edit to region1, and roll the wal.
311       addEdits(wal, hri1, t1, 2, sequenceId1);
312       // tests partial flush: roll on a partial flush, and ensure that wal is not archived.
313       wal.startCacheFlush(hri1.getEncodedNameAsBytes(), t1.getFamiliesKeys());
314       wal.rollWriter();
315       wal.completeCacheFlush(hri1.getEncodedNameAsBytes());
316       assertEquals(1, wal.getNumRolledLogFiles());
317     } finally {
318       if (wal != null) {
319         wal.close();
320       }
321     }
322   }
323 
324   /**
325    * Simulates WAL append ops for a region and tests
326    * {@link FSHLog#areAllRegionsFlushed(Map, Map, Map)} API.
327    * It compares the region sequenceIds with oldestFlushing and oldestUnFlushed entries.
328    * If a region's entries are larger than min of (oldestFlushing, oldestUnFlushed), then the
329    * region should be flushed before archiving this WAL.
330   */
331   @Test
332   public void testAllRegionsFlushed() {
333     LOG.debug("testAllRegionsFlushed");
334     Map<byte[], Long> oldestFlushingSeqNo = new HashMap<byte[], Long>();
335     Map<byte[], Long> oldestUnFlushedSeqNo = new HashMap<byte[], Long>();
336     Map<byte[], Long> seqNo = new HashMap<byte[], Long>();
337     // create a table
338     TableName t1 = TableName.valueOf("t1");
339     // create a region
340     HRegionInfo hri1 = new HRegionInfo(t1, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
341     // variables to mock region sequenceIds
342     final AtomicLong sequenceId1 = new AtomicLong(1);
343     // test empty map
344     assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
345     // add entries in the region
346     seqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.incrementAndGet());
347     oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get());
348     // should say region1 is not flushed.
349     assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
350     // test with entries in oldestFlushing map.
351     oldestUnFlushedSeqNo.clear();
352     oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), sequenceId1.get());
353     assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
354     // simulate region flush, i.e., clear oldestFlushing and oldestUnflushed maps
355     oldestFlushingSeqNo.clear();
356     oldestUnFlushedSeqNo.clear();
357     assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
358     // insert some large values for region1
359     oldestUnFlushedSeqNo.put(hri1.getEncodedNameAsBytes(), 1000l);
360     seqNo.put(hri1.getEncodedNameAsBytes(), 1500l);
361     assertFalse(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
362 
363     // tests when oldestUnFlushed/oldestFlushing contains larger value.
364     // It means region is flushed.
365     oldestFlushingSeqNo.put(hri1.getEncodedNameAsBytes(), 1200l);
366     oldestUnFlushedSeqNo.clear();
367     seqNo.put(hri1.getEncodedNameAsBytes(), 1199l);
368     assertTrue(FSHLog.areAllRegionsFlushed(seqNo, oldestFlushingSeqNo, oldestUnFlushedSeqNo));
369   }
370 
371   @Test(expected=IOException.class)
372   public void testFailedToCreateWALIfParentRenamed() throws IOException {
373     final String name = "testFailedToCreateWALIfParentRenamed";
374     FSHLog log = new FSHLog(fs, FSUtils.getRootDir(conf), name, HConstants.HREGION_OLDLOGDIR_NAME,
375         conf, null, true, null, null);
376     long filenum = System.currentTimeMillis();
377     Path path = log.computeFilename(filenum);
378     log.createWriterInstance(path);
379     Path parent = path.getParent();
380     path = log.computeFilename(filenum + 1);
381     Path newPath = new Path(parent.getParent(), parent.getName() + "-splitting");
382     fs.rename(parent, newPath);
383     log.createWriterInstance(path);
384     fail("It should fail to create the new WAL");
385   }
386 
387   /**
388    * Test flush for sure has a sequence id that is beyond the last edit appended.  We do this
389    * by slowing appends in the background ring buffer thread while in foreground we call
390    * flush.  The addition of the sync over HRegion in flush should fix an issue where flush was
391    * returning before all of its appends had made it out to the WAL (HBASE-11109).
392    * @throws IOException
393    * @see HBASE-11109
394    */
395   @Test
396   public void testFlushSequenceIdIsGreaterThanAllEditsInHFile() throws IOException {
397     String testName = "testFlushSequenceIdIsGreaterThanAllEditsInHFile";
398     final TableName tableName = TableName.valueOf(testName);
399     final HRegionInfo hri = new HRegionInfo(tableName);
400     final byte[] rowName = tableName.getName();
401     final HTableDescriptor htd = new HTableDescriptor(tableName);
402     htd.addFamily(new HColumnDescriptor("f"));
403     HRegion r = HRegion.createHRegion(hri, TEST_UTIL.getDefaultRootDirPath(),
404       TEST_UTIL.getConfiguration(), htd);
405     HRegion.closeHRegion(r);
406     final int countPerFamily = 10;
407     final MutableBoolean goslow = new MutableBoolean(false);
408     // subclass and doctor a method.
409     FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDefaultRootDirPath(),
410         testName, conf) {
411       @Override
412       void atHeadOfRingBufferEventHandlerAppend() {
413         if (goslow.isTrue()) {
414           Threads.sleep(100);
415           LOG.debug("Sleeping before appending 100ms");
416         }
417         super.atHeadOfRingBufferEventHandlerAppend();
418       }
419     };
420     HRegion region = HRegion.openHRegion(TEST_UTIL.getConfiguration(),
421       TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(), hri, htd, wal);
422     EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
423     try {
424       List<Put> puts = null;
425       for (HColumnDescriptor hcd: htd.getFamilies()) {
426         puts =
427           TestWALReplay.addRegionEdits(rowName, hcd.getName(), countPerFamily, ee, region, "x");
428       }
429 
430       // Now assert edits made it in.
431       final Get g = new Get(rowName);
432       Result result = region.get(g);
433       assertEquals(countPerFamily * htd.getFamilies().size(), result.size());
434 
435       // Construct a WALEdit and add it a few times to the WAL.
436       WALEdit edits = new WALEdit();
437       for (Put p: puts) {
438         CellScanner cs = p.cellScanner();
439         while (cs.advance()) {
440           edits.add(cs.current());
441         }
442       }
443       // Add any old cluster id.
444       List<UUID> clusterIds = new ArrayList<UUID>();
445       clusterIds.add(UUID.randomUUID());
446       // Now make appends run slow.
447       goslow.setValue(true);
448       for (int i = 0; i < countPerFamily; i++) {
449         final HRegionInfo info = region.getRegionInfo();
450         final WALKey logkey = new WALKey(info.getEncodedNameAsBytes(), tableName,
451             System.currentTimeMillis(), clusterIds, -1, -1);
452         wal.append(htd, info, logkey, edits, region.getSequenceId(), true, null);
453       }
454       region.flush(true);
455       // FlushResult.flushSequenceId is not visible here so go get the current sequence id.
456       long currentSequenceId = region.getSequenceId().get();
457       // Now release the appends
458       goslow.setValue(false);
459       synchronized (goslow) {
460         goslow.notifyAll();
461       }
462       assertTrue(currentSequenceId >= region.getSequenceId().get());
463     } finally {
464       region.close(true);
465       wal.close();
466     }
467   }
468 
469   @Test
470   public void testSyncRunnerIndexOverflow() throws IOException, NoSuchFieldException,
471       SecurityException, IllegalArgumentException, IllegalAccessException {
472     final String name = "testSyncRunnerIndexOverflow";
473     FSHLog log =
474         new FSHLog(fs, FSUtils.getRootDir(conf), name, HConstants.HREGION_OLDLOGDIR_NAME, conf,
475             null, true, null, null);
476     try {
477       Field ringBufferEventHandlerField = FSHLog.class.getDeclaredField("ringBufferEventHandler");
478       ringBufferEventHandlerField.setAccessible(true);
479       FSHLog.RingBufferEventHandler ringBufferEventHandler =
480           (FSHLog.RingBufferEventHandler) ringBufferEventHandlerField.get(log);
481       Field syncRunnerIndexField =
482           FSHLog.RingBufferEventHandler.class.getDeclaredField("syncRunnerIndex");
483       syncRunnerIndexField.setAccessible(true);
484       syncRunnerIndexField.set(ringBufferEventHandler, Integer.MAX_VALUE - 1);
485       HTableDescriptor htd =
486           new HTableDescriptor(TableName.valueOf("t1")).addFamily(new HColumnDescriptor("row"));
487       HRegionInfo hri =
488           new HRegionInfo(htd.getTableName(), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
489       final AtomicLong sequenceId1 = new AtomicLong(1);
490       for (int i = 0; i < 10; i++) {
491         addEdits(log, hri, htd, 1, sequenceId1);
492       }
493     } finally {
494       log.close();
495     }
496   }
497 }