diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 17d26fa4979..9437d99a835 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -960,7 +960,6 @@ Server { return; } LOG.info("Forcing splitLog and expire of " + sn); - fileSystemManager.splitMetaLog(sn); fileSystemManager.splitLog(sn); serverManager.expireServer(sn); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java index e53f6ed6db4..904912fa28b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java @@ -32,7 +32,6 @@ 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.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 serverNames) throws IOException { long splitTime = 0, splitLogSize = 0; - List serverNames = new ArrayList(); - serverNames.add(serverName); - List 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 getLogDirs(final List serverNames) throws IOException { List logDirs = new ArrayList(); 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 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 serverNames, PathFilter filter) throws IOException { - long splitTime = 0, splitLogSize = 0; - List 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); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java index e8cb5dcc55b..3abf785c919 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java @@ -40,7 +40,6 @@ 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.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.monitoring.MonitoredTask; 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 ZooKeeperListener { } } - private FileStatus[] getFileList(List logDirs, PathFilter filter) throws IOException { + private FileStatus[] getFileList(List logDirs) throws IOException { List fileStatus = new ArrayList(); for (Path hLogDir : logDirs) { this.fs = hLogDir.getFileSystem(conf); @@ -204,7 +202,8 @@ public class SplitLogManager extends ZooKeeperListener { 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 ZooKeeperListener { 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 ZooKeeperListener { * @return cumulative size of the logfiles split */ public long splitLogDistributed(final List 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 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(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java index 0f62005266d..eb8ec22b51b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java @@ -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 -ROOT-, @@ -38,7 +32,7 @@ import org.apache.zookeeper.KeeperException; 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 extends ServerShutdownHandler { } @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 - *

- * 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; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java index 1239b293e04..f325708fb35 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java @@ -55,10 +55,10 @@ import org.apache.zookeeper.KeeperException; @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, @@ -90,6 +90,69 @@ public class ServerShutdownHandler extends EventHandler { } } + /** + * Before assign the ROOT region, ensure it haven't + * been assigned by other place + *

+ * 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 -ROOT- */ @@ -125,13 +188,43 @@ public class ServerShutdownHandler extends EventHandler { 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: diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 72d2b6d2b95..6c5604769a7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -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 ClientProtocol, 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 ClientProtocol, // 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 ClientProtocol, "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 ClientProtocol, 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 ClientProtocol, 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 ClientProtocol, return listeners; } - protected List getMetaWALActionListeners() { - List listeners = new ArrayList(); - // 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 ClientProtocol, */ 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 ClientProtocol, 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 ClientProtocol, 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 ClientProtocol, 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(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java index 9488749c11e..20df71413e1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java @@ -24,7 +24,6 @@ import org.apache.hadoop.classification.InterfaceAudience; 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 implements WALActionsListener { 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 implements WALActionsListener { 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 implements WALActionsListener { } } - protected HLog getWAL() throws IOException { - return this.services.getWAL(null); - } - @Override public void preLogRoll(Path oldPath, Path newPath) throws IOException { // Not interested diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java deleted file mode 100644 index dcbbdd69de4..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaLogRoller.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * 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; - -import java.io.IOException; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.regionserver.wal.HLog; - -@InterfaceAudience.Private -class MetaLogRoller extends LogRoller { - public MetaLogRoller(Server server, RegionServerServices services) { - super(server, services); - } - @Override - protected HLog getWAL() throws IOException { - return services.getWAL(HRegionInfo.FIRST_META_REGIONINFO); - } -} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java index e40871fd222..422df5001ad 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java @@ -23,7 +23,6 @@ import java.util.concurrent.ConcurrentMap; 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 extends OnlineRegions { */ 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. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java index f91bfe05a5e..2c94060a18e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.executor.EventHandler; 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.KeeperException; 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 EventHandler { // 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() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 0a387eed507..4f08fe8c92d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -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 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 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 diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java index 75e2371b648..0b323dff89e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java @@ -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 diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java index eaee4346033..6600d476e2a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java @@ -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 listeners, - final String prefix) throws IOException { - return new FSHLog(fs, root, logName, HConstants.HREGION_OLDLOGDIR_NAME, - conf, listeners, false, prefix, true); - } /* * WAL Reader diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java index 3a248016033..4eaea71b953 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java @@ -49,7 +49,6 @@ 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.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); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java index e5eb0d91120..7888aba57b5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java @@ -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; - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 9300b19b02d..8889dd1d745 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -273,6 +273,12 @@ class MockRegionServer implements AdminProtocol, ClientProtocol, RegionServerSer return false; } + @Override + public HLog getWAL() { + // TODO Auto-generated method stub + return null; + } + @Override public CompactionRequestor getCompactionRequester() { // TODO Auto-generated method stub @@ -493,10 +499,4 @@ class MockRegionServer implements AdminProtocol, ClientProtocol, RegionServerSer // TODO Auto-generated method stub return null; } - - @Override - public HLog getWAL(HRegionInfo regionInfo) throws IOException { - // TODO Auto-generated method stub - return null; - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java index 52cf4548558..108e64895bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java @@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentSkipListMap; 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; @@ -89,6 +88,11 @@ public class MockRegionServerServices implements RegionServerServices { return this.stopping; } + @Override + public HLog getWAL() { + return null; + } + @Override public RpcServer getRpcServer() { return null; @@ -166,10 +170,4 @@ public class MockRegionServerServices implements RegionServerServices { 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