HBASE-10888 Enable distributed log replay as default

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1584346 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2014-04-03 19:39:55 +00:00
parent e124054e4e
commit 6892ae258d
8 changed files with 31 additions and 8 deletions

View File

@ -505,7 +505,7 @@ public class RecoverableZooKeeper {
Bytes.toStringBinary(data));
throw e;
}
LOG.info("Node " + path + " already exists");
LOG.debug("Node " + path + " already exists");
throw e;
case CONNECTIONLOSS:

View File

@ -794,7 +794,10 @@ public final class HConstants {
/** Conf key that enables unflushed WAL edits directly being replayed to region servers */
public static final String DISTRIBUTED_LOG_REPLAY_KEY = "hbase.master.distributed.log.replay";
public static final boolean DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG = false;
/**
* Default 'distributed log replay' as true since hbase 0.99.0
*/
public static final boolean DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG = true;
public static final String DISALLOW_WRITES_IN_RECOVERING =
"hbase.regionserver.disallow.writes.when.recovering";
public static final boolean DEFAULT_DISALLOW_WRITES_IN_RECOVERING_CONFIG = false;

View File

@ -219,6 +219,17 @@ possible configurations would overwhelm and obscure the important.
<value>org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter</value>
<description>The HLog file writer implementation.</description>
</property>
<property>
<name>hbase.master.distributed.log.replay</name>
<value>true</value>
<description>Enable 'distributed log replay' as default engine splitting
WAL files on server crash. This default is new in hbase 1.0. To fall
back to the old mode 'distributed log splitter', set the value to
'false'. 'Disributed log replay' improves MTTR because it does not
write intermediate files. 'DLR' required that 'hfile.format.version'
be set to version 3 or higher.
</description>
</property>
<property>
<name>hbase.regionserver.global.memstore.size</name>
<value>0.4</value>

View File

@ -115,7 +115,7 @@ public class DeadServer {
}
public synchronized void finish(ServerName sn) {
LOG.info("Finished processing " + sn);
LOG.debug("Finished processing " + sn);
this.numProcessing--;
}

View File

@ -628,7 +628,7 @@ public class SplitLogManager extends ZooKeeperListener {
try {
if (regions == null) {
// remove all children under /home/recovering-regions
LOG.info("Garbage collecting all recovering regions.");
LOG.debug("Garbage collecting all recovering region znodes");
ZKUtil.deleteChildrenRecursively(watcher, watcher.recoveringRegionsZNode);
} else {
for (String curRegion : regions) {

View File

@ -88,7 +88,7 @@ public class CreateTableHandler extends EventHandler {
int timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
// Need hbase:meta availability to create a table
try {
if(catalogTracker.waitForMeta(timeout) == null) {
if (catalogTracker.waitForMeta(timeout) == null) {
throw new NotAllMetaRegionsOnlineException();
}
} catch (InterruptedException e) {
@ -114,7 +114,9 @@ public class CreateTableHandler extends EventHandler {
// Use enabling state to tell if there is already a request for the same
// table in progress. This will introduce a new zookeeper call. Given
// createTable isn't a frequent operation, that should be ok.
//TODO: now that we have table locks, re-evaluate above
// TODO: now that we have table locks, re-evaluate above -- table locks are not enough.
// We could have cleared the hbase.rootdir and not zk. How can we detect this case?
// Having to clean zk AND hdfs is awkward.
try {
if (!this.assignmentManager.getZKTable().checkAndSetEnablingTable(tableName)) {
throw new TableExistsException(tableName);

View File

@ -83,6 +83,6 @@ public class LogReplayHandler extends EventHandler {
this.deadServers.finish(serverName);
}
// logReplay is the last step of SSH so log a line to indicate that
LOG.info("Finished processing of shutdown of " + serverName);
LOG.info("Finished processing shutdown of " + serverName);
}
}

View File

@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.master.SplitLogManager;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@ -1972,7 +1973,13 @@ public class HLogSplitter {
* @return true when distributed log replay is turned on
*/
public static boolean isDistributedLogReplay(Configuration conf) {
return conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
boolean dlr = conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
int version = conf.getInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
if (LOG.isDebugEnabled()) {
LOG.debug("Distributed log replay=" + dlr + ", " + HFile.FORMAT_VERSION_KEY + "=" + version);
}
// For distributed log replay, hfile version must be 3 at least; we need tag support.
return dlr && (version >= 3);
}
}