You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/01/11 20:08:23 UTC

svn commit: r1432234 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/master/ main/java/org/apache/hadoop/hbase/master/handler/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/regionserver/handler/...

Author: stack
Date: Fri Jan 11 19:08:22 2013
New Revision: 1432234

URL: http://svn.apache.org/viewvc?rev=1432234&view=rev
Log:
HBASE-7213 Have HLog files for .META. edits only; REVERT

Removed:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java
Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Fri Jan 11 19:08:22 2013
@@ -960,7 +960,6 @@ Server {
       return;
     }
     LOG.info("Forcing splitLog and expire of " + sn);
-    fileSystemManager.splitMetaLog(sn);
     fileSystemManager.splitLog(sn);
     serverManager.expireServer(sn);
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Fri Jan 11 19:08:22 2013
@@ -32,7 +32,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -85,18 +84,6 @@ public class MasterFileSystem {
   final SplitLogManager splitLogManager;
   private final MasterServices services;
 
-  private final static PathFilter META_FILTER = new PathFilter() {
-    public boolean accept(Path p) {
-      return HLogUtil.isMetaFile(p);
-    }
-  };
-
-  private final static PathFilter NON_META_FILTER = new PathFilter() {
-    public boolean accept(Path p) {
-      return !HLogUtil.isMetaFile(p);
-    }
-  };
-
   public MasterFileSystem(Server master, MasterServices services,
       MetricsMaster metricsMaster, boolean masterRecovery)
   throws IOException {
@@ -242,8 +229,7 @@ public class MasterFileSystem {
                 + " belongs to an existing region server");
           }
         }
