You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jd...@apache.org on 2010/05/19 03:33:28 UTC

svn commit: r945993 - in /hbase/trunk: ./ core/src/main/java/org/apache/hadoop/hbase/master/ core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ core/src/main/java/org/apache/hadoop/hbase/zookeeper/ core/src/test/java/org/apache/hadoop/hbase/ ...

Author: jdcryans
Date: Wed May 19 01:33:27 2010
New Revision: 945993

URL: http://svn.apache.org/viewvc?rev=945993&view=rev
Log:
HBASE-2527  Add the ability to easily extend some HLog actions
HBASE-2534  Recursive deletes and misc improvements to ZKW

Added:
    hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/LogActionsListener.java
    hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogActionsListener.java
Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java
    hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
    hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java
    hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=945993&r1=945992&r2=945993&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed May 19 01:33:27 2010
@@ -608,7 +608,7 @@ Release 0.21.0 - Unreleased
    	           by filter (Juhani Connolly via Ryan)
    HBASE-2566  Remove 'lib' dir; it only has libthrift and that is being
                pulled from http://people.apache.org/~rawson/repo/....
-
+   HBASE-2534  Recursive deletes and misc improvements to ZKW
 
   NEW FEATURES
    HBASE-1961  HBase EC2 scripts
@@ -642,6 +642,7 @@ Release 0.21.0 - Unreleased
    HBASE-2473  Add to admin create table start and end key params and
                desired number of regions
    HBASE-2529  Make OldLogsCleaner easier to extend
+   HBASE-2527  Add the ability to easily extend some HLog actions
 
   OPTIMIZATIONS
    HBASE-410   [testing] Speed up the test suite

Modified: hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java
URL: http://svn.apache.org/viewvc/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java?rev=945993&r1=945992&r2=945993&view=diff
==============================================================================
--- hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java (original)
+++ hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/master/LogCleanerDelegate.java Wed May 19 01:33:27 2010
@@ -38,3 +38,4 @@ public interface LogCleanerDelegate exte
    */
   public boolean isLogDeletable(Path filePath);
 }
+

Modified: hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
URL: http://svn.apache.org/viewvc/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java?rev=945993&r1=945992&r2=945993&view=diff
==============================================================================
--- hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (original)
+++ hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java Wed May 19 01:33:27 2010
@@ -125,6 +125,11 @@ public class HLog implements HConstants,
   private final int flushlogentries;
   private final AtomicInteger unflushedEntries = new AtomicInteger(0);
   private final Path oldLogDir;
+  private final List<LogActionsListener> actionListeners =
+      Collections.synchronizedList(new ArrayList<LogActionsListener>());
+
+  private static Class logWriterClass;
+  private static Class logReaderClass;
 
   private OutputStream hdfs_out;     // OutputStream associated with the current SequenceFile.writer
   private int initialReplication;    // initial replication factor of SequenceFile.writer
@@ -245,20 +250,39 @@ public class HLog implements HConstants,
   }
 
   /**
+   * HLog creating with a null actions listener.
+   *
+   * @param fs filesystem handle
+   * @param dir path to where hlogs are stored
+   * @param oldLogDir path to where hlogs are archived
+   * @param conf configuration to use
+   * @param listener listerner used to request log rolls
+   * @throws IOException
+   */
+  public HLog(final FileSystem fs, final Path dir, final Path oldLogDir,
+              final Configuration conf, final LogRollListener listener)
+  throws IOException {
+    this(fs, dir, oldLogDir, conf, listener, null);
+  }
+
+  /**
    * Create an edit log at the given <code>dir</code> location.
    *
    * You should never have to load an existing log. If there is a log at
    * startup, it should have already been processed and deleted by the time the
    * HLog object is started up.
    *
-   * @param fs
-   * @param dir
-   * @param conf
-   * @param listener
+   * @param fs filesystem handle
+   * @param dir path to where hlogs are stored
+   * @param oldLogDir path to where hlogs are archived
+   * @param conf configuration to use
+   * @param listener listerner used to request log rolls
+   * @param actionListener optional listener for hlog actions like archiving
    * @throws IOException
    */
   public HLog(final FileSystem fs, final Path dir, final Path oldLogDir,
-              final Configuration conf, final LogRollListener listener)
+              final Configuration conf, final LogRollListener listener,
+              final LogActionsListener actionListener)
   throws IOException {
     super();
     this.fs = fs;
@@ -289,6 +313,9 @@ public class HLog implements HConstants,
       ", enabled=" + this.enabled +
       ", flushlogentries=" + this.flushlogentries +
       ", optionallogflushinternal=" + this.optionalFlushInterval + "ms");
+    if (actionListener != null) {
+      addLogActionsListerner(actionListener);
+    }
     // rollWriter sets this.hdfs_out if it can.
     rollWriter();
 
