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     // faster failover with cluster.shutdown();fs.close() idiom
106     TEST_UTIL.getConfiguration()
107         .setInt("ipc.client.connect.max.retries", 1);
108     TEST_UTIL.getConfiguration().setInt(
109         "dfs.client.block.recovery.retries", 1);
110     TEST_UTIL.startMiniCluster(3);
111 
112     conf = TEST_UTIL.getConfiguration();
113     cluster = TEST_UTIL.getDFSCluster();
114     fs = cluster.getFileSystem();
115 
116     hbaseDir = new Path(TEST_UTIL.getConfiguration().get("hbase.rootdir"));
117     oldLogDir = new Path(hbaseDir, ".oldlogs");
118     dir = new Path(hbaseDir, getName());
119   }
120   private static String getName() {
121     // TODO Auto-generated method stub
122     return "TestHLog";
123   }
124 
125   /**
126    * Just write multiple logs then split.  Before fix for HADOOP-2283, this
127    * would fail.
128    * @throws IOException
129    */
130   @Test
131   public void testSplit() throws IOException {
132 
133     final byte [] tableName = Bytes.toBytes(getName());
134     final byte [] rowName = tableName;
135     Path logdir = new Path(dir, HConstants.HREGION_LOGDIR_NAME);
136     HLog log = new HLog(fs, logdir, oldLogDir, conf, null);
137     final int howmany = 3;
138     HRegionInfo[] infos = new HRegionInfo[3];
139     for(int i = 0; i < howmany; i++) {
140       infos[i] = new HRegionInfo(new HTableDescriptor(tableName),
141                 Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false);
142     }
143     // Add edits for three regions.
144     try {
145       for (int ii = 0; ii < howmany; ii++) {
146         for (int i = 0; i < howmany; i++) {
147 
148           for (int j = 0; j < howmany; j++) {
149             WALEdit edit = new WALEdit();
150             byte [] family = Bytes.toBytes("column");
151             byte [] qualifier = Bytes.toBytes(Integer.toString(j));
152             byte [] column = Bytes.toBytes("column:" + Integer.toString(j));
153             edit.add(new KeyValue(rowName, family, qualifier,
154                 System.currentTimeMillis(), column));
155             LOG.info("Region " + i + ": " + edit);
156             log.append(infos[i], tableName, edit,
157               System.currentTimeMillis());
158           }
159         }
160         log.rollWriter();
161       }
162       log.close();
163       Path splitsdir = new Path(dir, "splits");
164       List<Path> splits =
165         HLog.splitLog(splitsdir, logdir, oldLogDir, fs, conf);
166       verifySplits(splits, howmany);
167       log = null;
168     } finally {
169       if (log != null) {
170         log.closeAndDelete();
171       }
172     }
173   }
174 
175   /**
176    * Test new HDFS-265 sync.
177    * @throws Exception
178    */
179   @Test
180   public void Broken_testSync() throws Exception {
181     byte [] bytes = Bytes.toBytes(getName());
182     // First verify that using streams all works.
183     Path p = new Path(dir, getName() + ".fsdos");
184     FSDataOutputStream out = fs.create(p);
185     out.write(bytes);
186     out.sync();
187     FSDataInputStream in = fs.open(p);
188     assertTrue(in.available() > 0);
189     byte [] buffer = new byte [1024];
190     int read = in.read(buffer);
191     assertEquals(bytes.length, read);
192     out.close();
193     in.close();
194     Path subdir = new Path(dir, "hlogdir");
195     HLog wal = new HLog(fs, subdir, oldLogDir, conf, null);
196     final int total = 20;
197 
198     HRegionInfo info = new HRegionInfo(new HTableDescriptor(bytes),
199                 null,null, false);
200 
201     for (int i = 0; i < total; i++) {
202       WALEdit kvs = new WALEdit();
203       kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
204       wal.append(info, bytes, kvs, System.currentTimeMillis());
205     }
206     // Now call sync and try reading.  Opening a Reader before you sync just
207     // gives you EOFE.
208     wal.sync();
209     // Open a Reader.
210     Path walPath = wal.computeFilename();
211     HLog.Reader reader = HLog.getReader(fs, walPath, conf);
212     int count = 0;
213     HLog.Entry entry = new HLog.Entry();
214     while ((entry = reader.next(entry)) != null) count++;
215     assertEquals(total, count);
216     reader.close();
217     // Add test that checks to see that an open of a Reader works on a file
218     // that has had a sync done on it.
219     for (int i = 0; i < total; i++) {
220       WALEdit kvs = new WALEdit();
221       kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes));
222       wal.append(info, bytes, kvs, System.currentTimeMillis());
223     }
224     reader = HLog.getReader(fs, walPath, conf);
225     count = 0;
226     while((entry = reader.next(entry)) != null) count++;
227     assertTrue(count >= total);
228     reader.close();
229     // If I sync, should see double the edits.
230     wal.sync();
231     reader = HLog.getReader(fs, walPath, conf);
232     count = 0;
233     while((entry = reader.next(entry)) != null) count++;
234     assertEquals(total * 2, count);
235     // Now do a test that ensures stuff works when we go over block boundary,
236     // especially that we return good length on file.
237     final byte [] value = new byte[1025 * 1024];  // Make a 1M value.
238     for (int i = 0; i < total; i++) {
239       WALEdit kvs = new WALEdit();
240       kvs.add(new KeyValue(Bytes.toBytes(i), bytes, value));
241       wal.append(info, bytes, kvs, System.currentTimeMillis());
242     }
243     // Now I should have written out lots of blocks.  Sync then read.
244     wal.sync();
245     reader = HLog.getReader(fs, walPath, conf);
246     count = 0;
247     while((entry = reader.next(entry)) != null) count++;
248     assertEquals(total * 3, count);
249     reader.close();
250     // Close it and ensure that closed, Reader gets right length also.
251     wal.close();
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   }
258 
259   /**
260    * Test the findMemstoresWithEditsOlderThan method.
261    * @throws IOException
262    */
263   @Test
264   public void testFindMemstoresWithEditsOlderThan() throws IOException {
265     Map<byte [], Long> regionsToSeqids = new HashMap<byte [], Long>();
266     for (int i = 0; i < 10; i++) {
267       Long l = Long.valueOf(i);
268       regionsToSeqids.put(l.toString().getBytes(), l);
269     }
270     byte [][] regions =
271       HLog.findMemstoresWithEditsOlderThan(1, regionsToSeqids);
272     assertEquals(1, regions.length);
273     assertTrue(Bytes.equals(regions[0], "0".getBytes()));
274     regions = HLog.findMemstoresWithEditsOlderThan(3, regionsToSeqids);
275     int count = 3;
276     assertEquals(count, regions.length);
277     // Regions returned are not ordered.
278     for (int i = 0; i < count; i++) {
279       assertTrue(Bytes.equals(regions[i], "0".getBytes()) ||
280         Bytes.equals(regions[i], "1".getBytes()) ||
281         Bytes.equals(regions[i], "2".getBytes()));
282     }
283   }
284 
285   private void verifySplits(List<Path> splits, final int howmany)
286   throws IOException {
287     assertEquals(howmany, splits.size());
288     for (int i = 0; i < splits.size(); i++) {
289       LOG.info("Verifying=" + splits.get(i));
290       HLog.Reader reader = HLog.getReader(fs, splits.get(i), conf);
291       try {
292         int count = 0;
293         String previousRegion = null;
294         long seqno = -1;
295         HLog.Entry entry = new HLog.Entry();
296         while((entry = reader.next(entry)) != null) {
297           HLogKey key = entry.getKey();
298           String region = Bytes.toString(key.getRegionName());
299           // Assert that all edits are for same region.
300           if (previousRegion != null) {
301             assertEquals(previousRegion, region);
302           }
303           LOG.info("oldseqno=" + seqno + ", newseqno=" + key.getLogSeqNum());
304           assertTrue(seqno < key.getLogSeqNum());
305           seqno = key.getLogSeqNum();
306           previousRegion = region;
307           count++;
308         }
309         assertEquals(howmany * howmany, count);
310       } finally {
311         reader.close();
312       }
313     }
314   }
315   
316   // For this test to pass, requires: 
317   // 1. HDFS-200 (append support)
318   // 2. HDFS-988 (SafeMode should freeze file operations 
319   //              [FSNamesystem.nextGenerationStampForBlock])
320   // 3. HDFS-142 (on restart, maintain pendingCreates) 
321   @Test
322   public void testAppendClose() throws Exception {
323     byte [] tableName = Bytes.toBytes(getName());
324     HRegionInfo regioninfo = new HRegionInfo(new HTableDescriptor(tableName),
325         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
326     Path subdir = new Path(dir, "hlogdir");
327     Path archdir = new Path(dir, "hlogdir_archive");
328     HLog wal = new HLog(fs, subdir, archdir, conf, null);
329     final int total = 20;
330 
331     for (int i = 0; i < total; i++) {
332       WALEdit kvs = new WALEdit();
333       kvs.add(new KeyValue(Bytes.toBytes(i), tableName, tableName));
334       wal.append(regioninfo, tableName, kvs, System.currentTimeMillis());
335     }
336     // Now call sync to send the data to HDFS datanodes
337     wal.sync(true);
338      int namenodePort = cluster.getNameNodePort();
339     final Path walPath = wal.computeFilename();
340     
341 
342     // Stop the cluster.  (ensure restart since we're sharing MiniDFSCluster)
343     try {
344       cluster.getNameNode().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
345       cluster.shutdown();
346       try {
347         // wal.writer.close() will throw an exception, 
348         // but still call this since it closes the LogSyncer thread first
349         wal.close();
350       } catch (IOException e) {
351         LOG.info(e);
352       }
353       fs.close(); // closing FS last so DFSOutputStream can't call close
354       LOG.info("STOPPED first instance of the cluster");
355     } finally {
356       // Restart the cluster
357       while (cluster.isClusterUp()){
358         LOG.error("Waiting for cluster to go down");
359         Thread.sleep(1000);
360       }
361       cluster = new MiniDFSCluster(namenodePort, conf, 5, false, true, true, null, null, null, null);
362       cluster.waitActive();
363       fs = cluster.getFileSystem();
364       LOG.info("START second instance.");
365     }
366 
367     // set the lease period to be 1 second so that the
368     // namenode triggers lease recovery upon append request
369     Method setLeasePeriod = cluster.getClass()
370       .getDeclaredMethod("setLeasePeriod", new Class[]{Long.TYPE, Long.TYPE});
371     setLeasePeriod.setAccessible(true);
372     setLeasePeriod.invoke(cluster, 
373                           new Object[]{new Long(1000), new Long(1000)});
374     try {
375       Thread.sleep(1000);
376     } catch (InterruptedException e) {
377       LOG.info(e);
378     }
379     
380     // Now try recovering the log, like the HMaster would do
381     final FileSystem recoveredFs = fs;
382     final Configuration rlConf = conf;
383     
384     class RecoverLogThread extends Thread {
385       public Exception exception = null;
386       public void run() {
387           try {
388             FSUtils.recoverFileLease(recoveredFs, walPath, rlConf);
389           } catch (IOException e) {
390             exception = e;
391           }
392       }
393     }
394 
395     RecoverLogThread t = new RecoverLogThread();
396     t.start();
397     // Timeout after 60 sec. Without correct patches, would be an infinite loop
398     t.join(60 * 1000);
399     if(t.isAlive()) {
400       t.interrupt();
401       throw new Exception("Timed out waiting for HLog.recoverLog()");
402     }
403 
404     if (t.exception != null)
405       throw t.exception;
406 
407     // Make sure you can read all the content
408     SequenceFile.Reader reader 
409       = new SequenceFile.Reader(fs, walPath, conf);
410     int count = 0;
411     HLogKey key = HLog.newKey(conf);
412     WALEdit val = new WALEdit();
413     while (reader.next(key, val)) {
414       count++;
415       assertTrue("Should be one KeyValue per WALEdit",
416                  val.getKeyValues().size() == 1);
417     }
418     assertEquals(total, count);
419     reader.close();
420   }
421 
422   /**
423    * Tests that we can write out an edit, close, and then read it back in again.
424    * @throws IOException
425    */
426   @Test
427   public void testEditAdd() throws IOException {
428     final int COL_COUNT = 10;
429     final byte [] tableName = Bytes.toBytes("tablename");
430     final byte [] row = Bytes.toBytes("row");
431     HLog.Reader reader = null;
432     HLog log = new HLog(fs, dir, oldLogDir, conf, null);
433     try {
434       // Write columns named 1, 2, 3, etc. and then values of single byte
435       // 1, 2, 3...
436       long timestamp = System.currentTimeMillis();
437       WALEdit cols = new WALEdit();
438       for (int i = 0; i < COL_COUNT; i++) {
439         cols.add(new KeyValue(row, Bytes.toBytes("column"),
440             Bytes.toBytes(Integer.toString(i)),
441           timestamp, new byte[] { (byte)(i + '0') }));
442       }
443       HRegionInfo info = new HRegionInfo(new HTableDescriptor(tableName),
444         row,Bytes.toBytes(Bytes.toString(row) + "1"), false);
445       final byte [] regionName = info.getRegionName();
446       log.append(info, tableName, cols, System.currentTimeMillis());
447       long logSeqId = log.startCacheFlush();
448       log.completeCacheFlush(regionName, tableName, logSeqId, info.isMetaRegion());
449       log.close();
450       Path filename = log.computeFilename();
451       log = null;
452       // Now open a reader on the log and assert append worked.
453       reader = HLog.getReader(fs, filename, conf);
454       // Above we added all columns on a single row so we only read one
455       // entry in the below... thats why we have '1'.
456       for (int i = 0; i < 1; i++) {
457         HLog.Entry entry = reader.next(null);
458         if (entry == null) break;
459         HLogKey key = entry.getKey();
460         WALEdit val = entry.getEdit();
461         assertTrue(Bytes.equals(regionName, key.getRegionName()));
462         assertTrue(Bytes.equals(tableName, key.getTablename()));
463         KeyValue kv = val.getKeyValues().get(0);
464         assertTrue(Bytes.equals(row, kv.getRow()));
465         assertEquals((byte)(i + '0'), kv.getValue()[0]);
466         System.out.println(key + " " + val);
467       }
468       HLog.Entry entry = null;
469       while ((entry = reader.next(null)) != null) {
470         HLogKey key = entry.getKey();
471         WALEdit val = entry.getEdit();
472         // Assert only one more row... the meta flushed row.
473         assertTrue(Bytes.equals(regionName, key.getRegionName()));
474         assertTrue(Bytes.equals(tableName, key.getTablename()));
475         KeyValue kv = val.getKeyValues().get(0);
476         assertTrue(Bytes.equals(HLog.METAROW, kv.getRow()));
477         assertTrue(Bytes.equals(HLog.METAFAMILY, kv.getFamily()));
478         assertEquals(0, Bytes.compareTo(HLog.COMPLETE_CACHE_FLUSH,
479           val.getKeyValues().get(0).getValue()));
480         System.out.println(key + " " + val);
481       }
482     } finally {
483       if (log != null) {
484         log.closeAndDelete();
485       }
486       if (reader != null) {
487         reader.close();
488       }
489     }
490   }
491 
492   /**
493    * @throws IOException
494    */
495   @Test
496   public void testAppend() throws IOException {
497     final int COL_COUNT = 10;
498     final byte [] tableName = Bytes.toBytes("tablename");
499     final byte [] row = Bytes.toBytes("row");
500     Reader reader = null;
501     HLog log = new HLog(fs, dir, oldLogDir, conf, null);
502     try {
503       // Write columns named 1, 2, 3, etc. and then values of single byte
504       // 1, 2, 3...
505       long timestamp = System.currentTimeMillis();
506       WALEdit cols = new WALEdit();
507       for (int i = 0; i < COL_COUNT; i++) {
508         cols.add(new KeyValue(row, Bytes.toBytes("column"),
509           Bytes.toBytes(Integer.toString(i)),
510           timestamp, new byte[] { (byte)(i + '0') }));
511       }
512       HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
513           HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
514       log.append(hri, tableName, cols, System.currentTimeMillis());
515       long logSeqId = log.startCacheFlush();
516       log.completeCacheFlush(hri.getRegionName(), tableName, logSeqId, false);
517       log.close();
518       Path filename = log.computeFilename();
519       log = null;
520       // Now open a reader on the log and assert append worked.
521       reader = HLog.getReader(fs, filename, conf);
522       HLog.Entry entry = reader.next();
523       assertEquals(COL_COUNT, entry.getEdit().size());
524       int idx = 0;
525       for (KeyValue val : entry.getEdit().getKeyValues()) {
526         assertTrue(Bytes.equals(hri.getRegionName(),
527           entry.getKey().getRegionName()));
528         assertTrue(Bytes.equals(tableName, entry.getKey().getTablename()));
529         assertTrue(Bytes.equals(row, val.getRow()));
530         assertEquals((byte)(idx + '0'), val.getValue()[0]);
531         System.out.println(entry.getKey() + " " + val);
532         idx++;
533       }
534 
535       // Get next row... the meta flushed row.
536       entry = reader.next();
537       assertEquals(1, entry.getEdit().size());
538       for (KeyValue val : entry.getEdit().getKeyValues()) {
539         assertTrue(Bytes.equals(hri.getRegionName(),
540           entry.getKey().getRegionName()));
541         assertTrue(Bytes.equals(tableName, entry.getKey().getTablename()));
542         assertTrue(Bytes.equals(HLog.METAROW, val.getRow()));
543         assertTrue(Bytes.equals(HLog.METAFAMILY, val.getFamily()));
544         assertEquals(0, Bytes.compareTo(HLog.COMPLETE_CACHE_FLUSH,
545           val.getValue()));
546         System.out.println(entry.getKey() + " " + val);
547       }
548     } finally {
549       if (log != null) {
550         log.closeAndDelete();
551       }
552       if (reader != null) {
553         reader.close();
554       }
555     }
556   }
557 
558   /**
559    * Test that we can visit entries before they are appended
560    * @throws Exception
561    */
562   @Test
563   public void testVisitors() throws Exception {
564     final int COL_COUNT = 10;
565     final byte [] tableName = Bytes.toBytes("tablename");
566     final byte [] row = Bytes.toBytes("row");
567     HLog log = new HLog(fs, dir, oldLogDir, conf, null);
568     DumbLogEntriesVisitor visitor = new DumbLogEntriesVisitor();
569     log.addLogEntryVisitor(visitor);
570     long timestamp = System.currentTimeMillis();
571     HRegionInfo hri = new HRegionInfo(new HTableDescriptor(tableName),
572         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
573     for (int i = 0; i < COL_COUNT; i++) {
574       WALEdit cols = new WALEdit();
575       cols.add(new KeyValue(row, Bytes.toBytes("column"),
576           Bytes.toBytes(Integer.toString(i)),
577           timestamp, new byte[]{(byte) (i + '0')}));
578       log.append(hri, tableName, cols, System.currentTimeMillis());
579     }
580     assertEquals(COL_COUNT, visitor.increments);
581     log.removeLogEntryVisitor(visitor);
582     WALEdit cols = new WALEdit();
583     cols.add(new KeyValue(row, Bytes.toBytes("column"),
584         Bytes.toBytes(Integer.toString(11)),
585         timestamp, new byte[]{(byte) (11 + '0')}));
586     log.append(hri, tableName, cols, System.currentTimeMillis());
587     assertEquals(COL_COUNT, visitor.increments);
588   }
589 
590   static class DumbLogEntriesVisitor implements LogEntryVisitor {
591 
592     int increments = 0;
593 
594     @Override
595     public void visitLogEntryBeforeWrite(HRegionInfo info, HLogKey logKey,
596                                          WALEdit logEdit) {
597       increments++;
598     }
599   }
600 }