-        splitLog(serverNames, META_FILTER);
-        splitLog(serverNames, NON_META_FILTER);
+        splitLog(serverNames);
         retrySplitting = false;
       } catch (IOException ioe) {
         LOG.warn("Failed splitting of " + serverNames, ioe);
@@ -272,30 +258,8 @@ public class MasterFileSystem {
     splitLog(serverNames);
   }
 
-  /**
-   * Specialized method to handle the splitting for .META. HLog
-   * @param serverName
-   * @throws IOException
-   */
-  public void splitMetaLog(final ServerName serverName) throws IOException {
+  public void splitLog(final List<ServerName> serverNames) throws IOException {
     long splitTime = 0, splitLogSize = 0;
-    List<ServerName> serverNames = new ArrayList<ServerName>();
-    serverNames.add(serverName);
-    List<Path> logDirs = getLogDirs(serverNames);
-    if (logDirs.isEmpty()) {
-      LOG.info("No .META. logs to split");
-      return;
-    }
-    splitLogManager.handleDeadWorkers(serverNames);
-    splitTime = EnvironmentEdgeManager.currentTimeMillis();
-    splitLogSize = splitLogManager.splitLogDistributed(logDirs, META_FILTER);
-    splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime;
-    if (this.metricsMaster != null) {
-      this.metricsMaster.addSplit(splitTime, splitLogSize);
-    }
-  }
-
-  private List<Path> getLogDirs(final List<ServerName> serverNames) throws IOException {
     List<Path> logDirs = new ArrayList<Path>();
     for (ServerName serverName: serverNames) {
       Path logDir = new Path(this.rootdir, HLogUtil.getHLogDirectoryName(serverName.toString()));
@@ -313,23 +277,6 @@ public class MasterFileSystem {
       }
       logDirs.add(splitDir);
     }
-    return logDirs;
-  }
-
-  public void splitLog(final List<ServerName> serverNames) throws IOException {
-    splitLog(serverNames, NON_META_FILTER);
-  }
-
-  /**
-   * This method is the base split method that splits HLog files matching a filter.
-   * Callers should pass the appropriate filter for .META. and non-meta HLogs.
-   * @param serverNames
-   * @param filter
-   * @throws IOException
-   */
-  public void splitLog(final List<ServerName> serverNames, PathFilter filter) throws IOException {
-    long splitTime = 0, splitLogSize = 0;
-    List<Path> logDirs = getLogDirs(serverNames);
 
     if (logDirs.isEmpty()) {
       LOG.info("No logs to split");
@@ -339,7 +286,7 @@ public class MasterFileSystem {
     if (distributedLogSplitting) {
       splitLogManager.handleDeadWorkers(serverNames);
       splitTime = EnvironmentEdgeManager.currentTimeMillis();
-      splitLogSize = splitLogManager.splitLogDistributed(logDirs,filter);
+      splitLogSize = splitLogManager.splitLogDistributed(logDirs);
       splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime;
     } else {
       for(Path logDir: logDirs){
@@ -423,8 +370,7 @@ public class MasterFileSystem {
     // Make sure cluster ID exists
     if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
         HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
-      FSUtils.setClusterId(fs, rd, new ClusterId(), c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
-        10 * 1000));
+      FSUtils.setClusterId(fs, rd, new ClusterId(), c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
     }
     clusterId = FSUtils.getClusterId(fs, rd);
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java Fri Jan 11 19:08:22 2013
@@ -40,7 +40,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.SplitLogCounters;
 import org.apache.hadoop.hbase.DeserializationException;
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.monitorin
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
-import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
@@ -196,7 +194,7 @@ public class SplitLogManager extends Zoo
     }
   }
 
-  private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
+  private FileStatus[] getFileList(List<Path> logDirs) throws IOException {
     List<FileStatus> fileStatus = new ArrayList<FileStatus>();
     for (Path hLogDir : logDirs) {
       this.fs = hLogDir.getFileSystem(conf);
@@ -204,7 +202,8 @@ public class SplitLogManager extends Zoo
         LOG.warn(hLogDir + " doesn't exist. Nothing to do!");
         continue;
       }
-      FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, filter);
+      // TODO filter filenames?
+      FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, null);
       if (logfiles == null || logfiles.length == 0) {
         LOG.info(hLogDir + " is empty dir, no logs to split");
       } else {
@@ -229,7 +228,6 @@ public class SplitLogManager extends Zoo
     logDirs.add(logDir);
     return splitLogDistributed(logDirs);
   }
-
   /**
    * The caller will block until all the log files of the given region server
    * have been processed - successfully split or an error is encountered - by an
@@ -241,25 +239,9 @@ public class SplitLogManager extends Zoo
    * @return cumulative size of the logfiles split
    */
   public long splitLogDistributed(final List<Path> logDirs) throws IOException {
-    return splitLogDistributed(logDirs, null);
-  }
-
-  /**
-   * The caller will block until all the META log files of the given region server
-   * have been processed - successfully split or an error is encountered - by an
-   * available worker region server. This method must only be called after the
-   * region servers have been brought online.
-   *
-   * @param logDirs List of log dirs to split
-   * @param filter the Path filter to select specific files for considering
-   * @throws IOException If there was an error while splitting any log file
-   * @return cumulative size of the logfiles split
-   */
-  public long splitLogDistributed(final List<Path> logDirs, PathFilter filter) 
-      throws IOException {
     MonitoredTask status = TaskMonitor.get().createStatus(
           "Doing distributed log split in " + logDirs);
-    FileStatus[] logfiles = getFileList(logDirs, filter);
+    FileStatus[] logfiles = getFileList(logDirs);
     status.setStatus("Checking directory contents...");
     LOG.debug("Scheduling batch of logs to split");
     SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java Fri Jan 11 19:08:22 2013
@@ -18,17 +18,11 @@
  */
 package org.apache.hadoop.hbase.master.handler;
 
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.DeadServer;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * Shutdown handler for the server hosting <code>-ROOT-</code>,
@@ -38,7 +32,7 @@ import org.apache.zookeeper.KeeperExcept
 public class MetaServerShutdownHandler extends ServerShutdownHandler {
   private final boolean carryingRoot;
   private final boolean carryingMeta;
-  private static final Log LOG = LogFactory.getLog(MetaServerShutdownHandler.class);
+
   public MetaServerShutdownHandler(final Server server,
       final MasterServices services,
       final DeadServer deadServers, final ServerName serverName,
@@ -50,118 +44,11 @@ public class MetaServerShutdownHandler e
   }
 
   @Override
-  public void process() throws IOException {
-    try {
-      LOG.info("Splitting META logs for " + serverName);
-      if (this.shouldSplitHlog) {
-        this.services.getMasterFileSystem().splitMetaLog(serverName);
-      }
-    } catch (IOException ioe) {
-      this.services.getExecutorService().submit(this);
-      this.deadServers.add(serverName);
-      throw new IOException("failed log splitting for " +
-          serverName + ", will retry", ioe);
-    }
-
-    // Assign root and meta if we were carrying them.
-    if (isCarryingRoot()) { // -ROOT-
-      // Check again: region may be assigned to other where because of RIT
-      // timeout
-      if (this.services.getAssignmentManager().isCarryingRoot(serverName)) {
-        LOG.info("Server " + serverName
-            + " was carrying ROOT. Trying to assign.");
-        this.services.getAssignmentManager().regionOffline(
-            HRegionInfo.ROOT_REGIONINFO);
-        verifyAndAssignRootWithRetries();
-      } else {
-        LOG.info("ROOT has been assigned to otherwhere, skip assigning.");
-      }
-    }
-
-    // Carrying meta?
-    if (isCarryingMeta()) {
-      // Check again: region may be assigned to other where because of RIT
-      // timeout
-      if (this.services.getAssignmentManager().isCarryingMeta(serverName)) {
-        LOG.info("Server " + serverName
-            + " was carrying META. Trying to assign.");
-        this.services.getAssignmentManager().regionOffline(
-            HRegionInfo.FIRST_META_REGIONINFO);
-        this.services.getAssignmentManager().assignMeta();
-      } else {
-        LOG.info("META has been assigned to otherwhere, skip assigning.");
-      }
-
-    }
-    super.process();
-  }
-  /**
-   * Before assign the ROOT region, ensure it haven't
-   *  been assigned by other place
-   * <p>
-   * Under some scenarios, the ROOT region can be opened twice, so it seemed online
-   * in two regionserver at the same time.
-   * If the ROOT region has been assigned, so the operation can be canceled.
-   * @throws InterruptedException
-   * @throws IOException
-   * @throws KeeperException
-   */
-  private void verifyAndAssignRoot()
-  throws InterruptedException, IOException, KeeperException {
-    long timeout = this.server.getConfiguration().
-      getLong("hbase.catalog.verification.timeout", 1000);
-    if (!this.server.getCatalogTracker().verifyRootRegionLocation(timeout)) {
-      this.services.getAssignmentManager().assignRoot();
-    } else if (serverName.equals(server.getCatalogTracker().getRootLocation())) {
-      throw new IOException("-ROOT- is onlined on the dead server "
-          + serverName);
-    } else {
-      LOG.info("Skip assigning -ROOT-, because it is online on the "
-          + server.getCatalogTracker().getRootLocation());
-    }
-  }
-
-  /**
-   * Failed many times, shutdown processing
-   * @throws IOException
-   */
-  private void verifyAndAssignRootWithRetries() throws IOException {
-    int iTimes = this.server.getConfiguration().getInt(
-        "hbase.catalog.verification.retries", 10);
-
-    long waitTime = this.server.getConfiguration().getLong(
-        "hbase.catalog.verification.timeout", 1000);
-
-    int iFlag = 0;
-    while (true) {
-      try {
-        verifyAndAssignRoot();
-        break;
-      } catch (KeeperException e) {
-        this.server.abort("In server shutdown processing, assigning root", e);
-        throw new IOException("Aborting", e);
-      } catch (Exception e) {
-        if (iFlag >= iTimes) {
-          this.server.abort("verifyAndAssignRoot failed after" + iTimes
-              + " times retries, aborting", e);
-          throw new IOException("Aborting", e);
-        }
-        try {
-          Thread.sleep(waitTime);
-        } catch (InterruptedException e1) {
-          LOG.warn("Interrupted when is the thread sleep", e1);
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted", e1);
-        }
-        iFlag++;
-      }
-    }
-  }
-
   boolean isCarryingRoot() {
     return this.carryingRoot;
   }
 
+  @Override
   boolean isCarryingMeta() {
     return this.carryingMeta;
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java Fri Jan 11 19:08:22 2013
@@ -55,10 +55,10 @@ import org.apache.zookeeper.KeeperExcept
 @InterfaceAudience.Private
 public class ServerShutdownHandler extends EventHandler {
   private static final Log LOG = LogFactory.getLog(ServerShutdownHandler.class);
-  protected final ServerName serverName;
-  protected final MasterServices services;
-  protected final DeadServer deadServers;
-  protected final boolean shouldSplitHlog; // whether to split HLog or not
+  private final ServerName serverName;
+  private final MasterServices services;
+  private final DeadServer deadServers;
+  private final boolean shouldSplitHlog; // whether to split HLog or not
 
   public ServerShutdownHandler(final Server server, final MasterServices services,
       final DeadServer deadServers, final ServerName serverName,
@@ -91,6 +91,69 @@ public class ServerShutdownHandler exten
   }
 
   /**
+   * Before assign the ROOT region, ensure it haven't
+   *  been assigned by other place
+   * <p>
+   * Under some scenarios, the ROOT region can be opened twice, so it seemed online
+   * in two regionserver at the same time.
+   * If the ROOT region has been assigned, so the operation can be canceled.
+   * @throws InterruptedException
+   * @throws IOException
+   * @throws KeeperException
+   */
+  private void verifyAndAssignRoot()
+  throws InterruptedException, IOException, KeeperException {
+    long timeout = this.server.getConfiguration().
+      getLong("hbase.catalog.verification.timeout", 1000);
+    if (!this.server.getCatalogTracker().verifyRootRegionLocation(timeout)) {
+      this.services.getAssignmentManager().assignRoot();
+    } else if (serverName.equals(server.getCatalogTracker().getRootLocation())) {
+      throw new IOException("-ROOT- is onlined on the dead server "
+          + serverName);
+    } else {
+      LOG.info("Skip assigning -ROOT-, because it is online on the "
+          + server.getCatalogTracker().getRootLocation());
+    }
+  }
+
+  /**
+   * Failed many times, shutdown processing
+   * @throws IOException
+   */
+  private void verifyAndAssignRootWithRetries() throws IOException {
+    int iTimes = this.server.getConfiguration().getInt(
+        "hbase.catalog.verification.retries", 10);
+
+    long waitTime = this.server.getConfiguration().getLong(
+        "hbase.catalog.verification.timeout", 1000);
+
+    int iFlag = 0;
+    while (true) {
+      try {
+        verifyAndAssignRoot();
+        break;
+      } catch (KeeperException e) {
+        this.server.abort("In server shutdown processing, assigning root", e);
+        throw new IOException("Aborting", e);
+      } catch (Exception e) {
+        if (iFlag >= iTimes) {
+          this.server.abort("verifyAndAssignRoot failed after" + iTimes
+              + " times retries, aborting", e);
+          throw new IOException("Aborting", e);
+        }
+        try {
+          Thread.sleep(waitTime);
+        } catch (InterruptedException e1) {
+          LOG.warn("Interrupted when is the thread sleep", e1);
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted", e1);
+        }
+        iFlag++;
+      }
+    }
+  }
+
+  /**
    * @return True if the server we are processing was carrying <code>-ROOT-</code>
    */
   boolean isCarryingRoot() {
@@ -125,13 +188,43 @@ public class ServerShutdownHandler exten
           LOG.info("Skipping log splitting for " + serverName);
         }
       } catch (IOException ioe) {
-        //typecast to SSH so that we make sure that it is the SSH instance that
-        //gets submitted as opposed to MSSH or some other derived instance of SSH
-        this.services.getExecutorService().submit((ServerShutdownHandler)this);
+        this.services.getExecutorService().submit(this);
         this.deadServers.add(serverName);
         throw new IOException("failed log splitting for " +
           serverName + ", will retry", ioe);
       }
+
+      // Assign root and meta if we were carrying them.
+      if (isCarryingRoot()) { // -ROOT-
+        // Check again: region may be assigned to other where because of RIT
+        // timeout
+        if (this.services.getAssignmentManager().isCarryingRoot(serverName)) {
+          LOG.info("Server " + serverName
+              + " was carrying ROOT. Trying to assign.");
+          this.services.getAssignmentManager().regionOffline(
+              HRegionInfo.ROOT_REGIONINFO);
+          verifyAndAssignRootWithRetries();
+        } else {
+          LOG.info("ROOT has been assigned to otherwhere, skip assigning.");
+        }
+      }
+
+      // Carrying meta?
+      if (isCarryingMeta()) {
+        // Check again: region may be assigned to other where because of RIT
+        // timeout
+        if (this.services.getAssignmentManager().isCarryingMeta(serverName)) {
+          LOG.info("Server " + serverName
+              + " was carrying META. Trying to assign.");
+          this.services.getAssignmentManager().regionOffline(
+              HRegionInfo.FIRST_META_REGIONINFO);
+          this.services.getAssignmentManager().assignMeta();
+        } else {
+          LOG.info("META has been assigned to otherwhere, skip assigning.");
+        }
+       
+      }
+
       // We don't want worker thread in the MetaServerShutdownHandler
       // executor pool to block by waiting availability of -ROOT-
       // and .META. server. Otherwise, it could run into the following issue:

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Jan 11 19:08:22 2013
@@ -29,7 +29,6 @@ import java.lang.reflect.Method;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -329,7 +328,6 @@ public class  HRegionServer implements C
   RpcServer rpcServer;
 
   private final InetSocketAddress isa;
-  private UncaughtExceptionHandler uncaughtExceptionHandler;
 
   // Info server. Default access so can be used by unit tests. REGIONSERVER
   // is name of the webapp and the attribute name used stuffing this instance
@@ -359,12 +357,7 @@ public class  HRegionServer implements C
   // HLog and HLog roller. log is protected rather than private to avoid
   // eclipse warning when accessed by inner classes
   protected volatile HLog hlog;
-  // The meta updates are written to a different hlog. If this
-  // regionserver holds meta, then this field will be non-null.
-  protected volatile HLog hlogForMeta;
-
   LogRoller hlogRoller;
-  LogRoller metaHLogRoller;
 
   // flag set after we're done setting up server threads (used for testing)
   protected volatile boolean isOnline;
@@ -525,11 +518,6 @@ public class  HRegionServer implements C
       "hbase.regionserver.kerberos.principal", this.isa.getHostName());
     regionServerAccounting = new RegionServerAccounting();
     cacheConfig = new CacheConfig(conf);
-    uncaughtExceptionHandler = new UncaughtExceptionHandler() {
-      public void uncaughtException(Thread t, Throwable e) {
-        abort("Uncaught exception in service thread " + t.getName(), e);
-      }
-    };
     this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
   }
 
@@ -943,7 +931,6 @@ public class  HRegionServer implements C
     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
     if (this.hlogRoller != null) this.hlogRoller.interruptIfNecessary();
-    if (this.metaHLogRoller != null) this.metaHLogRoller.interruptIfNecessary();
     if (this.compactionChecker != null)
       this.compactionChecker.interrupt();
     if (this.healthCheckChore != null) {
@@ -1418,21 +1405,6 @@ public class  HRegionServer implements C
     return instantiateHLog(rootDir, logName);
   }
 
-  private HLog getMetaWAL() throws IOException {
-    if (this.hlogForMeta == null) {
-      final String logName
-      = HLogUtil.getHLogDirectoryName(this.serverNameFromMasterPOV.toString());
-
-      Path logdir = new Path(rootDir, logName);
-      if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
-
-      this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(), 
-          rootDir, logName, this.conf, getMetaWALActionListeners(), 
-          this.serverNameFromMasterPOV.toString());
-    }
-    return this.hlogForMeta;
-  }
-
   /**
    * Called by {@link #setupWALAndReplication()} creating WAL instance.
    * @param rootdir
@@ -1464,17 +1436,6 @@ public class  HRegionServer implements C
     return listeners;
   }
 
-  protected List<WALActionsListener> getMetaWALActionListeners() {
-    List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
-    // Log roller.
-    this.metaHLogRoller = new MetaLogRoller(this, this);
-    String n = Thread.currentThread().getName();
-    Threads.setDaemonThreadRunning(this.metaHLogRoller.getThread(), 
-        n + ".META.logRoller", uncaughtExceptionHandler);
-    listeners.add(this.metaHLogRoller);
-    return listeners;
-  }
-
   protected LogRoller getLogRoller() {
     return hlogRoller;
   }
@@ -1504,6 +1465,12 @@ public class  HRegionServer implements C
    */
   private void startServiceThreads() throws IOException {
     String n = Thread.currentThread().getName();
+    UncaughtExceptionHandler handler = new UncaughtExceptionHandler() {
+      public void uncaughtException(Thread t, Throwable e) {
+        abort("Uncaught exception in service thread " + t.getName(), e);
+      }
+    };
+
     // Start executor services
     this.service = new ExecutorService(getServerName().toString());
     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
@@ -1519,15 +1486,14 @@ public class  HRegionServer implements C
     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
 
-    Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), n + ".logRoller", uncaughtExceptionHandler);
+    Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), n + ".logRoller", handler);
     Threads.setDaemonThreadRunning(this.cacheFlusher.getThread(), n + ".cacheFlusher",
-      uncaughtExceptionHandler);
+      handler);
     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), n +
