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