1   /**
2    * Copyright 2007 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.assertFalse;
24  import static org.junit.Assert.assertNotNull;
25  import static org.junit.Assert.assertTrue;
26  
27  import java.io.EOFException;
28  import java.io.IOException;
29  import java.io.OutputStream;
30  import java.lang.reflect.InvocationTargetException;
31  import java.lang.reflect.Method;
32  import java.util.ArrayList;
33  import java.util.HashSet;
34  import java.util.List;
35  import java.util.Set;
36  
37  import org.apache.commons.logging.Log;
38  import org.apache.commons.logging.LogFactory;
39  import org.apache.commons.logging.impl.Log4JLogger;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.HBaseTestingUtility;
43  import org.apache.hadoop.hbase.HColumnDescriptor;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HTableDescriptor;
47  import org.apache.hadoop.hbase.KeyValue;
48  import org.apache.hadoop.hbase.MiniHBaseCluster;
49  import org.apache.hadoop.hbase.client.HBaseAdmin;
50  import org.apache.hadoop.hbase.client.HTable;
51  import org.apache.hadoop.hbase.client.Put;
52  import org.apache.hadoop.hbase.client.Result;
53  import org.apache.hadoop.hbase.client.ResultScanner;
54  import org.apache.hadoop.hbase.client.Scan;
55  import org.apache.hadoop.hbase.regionserver.HRegion;
56  import org.apache.hadoop.hbase.regionserver.HRegionServer;
57  import org.apache.hadoop.hbase.util.Bytes;
58  import org.apache.hadoop.hbase.util.FSUtils;
59  import org.apache.hadoop.hdfs.DFSClient;
60  import org.apache.hadoop.hdfs.MiniDFSCluster;
61  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
62  import org.apache.hadoop.hdfs.server.datanode.DataNode;
63  import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
64  import org.apache.log4j.Level;
65  import org.junit.After;
66  import org.junit.Before;
67  import org.junit.BeforeClass;
68  import org.junit.Test;
69  
70  /**
71   * Test log deletion as logs are rolled.
72   */
73  public class TestLogRolling  {
74    private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
75    private HRegionServer server;
76    private HLog log;
77    private String tableName;
78    private byte[] value;
79    private FileSystem fs;
80    private MiniDFSCluster dfsCluster;
81    private HBaseAdmin admin;
82    private MiniHBaseCluster cluster;
83    private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
84  
85   // verbose logging on classes that are touched in these tests
86   {
87     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
88     ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
89     ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem"))
90       .getLogger().setLevel(Level.ALL);
91     ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL);
92     ((Log4JLogger)HRegionServer.LOG).getLogger().setLevel(Level.ALL);
93     ((Log4JLogger)HRegion.LOG).getLogger().setLevel(Level.ALL);
94     ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL);
95   }
96  
97    /**
98     * constructor
99     * @throws Exception
100    */
101   public TestLogRolling()  {
102     this.server = null;
103     this.log = null;
104     this.tableName = null;
105 
106     String className = this.getClass().getName();
107     StringBuilder v = new StringBuilder(className);
108     while (v.length() < 1000) {
109       v.append(className);
110     }
111     this.value = Bytes.toBytes(v.toString());
112   }
113 
114   // Need to override this setup so we can edit the config before it gets sent
115  // to the HDFS & HBase cluster startup.
116  @BeforeClass
117   public static void setUpBeforeClass() throws Exception {
118     /**** configuration for testLogRolling ****/
119     // Force a region split after every 768KB
120     TEST_UTIL.getConfiguration().setLong("hbase.hregion.max.filesize", 768L * 1024L);
121 
122     // We roll the log after every 32 writes
123     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32);
124 
125     TEST_UTIL.getConfiguration().setInt(
126         "hbase.regionserver.logroll.errors.tolerated", 2);
127     TEST_UTIL.getConfiguration().setInt("ipc.ping.interval", 10 * 1000);
128     TEST_UTIL.getConfiguration().setInt("ipc.socket.timeout", 10 * 1000);
129     TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000);
130 
131     // For less frequently updated regions flush after every 2 flushes
132     TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount", 2);
133 
134     // We flush the cache after every 8192 bytes
135     TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.flush.size", 8192);
136 
137     // Increase the amount of time between client retries
138     TEST_UTIL.getConfiguration().setLong("hbase.client.pause", 10 * 1000);
139 
140     // Reduce thread wake frequency so that other threads can get
141     // a chance to run.
142     TEST_UTIL.getConfiguration().setInt(HConstants.THREAD_WAKE_FREQUENCY, 2 * 1000);
143 
144    /**** configuration for testLogRollOnDatanodeDeath ****/
145    // make sure log.hflush() calls syncFs() to open a pipeline
146     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
147    // lower the namenode & datanode heartbeat so the namenode
148    // quickly detects datanode failures
149     TEST_UTIL.getConfiguration().setInt("heartbeat.recheck.interval", 5000);
150     TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
151    // the namenode might still try to choose the recently-dead datanode
152    // for a pipeline, so try to a new pipeline multiple times
153     TEST_UTIL.getConfiguration().setInt("dfs.client.block.write.retries", 30);
154     TEST_UTIL.getConfiguration().setInt(
155         "hbase.regionserver.hlog.tolerable.lowreplication", 2);
156     TEST_UTIL.getConfiguration().setInt(
157         "hbase.regionserver.hlog.lowreplication.rolllimit", 3);
158   }
159 
160   @Before
161   public void setUp() throws Exception {
162     TEST_UTIL.startMiniCluster(2);
163 
164     cluster = TEST_UTIL.getHBaseCluster();
165     dfsCluster = TEST_UTIL.getDFSCluster();
166     fs = TEST_UTIL.getTestFileSystem();
167     admin = TEST_UTIL.getHBaseAdmin();
168 
169     // disable region rebalancing (interferes with log watching)
170     cluster.getMaster().balanceSwitch(false);
171   }
172 
173   @After
174   public void tearDown() throws Exception  {
175     TEST_UTIL.shutdownMiniCluster();
176   }
177 
178   private void startAndWriteData() throws IOException {
179     // When the META table can be opened, the region servers are running
180     new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
181     this.server = cluster.getRegionServerThreads().get(0).getRegionServer();
182     this.log = server.getWAL();
183 
184     // Create the test table and open it
185     HTableDescriptor desc = new HTableDescriptor(tableName);
186     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
187     admin.createTable(desc);
188     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
189 
190     server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
191     this.log = server.getWAL();
192     for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
193       Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
194       put.add(HConstants.CATALOG_FAMILY, null, value);
195       table.put(put);
196       if (i % 32 == 0) {
197         // After every 32 writes sleep to let the log roller run
198         try {
199           Thread.sleep(2000);
200         } catch (InterruptedException e) {
201           // continue
202         }
203       }
204     }
205   }
206 
207   /**
208    * Tests that logs are deleted
209    * @throws IOException
210    * @throws FailedLogCloseException
211    */
212   @Test
213   public void testLogRolling() throws FailedLogCloseException, IOException {
214     this.tableName = getName();
215       startAndWriteData();
216       LOG.info("after writing there are " + log.getNumLogFiles() + " log files");
217 
218       // flush all regions
219 
220       List<HRegion> regions =
221         new ArrayList<HRegion>(server.getOnlineRegionsLocalContext());
222       for (HRegion r: regions) {
223         r.flushcache();
224       }
225 
226       // Now roll the log
227       log.rollWriter();
228 
229       int count = log.getNumLogFiles();
230       LOG.info("after flushing all regions and rolling logs there are " +
231           log.getNumLogFiles() + " log files");
232       assertTrue(("actual count: " + count), count <= 2);
233   }
234 
235   private static String getName() {
236     return "TestLogRolling";
237   }
238 
239   void writeData(HTable table, int rownum) throws IOException {
240     Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", rownum)));
241     put.add(HConstants.CATALOG_FAMILY, null, value);
242     table.put(put);
243 
244     // sleep to let the log roller run (if it needs to)
245     try {
246       Thread.sleep(2000);
247     } catch (InterruptedException e) {
248       // continue
249     }
250   }
251 
252   void batchWriteAndWait(HTable table, int start, boolean expect, int timeout)
253       throws IOException {
254     for (int i = 0; i < 10; i++) {
255       Put put = new Put(Bytes.toBytes("row"
256           + String.format("%1$04d", (start + i))));
257       put.add(HConstants.CATALOG_FAMILY, null, value);
258       table.put(put);
259     }
260     long startTime = System.currentTimeMillis();
261     long remaining = timeout;
262     while (remaining > 0) {
263       if (log.isLowReplicationRollEnabled() == expect) {
264         break;
265       } else {
266         try {
267           Thread.sleep(200);
268         } catch (InterruptedException e) {
269           // continue
270         }
271         remaining = timeout - (System.currentTimeMillis() - startTime);
272       }
273     }
274   }
275   
276   /**
277    * Give me the HDFS pipeline for this log file
278    */
279   DatanodeInfo[] getPipeline(HLog log) throws IllegalArgumentException,
280       IllegalAccessException, InvocationTargetException {
281     OutputStream stm = log.getOutputStream();
282     Method getPipeline = null;
283     for (Method m : stm.getClass().getDeclaredMethods()) {
284       if (m.getName().endsWith("getPipeline")) {
285         getPipeline = m;
286         getPipeline.setAccessible(true);
287         break;
288       }
289     }
290 
291     assertTrue("Need DFSOutputStream.getPipeline() for this test",
292         null != getPipeline);
293     Object repl = getPipeline.invoke(stm, new Object[] {} /* NO_ARGS */);
294     return (DatanodeInfo[]) repl;
295   }
296 
297   /**
298    * Tests that logs are rolled upon detecting datanode death
299    * Requires an HDFS jar with HDFS-826 & syncFs() support (HDFS-200)
300    * @throws IOException
301    * @throws InterruptedException
302    * @throws InvocationTargetException 
303    * @throws IllegalAccessException
304    * @throws IllegalArgumentException 
305     */
306   @Test
307   public void testLogRollOnDatanodeDeath() throws IOException,
308       InterruptedException, IllegalArgumentException, IllegalAccessException,
309       InvocationTargetException {
310     assertTrue("This test requires HLog file replication.",
311       fs.getDefaultReplication() > 1);
312     LOG.info("Replication=" + fs.getDefaultReplication());
313     // When the META table can be opened, the region servers are running
314     new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
315 
316     this.server = cluster.getRegionServer(0);
317     this.log = server.getWAL();
318 
319     // Create the test table and open it
320     String tableName = getName();
321     HTableDescriptor desc = new HTableDescriptor(tableName);
322     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
323 
324     admin.createTable(desc);
325     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
326 
327     server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
328     this.log = server.getWAL();
329 
330     assertTrue("Need HDFS-826 for this test", log.canGetCurReplicas());
331     // don't run this test without append support (HDFS-200 & HDFS-142)
332     assertTrue("Need append support for this test", FSUtils
333         .isAppendSupported(TEST_UTIL.getConfiguration()));
334 
335     // add up the datanode count, to ensure proper replication when we kill 1
336     dfsCluster
337         .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
338     dfsCluster.waitActive();
339     assertTrue(dfsCluster.getDataNodes().size() >= fs.getDefaultReplication() + 1);
340 
341     writeData(table, 2);
342 
343     table.setAutoFlush(true);
344 
345     long curTime = System.currentTimeMillis();
346     long oldFilenum = log.getFilenum();
347     assertTrue("Log should have a timestamp older than now",
348         curTime > oldFilenum && oldFilenum != -1);
349 
350     assertTrue("The log shouldn't have rolled yet", oldFilenum == log.getFilenum());
351     DatanodeInfo[] pipeline = getPipeline(log);
352     assertTrue(pipeline.length == fs.getDefaultReplication());
353 
354     // kill a datanode in the pipeline to force a log roll on the next sync()
355     assertTrue(dfsCluster.stopDataNode(pipeline[0].getName()) != null);
356     Thread.sleep(10000);
357     // this write should succeed, but trigger a log roll
358     writeData(table, 2);
359     long newFilenum = log.getFilenum();
360 
361     assertTrue("Missing datanode should've triggered a log roll",
362         newFilenum > oldFilenum && newFilenum > curTime);
363 
364     // write some more log data (this should use a new hdfs_out)
365     writeData(table, 3);
366     assertTrue("The log should not roll again.", log.getFilenum() == newFilenum);
367     // kill another datanode in the pipeline, so the replicas will be lower than
368     // the configured value 2.
369     assertTrue(dfsCluster.stopDataNode(pipeline[1].getName()) != null);
370     Thread.sleep(10000);
371     batchWriteAndWait(table, 3, false, 10000);
372     assertTrue("LowReplication Roller should've been disabled",
373         !log.isLowReplicationRollEnabled());
374     dfsCluster
375         .startDataNodes(TEST_UTIL.getConfiguration(), 1, true, null, null);
376     dfsCluster.waitActive();
377     // Force roll writer. The new log file will have the default replications,
378     // and the LowReplication Roller will be enabled.
379     log.rollWriter(true);
380     batchWriteAndWait(table, 13, true, 10000);
381     assertTrue("New log file should have the default replication",
382         log.getLogReplication() == fs.getDefaultReplication());
383     assertTrue("LowReplication Roller should've been enabled",
384         log.isLowReplicationRollEnabled());
385   }
386 
387   /**
388    * Test that HLog is rolled when all data nodes in the pipeline have been
389    * restarted.
390    * @throws Exception
391    */
392   @Test
393   public void testLogRollOnPipelineRestart() throws Exception {
394     LOG.info("Starting testLogRollOnPipelineRestart");
395     assertTrue("This test requires HLog file replication.",
396       fs.getDefaultReplication() > 1);
397     LOG.info("Replication=" + fs.getDefaultReplication());
398     // When the META table can be opened, the region servers are running
399     new HTable(TEST_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
400 
401     this.server = cluster.getRegionServer(0);
402     this.log = server.getWAL();
403 
404     // Create the test table and open it
405     String tableName = getName();
406     HTableDescriptor desc = new HTableDescriptor(tableName);
407     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
408 
409     admin.createTable(desc);
410     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
411 
412     server = TEST_UTIL.getRSForFirstRegionInTable(Bytes.toBytes(tableName));
413     this.log = server.getWAL();
414     final List<Path> paths = new ArrayList<Path>();
415     paths.add(log.computeFilename());
416     log.registerWALActionsListener(new WALActionsListener() {
417       @Override
418       public void logRolled(Path newFile) {
419         paths.add(newFile);
420       }
421       @Override
422       public void logRollRequested() {}
423       @Override
424       public void logCloseRequested() {}
425       @Override
426       public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
427           WALEdit logEdit) {}
428       @Override
429       public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey,
430           WALEdit logEdit) {}
431     });
432 
433     assertTrue("Need HDFS-826 for this test", log.canGetCurReplicas());
434     // don't run this test without append support (HDFS-200 & HDFS-142)
435     assertTrue("Need append support for this test", FSUtils
436         .isAppendSupported(TEST_UTIL.getConfiguration()));
437 
438     writeData(table, 1002);
439 
440     table.setAutoFlush(true);
441 
442     long curTime = System.currentTimeMillis();
443     long oldFilenum = log.getFilenum();
444     assertTrue("Log should have a timestamp older than now",
445         curTime > oldFilenum && oldFilenum != -1);
446 
447     assertTrue("The log shouldn't have rolled yet", oldFilenum == log.getFilenum());
448 
449     // roll all datanodes in the pipeline
450     dfsCluster.restartDataNodes();
451     Thread.sleep(10000);
452     dfsCluster.waitActive();
453     LOG.info("Data Nodes restarted");
454 
455     //this.log.sync();
456     // this write should succeed, but trigger a log roll
457     writeData(table, 1003);
458     long newFilenum = log.getFilenum();
459 
460     assertTrue("Missing datanode should've triggered a log roll",
461         newFilenum > oldFilenum && newFilenum > curTime);
462 
463     //this.log.sync();
464     writeData(table, 1004);
465 
466     // roll all datanode again
467     dfsCluster.restartDataNodes();
468     Thread.sleep(10000);
469     dfsCluster.waitActive();
470     LOG.info("Data Nodes restarted");
471 
472     // this write should succeed, but trigger a log roll
473     writeData(table, 1005);
474 
475     // force a log roll to read back and verify previously written logs
476     log.rollWriter(true);
477 
478     // read back the data written
479     Set<String> loggedRows = new HashSet<String>();
480     for (Path p : paths) {
481       LOG.debug("Reading HLog "+FSUtils.getPath(p));
482       HLog.Reader reader = null;
483       try {
484         reader = HLog.getReader(fs, p, TEST_UTIL.getConfiguration());
485         HLog.Entry entry;
486         while ((entry = reader.next()) != null) {
487           LOG.debug("#"+entry.getKey().getLogSeqNum()+": "+entry.getEdit().getKeyValues());
488           for (KeyValue kv : entry.getEdit().getKeyValues()) {
489             loggedRows.add(Bytes.toStringBinary(kv.getRow()));
490           }
491         }
492       } catch (EOFException e) {
493         LOG.debug("EOF reading file "+FSUtils.getPath(p));
494       } finally {
495         if (reader != null) reader.close();
496       }
497     }
498 
499     // verify the written rows are there
500     assertTrue(loggedRows.contains("row1002"));
501     assertTrue(loggedRows.contains("row1003"));
502     assertTrue(loggedRows.contains("row1004"));
503     assertTrue(loggedRows.contains("row1005"));
504 
505     // flush all regions
506     List<HRegion> regions =
507         new ArrayList<HRegion>(server.getOnlineRegionsLocalContext());
508     for (HRegion r: regions) {
509       r.flushcache();
510     }
511 
512     ResultScanner scanner = table.getScanner(new Scan());
513     try {
514       for (int i=2; i<=5; i++) {
515         Result r = scanner.next();
516         assertNotNull(r);
517         assertFalse(r.isEmpty());
518         assertEquals("row100"+i, Bytes.toString(r.getRow()));
519       }
520     } finally {
521       scanner.close();
522     }
523   }
524 }