-      ".compactionChecker", uncaughtExceptionHandler);
+      ".compactionChecker", handler);
     if (this.healthCheckChore != null) {
     Threads
-        .setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker", 
-            uncaughtExceptionHandler);
+        .setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker", handler);
     }
 
     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
@@ -1608,31 +1574,11 @@ public class  HRegionServer implements C
       stop("One or more threads are no longer alive -- stop");
       return false;
     }
-    if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
-      stop("Meta HLog roller thread is no longer alive -- stop");
-      return false;
-    }
     return true;
   }
 
-  public HLog getWAL() {
-    try {
-      return getWAL(null);
-    } catch (IOException e) {
-      LOG.warn("getWAL threw exception " + e);
-      return null; 
-    }
-  }
-
   @Override
-  public HLog getWAL(HRegionInfo regionInfo) throws IOException {
-    //TODO: at some point this should delegate to the HLogFactory
-    //currently, we don't care about the region as much as we care about the 
-    //table.. (hence checking the tablename below)
-    if (regionInfo != null && 
-        Arrays.equals(regionInfo.getTableName(), HConstants.META_TABLE_NAME)) {
-      return getMetaWAL();
-    }
+  public HLog getWAL() {
     return this.hlog;
   }
 
@@ -1779,9 +1725,6 @@ public class  HRegionServer implements C
     if (this.hlogRoller != null) {
       Threads.shutdown(this.hlogRoller.getThread());
     }
-    if (this.metaHLogRoller != null) {
-      Threads.shutdown(this.metaHLogRoller.getThread());
-    }
     if (this.compactSplitThread != null) {
       this.compactSplitThread.join();
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java Fri Jan 11 19:08:22 2013
@@ -24,7 +24,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@@ -48,7 +47,7 @@ class LogRoller extends HasThread implem
   private final ReentrantLock rollLock = new ReentrantLock();
   private final AtomicBoolean rollLog = new AtomicBoolean(false);
   private final Server server;
-  protected final RegionServerServices services;
+  private final RegionServerServices services;
   private volatile long lastrolltime = System.currentTimeMillis();
   // Period to roll log.
   private final long rollperiod;
@@ -93,7 +92,7 @@ class LogRoller extends HasThread implem
       try {
         this.lastrolltime = now;
         // This is array of actual region names.
-        byte [][] regionsToFlush = getWAL().rollWriter(rollLog.get());
+        byte [][] regionsToFlush = this.services.getWAL().rollWriter(rollLog.get());
         if (regionsToFlush != null) {
           for (byte [] r: regionsToFlush) scheduleFlush(r);
         }
@@ -160,10 +159,6 @@ class LogRoller extends HasThread implem
     }
   }
 
-  protected HLog getWAL() throws IOException {
-    return this.services.getWAL(null);
-  }
-
   @Override
   public void preLogRoll(Path oldPath, Path newPath) throws IOException {
     // Not interested

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java Fri Jan 11 19:08:22 2013
@@ -23,7 +23,6 @@ import java.util.concurrent.ConcurrentMa
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
@@ -39,9 +38,8 @@ public interface RegionServerServices ex
    */
   public boolean isStopping();
 
-  /** @return the HLog for a particular region. Pass null for getting the 
-   * default (common) WAL */
-  public HLog getWAL(HRegionInfo regionInfo) throws IOException;
+  /** @return the HLog */
+  public HLog getWAL();
 
   /**
    * @return Implementation of {@link CompactionRequestor} or null.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java Fri Jan 11 19:08:22 2013
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.executor.
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
 import org.apache.zookeeper.KeeperException;
@@ -45,7 +44,7 @@ import org.apache.zookeeper.KeeperExcept
 public class OpenRegionHandler extends EventHandler {
   private static final Log LOG = LogFactory.getLog(OpenRegionHandler.class);
 
-  protected final RegionServerServices rsServices;
+  private final RegionServerServices rsServices;
 
   private final HRegionInfo regionInfo;
   private final HTableDescriptor htd;
@@ -425,8 +424,7 @@ public class OpenRegionHandler extends E
       // Instantiate the region.  This also periodically tickles our zk OPENING
       // state so master doesn't timeout this region in transition.
       region = HRegion.openHRegion(this.regionInfo, this.htd,
-          this.rsServices.getWAL(this.regionInfo), 
-          this.server.getConfiguration(),
+          this.rsServices.getWAL(), this.server.getConfiguration(),
           this.rsServices,
         new CancelableProgressable() {
           public boolean progress() {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java Fri Jan 11 19:08:22 2013
@@ -154,8 +154,6 @@ class FSHLog implements HLog, Syncable {
 
   private final AtomicLong logSeqNum = new AtomicLong(0);
 
-  private boolean forMeta = false;
-
   // The timestamp (in ms) when the log file was created.
   private volatile long filenum = -1;
 
@@ -213,15 +211,15 @@ class FSHLog implements HLog, Syncable {
    *
    * @param fs filesystem handle
    * @param root path for stored and archived hlogs
-   * @param logDir dir where hlogs are stored
+   * @param logName dir where hlogs are stored
    * @param conf configuration to use
    * @throws IOException
    */
-  public FSHLog(final FileSystem fs, final Path root, final String logDir,
+  public FSHLog(final FileSystem fs, final Path root, final String logName,
                 final Configuration conf)
   throws IOException {
-    this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, 
-        conf, null, true, null, false);
+    this(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, 
+        conf, null, true, null);
   }
   
   /**
@@ -229,16 +227,16 @@ class FSHLog implements HLog, Syncable {
    *
    * @param fs filesystem handle
    * @param root path for stored and archived hlogs
-   * @param logDir dir where hlogs are stored
-   * @param oldLogDir dir where hlogs are archived
+   * @param logName dir where hlogs are stored
+   * @param oldLogName dir where hlogs are archived
    * @param conf configuration to use
    * @throws IOException
    */
-  public FSHLog(final FileSystem fs, final Path root, final String logDir,
-                final String oldLogDir, final Configuration conf)
+  public FSHLog(final FileSystem fs, final Path root, final String logName,
+                final String oldLogName, final Configuration conf)
   throws IOException {
-    this(fs, root, logDir, oldLogDir, 
-        conf, null, true, null, false);
+    this(fs, root, logName, oldLogName, 
+        conf, null, true, null);
   }
 
   /**
@@ -250,7 +248,7 @@ class FSHLog implements HLog, Syncable {
    *
    * @param fs filesystem handle
    * @param root path for stored and archived hlogs
-   * @param logDir dir where hlogs are stored
+   * @param logName dir where hlogs are stored
    * @param conf configuration to use
    * @param listeners Listeners on WAL events. Listeners passed here will
    * be registered before we do anything else; e.g. the
@@ -260,11 +258,11 @@ class FSHLog implements HLog, Syncable {
    *        If prefix is null, "hlog" will be used
    * @throws IOException
    */
-  public FSHLog(final FileSystem fs, final Path root, final String logDir,
+  public FSHLog(final FileSystem fs, final Path root, final String logName,
       final Configuration conf, final List<WALActionsListener> listeners,
       final String prefix) throws IOException {
-    this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, 
-        conf, listeners, true, prefix, false);
+    this(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, 
+        conf, listeners, true, prefix);
   }
 
   /**
@@ -276,8 +274,7 @@ class FSHLog implements HLog, Syncable {
    *
    * @param fs filesystem handle
    * @param root path to where logs and oldlogs
-   * @param logDir dir where hlogs are stored
-   * @param oldLogDir dir where hlogs are archived
+   * @param oldLogName path to where hlogs are archived
    * @param conf configuration to use
    * @param listeners Listeners on WAL events. Listeners passed here will
    * be registered before we do anything else; e.g. the
@@ -286,20 +283,18 @@ class FSHLog implements HLog, Syncable {
    * @param prefix should always be hostname and port in distributed env and
    *        it will be URL encoded before being used.
    *        If prefix is null, "hlog" will be used
-   * @param forMeta if this hlog is meant for meta updates
    * @throws IOException
    */
-  public FSHLog(final FileSystem fs, final Path root, final String logDir,
-      final String oldLogDir, final Configuration conf, 
+  private FSHLog(final FileSystem fs, final Path root, final String logName,
+      final String oldLogName, final Configuration conf, 
       final List<WALActionsListener> listeners,
-      final boolean failIfLogDirExists, final String prefix, boolean forMeta)
+      final boolean failIfLogDirExists, final String prefix)
   throws IOException {
     super();
     this.fs = fs;
     this.rootDir = root;
-    this.dir = new Path(this.rootDir, logDir);
-    this.oldLogDir = new Path(this.rootDir, oldLogDir);
-    this.forMeta = forMeta;
+    this.dir = new Path(this.rootDir, logName);
+    this.oldLogDir = new Path(this.rootDir, oldLogName);
     this.conf = conf;
    
     if (listeners != null) {
@@ -338,16 +333,15 @@ class FSHLog implements HLog, Syncable {
     // If prefix is null||empty then just name it hlog
     this.prefix = prefix == null || prefix.isEmpty() ?
         "hlog" : URLEncoder.encode(prefix, "UTF8");
-
-    boolean dirExists = false;
-    if (failIfLogDirExists && (dirExists = this.fs.exists(dir))) {
+   
+    if (failIfLogDirExists && this.fs.exists(dir)) {
       throw new IOException("Target HLog directory already exists: " + dir);
     }
-    if (!dirExists && !fs.mkdirs(dir)) {
+    if (!fs.mkdirs(dir)) {
       throw new IOException("Unable to mkdir " + dir);
     }
 
-    if (!fs.exists(this.oldLogDir)) {
+    if (!fs.exists(oldLogDir)) {
       if (!fs.mkdirs(this.oldLogDir)) {
         throw new IOException("Unable to mkdir " + this.oldLogDir);
       }
@@ -489,7 +483,6 @@ class FSHLog implements HLog, Syncable {
       long currentFilenum = this.filenum;
       Path oldPath = null;
       if (currentFilenum > 0) {
-        //computeFilename  will take care of meta hlog filename
         oldPath = computeFilename(currentFilenum);
       }
       this.filenum = System.currentTimeMillis();
@@ -568,9 +561,6 @@ class FSHLog implements HLog, Syncable {
    */
   protected Writer createWriterInstance(final FileSystem fs, final Path path,
       final Configuration conf) throws IOException {
-    if (forMeta) {
-      //TODO: set a higher replication for the hlog files (HBASE-6773)
-    }
     return HLogFactory.createWriter(fs, path, conf);
   }
 
@@ -739,11 +729,7 @@ class FSHLog implements HLog, Syncable {
     if (filenum < 0) {
       throw new RuntimeException("hlog file number can't be < 0");
     }
-    String child = prefix + "." + filenum;
-    if (forMeta) {
-      child += HLog.META_HLOG_FILE_EXTN;
-    }
-    return new Path(dir, child);
+    return new Path(dir, prefix + "." + filenum);
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java Fri Jan 11 19:08:22 2013
@@ -50,8 +50,6 @@ public interface HLog {
   /** File Extension used while splitting an HLog into regions (HBASE-2312) */
   public static final String SPLITTING_EXT = "-splitting";
   public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
-  /** The META region's HLog filename extension */
-  public static final String META_HLOG_FILE_EXTN = ".meta";
 
   /*
    * Name of directory that holds recovered edits written by the wal log

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java Fri Jan 11 19:08:22 2013
@@ -26,9 +26,9 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
 import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
 
@@ -50,13 +50,6 @@ public class HLogFactory {
         final String prefix) throws IOException {
       return new FSHLog(fs, root, logName, conf, listeners, prefix);
     }
-
-    public static HLog createMetaHLog(final FileSystem fs, final Path root, final String logName,
-        final Configuration conf, final List<WALActionsListener> listeners,
-        final String prefix) throws IOException {
-      return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, 
-            conf, listeners, false, prefix, true);
-    }
     
     /*
      * WAL Reader

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java Fri Jan 11 19:08:22 2013
@@ -49,7 +49,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
@@ -303,11 +302,6 @@ public class HLogSplitter {
             + ": " + logPath + ", length=" + logLength);
         Reader in = null;
         try {
-          //actually, for meta-only hlogs, we don't need to go thru the process
-          //of parsing and segregating by regions since all the logs are for
-          //meta only. However, there is a sequence number that can be obtained
-          //only by parsing.. so we parse for all files currently
-          //TODO: optimize this part somehow
           in = getReader(fs, log, conf, skipErrors);
           if (in != null) {
             parseHLog(in, logPath, entryBuffers, fs, conf, skipErrors);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java Fri Jan 11 19:08:22 2013
@@ -76,8 +76,7 @@ public class HLogUtil {
   /**
    * Pattern used to validate a HLog file name
    */
-  private static final Pattern pattern = 
-      Pattern.compile(".*\\.\\d*("+HLog.META_HLOG_FILE_EXTN+")*");
+  private static final Pattern pattern = Pattern.compile(".*\\.\\d*");
 
   /**
    * @param filename
@@ -313,11 +312,4 @@ public class HLogUtil {
     }
     return filesSorted;
   }
-
-  public static boolean isMetaFile(Path p) {
-    if (p.getName().endsWith(HLog.META_HLOG_FILE_EXTN)) {
-      return true;
-    }
-    return false;
-  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java Fri Jan 11 19:08:22 2013
@@ -274,6 +274,12 @@ class MockRegionServer implements AdminP
   }
 
   @Override
+  public HLog getWAL() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
   public CompactionRequestor getCompactionRequester() {
     // TODO Auto-generated method stub
     return null;
@@ -493,10 +499,4 @@ class MockRegionServer implements AdminP
     // TODO Auto-generated method stub
     return null;
   }
-
-  @Override
-  public HLog getWAL(HRegionInfo regionInfo) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java?rev=1432234&r1=1432233&r2=1432234&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java Fri Jan 11 19:08:22 2013
@@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentSk
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -90,6 +89,11 @@ public class MockRegionServerServices im
   }
 
   @Override
+  public HLog getWAL() {
+    return null;
+  }
+
+  @Override
   public RpcServer getRpcServer() {
     return null;
   }
@@ -166,10 +170,4 @@ public class MockRegionServerServices im
   public Leases getLeases() {
     return null;
   }
-
-  @Override
-  public HLog getWAL(HRegionInfo regionInfo) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
-  }
 }
\ No newline at end of file