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.assertTrue;
24  import static org.junit.Assert.assertNotNull;
25  import static org.junit.Assert.fail;
26  
27  import java.io.IOException;
28  import java.lang.reflect.Method;
29  import java.util.HashMap;
30  import java.util.List;
31  import java.util.Map;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.commons.logging.impl.Log4JLogger;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FSDataInputStream;
38  import org.apache.hadoop.fs.FSDataOutputStream;
39  import org.apache.hadoop.fs.FileStatus;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.*;
43  import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.FSUtils;
46  import org.apache.hadoop.hbase.Coprocessor;
47  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
48  import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
49  import org.apache.hadoop.hdfs.DFSClient;
50  import org.apache.hadoop.hdfs.DistributedFileSystem;
51  import org.apache.hadoop.hdfs.MiniDFSCluster;
52  import org.apache.hadoop.hdfs.protocol.FSConstants;
53  import org.apache.hadoop.hdfs.server.datanode.DataNode;
54  import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
55  import org.apache.hadoop.io.SequenceFile;
56  import org.apache.log4j.Level;
57  import org.junit.After;
58  import org.junit.AfterClass;
59  import org.junit.Before;
60  import org.junit.BeforeClass;
61  import org.junit.Test;
62  import org.junit.experimental.categories.Category;
63  
64  /** JUnit test case for HLog */
65  @Category(LargeTests.class)
66  public class TestHLog  {
67    private static final Log LOG = LogFactory.getLog(TestHLog.class);
68    {
69      ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
70      ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
71      ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.server.namenode.FSNamesystem"))
72        .getLogger().setLevel(Level.ALL);
73      ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL);
74      ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL);
75    }
76  
77    private static Configuration conf;
78    private static FileSystem fs;
79    private static Path dir;
80    private static MiniDFSCluster cluster;
81    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
82    private static Path hbaseDir;
83    private static Path oldLogDir;
84  
85    @Before
86    public void setUp() throws Exception {
87  
88      FileStatus[] entries = fs.listStatus(new Path("/"));
89      for (FileStatus dir : entries) {
90        fs.delete(dir.getPath(), true);
91      }
92  
93    }
94  
95    @After
96    public void tearDown() throws Exception {
97    }
98  
99    @BeforeClass
100   public static void setUpBeforeClass() throws Exception {
101     // Make block sizes small.
102     TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
103     // needed for testAppendClose()
104     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
105     // quicker heartbeat interval for faster DN death notification
106     TEST_UTIL.getConfiguration().setInt("heartbeat.recheck.interval", 5000);
107     TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
108     TEST_UTIL.getConfiguration().setInt("dfs.socket.timeout", 5000);
109     // faster failover with cluster.shutdown();fs.close() idiom
110     TEST_UTIL.getConfiguration()
111         .setInt("ipc.client.connect.max.retries", 1);
112     TEST_UTIL.getConfiguration().setInt(
113         "dfs.client.block.recovery.retries", 1);
114     TEST_UTIL.getConfiguration().setInt(
115       "ipc.client.connection.maxidletime", 500);
116     TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
117         SampleRegionWALObserver.class.getName());
118     TEST_UTIL.startMiniDFSCluster(3);
119 
120     conf = TEST_UTIL.getConfiguration();
121     cluster = TEST_UTIL.getDFSCluster();
122     fs = cluster.getFileSystem();
123 
124     hbaseDir = TEST_UTIL.createRootDir();
125     oldLogDir = new Path(hbaseDir, ".oldlogs");
126     dir = new Path(hbaseDir, getName());
127   }
128   @AfterClass
129   public static void tearDownAfterClass() throws Exception {
130     TEST_UTIL.shutdownMiniCluster();
131   }
132 
133   private static String getName() {
134     // TODO Auto-generated method stub
135     return "TestHLog";
136   }
137 
138   /**
139    * Test that with three concurrent threads we still write edits in sequence
140    * edit id order.
141    * @throws Exception
142    */
143   @Test
144   public void testMaintainOrderWithConcurrentWrites() throws Exception {
145     // Run the HPE tool with three threads writing 3000 edits each concurrently.
146     // When done, verify that all edits were written and that the order in the
147     // WALs is of ascending edit sequence ids.
148     int errCode =
149       HLogPerformanceEvaluation.innerMain(new String [] {"-threads", "3", "-verify", "-iterations", "3000"});
150     assertEquals(0, errCode);
151   }
152 
153   /**
154    * Just write multiple logs then split.  Before fix for HADOOP-2283, this
155    * would fail.
156    * @throws IOException
157    */
158   @Test
159   public void testSplit() throws IOException {
160 
161     final byte [] tableName = Bytes.toBytes(getName());
162     final byte [] rowName = tableName;
163     Path logdir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
164     HLog log = new HLog(fs, logdir, oldLogDir, conf);
165     final int howmany = 3;
166     HRegionInfo[] infos = new HRegionInfo[3];
167     Path tabledir = new Path(hbaseDir, getName());
168     fs.mkdirs(tabledir);
169     for(int i = 0; i < howmany; i++) {
170       infos[i] = new HRegionInfo(tableName,
171                 Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false);
172       fs.mkdirs(new Path(tabledir, infos[i].getEncodedName()));
173       LOG.info("allo " + new Path(tabledir, infos[i].getEncodedName()).toString());
174     }
175     HTableDescriptor htd = new HTableDescriptor(tableName);
176     htd.addFamily(new HColumnDescriptor("column"));
177 
178     // Add edits for three regions.
179     try {
180       for (int ii = 0; ii < howmany; ii++) {
181         for (int i = 0; i < howmany; i++) {
182 
183           for (int j = 0; j < howmany; j++) {
184             WALEdit edit = new WALEdit();
185             byte [] family = Bytes.toBytes("column");
186             byte [] qualifier = Bytes.toBytes(Integer.toString(j));
187             byte [] column = Bytes.toBytes("column:" + Integer.toString(j));
188             edit.add(new KeyValue(rowName, family, qualifier,
189                 System.currentTimeMillis(), column));
190             LOG.info("Region " + i + ": " + edit);
191             log.append(infos[i], tableName, edit,
192               System.currentTimeMillis(), htd);
193           }
194         }
195         log.rollWriter();
196       }
197       log.close();
198       HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
199           hbaseDir, logdir, this.oldLogDir, this.fs);
200       List<Path> splits =
201         logSplitter.splitLog();
202       verifySplits(splits, howmany);
203       log = null;
204     } finally {
205       if (log != null) {
206         log.closeAndDelete();
207       }
208     }
209   }
210 
211   /**
212    * Test new HDFS-265 sync.
213    * @throws Exception
214    */
215   @Test
216   public void Broken_testSync() throws Exception {
217     byte [] bytes = Bytes.toBytes(getName());
218     // First verify that using streams all works.
219     Path p = new Path(dir, getName() + ".fsdos");
220     FSDataOutputStream out = fs.create(p);
221     out.write(bytes);
222     Method syncMethod = null;
223     try {
224       syncMethod = out.getClass().getMethod("hflush", new Class<?> []{});
225     } catch (NoSuchMethodException e) {
226       try {
227         syncMethod = out.getClass().getMethod("sync", new Class<?> []{});
228       } catch (NoSuchMethodException ex) {
229         fail("This version of Hadoop supports neither Syncable.sync() " +
230             "nor Syncable.hflush().");
231       }
232     }
233     syncMethod.invoke(out, new Object[]{});
234     FSDataInputStream in = fs.open(p);
235     assertTrue(in.available() > 0);
236     byte [] buffer = new byte [1024];
237     int read = in.read(buffer);
238     assertEquals(bytes.length, read);
239     out.close();
240     in.close();
241     Path subdir = new Path(dir, "hlogdir");
242     HLog wal = new HLog(fs, subdir, oldLogDir, conf);
243     final int total = 20;
244     HLog.Reader reader = null;
245 
246     try {
247       HRegionInfo info = new HRegionInfo(bytes,
248                   null,null, false);
249       HTableDescriptor htd = new HTableDescriptor();
250       htd.addFamily(new HColumnDescriptor(bytes));
251 
252       for (int i = 0; i < total; i++) {
253         WALEdit kvs = new WALEdit();
254         kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
255         wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
256       }
257       // Now call sync and try reading.  Opening a Reader before you sync just
258       // gives you EOFE.
259       wal.sync();
260       // Open a Reader.
261       Path walPath = wal.computeFilename();
262       reader = HLog.getReader(fs, walPath, conf);
263       int count = 0;
264       HLog.Entry entry = new HLog.Entry();
265       while ((entry = reader.next(entry)) != null) count++;
266       assertEquals(total, count);
267       reader.close();
268       // Add test that checks to see that an open of a Reader works on a file
269       // that has had a sync done on it.
270       for (int i = 0; i < total; i++) {
271         WALEdit kvs = new WALEdit();
272         kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
273         wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
274       }
275       reader = HLog.getReader(fs, walPath, conf);
276       count = 0;
277       while((entry = reader.next(entry)) != null) count++;
278       assertTrue(count >= total);
279       reader.close();
280       // If I sync, should see double the edits.
281       wal.sync();
282       reader = HLog.getReader(fs, walPath, conf);
283       count = 0;
284       while((entry = reader.next(entry)) != null) count++;
285       assertEquals(total * 2, count);
286       // Now do a test that ensures stuff works when we go over block boundary,
287       // especially that we return good length on file.
288       final byte [] value = new byte[1025 * 1024];  // Make a 1M value.
289       for (int i = 0; i < total; i++) {
290         WALEdit kvs = new WALEdit();
291         kvs.add(new KeyValue(Bytes.toBytes(i), bytes, value));
292         wal.append(info, bytes, kvs, System.currentTimeMillis(), htd);
293       }
294       // Now I should have written out lots of blocks.  Sync then read.
295       wal.sync();
296       reader = HLog.getReader(fs, walPath, conf);
297       count = 0;
298       while((entry = reader.next(entry)) != null) count++;
299       assertEquals(total * 3, count);
300       reader.close();
301       // Close it and ensure that closed, Reader gets right length also.
302       wal.close();
303       reader = HLog.getReader(fs, walPath, conf);
304       count = 0;
305       while((entry = reader.next(entry)) != null) count++;
306       assertEquals(total * 3, count);
307       reader.close();
308     } finally {
309       if (wal != null) wal.closeAndDelete();
310       if (reader != null) reader.close();
311     }
312   }
313 
314   /**
315    * Test the findMemstoresWithEditsEqualOrOlderThan method.
316    * @throws IOException
317    */
318   @Test
319   public void testFindMemstoresWithEditsEqualOrOlderThan() throws IOException {
320     Map<byte [], Long> regionsToSeqids = new HashMap<byte [], Long>();
321     for (int i = 0; i < 10; i++) {
322       Long l = Long.valueOf(i);
323       regionsToSeqids.put(l.toString().getBytes(), l);
324     }
325     byte [][] regions =
326       HLog.findMemstoresWithEditsEqualOrOlderThan(1, regionsToSeqids);
327     assertEquals(2, regions.length);
328     assertTrue(Bytes.equals(regions[0], "0".getBytes()) ||
329         Bytes.equals(regions[0], "1".getBytes()));
330     regions = HLog.findMemstoresWithEditsEqualOrOlderThan(3, regionsToSeqids);
331     int count = 4;
332     assertEquals(count, regions.length);
333     // Regions returned are not ordered.
334     for (int i = 0; i < count; i++) {
335       assertTrue(Bytes.equals(regions[i], "0".getBytes()) ||
336         Bytes.equals(regions[i], "1".getBytes()) ||
337         Bytes.equals(regions[i], "2".getBytes()) ||
338         Bytes.equals(regions[i], "3".getBytes()));
339     }
340   }
341 
342   private void verifySplits(List<Path> splits, final int howmany)
343   throws IOException {
344     assertEquals(howmany, splits.size());
345     for (int i = 0; i < splits.size(); i++) {
346       LOG.info("Verifying=" + splits.get(i));
347       HLog.Reader reader = HLog.getReader(fs, splits.get(i), conf);
348       try {
349         int count = 0;
350         String previousRegion = null;
351         long seqno = -1;
352         HLog.Entry entry = new HLog.Entry();
353         while((entry = reader.next(entry)) != null) {
354           HLogKey key = entry.getKey();
355           String region = Bytes.toString(key.getEncodedRegionName());
356           // Assert that all edits are for same region.
357           if (previousRegion != null) {
358             assertEquals(previousRegion, region);
359           }
360           LOG.info("oldseqno=" + seqno + ", newseqno=" + key.getLogSeqNum());
361           assertTrue(seqno < key.getLogSeqNum());
362           seqno = key.getLogSeqNum();
363           previousRegion = region;
364           count++;
365         }
366         assertEquals(howmany * howmany, count);
367       } finally {
368         reader.close();
369       }
370     }
371   }
372   
373   // For this test to pass, requires:
374   // 1. HDFS-200 (append support)
375   // 2. HDFS-988 (SafeMode should freeze file operations
376   //              [FSNamesystem.nextGenerationStampForBlock])
377   // 3. HDFS-142 (on restart, maintain pendingCreates)
378   @Test
379   public void testAppendClose() throws Exception {
380     byte [] tableName = Bytes.toBytes(getName());
381     HRegionInfo regioninfo = new HRegionInfo(tableName,
382              HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
383     Path subdir = new Path(dir, "hlogdir");
384     Path archdir = new Path(dir, "hlogdir_archive");
385     HLog wal = new HLog(fs, subdir, archdir, conf);
386     final int total = 20;
387 
388     HTableDescriptor htd = new HTableDescriptor();
389     htd.addFamily(new HColumnDescriptor(tableName));
390 
391     for (int i = 0; i < total; i++) {
392       WALEdit kvs = new WALEdit();
393       kvs.add(new KeyValue(Bytes.toBytes(i), tableName, tableName));
394       wal.append(regioninfo, tableName, kvs, System.currentTimeMillis(), htd);
395     }
396     // Now call sync to send the data to HDFS datanodes
397     wal.sync();
398      int namenodePort = cluster.getNameNodePort();
399     final Path walPath = wal.computeFilename();
400     
401 
402     // Stop the cluster.  (ensure restart since we're sharing MiniDFSCluster)
403     try {
404       DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
405       dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER);
406       cluster.shutdown();
407       try {
408         // wal.writer.close() will throw an exception,
409         // but still call this since it closes the LogSyncer thread first
410         wal.close();
411       } catch (IOException e) {
412         LOG.info(e);
413       }
414       fs.close(); // closing FS last so DFSOutputStream can't call close
415       LOG.info("STOPPED first instance of the cluster");
416     } finally {
417       // Restart the cluster
418       while (cluster.isClusterUp()){
419         LOG.error("Waiting for cluster to go down");
420         Thread.sleep(1000);
421       }
422 
423       // Workaround a strange issue with Hadoop's RPC system - if we don't
424       // sleep here, the new datanodes will pick up a cached IPC connection to
425       // the old (dead) NN and fail to start. Sleeping 2 seconds goes past
426       // the idle time threshold configured in the conf above
427       Thread.sleep(2000);
428 
429       cluster = new MiniDFSCluster(namenodePort, conf, 5, false, true, true, null, null, null, null);
430       TEST_UTIL.setDFSCluster(cluster);
431       cluster.waitActive();
432       fs = cluster.getFileSystem();
433       LOG.info("START second instance.");
434     }
435 
436     // set the lease period to be 1 second so that the
437     // namenode triggers lease recovery upon append request
438     Method setLeasePeriod = cluster.getClass()
439       .getDeclaredMethod("setLeasePeriod", new Class[]{Long.TYPE, Long.TYPE});
440     setLeasePeriod.setAccessible(true);
441     setLeasePeriod.invoke(cluster,
442                           new Object[]{new Long(1000), new Long(1000)});
443     try {
444       Thread.sleep(1000);
445     } catch (InterruptedException e) {
446       LOG.info(e);
447     }
448     
449     // Now try recovering the log, like the HMaster would do
450     final FileSystem recoveredFs = fs;
451     final Configuration rlConf = conf;
452     
453     class RecoverLogThread extends Thread {
454       public Exception exception = null;
455       public void run() {
456           try {
457             FSUtils.getInstance(fs, rlConf)
458               .recoverFileLease(recoveredFs, walPath, rlConf);
459           } catch (IOException e) {
460             exception = e;
461           }
462       }
463     }
464 
465     RecoverLogThread t = new RecoverLogThread();
466     t.start();
467     // Timeout after 60 sec. Without correct patches, would be an infinite loop
468     t.join(60 * 1000);
469     if(t.isAlive()) {
470       t.interrupt();
471       throw new Exception("Timed out waiting for HLog.recoverLog()");
472     }
473 
474     if (t.exception != null)
475       throw t.exception;
476 
477     // Make sure you can read all the content
478     HLog.Reader reader = HLog.getReader(this.fs, walPath, this.conf);
479     int count = 0;
480     HLog.Entry entry = new HLog.Entry();
481     while (reader.next(entry) != null) {
482       count++;
483       assertTrue("Should be one KeyValue per WALEdit",
484                  entry.getEdit().getKeyValues().size() == 1);
485     }
486     assertEquals(total, count);
487     reader.close();
488 
489     // Reset the lease period
490     setLeasePeriod.invoke(cluster, new Object[]{new Long(60000), new Long(3600000)});
491   }
492 
493   /**
494    * Tests that we can write out an edit, close, and then read it back in again.
495    * @throws IOException
496    */
497   @Test
498   public void testEditAdd() throws IOException {
499     final int COL_COUNT = 10;
500     final byte [] tableName = Bytes.toBytes("tablename");
501     final byte [] row = Bytes.toBytes("row");
502     HLog.Reader reader = null;
503     HLog log = null;
504     try {
505       log = new HLog(fs, dir, oldLogDir, conf);
506       // Write columns named 1, 2, 3, etc. and then values of single byte
507       // 1, 2, 3...
508       long timestamp = System.currentTimeMillis();
509       WALEdit cols = new WALEdit();
510       for (int i = 0; i < COL_COUNT; i++) {
511         cols.add(new KeyValue(row, Bytes.toBytes("column"),
512             Bytes.toBytes(Integer.toString(i)),
513           timestamp, new byte[] { (byte)(i + '0') }));
514       }
515       HRegionInfo info = new HRegionInfo(tableName,
516         row,Bytes.toBytes(Bytes.toString(row) + "1"), false);
517       HTableDescriptor htd = new HTableDescriptor();
518       htd.addFamily(new HColumnDescriptor("column"));
519 
520       log.append(info, tableName, cols, System.currentTimeMillis(), htd);
521       long logSeqId = log.startCacheFlush(info.getEncodedNameAsBytes());
522       log.completeCacheFlush(info.getEncodedNameAsBytes(), tableName, logSeqId,
523           info.isMetaRegion());
524       log.close();
525       Path filename = log.computeFilename();
526       log = null;
527       // Now open a reader on the log and assert append worked.
528       reader = HLog.getReader(fs, filename, conf);
529       // Above we added all columns on a single row so we only read one
530       // entry in the below... thats why we have '1'.
531       for (int i = 0; i < 1; i++) {
532         HLog.Entry entry = reader.next(null);
533         if (entry == null) break;
534         HLogKey key = entry.getKey();
535         WALEdit val = entry.getEdit();
536         assertTrue(Bytes.equals(info.getEncodedNameAsBytes(), key.getEncodedRegionName()));
537         assertTrue(Bytes.equals(tableName, key.getTablename()));
538         KeyValue kv = val.getKeyValues().get(0);
539         assertTrue(Bytes.equals(row, kv.getRow()));
540         assertEquals((byte)(i + '0'), kv.getValue()[0]);
541         System.out.println(key + " " + val);
542       }
543       HLog.Entry entry = null;
544       while ((entry = reader.next(null)) != null) {
545         HLogKey key = entry.getKey();
546         WALEdit val = entry.getEdit();
547         // Assert only one more row... the meta flushed row.
548         assertTrue(Bytes.equals(info.getEncodedNameAsBytes(), key.getEncodedRegionName()));
549         assertTrue(Bytes.equals(tableName, key.getTablename()));
550         KeyValue kv = val.getKeyValues().get(0);
551         assertTrue(Bytes.equals(HLog.METAROW, kv.getRow()));
552         assertTrue(Bytes.equals(HLog.METAFAMILY, kv.getFamily()));
553         assertEquals(0, Bytes.compareTo(HLog.COMPLETE_CACHE_FLUSH,
554           val.getKeyValues().get(0).getValue()));
555         System.out.println(key + " " + val);
556       }
557     } finally {
558       if (log != null) {
559         log.closeAndDelete();
560       }
561       if (reader != null) {
562         reader.close();
563       }
564     }
565   }
566 
567   /**
568    * @throws IOException
569    */
570   @Test
571   public void testAppend() throws IOException {
572     final int COL_COUNT = 10;
573     final byte [] tableName = Bytes.toBytes("tablename");
574     final byte [] row = Bytes.toBytes("row");
575     Reader reader = null;
576     HLog log = new HLog(fs, dir, oldLogDir, conf);
577     try {
578       // Write columns named 1, 2, 3, etc. and then values of single byte
579       // 1, 2, 3...
580       long timestamp = System.currentTimeMillis();
581       WALEdit cols = new WALEdit();
582       for (int i = 0; i < COL_COUNT; i++) {
583         cols.add(new KeyValue(row, Bytes.toBytes("column"),
584           Bytes.toBytes(Integer.toString(i)),
585           timestamp, new byte[] { (byte)(i + '0') }));
586       }
587       HRegionInfo hri = new HRegionInfo(tableName,
588           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
589       HTableDescriptor htd = new HTableDescriptor();
590       htd.addFamily(new HColumnDescriptor("column"));
591       log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
592       long logSeqId = log.startCacheFlush(hri.getEncodedNameAsBytes());
593       log.completeCacheFlush(hri.getEncodedNameAsBytes(), tableName, logSeqId, false);
594       log.close();
595       Path filename = log.computeFilename();
596       log = null;
597       // Now open a reader on the log and assert append worked.
598       reader = HLog.getReader(fs, filename, conf);
599       HLog.Entry entry = reader.next();
600       assertEquals(COL_COUNT, entry.getEdit().size());
601       int idx = 0;
602       for (KeyValue val : entry.getEdit().getKeyValues()) {
603         assertTrue(Bytes.equals(hri.getEncodedNameAsBytes(),
604           entry.getKey().getEncodedRegionName()));
605         assertTrue(Bytes.equals(tableName, entry.getKey().getTablename()));
606         assertTrue(Bytes.equals(row, val.getRow()));
607         assertEquals((byte)(idx + '0'), val.getValue()[0]);
608         System.out.println(entry.getKey() + " " + val);
609         idx++;
610       }
611 
612       // Get next row... the meta flushed row.
613       entry = reader.next();
614       assertEquals(1, entry.getEdit().size());
615       for (KeyValue val : entry.getEdit().getKeyValues()) {
616         assertTrue(Bytes.equals(hri.getEncodedNameAsBytes(),
617           entry.getKey().getEncodedRegionName()));
618         assertTrue(Bytes.equals(tableName, entry.getKey().getTablename()));
619         assertTrue(Bytes.equals(HLog.METAROW, val.getRow()));
620         assertTrue(Bytes.equals(HLog.METAFAMILY, val.getFamily()));
621         assertEquals(0, Bytes.compareTo(HLog.COMPLETE_CACHE_FLUSH,
622           val.getValue()));
623         System.out.println(entry.getKey() + " " + val);
624       }
625     } finally {
626       if (log != null) {
627         log.closeAndDelete();
628       }
629       if (reader != null) {
630         reader.close();
631       }
632     }
633   }
634 
635   /**
636    * Test that we can visit entries before they are appended
637    * @throws Exception
638    */
639   @Test
640   public void testVisitors() throws Exception {
641     final int COL_COUNT = 10;
642     final byte [] tableName = Bytes.toBytes("tablename");
643     final byte [] row = Bytes.toBytes("row");
644     HLog log = new HLog(fs, dir, oldLogDir, conf);
645     try {
646       DumbWALActionsListener visitor = new DumbWALActionsListener();
647       log.registerWALActionsListener(visitor);
648       long timestamp = System.currentTimeMillis();
649       HTableDescriptor htd = new HTableDescriptor();
650       htd.addFamily(new HColumnDescriptor("column"));
651 
652       HRegionInfo hri = new HRegionInfo(tableName,
653           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
654       for (int i = 0; i < COL_COUNT; i++) {
655         WALEdit cols = new WALEdit();
656         cols.add(new KeyValue(row, Bytes.toBytes("column"),
657             Bytes.toBytes(Integer.toString(i)),
658             timestamp, new byte[]{(byte) (i + '0')}));
659         log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
660       }
661       assertEquals(COL_COUNT, visitor.increments);
662       log.unregisterWALActionsListener(visitor);
663       WALEdit cols = new WALEdit();
664       cols.add(new KeyValue(row, Bytes.toBytes("column"),
665           Bytes.toBytes(Integer.toString(11)),
666           timestamp, new byte[]{(byte) (11 + '0')}));
667       log.append(hri, tableName, cols, System.currentTimeMillis(), htd);
668       assertEquals(COL_COUNT, visitor.increments);
669     } finally {
670       if (log != null) log.closeAndDelete();
671     }
672   }
673 
674   @Test
675   public void testLogCleaning() throws Exception {
676     LOG.info("testLogCleaning");
677     final byte [] tableName = Bytes.toBytes("testLogCleaning");
678     final byte [] tableName2 = Bytes.toBytes("testLogCleaning2");
679 
680     HLog log = new HLog(fs, dir, oldLogDir, conf);
681     try {
682       HRegionInfo hri = new HRegionInfo(tableName,
683           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
684       HRegionInfo hri2 = new HRegionInfo(tableName2,
685           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
686 
687       // Add a single edit and make sure that rolling won't remove the file
688       // Before HBASE-3198 it used to delete it
689       addEdits(log, hri, tableName, 1);
690       log.rollWriter();
691       assertEquals(1, log.getNumLogFiles());
692 
693       // See if there's anything wrong with more than 1 edit
694       addEdits(log, hri, tableName, 2);
695       log.rollWriter();
696       assertEquals(2, log.getNumLogFiles());
697 
698       // Now mix edits from 2 regions, still no flushing
699       addEdits(log, hri, tableName, 1);
700       addEdits(log, hri2, tableName2, 1);
701       addEdits(log, hri, tableName, 1);
702       addEdits(log, hri2, tableName2, 1);
703       log.rollWriter();
704       assertEquals(3, log.getNumLogFiles());
705 
706       // Flush the first region, we expect to see the first two files getting
707       // archived
708       long seqId = log.startCacheFlush(hri.getEncodedNameAsBytes());
709       log.completeCacheFlush(hri.getEncodedNameAsBytes(), tableName, seqId, false);
710       log.rollWriter();
711       assertEquals(2, log.getNumLogFiles());
712 
713       // Flush the second region, which removes all the remaining output files
714       // since the oldest was completely flushed and the two others only contain
715       // flush information
716       seqId = log.startCacheFlush(hri2.getEncodedNameAsBytes());
717       log.completeCacheFlush(hri2.getEncodedNameAsBytes(), tableName2, seqId, false);
718       log.rollWriter();
719       assertEquals(0, log.getNumLogFiles());
720     } finally {
721       if (log != null) log.closeAndDelete();
722     }
723   }
724 
725   /**
726    * A loaded WAL coprocessor won't break existing HLog test cases.
727    */
728   @Test
729   public void testWALCoprocessorLoaded() throws Exception {
730     // test to see whether the coprocessor is loaded or not.
731     HLog log = new HLog(fs, dir, oldLogDir, conf);
732     try {
733       WALCoprocessorHost host = log.getCoprocessorHost();
734       Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class.getName());
735       assertNotNull(c);
736     } finally {
737       if (log != null) log.closeAndDelete();
738     }
739   }
740 
741   private void addEdits(HLog log, HRegionInfo hri, byte [] tableName,
742                         int times) throws IOException {
743     HTableDescriptor htd = new HTableDescriptor();
744     htd.addFamily(new HColumnDescriptor("row"));
745 
746     final byte [] row = Bytes.toBytes("row");
747     for (int i = 0; i < times; i++) {
748       long timestamp = System.currentTimeMillis();
749       WALEdit cols = new WALEdit();
750       cols.add(new KeyValue(row, row, row, timestamp, row));
751       log.append(hri, tableName, cols, timestamp, htd);
752     }
753   }
754 
755   static class DumbWALActionsListener implements WALActionsListener {
756     int increments = 0;
757 
758     @Override
759     public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
760                                          WALEdit logEdit) {
761       increments++;
762     }
763 
764     @Override
765     public void visitLogEntryBeforeWrite(HTableDescriptor htd, HLogKey logKey, WALEdit logEdit) {
766       //To change body of implemented methods use File | Settings | File Templates.
767       increments++;
768     }
769 
770     @Override
771     public void preLogRoll(Path oldFile, Path newFile) {
772       // TODO Auto-generated method stub
773     }
774 
775     @Override
776     public void postLogRoll(Path oldFile, Path newFile) {
777       // TODO Auto-generated method stub
778     }
779 
780     @Override
781     public void preLogArchive(Path oldFile, Path newFile) {
782       // TODO Auto-generated method stub
783     }
784 
785     @Override
786     public void postLogArchive(Path oldFile, Path newFile) {
787       // TODO Auto-generated method stub
788     }
789 
790     @Override
791     public void logRollRequested() {
792       // TODO Auto-generated method stub
793       
794     }
795 
796     @Override
797     public void logCloseRequested() {
798       // not interested
799     }
800   }
801 
802   @org.junit.Rule
803   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
804     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
805 }
806