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