@@ -406,6 +433,12 @@ public class HLog implements HConstants,
             ", filesize=" +
             this.fs.getFileStatus(oldFile).getLen() + ". ": "") +
           "New hlog " + FSUtils.getPath(newPath));
+        // Tell our listeners that a new log was created
+        if (!this.actionListeners.isEmpty()) {
+          for (LogActionsListener list : this.actionListeners) {
+            list.logRolled(newPath);
+          }
+        }
         // Can we delete any of the old log files?
         if (this.outputfiles.size() > 0) {
           if (this.lastSeqWritten.size() <= 0) {
@@ -442,9 +475,12 @@ public class HLog implements HConstants,
     final Path path, Configuration conf)
   throws IOException {
     try {
-      Class c = Class.forName(conf.get("hbase.regionserver.hlog.reader.impl",
+      if (logReaderClass == null) {
+        logReaderClass = Class.forName(conf.get("hbase.regionserver.hlog.reader.impl",
         SequenceFileLogReader.class.getCanonicalName()));
-      HLog.Reader reader = (HLog.Reader) c.newInstance();
+      }
+
+      HLog.Reader reader = (HLog.Reader) logReaderClass.newInstance();
       reader.init(fs, path, conf);
       return reader;
     } catch (Exception e) {
@@ -466,9 +502,11 @@ public class HLog implements HConstants,
       final Path path, Configuration conf)
   throws IOException {
     try {
-      Class c = Class.forName(conf.get("hbase.regionserver.hlog.writer.impl",
+      if (logWriterClass == null) {
+        logWriterClass = Class.forName(conf.get("hbase.regionserver.hlog.writer.impl",
         SequenceFileLogWriter.class.getCanonicalName()));
-      HLog.Writer writer = (HLog.Writer) c.newInstance();
+      }
+      HLog.Writer writer = (HLog.Writer) logWriterClass.newInstance();
       writer.init(fs, path, conf);
       return writer;
     } catch (Exception e) {
@@ -602,6 +640,11 @@ public class HLog implements HConstants,
       " whose highest sequence/edit id is " + seqno + " to " +
       FSUtils.getPath(newPath));
     this.fs.rename(p, newPath);
+    if (!this.actionListeners.isEmpty()) {
+      for (LogActionsListener list : this.actionListeners) {
+        list.logArchived(p, newPath);
+      }
+    }
   }
 
   /**
@@ -1621,8 +1664,6 @@ public class HLog implements HConstants,
     return dirName.toString();
   }
 
-  // We create a new file name with a ts in front of it to make sure we almost
-  // certainly don't have a file name conflict.
   private static Path getHLogArchivePath(Path oldLogDir, Path p) {
     return new Path(oldLogDir, System.currentTimeMillis() + "." + p.getName());
   }
@@ -1633,6 +1674,23 @@ public class HLog implements HConstants,
   }
 
   /**
+   *
+   * @param list
+   */
+  public void addLogActionsListerner(LogActionsListener list) {
+    LOG.info("Adding a listener");
+    this.actionListeners.add(list);
+  }
+
+  /**
+   *
+   * @param list
+   */
+  public boolean removeLogActionsListener(LogActionsListener list) {
+    return this.actionListeners.remove(list);
+  }
+
+  /**
    * Pass one or more log file names and it will either dump out a text version
    * on <code>stdout</code> or split the specified log files.
    *

Added: hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/LogActionsListener.java
URL: http://svn.apache.org/viewvc/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/LogActionsListener.java?rev=945993&view=auto
==============================================================================
--- hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/LogActionsListener.java (added)
+++ hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/LogActionsListener.java Wed May 19 01:33:27 2010
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Interface that defines all actions that can be listened to coming
+ * from the HLog. The calls are done in sync with what happens over in the
+ * HLog so make sure your implementation is fast.
+ */
+public interface LogActionsListener {
+
+  /**
+   * Notify the listener that a new file is available
+   * @param newFile the path to the new hlog
+   */
+  public void logRolled(Path newFile);
+
+  /**
+   * Notify that the following log moved
+   * @param oldPath the old path
+   * @param newPath the new path
+   */
+  public void logArchived(Path oldPath, Path newPath);
+}

Modified: hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java?rev=945993&r1=945992&r2=945993&view=diff
==============================================================================
--- hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java (original)
+++ hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java Wed May 19 01:33:27 2010
@@ -397,7 +397,12 @@ public class ZooKeeperWrapper implements
     return new HServerAddress(addressString);
   }
 
-  private boolean ensureExists(final String znode) {
+  /**
+   * Make sure this znode exists by creating it if it's missing
+   * @param znode full path to znode
+   * @return true if it works
+   */
+  public boolean ensureExists(final String znode) {
     try {
       Stat stat = zooKeeper.exists(znode, false);
       if (stat != null) {
@@ -439,8 +444,7 @@ public class ZooKeeperWrapper implements
     }
 
     try {
-      zooKeeper.delete(rootRegionZNode, -1);
-      LOG.debug("Deleted ZNode " + rootRegionZNode);
+      deleteZNode(rootRegionZNode);
       return true;
     } catch (KeeperException.NoNodeException e) {
       return true;    // ok, move on.
@@ -453,6 +457,41 @@ public class ZooKeeperWrapper implements
     return false;
   }
 
+  /**
+   * Unrecursive deletion of specified znode
+   * @param znode
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
+  public void deleteZNode(String znode)
+      throws KeeperException, InterruptedException {
+    deleteZNode(znode, false);
+  }
+
+  /**
+   * Optionnally recursive deletion of specified znode
+   * @param znode
+   * @param recursive
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
+  public void deleteZNode(String znode, boolean recursive)
+    throws KeeperException, InterruptedException {
+    if (recursive) {
+      LOG.info("deleteZNode get children for " + znode);
+      List<String> znodes = this.zooKeeper.getChildren(znode, false);
+      if (znodes.size() > 0) {
+        for (String child : znodes) {
+          String childFullPath = getZNode(znode, child);
+          LOG.info("deleteZNode recursive call " + childFullPath);
+          this.deleteZNode(childFullPath, true);
+        }
+      }
+    }
+    this.zooKeeper.delete(znode, -1);
+    LOG.debug("Deleted ZNode " + znode);
+  }
+
   private boolean createRootRegionLocation(String address) {
     byte[] data = Bytes.toBytes(address);
     try {
@@ -589,6 +628,15 @@ public class ZooKeeperWrapper implements
   }
 
   /**
+   * Scans the regions servers directory and sets a watch on each znode
+   * @param watcher a watch to use for each znode
+   * @return A list of server addresses
+   */
+  public List<HServerAddress> scanRSDirectory(Watcher watcher) {
+    return scanAddressDirectory(rsZNode, watcher);
+  }
+
+  /**
    * Method used to make sure the region server directory is empty.
    *
    */
@@ -676,11 +724,25 @@ public class ZooKeeperWrapper implements
     return list;
   }
 
+  /**
+   * List all znodes in the specified path and set a watcher on each
+   * @param znode path to list
+   * @param watcher watch to set, can be null
+   * @return a list of all the znodes
+   */
   public List<String> listZnodes(String znode, Watcher watcher) {
     List<String> nodes = null;
     try {
       if (checkExistenceOf(znode)) {
-        nodes = zooKeeper.getChildren(znode, watcher);
+        if (watcher == null) {
+          nodes = zooKeeper.getChildren(znode, false);
+        } else {
+          nodes = zooKeeper.getChildren(znode, watcher);
+          for (String node : nodes) {
+            getDataAndWatch(znode, node, watcher);
+          }
+        }
+
       }
     } catch (KeeperException e) {
       LOG.warn("Failed to read " + znode + " znode in ZooKeeper: " + e);
@@ -710,21 +772,46 @@ public class ZooKeeperWrapper implements
     return data;
   }
 
+  /**
+   * Write a znode and fail if it already exists
+   * @param parentPath parent path to the new znode
+   * @param child name of the znode
+   * @param strData data to insert
+   * @throws InterruptedException
+   * @throws KeeperException
+   */
   public void writeZNode(String parentPath, String child, String strData)
       throws InterruptedException, KeeperException {
+    writeZNode(parentPath, child, strData, false);
+  }
+
+
+  /**
+   * Write (and optionally over-write) a znode
+   * @param parentPath parent path to the new znode
+   * @param child name of the znode
+   * @param strData data to insert
+   * @param failOnWrite true if an exception should be returned if the znode
+   * already exists, false if it should be overwritten
+   * @throws InterruptedException
+   * @throws KeeperException
+   */
+  public void writeZNode(String parentPath, String child, String strData,
+      boolean failOnWrite) throws InterruptedException, KeeperException {
     String path = joinPath(parentPath, child);
     if (!ensureExists(parentPath)) {
       LOG.error("unable to ensure parent exists: " + parentPath);
     }
     byte[] data = Bytes.toBytes(strData);
-    try {
+    Stat stat = this.zooKeeper.exists(path, false);
+    if (failOnWrite || stat == null) {
       this.zooKeeper.create(path, data,
           Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
       LOG.debug("Created " + path);
-      } catch (KeeperException.NodeExistsException ex) {
-        this.zooKeeper.setData(path, data, -1);
-        LOG.debug("Updated " + path);
-      }
+    } else {
+      this.zooKeeper.setData(path, data, -1);
+      LOG.debug("Updated " + path);
+    }
   }
 
   public static String getZookeeperClusterKey(Configuration conf) {
@@ -732,5 +819,12 @@ public class ZooKeeperWrapper implements
           conf.get(ZOOKEEPER_ZNODE_PARENT);
   }
 
+  /**
+   * Get the path of this region server's znode
+   * @return path to znode
+   */
+  public String getRsZNode() {
+    return this.rsZNode;
+  }
 
 }

Modified: hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=945993&r1=945992&r2=945993&view=diff
==============================================================================
--- hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Wed May 19 01:33:27 2010
@@ -22,16 +22,29 @@ package org.apache.hadoop.hbase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
-import org.junit.*;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
 public class TestZooKeeper {
   private final Log LOG = LogFactory.getLog(this.getClass());
 
@@ -163,4 +176,25 @@ public class TestZooKeeper {
       fail();
     }
   }
+
+  /**
+   * Create a bunch of znodes in a hierarchy, try deleting one that has childs
+   * (it will fail), then delete it recursively, then delete the last znode
+   * @throws Exception
+   */
+  @Test
+  public void testZNodeDeletes() throws Exception {
+    ZooKeeperWrapper zkw = new ZooKeeperWrapper(conf, EmptyWatcher.instance);
+    zkw.ensureExists("/l1/l2/l3/l4");
+    try {
+      zkw.deleteZNode("/l1/l2");
+      fail("We should not be able to delete if znode has childs");
+    } catch (KeeperException ex) {
+      assertNotNull(zkw.getData("/l1/l2/l3", "l4"));
+    }
+    zkw.deleteZNode("/l1/l2", true);
+    assertNull(zkw.getData("/l1/l2/l3", "l4"));
+    zkw.deleteZNode("/l1");
+    assertNull(zkw.getData("/l1", "l2"));
+  }
 }
\ No newline at end of file

Added: hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogActionsListener.java
URL: http://svn.apache.org/viewvc/hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogActionsListener.java?rev=945993&view=auto
==============================================================================
--- hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogActionsListener.java (added)
+++ hbase/trunk/core/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogActionsListener.java Wed May 19 01:33:27 2010
@@ -0,0 +1,126 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+/**
+ * Test that the actions are called while playing with an HLog
+ */
+public class TestLogActionsListener {
+
+  protected static final Log LOG =
+      LogFactory.getLog(TestLogActionsListener.class);
+
+  private final static HBaseTestingUtility TEST_UTIL =
+      new HBaseTestingUtility();
+
+  private final static byte[] SOME_BYTES =  Bytes.toBytes("t");
+  private static FileSystem fs;
+  private static Path oldLogDir;
+  private static Path logDir;
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    conf.setInt("hbase.regionserver.maxlogs", 5);
+    fs = FileSystem.get(conf);
+    oldLogDir = new Path(TEST_UTIL.getTestDir(),
+        HConstants.HREGION_OLDLOGDIR_NAME);
+    logDir = new Path(TEST_UTIL.getTestDir(),
+        HConstants.HREGION_LOGDIR_NAME);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    fs.delete(logDir, true);
+    fs.delete(oldLogDir, true);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    setUp();
+  }
+
+  /**
+   * Add a bunch of dummy data and roll the logs every two insert. We
+   * should end up with 10 rolled files (plus the roll called in
+   * the constructor). Also test adding a listener while it's running.
+   */
+  @Test
+  public void testActionListener() throws Exception {
+    DummyLogActionsListener list = new DummyLogActionsListener();
+    DummyLogActionsListener laterList = new DummyLogActionsListener();
+    HLog hlog = new HLog(fs, logDir, oldLogDir, conf, null, list);
+    HRegionInfo hri = new HRegionInfo(new HTableDescriptor(SOME_BYTES),
+        SOME_BYTES, SOME_BYTES, false);
+
+    for (int i = 0; i < 20; i++) {
+      byte[] b = Bytes.toBytes(i+"");
+      KeyValue kv = new KeyValue(b,b,b);
+      WALEdit edit = new WALEdit();
+      edit.add(kv);
+      HLogKey key = new HLogKey(b,b, 0, 0);
+      hlog.append(hri, key, edit);
+      if (i == 10) {
+        hlog.addLogActionsListerner(laterList);
+      }
+      if (i % 2 == 0) {
+        hlog.rollWriter();
+      }
+    }
+    assertEquals(11, list.logRollCounter);
+    assertEquals(5, laterList.logRollCounter);
+  }
+
+  /**
+   * Just counts when methods are called
+   */
+  static class DummyLogActionsListener implements LogActionsListener {
+
+    public int logRollCounter = 0;
+
+    @Override
+    public void logRolled(Path newFile) {
+      logRollCounter++;
+    }
+
+    @Override
+    public void logArchived(Path oldPath, Path newPath) {
+      // This one is a bit tricky to test since it involves seq numbers
+    }
+  }
+}