HBASE-7006 [MTTR] Improve Region Server Recovery Time - Distributed Log Replay

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1482675 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-05-15 04:24:02 +00:00
parent 6bf0abbec9
commit cb53db35f3
58 changed files with 4159 additions and 705 deletions

View File

@ -185,6 +185,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
private byte[] tableName = null; private byte[] tableName = null;
private String tableNameAsString = null; private String tableNameAsString = null;
// when a region is in recovering state, it can only accept writes not reads
private volatile boolean recovering = false;
/** HRegionInfo for root region */ /** HRegionInfo for root region */
public static final HRegionInfo ROOT_REGIONINFO = public static final HRegionInfo ROOT_REGIONINFO =
new HRegionInfo(0L, Bytes.toBytes("-ROOT-")); new HRegionInfo(0L, Bytes.toBytes("-ROOT-"));
@ -293,6 +296,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
this.startKey = startKey == null? this.startKey = startKey == null?
HConstants.EMPTY_START_ROW: startKey.clone(); HConstants.EMPTY_START_ROW: startKey.clone();
this.tableName = tableName.clone(); this.tableName = tableName.clone();
this.recovering = false;
setHashCode(); setHashCode();
} }
@ -313,6 +317,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
this.hashCode = other.hashCode(); this.hashCode = other.hashCode();
this.encodedName = other.getEncodedName(); this.encodedName = other.getEncodedName();
this.tableName = other.tableName; this.tableName = other.tableName;
this.recovering = other.isRecovering();
} }
@ -596,6 +601,20 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
this.split = split; this.split = split;
} }
/**
* @return True if current region is in recovering
*/
public boolean isRecovering() {
return this.recovering;
}
/**
* @param newState set recovering state
*/
public void setRecovering(boolean newState) {
this.recovering = newState;
}
/** /**
* @return True if this region is offline. * @return True if this region is offline.
*/ */
@ -833,6 +852,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
} }
builder.setOffline(info.isOffline()); builder.setOffline(info.isOffline());
builder.setSplit(info.isSplit()); builder.setSplit(info.isSplit());
builder.setRecovering(info.isRecovering());
return builder.build(); return builder.build();
} }
@ -865,6 +885,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
if (proto.hasOffline()) { if (proto.hasOffline()) {
hri.setOffline(proto.getOffline()); hri.setOffline(proto.getOffline());
} }
if (proto.hasRecovering()) {
hri.setRecovering(proto.getRecovering());
}
return hri; return hri;
} }

View File

@ -18,6 +18,14 @@
package org.apache.hadoop.hbase.client; package org.apache.hadoop.hbase.client;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.UUID;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
@ -31,14 +39,6 @@ import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.ClassSize;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.UUID;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public abstract class Mutation extends OperationWithAttributes implements Row, CellScannable, public abstract class Mutation extends OperationWithAttributes implements Row, CellScannable,
@ -63,6 +63,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
protected byte [] row = null; protected byte [] row = null;
protected long ts = HConstants.LATEST_TIMESTAMP; protected long ts = HConstants.LATEST_TIMESTAMP;
protected Durability durability = Durability.USE_DEFAULT; protected Durability durability = Durability.USE_DEFAULT;
// A Map sorted by column family. // A Map sorted by column family.
protected NavigableMap<byte [], List<? extends Cell>> familyMap = protected NavigableMap<byte [], List<? extends Cell>> familyMap =
new TreeMap<byte [], List<? extends Cell>>(Bytes.BYTES_COMPARATOR); new TreeMap<byte [], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);

View File

@ -178,7 +178,7 @@ public abstract class ServerCallable<T> implements Callable<T> {
if (t instanceof SocketTimeoutException || if (t instanceof SocketTimeoutException ||
t instanceof ConnectException || t instanceof ConnectException ||
t instanceof RetriesExhaustedException || t instanceof RetriesExhaustedException ||
getConnection().isDeadServer(location.getServerName())) { (location != null && getConnection().isDeadServer(location.getServerName()))) {
// if thrown these exceptions, we clear all the cache entries that // if thrown these exceptions, we clear all the cache entries that
// map to that slow/dead server; otherwise, let cache miss and ask // map to that slow/dead server; otherwise, let cache miss and ask
// .META. again to find the new location // .META. again to find the new location
@ -261,8 +261,10 @@ public abstract class ServerCallable<T> implements Callable<T> {
*/ */
protected static Throwable translateException(Throwable t) throws DoNotRetryIOException { protected static Throwable translateException(Throwable t) throws DoNotRetryIOException {
if (t instanceof UndeclaredThrowableException) { if (t instanceof UndeclaredThrowableException) {
if(t.getCause() != null) {
t = t.getCause(); t = t.getCause();
} }
}
if (t instanceof RemoteException) { if (t instanceof RemoteException) {
t = ((RemoteException)t).unwrapRemoteException(); t = ((RemoteException)t).unwrapRemoteException();
} }

View File

@ -0,0 +1,45 @@
/**
*
* 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.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown when a read request issued against a region which is in recovering state.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class RegionInRecoveryException extends NotServingRegionException {
private static final long serialVersionUID = 327302071153799L;
/** default constructor */
public RegionInRecoveryException() {
super();
}
/**
* Constructor
* @param s message
*/
public RegionInRecoveryException(String s) {
super(s);
}
}

View File

@ -431,6 +431,30 @@ public class ReplicationZookeeper implements Closeable {
return ProtobufUtil.prependPBMagic(bytes); return ProtobufUtil.prependPBMagic(bytes);
} }
/**
* @param position
* @return Serialized protobuf of <code>position</code> with pb magic prefix
* prepended suitable for use as content of an hlog position in a
* replication queue.
*/
public static byte[] positionToByteArray(
final long position) {
return ZKUtil.positionToByteArray(position);
}
/**
* @param lockOwner
* @return Serialized protobuf of <code>lockOwner</code> with pb magic prefix
* prepended suitable for use as content of an replication lock during
* region server fail over.
*/
static byte[] lockToByteArray(
final String lockOwner) {
byte[] bytes = ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build()
.toByteArray();
return ProtobufUtil.prependPBMagic(bytes);
}
/** /**
* @param bytes Content of a peer znode. * @param bytes Content of a peer znode.
* @return ClusterKey parsed from the passed bytes. * @return ClusterKey parsed from the passed bytes.
@ -476,6 +500,42 @@ public class ReplicationZookeeper implements Closeable {
} }
} }
/**
* @param bytes - Content of a HLog position znode.
* @return long - The current HLog position.
* @throws DeserializationException
*/
public static long parseHLogPositionFrom(
final byte[] bytes) throws DeserializationException {
return ZKUtil.parseHLogPositionFrom(bytes);
}
/**
* @param bytes - Content of a lock znode.
* @return String - The owner of the lock.
* @throws DeserializationException
*/
static String parseLockOwnerFrom(
final byte[] bytes) throws DeserializationException {
if (ProtobufUtil.isPBMagicPrefix(bytes)) {
int pblen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.ReplicationLock.Builder builder = ZooKeeperProtos.ReplicationLock
.newBuilder();
ZooKeeperProtos.ReplicationLock lock;
try {
lock = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
} catch (InvalidProtocolBufferException e) {
throw new DeserializationException(e);
}
return lock.getLockOwner();
} else {
if (bytes.length > 0) {
return Bytes.toString(bytes);
}
return "";
}
}
private boolean peerExists(String id) throws KeeperException { private boolean peerExists(String id) throws KeeperException {
return ZKUtil.checkExists(this.zookeeper, return ZKUtil.checkExists(this.zookeeper,
ZKUtil.joinZNode(this.peersZNode, id)) >= 0; ZKUtil.joinZNode(this.peersZNode, id)) >= 0;

View File

@ -376,6 +376,7 @@ public class ZKTable {
Set<String> allTables = new HashSet<String>(); Set<String> allTables = new HashSet<String>();
List<String> children = List<String> children =
ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
if(children == null) return allTables;
for (String child: children) { for (String child: children) {
ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(zkw, child); ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(zkw, child);
for (ZooKeeperProtos.Table.State expectedState: states) { for (ZooKeeperProtos.Table.State expectedState: states) {

View File

@ -27,6 +27,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent; import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
@ -50,6 +52,8 @@ import org.apache.zookeeper.proto.DeleteRequest;
import org.apache.zookeeper.proto.SetDataRequest; import org.apache.zookeeper.proto.SetDataRequest;
import org.apache.zookeeper.server.ZooKeeperSaslServer; import org.apache.zookeeper.server.ZooKeeperSaslServer;
import com.google.protobuf.InvalidProtocolBufferException;
import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
import java.io.BufferedReader; import java.io.BufferedReader;
@ -1815,4 +1819,39 @@ public class ZKUtil {
} }
} }
/**
* @param position
* @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable
* for use as content of an hlog position in a replication queue.
*/
public static byte[] positionToByteArray(final long position) {
byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
.build().toByteArray();
return ProtobufUtil.prependPBMagic(bytes);
}
/**
* @param bytes - Content of a HLog position znode.
* @return long - The current HLog position.
* @throws DeserializationException
*/
public static long parseHLogPositionFrom(final byte[] bytes) throws DeserializationException {
if (ProtobufUtil.isPBMagicPrefix(bytes)) {
int pblen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.ReplicationHLogPosition.Builder builder =
ZooKeeperProtos.ReplicationHLogPosition.newBuilder();
ZooKeeperProtos.ReplicationHLogPosition position;
try {
position = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
} catch (InvalidProtocolBufferException e) {
throw new DeserializationException(e);
}
return position.getPosition();
} else {
if (bytes.length > 0) {
return Bytes.toLong(bytes);
}
return 0;
}
}
} }

View File

@ -107,6 +107,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
public String balancerZNode; public String balancerZNode;
// znode containing the lock for the tables // znode containing the lock for the tables
public String tableLockZNode; public String tableLockZNode;
// znode containing the state of recovering regions
public String recoveringRegionsZNode;
// Certain ZooKeeper nodes need to be world-readable // Certain ZooKeeper nodes need to be world-readable
public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE = public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
@ -133,9 +135,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
/** /**
* Instantiate a ZooKeeper connection and watcher. * Instantiate a ZooKeeper connection and watcher.
* @param identifier string that is passed to RecoverableZookeeper to be used as
* identifier for this instance. Use null for default.
* @param conf * @param conf
* @param identifier string that is passed to RecoverableZookeeper to be used as identifier for
* this instance. Use null for default.
* @param abortable Can be null if there is on error there is no host to abort: e.g. client * @param abortable Can be null if there is on error there is no host to abort: e.g. client
* context. * context.
* @param canCreateBaseZNode * @param canCreateBaseZNode
@ -176,6 +178,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
ZKUtil.createAndFailSilent(this, splitLogZNode); ZKUtil.createAndFailSilent(this, splitLogZNode);
ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode); ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode);
ZKUtil.createAndFailSilent(this, tableLockZNode); ZKUtil.createAndFailSilent(this, tableLockZNode);
ZKUtil.createAndFailSilent(this, recoveringRegionsZNode);
} catch (KeeperException e) { } catch (KeeperException e) {
throw new ZooKeeperConnectionException( throw new ZooKeeperConnectionException(
prefix("Unexpected KeeperException creating base node"), e); prefix("Unexpected KeeperException creating base node"), e);
@ -227,6 +230,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
conf.get("zookeeper.znode.balancer", "balancer")); conf.get("zookeeper.znode.balancer", "balancer"));
tableLockZNode = ZKUtil.joinZNode(baseZNode, tableLockZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.tableLock", "table-lock")); conf.get("zookeeper.znode.tableLock", "table-lock"));
recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.recovering.regions", "recovering-regions"));
} }
/** /**

View File

@ -717,6 +717,14 @@ public final class HConstants {
public static final String DISTRIBUTED_LOG_SPLITTING_KEY = public static final String DISTRIBUTED_LOG_SPLITTING_KEY =
"hbase.master.distributed.log.splitting"; "hbase.master.distributed.log.splitting";
/** 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 = true;
/** Conf key that specifies timeout value to wait for a region ready */
public static final String LOG_REPLAY_WAIT_REGION_TIMEOUT =
"hbase.master.log.replay.wait.region.timeout";
/** /**
* The name of the configuration parameter that specifies * The name of the configuration parameter that specifies
* the number of bytes in a newly created checksum chunk. * the number of bytes in a newly created checksum chunk.
@ -767,6 +775,7 @@ public final class HConstants {
public static final int QOS_THRESHOLD = 10; public static final int QOS_THRESHOLD = 10;
public static final int HIGH_QOS = 100; public static final int HIGH_QOS = 100;
public static final int REPLICATION_QOS = 5; // normal_QOS < replication_QOS < high_QOS public static final int REPLICATION_QOS = 5; // normal_QOS < replication_QOS < high_QOS
public static final int REPLAY_QOS = 6; // REPLICATION_QOS < REPLAY_QOS < high_QOS
/** Directory under /hbase where archived hfiles are stored */ /** Directory under /hbase where archived hfiles are stored */
public static final String HFILE_ARCHIVE_DIRECTORY = ".archive"; public static final String HFILE_ARCHIVE_DIRECTORY = ".archive";

View File

@ -60,6 +60,8 @@ public interface MetricsMasterSource extends BaseSource {
static final String SNAPSHOT_TIME_NAME = "snapshotTime"; static final String SNAPSHOT_TIME_NAME = "snapshotTime";
static final String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime"; static final String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime";
static final String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime"; static final String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime";
static final String META_SPLIT_TIME_NAME = "metaHlogSplitTime";
static final String META_SPLIT_SIZE_NAME = "metaHlogSplitSize";
static final String CLUSTER_REQUESTS_NAME = "clusterRequests"; static final String CLUSTER_REQUESTS_NAME = "clusterRequests";
static final String RIT_COUNT_NAME = "ritCount"; static final String RIT_COUNT_NAME = "ritCount";
static final String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold"; static final String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
@ -78,7 +80,8 @@ public interface MetricsMasterSource extends BaseSource {
static final String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()"; static final String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()";
static final String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()"; static final String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()";
static final String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()"; static final String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()";
static final String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()";
static final String META_SPLIT_SIZE_DESC = "Size of META HLog files being split";
/** /**
* Increment the number of requests the cluster has seen. * Increment the number of requests the cluster has seen.
@ -117,4 +120,9 @@ public interface MetricsMasterSource extends BaseSource {
void updateSnapshotCloneTime(long time); void updateSnapshotCloneTime(long time);
void updateSnapshotRestoreTime(long time); void updateSnapshotRestoreTime(long time);
void updateMetaWALSplitTime(long time);
void updateMetaWALSplitSize(long size);
} }

View File

@ -80,6 +80,13 @@ public interface MetricsRegionServerSource extends BaseSource {
*/ */
void updateAppend(long t); void updateAppend(long t);
/**
* Update the Replay time histogram.
*
* @param t time it took
*/
void updateReplay(long t);
/** /**
* Increment the number of slow Puts that have happened. * Increment the number of slow Puts that have happened.
*/ */
@ -188,6 +195,7 @@ public interface MetricsRegionServerSource extends BaseSource {
static final String INCREMENT_KEY = "increment"; static final String INCREMENT_KEY = "increment";
static final String MUTATE_KEY = "mutate"; static final String MUTATE_KEY = "mutate";
static final String APPEND_KEY = "append"; static final String APPEND_KEY = "append";
static final String REPLAY_KEY = "replay";
static final String SCAN_NEXT_KEY = "scanNext"; static final String SCAN_NEXT_KEY = "scanNext";
static final String SLOW_MUTATE_KEY = "slowPutCount"; static final String SLOW_MUTATE_KEY = "slowPutCount";
static final String SLOW_GET_KEY = "slowGetCount"; static final String SLOW_GET_KEY = "slowGetCount";

View File

@ -0,0 +1,72 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.hadoop.hbase.metrics.BaseSource;
/**
* Interface of the source that will export metrics about log replay statistics when recovering a
* region server in distributedLogReplay mode
*/
public interface MetricsEditsReplaySource extends BaseSource {
/**
* The name of the metrics
*/
static final String METRICS_NAME = "replay";
/**
* The name of the metrics context that metrics will be under.
*/
static final String METRICS_CONTEXT = "regionserver";
/**
* Description
*/
static final String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog Edits Replay";
/**
* The name of the metrics context that metrics will be under in jmx
*/
static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
static final String REPLAY_TIME_NAME = "replayTime";
static final String REPLAY_TIME_DESC = "Time an replay operation took.";
static final String REPLAY_BATCH_SIZE_NAME = "replayBatchSize";
static final String REPLAY_BATCH_SIZE_DESC = "Number of changes in each replay batch.";
static final String REPLAY_DATA_SIZE_NAME = "replayDataSize";
static final String REPLAY_DATA_SIZE_DESC = "Size (in bytes) of the data of each replay.";
/**
* Add the time a replay command took
*/
void updateReplayTime(long time);
/**
* Add the batch size of each replay
*/
void updateReplayBatchSize(long size);
/**
* Add the payload data size of each replay
*/
void updateReplayDataSize(long size);
}

View File

@ -48,6 +48,8 @@ public class MetricsMasterSourceImpl
private MetricMutableStat snapshotTimeHisto; private MetricMutableStat snapshotTimeHisto;
private MetricMutableStat snapshotCloneTimeHisto; private MetricMutableStat snapshotCloneTimeHisto;
private MetricMutableStat snapshotRestoreTimeHisto; private MetricMutableStat snapshotRestoreTimeHisto;
private MetricMutableHistogram metaSplitTimeHisto;
private MetricMutableHistogram metaSplitSizeHisto;
public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) { public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, masterWrapper); this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, masterWrapper);
@ -77,6 +79,8 @@ public class MetricsMasterSourceImpl
SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true); SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true);
snapshotRestoreTimeHisto = metricsRegistry.newStat( snapshotRestoreTimeHisto = metricsRegistry.newStat(
SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true); SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true);
metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
} }
public void incRequests(final int inc) { public void incRequests(final int inc) {
@ -120,6 +124,16 @@ public class MetricsMasterSourceImpl
snapshotRestoreTimeHisto.add(time); snapshotRestoreTimeHisto.add(time);
} }
@Override
public void updateMetaWALSplitTime(long time) {
metaSplitTimeHisto.add(time);
}
@Override
public void updateMetaWALSplitSize(long size) {
metaSplitSizeHisto.add(size);
}
/** /**
* Method to export all the metrics. * Method to export all the metrics.
* *

View File

@ -38,6 +38,7 @@ public class MetricsRegionServerSourceImpl
private final MetricHistogram getHisto; private final MetricHistogram getHisto;
private final MetricHistogram incrementHisto; private final MetricHistogram incrementHisto;
private final MetricHistogram appendHisto; private final MetricHistogram appendHisto;
private final MetricHistogram replayHisto;
private final MetricMutableCounterLong slowPut; private final MetricMutableCounterLong slowPut;
private final MetricMutableCounterLong slowDelete; private final MetricMutableCounterLong slowDelete;
private final MetricMutableCounterLong slowGet; private final MetricMutableCounterLong slowGet;
@ -70,6 +71,8 @@ public class MetricsRegionServerSourceImpl
appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY); appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY);
slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l); slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l);
replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY);
} }
@Override @Override
@ -97,6 +100,11 @@ public class MetricsRegionServerSourceImpl
appendHisto.add(t); appendHisto.add(t);
} }
@Override
public void updateReplay(long t) {
replayHisto.add(t);
}
@Override @Override
public void incrSlowPut() { public void incrSlowPut() {
slowPut.incr(); slowPut.incr();

View File

@ -0,0 +1,75 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.lib.MetricMutableHistogram;
/**
* Hadoop1 implementation of MetricsMasterSource.
*
* Implements BaseSource through BaseSourceImpl, following the pattern
*/
public class MetricsEditsReplaySourceImpl
extends BaseSourceImpl implements MetricsEditsReplaySource {
private static final Log LOG = LogFactory.getLog(MetricsEditsReplaySourceImpl.class.getName());
private MetricMutableHistogram replayTimeHisto;
private MetricMutableHistogram replayBatchSizeHisto;
private MetricMutableHistogram replayDataSizeHisto;
public MetricsEditsReplaySourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
}
public MetricsEditsReplaySourceImpl(String metricsName,
String metricsDescription,
String metricsContext,
String metricsJmxContext) {
super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
}
@Override
public void init() {
super.init();
replayTimeHisto = metricsRegistry.newHistogram(REPLAY_TIME_NAME, REPLAY_TIME_DESC);
replayBatchSizeHisto =
metricsRegistry.newHistogram(REPLAY_BATCH_SIZE_NAME, REPLAY_BATCH_SIZE_DESC);
replayDataSizeHisto =
metricsRegistry.newHistogram(REPLAY_DATA_SIZE_NAME, REPLAY_DATA_SIZE_DESC);
}
@Override
public void updateReplayTime(long time) {
replayTimeHisto.add(time);
}
@Override
public void updateReplayBatchSize(long size) {
replayBatchSizeHisto.add(size);
}
@Override
public void updateReplayDataSize(long size) {
replayDataSizeHisto.add(size);
}
}

View File

@ -0,0 +1 @@
org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySourceImpl

View File

@ -45,6 +45,8 @@ public class MetricsMasterSourceImpl
private MutableStat snapshotTimeHisto; private MutableStat snapshotTimeHisto;
private MutableStat snapshotCloneTimeHisto; private MutableStat snapshotCloneTimeHisto;
private MutableStat snapshotRestoreTimeHisto; private MutableStat snapshotRestoreTimeHisto;
private MutableHistogram metaSplitTimeHisto;
private MutableHistogram metaSplitSizeHisto;
public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) { public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
this(METRICS_NAME, this(METRICS_NAME,
@ -79,6 +81,8 @@ public class MetricsMasterSourceImpl
SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true); SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true);
snapshotRestoreTimeHisto = metricsRegistry.newStat( snapshotRestoreTimeHisto = metricsRegistry.newStat(
SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true); SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true);
metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
} }
public void incRequests(final int inc) { public void incRequests(final int inc) {
@ -122,6 +126,16 @@ public class MetricsMasterSourceImpl
snapshotRestoreTimeHisto.add(time); snapshotRestoreTimeHisto.add(time);
} }
@Override
public void updateMetaWALSplitTime(long time) {
metaSplitTimeHisto.add(time);
}
@Override
public void updateMetaWALSplitSize(long size) {
metaSplitSizeHisto.add(size);
}
@Override @Override
public void getMetrics(MetricsCollector metricsCollector, boolean all) { public void getMetrics(MetricsCollector metricsCollector, boolean all) {

View File

@ -41,6 +41,7 @@ public class MetricsRegionServerSourceImpl
private final MetricHistogram getHisto; private final MetricHistogram getHisto;
private final MetricHistogram incrementHisto; private final MetricHistogram incrementHisto;
private final MetricHistogram appendHisto; private final MetricHistogram appendHisto;
private final MetricHistogram replayHisto;
private final MutableCounterLong slowPut; private final MutableCounterLong slowPut;
private final MutableCounterLong slowDelete; private final MutableCounterLong slowDelete;
@ -75,6 +76,8 @@ public class MetricsRegionServerSourceImpl
appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY); appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY);
slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l); slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l);
replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY);
} }
@Override @Override
@ -102,6 +105,11 @@ public class MetricsRegionServerSourceImpl
appendHisto.add(t); appendHisto.add(t);
} }
@Override
public void updateReplay(long t) {
replayHisto.add(t);
}
@Override @Override
public void incrSlowPut() { public void incrSlowPut() {
slowPut.incr(); slowPut.incr();

View File

@ -0,0 +1,74 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
/**
* Hadoop1 implementation of MetricsMasterSource. Implements BaseSource through BaseSourceImpl,
* following the pattern
*/
public class MetricsEditsReplaySourceImpl extends BaseSourceImpl implements
MetricsEditsReplaySource {
private static final Log LOG = LogFactory.getLog(MetricsEditsReplaySourceImpl.class.getName());
private MetricHistogram replayTimeHisto;
private MetricHistogram replayBatchSizeHisto;
private MetricHistogram replayDataSizeHisto;
public MetricsEditsReplaySourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
}
public MetricsEditsReplaySourceImpl(String metricsName,
String metricsDescription,
String metricsContext,
String metricsJmxContext) {
super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
}
@Override
public void init() {
super.init();
replayTimeHisto = metricsRegistry.newHistogram(REPLAY_TIME_NAME, REPLAY_TIME_DESC);
replayBatchSizeHisto = metricsRegistry.newHistogram(REPLAY_BATCH_SIZE_NAME,
REPLAY_BATCH_SIZE_DESC);
replayDataSizeHisto = metricsRegistry
.newHistogram(REPLAY_DATA_SIZE_NAME, REPLAY_DATA_SIZE_DESC);
}
@Override
public void updateReplayTime(long time) {
replayTimeHisto.add(time);
}
@Override
public void updateReplayBatchSize(long size) {
replayBatchSizeHisto.add(size);
}
@Override
public void updateReplayDataSize(long size) {
replayDataSizeHisto.add(size);
}
}

View File

@ -0,0 +1 @@
org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySourceImpl

View File

@ -14155,6 +14155,11 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse> done); com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse> done);
public abstract void replay(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse> done);
public abstract void rollWALWriter( public abstract void rollWALWriter(
com.google.protobuf.RpcController controller, com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request,
@ -14255,6 +14260,14 @@ public final class AdminProtos {
impl.replicateWALEntry(controller, request, done); impl.replicateWALEntry(controller, request, done);
} }
@java.lang.Override
public void replay(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse> done) {
impl.replay(controller, request, done);
}
@java.lang.Override @java.lang.Override
public void rollWALWriter( public void rollWALWriter(
com.google.protobuf.RpcController controller, com.google.protobuf.RpcController controller,
@ -14322,10 +14335,12 @@ public final class AdminProtos {
case 9: case 9:
return impl.replicateWALEntry(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request); return impl.replicateWALEntry(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request);
case 10: case 10:
return impl.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request); return impl.replay(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request);
case 11: case 11:
return impl.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request); return impl.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request);
case 12: case 12:
return impl.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request);
case 13:
return impl.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request); return impl.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request);
default: default:
throw new java.lang.AssertionError("Can't get here."); throw new java.lang.AssertionError("Can't get here.");
@ -14362,10 +14377,12 @@ public final class AdminProtos {
case 9: case 9:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance();
case 10: case 10:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance();
case 11: case 11:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance();
case 12: case 12:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance();
case 13:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance();
default: default:
throw new java.lang.AssertionError("Can't get here."); throw new java.lang.AssertionError("Can't get here.");
@ -14402,10 +14419,12 @@ public final class AdminProtos {
case 9: case 9:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance();
case 10: case 10:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance();
case 11: case 11:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance();
case 12: case 12:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance();
case 13:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance();
default: default:
throw new java.lang.AssertionError("Can't get here."); throw new java.lang.AssertionError("Can't get here.");
@ -14465,6 +14484,11 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse> done); com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse> done);
public abstract void replay(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse> done);
public abstract void rollWALWriter( public abstract void rollWALWriter(
com.google.protobuf.RpcController controller, com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request,
@ -14553,16 +14577,21 @@ public final class AdminProtos {
done)); done));
return; return;
case 10: case 10:
this.replay(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request,
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse>specializeCallback(
done));
return;
case 11:
this.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request, this.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request,
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse>specializeCallback( com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse>specializeCallback(
done)); done));
return; return;
case 11: case 12:
this.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request, this.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request,
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse>specializeCallback( com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse>specializeCallback(
done)); done));
return; return;
case 12: case 13:
this.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request, this.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request,
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse>specializeCallback( com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse>specializeCallback(
done)); done));
@ -14602,10 +14631,12 @@ public final class AdminProtos {
case 9: case 9:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance();
case 10: case 10:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance();
case 11: case 11:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance();
case 12: case 12:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance();
case 13:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance();
default: default:
throw new java.lang.AssertionError("Can't get here."); throw new java.lang.AssertionError("Can't get here.");
@ -14642,10 +14673,12 @@ public final class AdminProtos {
case 9: case 9:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance();
case 10: case 10:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance();
case 11: case 11:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance();
case 12: case 12:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance();
case 13:
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance();
default: default:
throw new java.lang.AssertionError("Can't get here."); throw new java.lang.AssertionError("Can't get here.");
@ -14818,12 +14851,27 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance())); org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance()));
} }
public void replay(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse> done) {
channel.callMethod(
getDescriptor().getMethods().get(10),
controller,
request,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(),
com.google.protobuf.RpcUtil.generalizeCallback(
done,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.class,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance()));
}
public void rollWALWriter( public void rollWALWriter(
com.google.protobuf.RpcController controller, com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse> done) { com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse> done) {
channel.callMethod( channel.callMethod(
getDescriptor().getMethods().get(10), getDescriptor().getMethods().get(11),
controller, controller,
request, request,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(), org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(),
@ -14838,7 +14886,7 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse> done) { com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse> done) {
channel.callMethod( channel.callMethod(
getDescriptor().getMethods().get(11), getDescriptor().getMethods().get(12),
controller, controller,
request, request,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(), org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(),
@ -14853,7 +14901,7 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse> done) { com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse> done) {
channel.callMethod( channel.callMethod(
getDescriptor().getMethods().get(12), getDescriptor().getMethods().get(13),
controller, controller,
request, request,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(), org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(),
@ -14920,6 +14968,11 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request) org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request)
throws com.google.protobuf.ServiceException; throws com.google.protobuf.ServiceException;
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse replay(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request)
throws com.google.protobuf.ServiceException;
public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter( public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter(
com.google.protobuf.RpcController controller, com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request) org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request)
@ -15063,12 +15116,24 @@ public final class AdminProtos {
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse replay(
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request)
throws com.google.protobuf.ServiceException {
return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse) channel.callBlockingMethod(
getDescriptor().getMethods().get(10),
controller,
request,
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance());
}
public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter( public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter(
com.google.protobuf.RpcController controller, com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request) org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request)
throws com.google.protobuf.ServiceException { throws com.google.protobuf.ServiceException {
return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse) channel.callBlockingMethod( return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse) channel.callBlockingMethod(
getDescriptor().getMethods().get(10), getDescriptor().getMethods().get(11),
controller, controller,
request, request,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance()); org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance());
@ -15080,7 +15145,7 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request) org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request)
throws com.google.protobuf.ServiceException { throws com.google.protobuf.ServiceException {
return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse) channel.callBlockingMethod( return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse) channel.callBlockingMethod(
getDescriptor().getMethods().get(11), getDescriptor().getMethods().get(12),
controller, controller,
request, request,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance()); org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance());
@ -15092,7 +15157,7 @@ public final class AdminProtos {
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request) org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request)
throws com.google.protobuf.ServiceException { throws com.google.protobuf.ServiceException {
return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse) channel.callBlockingMethod( return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse) channel.callBlockingMethod(
getDescriptor().getMethods().get(12), getDescriptor().getMethods().get(13),
controller, controller,
request, request,
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance()); org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance());
@ -15255,78 +15320,80 @@ public final class AdminProtos {
descriptor; descriptor;
static { static {
java.lang.String[] descriptorData = { java.lang.String[] descriptorData = {
"\n\013Admin.proto\032\013hbase.proto\032\tWAL.proto\"Q\n" + "\n\013Admin.proto\032\014Client.proto\032\013hbase.proto" +
"\024GetRegionInfoRequest\022 \n\006region\030\001 \002(\0132\020." + "\032\tWAL.proto\"Q\n\024GetRegionInfoRequest\022 \n\006r" +
"RegionSpecifier\022\027\n\017compactionState\030\002 \001(\010" + "egion\030\001 \002(\0132\020.RegionSpecifier\022\027\n\017compact" +
"\"\301\001\n\025GetRegionInfoResponse\022\037\n\nregionInfo" + "ionState\030\002 \001(\010\"\301\001\n\025GetRegionInfoResponse" +
"\030\001 \002(\0132\013.RegionInfo\022?\n\017compactionState\030\002" + "\022\037\n\nregionInfo\030\001 \002(\0132\013.RegionInfo\022?\n\017com" +
" \001(\0162&.GetRegionInfoResponse.CompactionS" + "pactionState\030\002 \001(\0162&.GetRegionInfoRespon" +
"tate\"F\n\017CompactionState\022\010\n\004NONE\020\000\022\t\n\005MIN" + "se.CompactionState\"F\n\017CompactionState\022\010\n" +
"OR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_AND_MINOR\020\003\"G\n\023" + "\004NONE\020\000\022\t\n\005MINOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_A" +
"GetStoreFileRequest\022 \n\006region\030\001 \002(\0132\020.Re" + "ND_MINOR\020\003\"G\n\023GetStoreFileRequest\022 \n\006reg" +
"gionSpecifier\022\016\n\006family\030\002 \003(\014\")\n\024GetStor", "ion\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006family\030\002 ",
"eFileResponse\022\021\n\tstoreFile\030\001 \003(\t\"\030\n\026GetO" + "\003(\014\")\n\024GetStoreFileResponse\022\021\n\tstoreFile" +
"nlineRegionRequest\":\n\027GetOnlineRegionRes" + "\030\001 \003(\t\"\030\n\026GetOnlineRegionRequest\":\n\027GetO" +
"ponse\022\037\n\nregionInfo\030\001 \003(\0132\013.RegionInfo\"\270" + "nlineRegionResponse\022\037\n\nregionInfo\030\001 \003(\0132" +
"\001\n\021OpenRegionRequest\0223\n\010openInfo\030\001 \003(\0132!" + "\013.RegionInfo\"\270\001\n\021OpenRegionRequest\0223\n\010op" +
".OpenRegionRequest.RegionOpenInfo\032n\n\016Reg" + "enInfo\030\001 \003(\0132!.OpenRegionRequest.RegionO" +
"ionOpenInfo\022\033\n\006region\030\001 \002(\0132\013.RegionInfo" + "penInfo\032n\n\016RegionOpenInfo\022\033\n\006region\030\001 \002(" +
"\022\034\n\024versionOfOfflineNode\030\002 \001(\r\022!\n\014favore" + "\0132\013.RegionInfo\022\034\n\024versionOfOfflineNode\030\002" +
"dNodes\030\003 \003(\0132\013.ServerName\"\234\001\n\022OpenRegion" + " \001(\r\022!\n\014favoredNodes\030\003 \003(\0132\013.ServerName\"" +
"Response\022<\n\014openingState\030\001 \003(\0162&.OpenReg" + "\234\001\n\022OpenRegionResponse\022<\n\014openingState\030\001" +
"ionResponse.RegionOpeningState\"H\n\022Region", " \003(\0162&.OpenRegionResponse.RegionOpeningS",
"OpeningState\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPEN" + "tate\"H\n\022RegionOpeningState\022\n\n\006OPENED\020\000\022\022" +
"ED\020\001\022\022\n\016FAILED_OPENING\020\002\"\232\001\n\022CloseRegion" + "\n\016ALREADY_OPENED\020\001\022\022\n\016FAILED_OPENING\020\002\"\232" +
"Request\022 \n\006region\030\001 \002(\0132\020.RegionSpecifie" + "\001\n\022CloseRegionRequest\022 \n\006region\030\001 \002(\0132\020." +
"r\022\034\n\024versionOfClosingNode\030\002 \001(\r\022\034\n\016trans" + "RegionSpecifier\022\034\n\024versionOfClosingNode\030" +
"itionInZK\030\003 \001(\010:\004true\022&\n\021destinationServ" + "\002 \001(\r\022\034\n\016transitionInZK\030\003 \001(\010:\004true\022&\n\021d" +
"er\030\004 \001(\0132\013.ServerName\"%\n\023CloseRegionResp" + "estinationServer\030\004 \001(\0132\013.ServerName\"%\n\023C" +
"onse\022\016\n\006closed\030\001 \002(\010\"M\n\022FlushRegionReque" + "loseRegionResponse\022\016\n\006closed\030\001 \002(\010\"M\n\022Fl" +
"st\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\025\n\r" + "ushRegionRequest\022 \n\006region\030\001 \002(\0132\020.Regio" +
"ifOlderThanTs\030\002 \001(\004\"=\n\023FlushRegionRespon" + "nSpecifier\022\025\n\rifOlderThanTs\030\002 \001(\004\"=\n\023Flu" +
"se\022\025\n\rlastFlushTime\030\001 \002(\004\022\017\n\007flushed\030\002 \001", "shRegionResponse\022\025\n\rlastFlushTime\030\001 \002(\004\022",
"(\010\"J\n\022SplitRegionRequest\022 \n\006region\030\001 \002(\013" + "\017\n\007flushed\030\002 \001(\010\"J\n\022SplitRegionRequest\022 " +
"2\020.RegionSpecifier\022\022\n\nsplitPoint\030\002 \001(\014\"\025" + "\n\006region\030\001 \002(\0132\020.RegionSpecifier\022\022\n\nspli" +
"\n\023SplitRegionResponse\"W\n\024CompactRegionRe" + "tPoint\030\002 \001(\014\"\025\n\023SplitRegionResponse\"W\n\024C" +
"quest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022" + "ompactRegionRequest\022 \n\006region\030\001 \002(\0132\020.Re" +
"\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025Compac" + "gionSpecifier\022\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003" +
"tRegionResponse\"t\n\023MergeRegionsRequest\022!" + " \001(\014\"\027\n\025CompactRegionResponse\"t\n\023MergeRe" +
"\n\007regionA\030\001 \002(\0132\020.RegionSpecifier\022!\n\007reg" + "gionsRequest\022!\n\007regionA\030\001 \002(\0132\020.RegionSp" +
"ionB\030\002 \002(\0132\020.RegionSpecifier\022\027\n\010forcible" + "ecifier\022!\n\007regionB\030\002 \002(\0132\020.RegionSpecifi" +
"\030\003 \001(\010:\005false\"\026\n\024MergeRegionsResponse\"7\n" + "er\022\027\n\010forcible\030\003 \001(\010:\005false\"\026\n\024MergeRegi" +
"\010WALEntry\022\024\n\003key\030\001 \002(\0132\007.WALKey\022\025\n\rkeyVa", "onsResponse\"7\n\010WALEntry\022\024\n\003key\030\001 \002(\0132\007.W",
"lueBytes\030\002 \003(\014\"4\n\030ReplicateWALEntryReque" + "ALKey\022\025\n\rkeyValueBytes\030\002 \003(\014\"4\n\030Replicat" +
"st\022\030\n\005entry\030\001 \003(\0132\t.WALEntry\"\033\n\031Replicat" + "eWALEntryRequest\022\030\n\005entry\030\001 \003(\0132\t.WALEnt" +
"eWALEntryResponse\"\026\n\024RollWALWriterReques" + "ry\"\033\n\031ReplicateWALEntryResponse\"\026\n\024RollW" +
"t\".\n\025RollWALWriterResponse\022\025\n\rregionToFl" + "ALWriterRequest\".\n\025RollWALWriterResponse" +
"ush\030\001 \003(\014\"#\n\021StopServerRequest\022\016\n\006reason" + "\022\025\n\rregionToFlush\030\001 \003(\014\"#\n\021StopServerReq" +
"\030\001 \002(\t\"\024\n\022StopServerResponse\"\026\n\024GetServe" + "uest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServerRespon" +
"rInfoRequest\"@\n\nServerInfo\022\037\n\nserverName" + "se\"\026\n\024GetServerInfoRequest\"@\n\nServerInfo" +
"\030\001 \002(\0132\013.ServerName\022\021\n\twebuiPort\030\002 \001(\r\"8" + "\022\037\n\nserverName\030\001 \002(\0132\013.ServerName\022\021\n\tweb" +
"\n\025GetServerInfoResponse\022\037\n\nserverInfo\030\001 " + "uiPort\030\002 \001(\r\"8\n\025GetServerInfoResponse\022\037\n" +
"\002(\0132\013.ServerInfo2\266\006\n\014AdminService\022>\n\rget", "\nserverInfo\030\001 \002(\0132\013.ServerInfo2\337\006\n\014Admin",
"RegionInfo\022\025.GetRegionInfoRequest\032\026.GetR" + "Service\022>\n\rgetRegionInfo\022\025.GetRegionInfo" +
"egionInfoResponse\022;\n\014getStoreFile\022\024.GetS" + "Request\032\026.GetRegionInfoResponse\022;\n\014getSt" +
"toreFileRequest\032\025.GetStoreFileResponse\022D" + "oreFile\022\024.GetStoreFileRequest\032\025.GetStore" +
"\n\017getOnlineRegion\022\027.GetOnlineRegionReque" + "FileResponse\022D\n\017getOnlineRegion\022\027.GetOnl" +
"st\032\030.GetOnlineRegionResponse\0225\n\nopenRegi" + "ineRegionRequest\032\030.GetOnlineRegionRespon" +
"on\022\022.OpenRegionRequest\032\023.OpenRegionRespo" + "se\0225\n\nopenRegion\022\022.OpenRegionRequest\032\023.O" +
"nse\0228\n\013closeRegion\022\023.CloseRegionRequest\032" + "penRegionResponse\0228\n\013closeRegion\022\023.Close" +
"\024.CloseRegionResponse\0228\n\013flushRegion\022\023.F" + "RegionRequest\032\024.CloseRegionResponse\0228\n\013f" +
"lushRegionRequest\032\024.FlushRegionResponse\022" + "lushRegion\022\023.FlushRegionRequest\032\024.FlushR" +
"8\n\013splitRegion\022\023.SplitRegionRequest\032\024.Sp", "egionResponse\0228\n\013splitRegion\022\023.SplitRegi",
"litRegionResponse\022>\n\rcompactRegion\022\025.Com" + "onRequest\032\024.SplitRegionResponse\022>\n\rcompa" +
"pactRegionRequest\032\026.CompactRegionRespons" + "ctRegion\022\025.CompactRegionRequest\032\026.Compac" +
"e\022;\n\014mergeRegions\022\024.MergeRegionsRequest\032" + "tRegionResponse\022;\n\014mergeRegions\022\024.MergeR" +
"\025.MergeRegionsResponse\022J\n\021replicateWALEn" + "egionsRequest\032\025.MergeRegionsResponse\022J\n\021" +
"try\022\031.ReplicateWALEntryRequest\032\032.Replica" + "replicateWALEntry\022\031.ReplicateWALEntryReq" +
"teWALEntryResponse\022>\n\rrollWALWriter\022\025.Ro" + "uest\032\032.ReplicateWALEntryResponse\022\'\n\006repl" +
"llWALWriterRequest\032\026.RollWALWriterRespon" + "ay\022\r.MultiRequest\032\016.MultiResponse\022>\n\rrol" +
"se\022>\n\rgetServerInfo\022\025.GetServerInfoReque" + "lWALWriter\022\025.RollWALWriterRequest\032\026.Roll" +
"st\032\026.GetServerInfoResponse\0225\n\nstopServer" + "WALWriterResponse\022>\n\rgetServerInfo\022\025.Get" +
"\022\022.StopServerRequest\032\023.StopServerRespons", "ServerInfoRequest\032\026.GetServerInfoRespons",
"eBA\n*org.apache.hadoop.hbase.protobuf.ge" + "e\0225\n\nstopServer\022\022.StopServerRequest\032\023.St" +
"neratedB\013AdminProtosH\001\210\001\001\240\001\001" "opServerResponseBA\n*org.apache.hadoop.hb" +
"ase.protobuf.generatedB\013AdminProtosH\001\210\001\001" +
"\240\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -15571,6 +15638,7 @@ public final class AdminProtos {
com.google.protobuf.Descriptors.FileDescriptor com.google.protobuf.Descriptors.FileDescriptor
.internalBuildGeneratedFileFrom(descriptorData, .internalBuildGeneratedFileFrom(descriptorData,
new com.google.protobuf.Descriptors.FileDescriptor[] { new com.google.protobuf.Descriptors.FileDescriptor[] {
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.getDescriptor(),
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
org.apache.hadoop.hbase.protobuf.generated.WALProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.WALProtos.getDescriptor(),
}, assigner); }, assigner);

View File

@ -3132,6 +3132,10 @@ public final class HBaseProtos {
// optional bool split = 6; // optional bool split = 6;
boolean hasSplit(); boolean hasSplit();
boolean getSplit(); boolean getSplit();
// optional bool recovering = 7;
boolean hasRecovering();
boolean getRecovering();
} }
public static final class RegionInfo extends public static final class RegionInfo extends
com.google.protobuf.GeneratedMessage com.google.protobuf.GeneratedMessage
@ -3222,6 +3226,16 @@ public final class HBaseProtos {
return split_; return split_;
} }
// optional bool recovering = 7;
public static final int RECOVERING_FIELD_NUMBER = 7;
private boolean recovering_;
public boolean hasRecovering() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
public boolean getRecovering() {
return recovering_;
}
private void initFields() { private void initFields() {
regionId_ = 0L; regionId_ = 0L;
tableName_ = com.google.protobuf.ByteString.EMPTY; tableName_ = com.google.protobuf.ByteString.EMPTY;
@ -3229,6 +3243,7 @@ public final class HBaseProtos {
endKey_ = com.google.protobuf.ByteString.EMPTY; endKey_ = com.google.protobuf.ByteString.EMPTY;
offline_ = false; offline_ = false;
split_ = false; split_ = false;
recovering_ = false;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -3268,6 +3283,9 @@ public final class HBaseProtos {
if (((bitField0_ & 0x00000020) == 0x00000020)) { if (((bitField0_ & 0x00000020) == 0x00000020)) {
output.writeBool(6, split_); output.writeBool(6, split_);
} }
if (((bitField0_ & 0x00000040) == 0x00000040)) {
output.writeBool(7, recovering_);
}
getUnknownFields().writeTo(output); getUnknownFields().writeTo(output);
} }
@ -3301,6 +3319,10 @@ public final class HBaseProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeBoolSize(6, split_); .computeBoolSize(6, split_);
} }
if (((bitField0_ & 0x00000040) == 0x00000040)) {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(7, recovering_);
}
size += getUnknownFields().getSerializedSize(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -3354,6 +3376,11 @@ public final class HBaseProtos {
result = result && (getSplit() result = result && (getSplit()
== other.getSplit()); == other.getSplit());
} }
result = result && (hasRecovering() == other.hasRecovering());
if (hasRecovering()) {
result = result && (getRecovering()
== other.getRecovering());
}
result = result && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -3387,6 +3414,10 @@ public final class HBaseProtos {
hash = (37 * hash) + SPLIT_FIELD_NUMBER; hash = (37 * hash) + SPLIT_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getSplit()); hash = (53 * hash) + hashBoolean(getSplit());
} }
if (hasRecovering()) {
hash = (37 * hash) + RECOVERING_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getRecovering());
}
hash = (29 * hash) + getUnknownFields().hashCode(); hash = (29 * hash) + getUnknownFields().hashCode();
return hash; return hash;
} }
@ -3515,6 +3546,8 @@ public final class HBaseProtos {
bitField0_ = (bitField0_ & ~0x00000010); bitField0_ = (bitField0_ & ~0x00000010);
split_ = false; split_ = false;
bitField0_ = (bitField0_ & ~0x00000020); bitField0_ = (bitField0_ & ~0x00000020);
recovering_ = false;
bitField0_ = (bitField0_ & ~0x00000040);
return this; return this;
} }
@ -3577,6 +3610,10 @@ public final class HBaseProtos {
to_bitField0_ |= 0x00000020; to_bitField0_ |= 0x00000020;
} }
result.split_ = split_; result.split_ = split_;
if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
to_bitField0_ |= 0x00000040;
}
result.recovering_ = recovering_;
result.bitField0_ = to_bitField0_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -3611,6 +3648,9 @@ public final class HBaseProtos {
if (other.hasSplit()) { if (other.hasSplit()) {
setSplit(other.getSplit()); setSplit(other.getSplit());
} }
if (other.hasRecovering()) {
setRecovering(other.getRecovering());
}
this.mergeUnknownFields(other.getUnknownFields()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -3680,6 +3720,11 @@ public final class HBaseProtos {
split_ = input.readBool(); split_ = input.readBool();
break; break;
} }
case 56: {
bitField0_ |= 0x00000040;
recovering_ = input.readBool();
break;
}
} }
} }
} }
@ -3821,6 +3866,27 @@ public final class HBaseProtos {
return this; return this;
} }
// optional bool recovering = 7;
private boolean recovering_ ;
public boolean hasRecovering() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
public boolean getRecovering() {
return recovering_;
}
public Builder setRecovering(boolean value) {
bitField0_ |= 0x00000040;
recovering_ = value;
onChanged();
return this;
}
public Builder clearRecovering() {
bitField0_ = (bitField0_ & ~0x00000040);
recovering_ = false;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:RegionInfo) // @@protoc_insertion_point(builder_scope:RegionInfo)
} }
@ -14719,60 +14785,60 @@ public final class HBaseProtos {
"hema\022&\n\rconfiguration\030\004 \003(\0132\017.NameString" + "hema\022&\n\rconfiguration\030\004 \003(\0132\017.NameString" +
"Pair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(\014" + "Pair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(\014" +
"\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022&\n" + "\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022&\n" +
"\rconfiguration\030\003 \003(\0132\017.NameStringPair\"s\n", "\rconfiguration\030\003 \003(\0132\017.NameStringPair\"\207\001",
"\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableNa" + "\n\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableN" +
"me\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 \001" + "ame\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 " +
"(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\"0\n\014Fa" + "\001(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\022\n\nr" +
"voredNodes\022 \n\013favoredNode\030\001 \003(\0132\013.Server" + "ecovering\030\007 \001(\010\"0\n\014FavoredNodes\022 \n\013favor" +
"Name\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001 \002(\0162$" + "edNode\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpec" +
".RegionSpecifier.RegionSpecifierType\022\r\n\005" + "ifier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Re" +
"value\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017\n\013RE" + "gionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regi" +
"GION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"\260\003\n" + "onSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCO" +
"\nRegionLoad\022)\n\017regionSpecifier\030\001 \002(\0132\020.R" + "DED_REGION_NAME\020\002\"\260\003\n\nRegionLoad\022)\n\017regi" +
"egionSpecifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstoref", "onSpecifier\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006s",
"iles\030\003 \001(\r\022\037\n\027storeUncompressedSizeMB\030\004 " + "tores\030\002 \001(\r\022\022\n\nstorefiles\030\003 \001(\r\022\037\n\027store" +
"\001(\r\022\027\n\017storefileSizeMB\030\005 \001(\r\022\026\n\016memstore" + "UncompressedSizeMB\030\004 \001(\r\022\027\n\017storefileSiz" +
"SizeMB\030\006 \001(\r\022\034\n\024storefileIndexSizeMB\030\007 \001" + "eMB\030\005 \001(\r\022\026\n\016memstoreSizeMB\030\006 \001(\r\022\034\n\024sto" +
"(\r\022\031\n\021readRequestsCount\030\010 \001(\004\022\032\n\022writeRe" + "refileIndexSizeMB\030\007 \001(\r\022\031\n\021readRequestsC" +
"questsCount\030\t \001(\004\022\032\n\022totalCompactingKVs\030" + "ount\030\010 \001(\004\022\032\n\022writeRequestsCount\030\t \001(\004\022\032" +
"\n \001(\004\022\033\n\023currentCompactedKVs\030\013 \001(\004\022\027\n\017ro" + "\n\022totalCompactingKVs\030\n \001(\004\022\033\n\023currentCom" +
"otIndexSizeKB\030\014 \001(\r\022\036\n\026totalStaticIndexS" + "pactedKVs\030\013 \001(\004\022\027\n\017rootIndexSizeKB\030\014 \001(\r" +
"izeKB\030\r \001(\r\022\036\n\026totalStaticBloomSizeKB\030\016 " + "\022\036\n\026totalStaticIndexSizeKB\030\r \001(\r\022\036\n\026tota" +
"\001(\r\022\032\n\022completeSequenceId\030\017 \001(\004\"\372\001\n\nServ" + "lStaticBloomSizeKB\030\016 \001(\r\022\032\n\022completeSequ" +
"erLoad\022\030\n\020numberOfRequests\030\001 \001(\r\022\035\n\025tota", "enceId\030\017 \001(\004\"\372\001\n\nServerLoad\022\030\n\020numberOfR",
"lNumberOfRequests\030\002 \001(\r\022\022\n\nusedHeapMB\030\003 " + "equests\030\001 \001(\r\022\035\n\025totalNumberOfRequests\030\002" +
"\001(\r\022\021\n\tmaxHeapMB\030\004 \001(\r\022 \n\013regionLoads\030\005 " + " \001(\r\022\022\n\nusedHeapMB\030\003 \001(\r\022\021\n\tmaxHeapMB\030\004 " +
"\003(\0132\013.RegionLoad\022\"\n\014coprocessors\030\006 \003(\0132\014" + "\001(\r\022 \n\013regionLoads\030\005 \003(\0132\013.RegionLoad\022\"\n" +
".Coprocessor\022\027\n\017reportStartTime\030\007 \001(\004\022\025\n" + "\014coprocessors\030\006 \003(\0132\014.Coprocessor\022\027\n\017rep" +
"\rreportEndTime\030\010 \001(\004\022\026\n\016infoServerPort\030\t" + "ortStartTime\030\007 \001(\004\022\025\n\rreportEndTime\030\010 \001(" +
" \001(\r\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 " + "\004\022\026\n\016infoServerPort\030\t \001(\r\"%\n\tTimeRange\022\014" +
"\001(\004\"0\n\006Filter\022\014\n\004name\030\001 \002(\t\022\030\n\020serialize" + "\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"0\n\006Filter\022\014\n\004na" +
"dFilter\030\002 \001(\014\"x\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016" + "me\030\001 \002(\t\022\030\n\020serializedFilter\030\002 \001(\014\"x\n\010Ke" +
"\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttim" + "yValue\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tq" +
"estamp\030\004 \001(\004\022\032\n\007keyType\030\005 \001(\0162\t.CellType", "ualifier\030\003 \002(\014\022\021\n\ttimestamp\030\004 \001(\004\022\032\n\007key",
"\022\r\n\005value\030\006 \001(\014\"?\n\nServerName\022\020\n\010hostNam" + "Type\030\005 \001(\0162\t.CellType\022\r\n\005value\030\006 \001(\014\"?\n\n" +
"e\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004" + "ServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n\004port\030\002 \001" +
"\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStr" + "(\r\022\021\n\tstartCode\030\003 \001(\004\"\033\n\013Coprocessor\022\014\n\004" +
"ingPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\r" + "name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004name\030\001 \002" +
"NameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001" + "(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair\022\014\n\004na" +
"(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006s" + "me\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBytesPai" +
"econd\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001" + "r\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n\rName" +
"(\t\022\r\n\005value\030\002 \001(\003\"\255\001\n\023SnapshotDescriptio" + "Int64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\255" +
"n\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\027\n\014creati" + "\001\n\023SnapshotDescription\022\014\n\004name\030\001 \002(\t\022\r\n\005" +
"onTime\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snapshot", "table\030\002 \001(\t\022\027\n\014creationTime\030\003 \001(\003:\0010\022.\n\004",
"Description.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005" + "type\030\004 \001(\0162\031.SnapshotDescription.Type:\005F" +
"\"\037\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\"\n\n\010Emp" + "LUSH\022\017\n\007version\030\005 \001(\005\"\037\n\004Type\022\014\n\010DISABLE" +
"tyMsg\"\032\n\007LongMsg\022\017\n\007longMsg\030\001 \002(\003\"&\n\rBig" + "D\020\000\022\t\n\005FLUSH\020\001\"\n\n\010EmptyMsg\"\032\n\007LongMsg\022\017\n" +
"DecimalMsg\022\025\n\rbigdecimalMsg\030\001 \002(\014\"1\n\004UUI" + "\007longMsg\030\001 \002(\003\"&\n\rBigDecimalMsg\022\025\n\rbigde" +
"D\022\024\n\014leastSigBits\030\001 \002(\004\022\023\n\013mostSigBits\030\002" + "cimalMsg\030\001 \002(\014\"1\n\004UUID\022\024\n\014leastSigBits\030\001" +
" \002(\004*`\n\010CellType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n" + " \002(\004\022\023\n\013mostSigBits\030\002 \002(\004*`\n\010CellType\022\013\n" +
"\n\006DELETE\020\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_" + "\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELET" +
"FAMILY\020\016\022\014\n\007MAXIMUM\020\377\001*r\n\013CompareType\022\010\n" + "E_COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM" +
"\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n" + "\020\377\001*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_" +
"\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GR", "EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GRE",
"EATER\020\005\022\t\n\005NO_OP\020\006B>\n*org.apache.hadoop." + "ATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006B" +
"hbase.protobuf.generatedB\013HBaseProtosH\001\240" + ">\n*org.apache.hadoop.hbase.protobuf.gene" +
"\001\001" "ratedB\013HBaseProtosH\001\240\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -14808,7 +14874,7 @@ public final class HBaseProtos {
internal_static_RegionInfo_fieldAccessorTable = new internal_static_RegionInfo_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RegionInfo_descriptor, internal_static_RegionInfo_descriptor,
new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", }, new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", "Recovering", },
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.class,
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder.class); org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder.class);
internal_static_FavoredNodes_descriptor = internal_static_FavoredNodes_descriptor =

View File

@ -24,6 +24,7 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true; option java_generate_equals_and_hash = true;
option optimize_for = SPEED; option optimize_for = SPEED;
import "Client.proto";
import "hbase.proto"; import "hbase.proto";
import "WAL.proto"; import "WAL.proto";
@ -234,6 +235,9 @@ service AdminService {
rpc replicateWALEntry(ReplicateWALEntryRequest) rpc replicateWALEntry(ReplicateWALEntryRequest)
returns(ReplicateWALEntryResponse); returns(ReplicateWALEntryResponse);
rpc replay(MultiRequest)
returns(MultiResponse);
rpc rollWALWriter(RollWALWriterRequest) rpc rollWALWriter(RollWALWriterRequest)
returns(RollWALWriterResponse); returns(RollWALWriterResponse);

View File

@ -81,6 +81,7 @@ message RegionInfo {
optional bytes endKey = 4; optional bytes endKey = 4;
optional bool offline = 5; optional bool offline = 5;
optional bool split = 6; optional bool split = 6;
optional bool recovering = 7;
} }
/** /**

View File

@ -422,7 +422,8 @@ public class AssignmentManager extends ZooKeeperListener {
return; return;
} }
boolean failover = !serverManager.getDeadServers().isEmpty(); boolean failover = (!serverManager.getDeadServers().isEmpty() || !serverManager
.getRequeuedDeadServers().isEmpty());
if (!failover) { if (!failover) {
// Run through all regions. If they are not assigned and not in RIT, then // Run through all regions. If they are not assigned and not in RIT, then
@ -2728,18 +2729,38 @@ public class AssignmentManager extends ZooKeeperListener {
*/ */
public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri) public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
throws IOException, InterruptedException { throws IOException, InterruptedException {
if (!regionStates.isRegionInTransition(hri)) return; waitOnRegionToClearRegionsInTransition(hri, -1L);
}
/**
* Wait on region to clear regions-in-transition or time out
* @param hri
* @param timeOut Milliseconds to wait for current region to be out of transition state.
* @return True when a region clears regions-in-transition before timeout otherwise false
* @throws IOException
* @throws InterruptedException
*/
public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
throws IOException, InterruptedException {
if (!regionStates.isRegionInTransition(hri)) return true;
RegionState rs = null; RegionState rs = null;
long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTimeMillis()
+ timeOut;
// There is already a timeout monitor on regions in transition so I // There is already a timeout monitor on regions in transition so I
// should not have to have one here too? // should not have to have one here too?
while(!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
LOG.info("Waiting on " + rs + " to clear regions-in-transition"); LOG.info("Waiting on " + rs + " to clear regions-in-transition");
while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
regionStates.waitForUpdate(100); regionStates.waitForUpdate(100);
if (EnvironmentEdgeManager.currentTimeMillis() > end) {
LOG.info("Timed out on waiting for region:" + hri.getEncodedName() + " to be assigned.");
return false;
}
} }
if (this.server.isStopped()) { if (this.server.isStopped()) {
LOG.info("Giving up wait on regions in " + LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
"transition because stoppable.isStopped is set"); return false;
} }
return true;
} }
/** /**

View File

@ -27,6 +27,7 @@ import java.net.UnknownHostException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -348,6 +349,14 @@ MasterServices, Server {
/** The health check chore. */ /** The health check chore. */
private HealthCheckChore healthCheckChore; private HealthCheckChore healthCheckChore;
/**
* is in distributedLogReplay mode. When true, SplitLogWorker directly replays WAL edits to newly
* assigned region servers instead of creating recovered.edits files.
*/
private final boolean distributedLogReplay;
/** flag used in test cases in order to simulate RS failures during master initialization */
private volatile boolean initializationBeforeMetaAssignment = false;
/** /**
* Initializes the HMaster. The steps are as follows: * Initializes the HMaster. The steps are as follows:
@ -451,6 +460,9 @@ MasterServices, Server {
clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass); clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
Threads.setDaemonThreadRunning(clusterStatusPublisherChore.getThread()); Threads.setDaemonThreadRunning(clusterStatusPublisherChore.getThread());
} }
distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
} }
/** /**
@ -769,18 +781,47 @@ MasterServices, Server {
this.assignmentManager.startTimeOutMonitor(); this.assignmentManager.startTimeOutMonitor();
} }
// TODO: Should do this in background rather than block master startup // get a list for previously failed RS which need log splitting work
status.setStatus("Splitting logs after master startup"); // we recover .META. region servers inside master initialization and
splitLogAfterStartup(this.fileSystemManager); // handle other failed servers in SSH in order to start up master node ASAP
Set<ServerName> previouslyFailedServers = this.fileSystemManager
.getFailedServersFromLogFolders();
// remove stale recovering regions from previous run
this.fileSystemManager.removeStaleRecoveringRegionsFromZK(previouslyFailedServers);
// log splitting for .META. server
ServerName oldMetaServerLocation = this.catalogTracker.getMetaLocation();
if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) {
splitMetaLogBeforeAssignment(oldMetaServerLocation);
// Note: we can't remove oldMetaServerLocation from previousFailedServers list because it
// may also host user regions
}
this.initializationBeforeMetaAssignment = true;
// Make sure meta assigned before proceeding. // Make sure meta assigned before proceeding.
if (!assignMeta(status)) return; status.setStatus("Assigning Meta Region");
assignMeta(status);
if (this.distributedLogReplay && oldMetaServerLocation != null
&& previouslyFailedServers.contains(oldMetaServerLocation)) {
// replay WAL edits mode need new .META. RS is assigned firstly
status.setStatus("replaying log for Meta Region");
this.fileSystemManager.splitMetaLog(oldMetaServerLocation);
}
enableServerShutdownHandler(); enableServerShutdownHandler();
status.setStatus("Submitting log splitting work for previously failed region servers");
// Master has recovered META region server and we put
// other failed region servers in a queue to be handled later by SSH
for (ServerName tmpServer : previouslyFailedServers) {
this.serverManager.processDeadServer(tmpServer, true);
}
// Update meta with new PB serialization if required. i.e migrate all HRI to PB serialization // Update meta with new PB serialization if required. i.e migrate all HRI to PB serialization
// in meta. This must happen before we assign all user regions or else the assignment will // in meta. This must happen before we assign all user regions or else the assignment will
// fail. // fail.
// TODO: Remove this after 0.96, when we do 0.98.
org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB
.updateMetaIfNecessary(this); .updateMetaIfNecessary(this);
@ -829,14 +870,6 @@ MasterServices, Server {
Threads.setDaemonThreadRunning(catalogJanitorChore.getThread()); Threads.setDaemonThreadRunning(catalogJanitorChore.getThread());
} }
/**
* Override to change master's splitLogAfterStartup. Used testing
* @param mfs
*/
protected void splitLogAfterStartup(final MasterFileSystem mfs) {
mfs.splitLogAfterStartup();
}
/** /**
* Create a {@link ServerManager} instance. * Create a {@link ServerManager} instance.
* @param master * @param master
@ -865,52 +898,66 @@ MasterServices, Server {
} }
/** /**
* Check <code>.META.</code> are assigned. If not, * Check <code>.META.</code> is assigned. If not, assign it.
* assign them. * @param status MonitoredTask
* @throws InterruptedException * @throws InterruptedException
* @throws IOException * @throws IOException
* @throws KeeperException * @throws KeeperException
* @return True if meta is healthy, assigned
*/ */
boolean assignMeta(MonitoredTask status) void assignMeta(MonitoredTask status)
throws InterruptedException, IOException, KeeperException { throws InterruptedException, IOException, KeeperException {
// Work on meta region
int assigned = 0; int assigned = 0;
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000); long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
boolean beingExpired = false;
// Work on .META. region. Is it in zk in transition?
status.setStatus("Assigning META region"); status.setStatus("Assigning META region");
assignmentManager.getRegionStates().createRegionState(
HRegionInfo.FIRST_META_REGIONINFO); assignmentManager.getRegionStates().createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
boolean rit = this.assignmentManager. boolean rit = this.assignmentManager
processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO); .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
ServerName currentMetaServer = null; boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
boolean metaRegionLocation = catalogTracker.verifyMetaRegionLocation(timeout);
if (!rit && !metaRegionLocation) { if (!rit && !metaRegionLocation) {
currentMetaServer = this.catalogTracker.getMetaLocation(); ServerName currentMetaServer = this.catalogTracker.getMetaLocation();
splitLogAndExpireIfOnline(currentMetaServer); if (currentMetaServer != null) {
this.assignmentManager.assignMeta(); beingExpired = expireIfOnline(currentMetaServer);
enableSSHandWaitForMeta(); }
if (beingExpired) {
splitMetaLogBeforeAssignment(currentMetaServer);
}
assignmentManager.assignMeta();
// Make sure a .META. location is set. // Make sure a .META. location is set.
if (!isMetaLocation()) return false; enableSSHandWaitForMeta();
// This guarantees that the transition assigning .META. has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
assigned++; assigned++;
if (beingExpired && this.distributedLogReplay) {
// In Replay WAL Mode, we need the new .META. server online
this.fileSystemManager.splitMetaLog(currentMetaServer);
}
} else if (rit && !metaRegionLocation) { } else if (rit && !metaRegionLocation) {
// Make sure a .META. location is set. // Make sure a .META. location is set.
if (!isMetaLocation()) return false; enableSSHandWaitForMeta();
// This guarantees that the transition assigning .META. has completed
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
assigned++; assigned++;
} else if (metaRegionLocation) { } else {
// Region already assigned. We didn't assign it. Add to in-memory state. // Region already assigned. We didn't assign it. Add to in-memory state.
this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO, this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
this.catalogTracker.getMetaLocation()); this.catalogTracker.getMetaLocation());
} }
enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME)); enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME));
LOG.info(".META. assigned=" + assigned + ", rit=" + rit + LOG.info(".META. assigned=" + assigned + ", rit=" + rit + ", location="
", location=" + catalogTracker.getMetaLocation()); + catalogTracker.getMetaLocation());
status.setStatus("META assigned."); status.setStatus("META assigned.");
return true; }
private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
if (this.distributedLogReplay) {
// In log replay mode, we mark META region as recovering in ZK
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
regions.add(HRegionInfo.FIRST_META_REGIONINFO);
this.fileSystemManager.prepareMetaLogReplay(currentMetaServer, regions);
} else {
// In recovered.edits mode: create recovered edits file for .META. server
this.fileSystemManager.splitMetaLog(currentMetaServer);
}
} }
private void enableSSHandWaitForMeta() throws IOException, InterruptedException { private void enableSSHandWaitForMeta() throws IOException, InterruptedException {
@ -921,24 +968,6 @@ MasterServices, Server {
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO); this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
} }
/**
* @return True if there a meta available
* @throws InterruptedException
*/
private boolean isMetaLocation() throws InterruptedException {
// Cycle up here in master rather than down in catalogtracker so we can
// check the master stopped flag every so often.
while (!this.stopped) {
try {
if (this.catalogTracker.waitForMeta(100) != null) break;
} catch (NotAllMetaRegionsOnlineException e) {
// Ignore. I know .META. is not online yet.
}
}
// We got here because we came of above loop.
return !this.stopped;
}
private void enableCatalogTables(String catalogTableName) { private void enableCatalogTables(String catalogTableName) {
if (!this.assignmentManager.getZKTable().isEnabledTable(catalogTableName)) { if (!this.assignmentManager.getZKTable().isEnabledTable(catalogTableName)) {
this.assignmentManager.setEnabledTable(catalogTableName); this.assignmentManager.setEnabledTable(catalogTableName);
@ -946,20 +975,19 @@ MasterServices, Server {
} }
/** /**
* Split a server's log and expire it if we find it is one of the online * Expire a server if we find it is one of the online servers.
* servers.
* @param sn ServerName to check. * @param sn ServerName to check.
* @return true when server <code>sn<code> is being expired by the function.
* @throws IOException * @throws IOException
*/ */
private void splitLogAndExpireIfOnline(final ServerName sn) private boolean expireIfOnline(final ServerName sn)
throws IOException { throws IOException {
if (sn == null || !serverManager.isServerOnline(sn)) { if (sn == null || !serverManager.isServerOnline(sn)) {
return; return false;
} }
LOG.info("Forcing splitLog and expire of " + sn); LOG.info("Forcing expire of " + sn);
fileSystemManager.splitMetaLog(sn);
fileSystemManager.splitLog(sn);
serverManager.expireServer(sn); serverManager.expireServer(sn);
return true;
} }
@Override @Override
@ -2235,6 +2263,14 @@ MasterServices, Server {
return this.serverShutdownHandlerEnabled; return this.serverShutdownHandlerEnabled;
} }
/**
* Report whether this master has started initialization and is about to do meta region assignment
* @return true if master is in initialization & about to assign META regions
*/
public boolean isInitializationStartsMetaRegionAssignment() {
return this.initializationBeforeMetaAssignment;
}
@Override @Override
public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest req) public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest req)
throws ServiceException { throws ServiceException {
@ -2678,4 +2714,5 @@ MasterServices, Server {
String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC); String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation); return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
} }
} }

View File

@ -20,7 +20,9 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.NavigableMap;
import java.util.Set; import java.util.Set;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
@ -34,26 +36,29 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.exceptions.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.exceptions.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.exceptions.OrphanHLogAfterSplitException;
import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.exceptions.OrphanHLogAfterSplitException;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.zookeeper.KeeperException;
/** /**
* This class abstracts a bunch of operations the HMaster needs to interact with * This class abstracts a bunch of operations the HMaster needs to interact with
@ -83,6 +88,7 @@ public class MasterFileSystem {
private final Path tempdir; private final Path tempdir;
// create the split log lock // create the split log lock
final Lock splitLogLock = new ReentrantLock(); final Lock splitLogLock = new ReentrantLock();
final boolean distributedLogReplay;
final boolean distributedLogSplitting; final boolean distributedLogSplitting;
final SplitLogManager splitLogManager; final SplitLogManager splitLogManager;
private final MasterServices services; private final MasterServices services;
@ -118,15 +124,14 @@ public class MasterFileSystem {
FSUtils.setFsDefault(conf, new Path(this.fs.getUri())); FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
// make sure the fs has the same conf // make sure the fs has the same conf
fs.setConf(conf); fs.setConf(conf);
this.distributedLogSplitting = this.splitLogManager = new SplitLogManager(master.getZooKeeper(), master.getConfiguration(),
conf.getBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, true); master, services, master.getServerName());
this.distributedLogSplitting = conf.getBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, true);
if (this.distributedLogSplitting) { if (this.distributedLogSplitting) {
this.splitLogManager = new SplitLogManager(master.getZooKeeper(),
master.getConfiguration(), master, services, master.getServerName());
this.splitLogManager.finishInitialization(masterRecovery); this.splitLogManager.finishInitialization(masterRecovery);
} else {
this.splitLogManager = null;
} }
this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
// setup the filesystem variable // setup the filesystem variable
// set up the archived logs path // set up the archived logs path
this.oldLogDir = createInitialFileSystemLayout(); this.oldLogDir = createInitialFileSystemLayout();
@ -212,21 +217,23 @@ public class MasterFileSystem {
} }
/** /**
* Inspect the log directory to recover any log file without * Inspect the log directory to find dead servers which need recovery work
* an active region server. * @return A set of ServerNames which aren't running but still have WAL files left in file system
*/ */
void splitLogAfterStartup() { Set<ServerName> getFailedServersFromLogFolders() {
boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors", boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors",
HLog.SPLIT_SKIP_ERRORS_DEFAULT); HLog.SPLIT_SKIP_ERRORS_DEFAULT);
Set<ServerName> serverNames = new HashSet<ServerName>();
Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME); Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
do { do {
if (master.isStopped()) { if (master.isStopped()) {
LOG.warn("Master stopped while splitting logs"); LOG.warn("Master stopped while trying to get failed servers.");
break; break;
} }
List<ServerName> serverNames = new ArrayList<ServerName>();
try { try {
if (!this.fs.exists(logsDirPath)) return; if (!this.fs.exists(logsDirPath)) return serverNames;
FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null); FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
// Get online servers after getting log folders to avoid log folder deletion of newly // Get online servers after getting log folders to avoid log folder deletion of newly
// checked in region servers . see HBASE-5916 // checked in region servers . see HBASE-5916
@ -235,7 +242,7 @@ public class MasterFileSystem {
if (logFolders == null || logFolders.length == 0) { if (logFolders == null || logFolders.length == 0) {
LOG.debug("No log files to split, proceeding..."); LOG.debug("No log files to split, proceeding...");
return; return serverNames;
} }
for (FileStatus status : logFolders) { for (FileStatus status : logFolders) {
String sn = status.getPath().getName(); String sn = status.getPath().getName();
@ -249,23 +256,19 @@ public class MasterFileSystem {
+ "to a known region server, splitting"); + "to a known region server, splitting");
serverNames.add(serverName); serverNames.add(serverName);
} else { } else {
LOG.info("Log folder " + status.getPath() LOG.info("Log folder " + status.getPath() + " belongs to an existing region server");
+ " belongs to an existing region server");
} }
} }
splitLog(serverNames, META_FILTER);
splitLog(serverNames, NON_META_FILTER);
retrySplitting = false; retrySplitting = false;
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.warn("Failed splitting of " + serverNames, ioe); LOG.warn("Failed getting failed servers to be recovered.", ioe);
if (!checkFileSystem()) { if (!checkFileSystem()) {
LOG.warn("Bad Filesystem, exiting"); LOG.warn("Bad Filesystem, exiting");
Runtime.getRuntime().halt(1); Runtime.getRuntime().halt(1);
} }
try { try {
if (retrySplitting) { if (retrySplitting) {
Thread.sleep(conf.getInt( Thread.sleep(conf.getInt("hbase.hlog.split.failure.retry.interval", 30 * 1000));
"hbase.hlog.split.failure.retry.interval", 30 * 1000));
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.warn("Interrupted, aborting since cannot return w/o splitting"); LOG.warn("Interrupted, aborting since cannot return w/o splitting");
@ -275,10 +278,12 @@ public class MasterFileSystem {
} }
} }
} while (retrySplitting); } while (retrySplitting);
return serverNames;
} }
public void splitLog(final ServerName serverName) throws IOException { public void splitLog(final ServerName serverName) throws IOException {
List<ServerName> serverNames = new ArrayList<ServerName>(); Set<ServerName> serverNames = new HashSet<ServerName>();
serverNames.add(serverName); serverNames.add(serverName);
splitLog(serverNames); splitLog(serverNames);
} }
@ -290,23 +295,20 @@ public class MasterFileSystem {
*/ */
public void splitMetaLog(final ServerName serverName) throws IOException { public void splitMetaLog(final ServerName serverName) throws IOException {
long splitTime = 0, splitLogSize = 0; long splitTime = 0, splitLogSize = 0;
List<ServerName> serverNames = new ArrayList<ServerName>(); Set<ServerName> serverNames = new HashSet<ServerName>();
serverNames.add(serverName); serverNames.add(serverName);
List<Path> logDirs = getLogDirs(serverNames); List<Path> logDirs = getLogDirs(serverNames);
if (logDirs.isEmpty()) {
LOG.info("No meta logs to split");
return;
}
splitLogManager.handleDeadWorkers(serverNames); splitLogManager.handleDeadWorkers(serverNames);
splitTime = EnvironmentEdgeManager.currentTimeMillis(); splitTime = EnvironmentEdgeManager.currentTimeMillis();
splitLogSize = splitLogManager.splitLogDistributed(logDirs, META_FILTER); splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, META_FILTER);
splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime; splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime;
if (this.metricsMaster != null) { if (this.metricsMaster != null) {
this.metricsMaster.addSplit(splitTime, splitLogSize); this.metricsMaster.addMetaWALSplit(splitTime, splitLogSize);
} }
} }
private List<Path> getLogDirs(final List<ServerName> serverNames) throws IOException { private List<Path> getLogDirs(final Set<ServerName> serverNames) throws IOException {
List<Path> logDirs = new ArrayList<Path>(); List<Path> logDirs = new ArrayList<Path>();
for (ServerName serverName: serverNames) { for (ServerName serverName: serverNames) {
Path logDir = new Path(this.rootdir, HLogUtil.getHLogDirectoryName(serverName.toString())); Path logDir = new Path(this.rootdir, HLogUtil.getHLogDirectoryName(serverName.toString()));
@ -327,30 +329,79 @@ public class MasterFileSystem {
return logDirs; return logDirs;
} }
public void splitLog(final List<ServerName> serverNames) throws IOException { /**
* Mark regions in recovering state when distributedLogReplay are set true
* @param serverNames Set of ServerNames to be replayed wals in order to recover changes contained
* in them
* @throws IOException
*/
public void prepareLogReplay(Set<ServerName> serverNames) throws IOException {
if (!this.distributedLogReplay) {
return;
}
// mark regions in recovering state
for (ServerName serverName : serverNames) {
NavigableMap<HRegionInfo, Result> regions = this.getServerUserRegions(serverName);
if (regions == null) {
continue;
}
try {
this.splitLogManager.markRegionsRecoveringInZK(serverName, regions.keySet());
} catch (KeeperException e) {
throw new IOException(e);
}
}
}
/**
* Mark meta regions in recovering state when distributedLogReplay are set true. The function is used
* when {@link #getServerUserRegions(ServerName)} can't be used in case meta RS is down.
* @param serverName
* @param regions
* @throws IOException
*/
public void prepareMetaLogReplay(ServerName serverName, Set<HRegionInfo> regions)
throws IOException {
if (!this.distributedLogReplay || (regions == null)) {
return;
}
// mark regions in recovering state
try {
this.splitLogManager.markRegionsRecoveringInZK(serverName, regions);
} catch (KeeperException e) {
throw new IOException(e);
}
}
public void splitLog(final Set<ServerName> serverNames) throws IOException {
splitLog(serverNames, NON_META_FILTER); splitLog(serverNames, NON_META_FILTER);
} }
/** /**
* This method is the base split method that splits HLog files matching a filter. * Wrapper function on {@link SplitLogManager#removeStaleRecoveringRegionsFromZK(Set)}
* Callers should pass the appropriate filter for meta and non-meta HLogs. * @param failedServers
* @throws KeeperException
*/
void removeStaleRecoveringRegionsFromZK(final Set<ServerName> failedServers)
throws KeeperException {
this.splitLogManager.removeStaleRecoveringRegionsFromZK(failedServers);
}
/**
* 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 serverNames
* @param filter * @param filter
* @throws IOException * @throws IOException
*/ */
public void splitLog(final List<ServerName> serverNames, PathFilter filter) throws IOException { public void splitLog(final Set<ServerName> serverNames, PathFilter filter) throws IOException {
long splitTime = 0, splitLogSize = 0; long splitTime = 0, splitLogSize = 0;
List<Path> logDirs = getLogDirs(serverNames); List<Path> logDirs = getLogDirs(serverNames);
if (logDirs.isEmpty()) {
LOG.info("No logs to split");
return;
}
if (distributedLogSplitting) { if (distributedLogSplitting) {
splitLogManager.handleDeadWorkers(serverNames); splitLogManager.handleDeadWorkers(serverNames);
splitTime = EnvironmentEdgeManager.currentTimeMillis(); splitTime = EnvironmentEdgeManager.currentTimeMillis();
splitLogSize = splitLogManager.splitLogDistributed(logDirs,filter); splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, filter);
splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime; splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime;
} else { } else {
for(Path logDir: logDirs){ for(Path logDir: logDirs){
@ -358,8 +409,8 @@ public class MasterFileSystem {
// one at a time // one at a time
this.splitLogLock.lock(); this.splitLogLock.lock();
try { try {
HLogSplitter splitter = HLogSplitter.createLogSplitter( HLogSplitter splitter = HLogSplitter.createLogSplitter(conf, rootdir, logDir, oldLogDir,
conf, rootdir, logDir, oldLogDir, this.fs); this.fs);
try { try {
// If FS is in safe mode, just wait till out of it. // If FS is in safe mode, just wait till out of it.
FSUtils.waitOnSafeMode(conf, conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 1000)); FSUtils.waitOnSafeMode(conf, conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 1000));
@ -380,9 +431,13 @@ public class MasterFileSystem {
} }
if (this.metricsMaster != null) { if (this.metricsMaster != null) {
if (filter == this.META_FILTER) {
this.metricsMaster.addMetaWALSplit(splitTime, splitLogSize);
} else {
this.metricsMaster.addSplit(splitTime, splitLogSize); this.metricsMaster.addSplit(splitTime, splitLogSize);
} }
} }
}
/** /**
* Get the rootdir. Make sure its wholesome and exists before returning. * Get the rootdir. Make sure its wholesome and exists before returning.
@ -648,4 +703,18 @@ public class MasterFileSystem {
this.services.getTableDescriptors().add(htd); this.services.getTableDescriptors().add(htd);
return htd; return htd;
} }
private NavigableMap<HRegionInfo, Result> getServerUserRegions(ServerName serverName)
throws IOException {
if (!this.master.isStopped()) {
try {
this.master.getCatalogTracker().waitForMeta();
return MetaReader.getServerUserRegions(this.master.getCatalogTracker(), serverName);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new IOException("Interrupted", e);
}
}
return null;
}
} }

View File

@ -181,4 +181,9 @@ public interface MasterServices extends Server {
public void dispatchMergingRegions(final HRegionInfo region_a, public void dispatchMergingRegions(final HRegionInfo region_a,
final HRegionInfo region_b, final boolean forcible) throws IOException; final HRegionInfo region_b, final boolean forcible) throws IOException;
/**
* @return true if master is initialized
*/
public boolean isInitialized();
} }

View File

@ -58,6 +58,16 @@ public class MetricsMaster {
masterSource.updateSplitSize(size); masterSource.updateSplitSize(size);
} }
/**
* Record a single instance of a split
* @param time time that the split took
* @param size length of original HLogs that were split
*/
public synchronized void addMetaWALSplit(long time, long size) {
masterSource.updateMetaWALSplitTime(time);
masterSource.updateMetaWALSplitSize(size);
}
/** /**
* @param inc How much to add to requests. * @param inc How much to add to requests.
*/ */

View File

@ -154,21 +154,21 @@ public class ServerManager {
private Set<ServerName> queuedDeadServers = new HashSet<ServerName>(); private Set<ServerName> queuedDeadServers = new HashSet<ServerName>();
/** /**
* Set of region servers which are dead and submitted to ServerShutdownHandler to * Set of region servers which are dead and submitted to ServerShutdownHandler to process but not
* process but not fully processed immediately. * fully processed immediately.
* <p> * <p>
* If one server died before assignment manager finished the failover cleanup, the server * If one server died before assignment manager finished the failover cleanup, the server will be
* will be added to this set and will be processed through calling * added to this set and will be processed through calling
* {@link ServerManager#processQueuedDeadServers()} by assignment manager. * {@link ServerManager#processQueuedDeadServers()} by assignment manager.
* <p> * <p>
* For all the region servers in this set, HLog split is already completed. * The Boolean value indicates whether log split is needed inside ServerShutdownHandler
* <p> * <p>
* ServerShutdownHandler processes a dead server submitted to the handler after * ServerShutdownHandler processes a dead server submitted to the handler after the handler is
* the handler is enabled. It may not be able to complete the processing because meta * enabled. It may not be able to complete the processing because meta is not yet online or master
* is not yet online or master is currently in startup mode. In this case, the dead * is currently in startup mode. In this case, the dead server will be parked in this set
* server will be parked in this set temporarily. * temporarily.
*/ */
private Set<ServerName> requeuedDeadServers = new HashSet<ServerName>(); private Map<ServerName, Boolean> requeuedDeadServers = new HashMap<ServerName, Boolean>();
/** /**
* Constructor. * Constructor.
@ -513,6 +513,10 @@ public class ServerManager {
} }
public synchronized void processDeadServer(final ServerName serverName) { public synchronized void processDeadServer(final ServerName serverName) {
this.processDeadServer(serverName, false);
}
public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitHlog) {
// When assignment manager is cleaning up the zookeeper nodes and rebuilding the // When assignment manager is cleaning up the zookeeper nodes and rebuilding the
// in-memory region states, region servers could be down. Meta table can and // in-memory region states, region servers could be down. Meta table can and
// should be re-assigned, log splitting can be done too. However, it is better to // should be re-assigned, log splitting can be done too. However, it is better to
@ -522,13 +526,14 @@ public class ServerManager {
// the handler threads and meta table could not be re-assigned in case // the handler threads and meta table could not be re-assigned in case
// the corresponding server is down. So we queue them up here instead. // the corresponding server is down. So we queue them up here instead.
if (!services.getAssignmentManager().isFailoverCleanupDone()) { if (!services.getAssignmentManager().isFailoverCleanupDone()) {
requeuedDeadServers.add(serverName); requeuedDeadServers.put(serverName, shouldSplitHlog);
return; return;
} }
this.deadservers.add(serverName); this.deadservers.add(serverName);
this.services.getExecutorService().submit(new ServerShutdownHandler( this.services.getExecutorService().submit(
this.master, this.services, this.deadservers, serverName, false)); new ServerShutdownHandler(this.master, this.services, this.deadservers, serverName,
shouldSplitHlog));
} }
/** /**
@ -541,18 +546,20 @@ public class ServerManager {
} }
Iterator<ServerName> serverIterator = queuedDeadServers.iterator(); Iterator<ServerName> serverIterator = queuedDeadServers.iterator();
while (serverIterator.hasNext()) { while (serverIterator.hasNext()) {
expireServer(serverIterator.next()); ServerName tmpServerName = serverIterator.next();
expireServer(tmpServerName);
serverIterator.remove(); serverIterator.remove();
requeuedDeadServers.remove(tmpServerName);
} }
if (!services.getAssignmentManager().isFailoverCleanupDone()) { if (!services.getAssignmentManager().isFailoverCleanupDone()) {
LOG.info("AssignmentManager hasn't finished failover cleanup"); LOG.info("AssignmentManager hasn't finished failover cleanup");
} }
serverIterator = requeuedDeadServers.iterator();
while (serverIterator.hasNext()) { for(ServerName tmpServerName : requeuedDeadServers.keySet()){
processDeadServer(serverIterator.next()); processDeadServer(tmpServerName, requeuedDeadServers.get(tmpServerName));
serverIterator.remove();
} }
requeuedDeadServers.clear();
} }
/* /*
@ -838,6 +845,14 @@ public class ServerManager {
return new HashSet<ServerName>(this.queuedDeadServers); return new HashSet<ServerName>(this.queuedDeadServers);
} }
/**
* @return A copy of the internal map of requeuedDeadServers servers and their corresponding
* splitlog need flag.
*/
Map<ServerName, Boolean> getRequeuedDeadServers() {
return Collections.unmodifiableMap(this.requeuedDeadServers);
}
public boolean isServerOnline(ServerName serverName) { public boolean isServerOnline(ServerName serverName) {
return serverName != null && onlineServers.containsKey(serverName); return serverName != null && onlineServers.containsKey(serverName);
} }
@ -851,7 +866,7 @@ public class ServerManager {
public synchronized boolean isServerDead(ServerName serverName) { public synchronized boolean isServerDead(ServerName serverName) {
return serverName == null || deadservers.isDeadServer(serverName) return serverName == null || deadservers.isDeadServer(serverName)
|| queuedDeadServers.contains(serverName) || queuedDeadServers.contains(serverName)
|| requeuedDeadServers.contains(serverName); || requeuedDeadServers.containsKey(serverName);
} }
public void shutdownCluster() { public void shutdownCluster() {

View File

@ -33,6 +33,7 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -43,16 +44,19 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Chore; import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters; import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.SplitLogTask; import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status; import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; 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.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
@ -118,6 +122,20 @@ public class SplitLogManager extends ZooKeeperListener {
private long unassignedTimeout; private long unassignedTimeout;
private long lastNodeCreateTime = Long.MAX_VALUE; private long lastNodeCreateTime = Long.MAX_VALUE;
public boolean ignoreZKDeleteForTesting = false; public boolean ignoreZKDeleteForTesting = false;
private volatile long lastRecoveringNodeCreationTime = Long.MAX_VALUE;
// When lastRecoveringNodeCreationTime is older than the following threshold, we'll check
// whether to GC stale recovering znodes
private long checkRecoveringTimeThreshold = 15000; // 15 seconds
private final Set<ServerName> failedRecoveringRegionDeletions = Collections
.synchronizedSet(new HashSet<ServerName>());
/**
* In distributedLogReplay mode, we need touch both splitlog and recovering-regions znodes in one
* operation. So the lock is used to guard such cases.
*/
protected final ReentrantLock recoveringRegionLock = new ReentrantLock();
final boolean distributedLogReplay;
private final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<String, Task>(); private final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<String, Task>();
private TimeoutMonitor timeoutMonitor; private TimeoutMonitor timeoutMonitor;
@ -181,10 +199,13 @@ public class SplitLogManager extends ZooKeeperListener {
LOG.info("timeout=" + timeout + ", unassigned timeout=" + unassignedTimeout); LOG.info("timeout=" + timeout + ", unassigned timeout=" + unassignedTimeout);
this.serverName = serverName; this.serverName = serverName;
this.timeoutMonitor = this.timeoutMonitor = new TimeoutMonitor(
new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper); conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper);
this.failedDeletions = Collections.synchronizedSet(new HashSet<String>()); this.failedDeletions = Collections.synchronizedSet(new HashSet<String>());
this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
LOG.info("distributedLogReplay = " + this.distributedLogReplay);
} }
public void finishInitialization(boolean masterRecovery) { public void finishInitialization(boolean masterRecovery) {
@ -244,7 +265,22 @@ public class SplitLogManager extends ZooKeeperListener {
* @return cumulative size of the logfiles split * @return cumulative size of the logfiles split
*/ */
public long splitLogDistributed(final List<Path> logDirs) throws IOException { public long splitLogDistributed(final List<Path> logDirs) throws IOException {
return splitLogDistributed(logDirs, null); if (logDirs.isEmpty()) {
return 0;
}
Set<ServerName> serverNames = new HashSet<ServerName>();
for (Path logDir : logDirs) {
try {
ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logDir);
if (serverName != null) {
serverNames.add(serverName);
}
} catch (IllegalArgumentException e) {
// ignore invalid format error.
LOG.warn("Cannot parse server name from " + logDir);
}
}
return splitLogDistributed(serverNames, logDirs, null);
} }
/** /**
@ -258,15 +294,15 @@ public class SplitLogManager extends ZooKeeperListener {
* @throws IOException If there was an error while splitting any log file * @throws IOException If there was an error while splitting any log file
* @return cumulative size of the logfiles split * @return cumulative size of the logfiles split
*/ */
public long splitLogDistributed(final List<Path> logDirs, PathFilter filter) public long splitLogDistributed(final Set<ServerName> serverNames, final List<Path> logDirs,
throws IOException { PathFilter filter) throws IOException {
MonitoredTask status = TaskMonitor.get().createStatus( MonitoredTask status = TaskMonitor.get().createStatus(
"Doing distributed log split in " + logDirs); "Doing distributed log split in " + logDirs);
FileStatus[] logfiles = getFileList(logDirs, filter); FileStatus[] logfiles = getFileList(logDirs, filter);
status.setStatus("Checking directory contents..."); status.setStatus("Checking directory contents...");
LOG.debug("Scheduling batch of logs to split"); LOG.debug("Scheduling batch of logs to split");
SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet(); SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet();
LOG.info("started splitting logs in " + logDirs); LOG.info("started splitting " + logfiles.length + " logs in " + logDirs);
long t = EnvironmentEdgeManager.currentTimeMillis(); long t = EnvironmentEdgeManager.currentTimeMillis();
long totalSize = 0; long totalSize = 0;
TaskBatch batch = new TaskBatch(); TaskBatch batch = new TaskBatch();
@ -283,6 +319,9 @@ public class SplitLogManager extends ZooKeeperListener {
} }
} }
waitForSplittingCompletion(batch, status); waitForSplittingCompletion(batch, status);
// remove recovering regions from ZK
this.removeRecoveringRegionsFromZK(serverNames);
if (batch.done != batch.installed) { if (batch.done != batch.installed) {
batch.isDead = true; batch.isDead = true;
SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet(); SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet();
@ -409,6 +448,171 @@ public class SplitLogManager extends ZooKeeperListener {
return count; return count;
} }
/**
* It removes recovering regions under /hbase/recovering-regions/[encoded region name] so that the
* region server hosting the region can allow reads to the recovered region
* @param serverNames servers which are just recovered
*/
private void removeRecoveringRegionsFromZK(final Set<ServerName> serverNames) {
if (!this.distributedLogReplay) {
// the function is only used in WALEdit direct replay mode
return;
}
int count = 0;
Set<String> recoveredServerNameSet = new HashSet<String>();
if (serverNames != null) {
for (ServerName tmpServerName : serverNames) {
recoveredServerNameSet.add(tmpServerName.getServerName());
}
}
try {
this.recoveringRegionLock.lock();
List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
if (tasks != null) {
for (String t : tasks) {
if (!ZKSplitLog.isRescanNode(watcher, t)) {
count++;
}
}
}
if (count == 0 && this.master.isInitialized()
&& !this.master.getServerManager().areDeadServersInProgress()) {
// no splitting work items left
deleteRecoveringRegionZNodes(null);
// reset lastRecoveringNodeCreationTime because we cleared all recovering znodes at
// this point.
lastRecoveringNodeCreationTime = Long.MAX_VALUE;
} else if (!recoveredServerNameSet.isEmpty()) {
// remove recovering regions which doesn't have any RS associated with it
List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
if (regions != null) {
for (String region : regions) {
String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
List<String> failedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
if (failedServers == null || failedServers.isEmpty()) {
ZKUtil.deleteNode(watcher, nodePath);
continue;
}
if (recoveredServerNameSet.containsAll(failedServers)) {
ZKUtil.deleteNodeRecursively(watcher, nodePath);
} else {
for (String failedServer : failedServers) {
if (recoveredServerNameSet.contains(failedServer)) {
String tmpPath = ZKUtil.joinZNode(nodePath, failedServer);
ZKUtil.deleteNode(watcher, tmpPath);
}
}
}
}
}
}
} catch (KeeperException ke) {
LOG.warn("removeRecoveringRegionsFromZK got zookeeper exception. Will retry", ke);
if (serverNames != null && !serverNames.isEmpty()) {
this.failedRecoveringRegionDeletions.addAll(serverNames);
}
} finally {
this.recoveringRegionLock.unlock();
}
}
/**
* It removes stale recovering regions under /hbase/recovering-regions/[encoded region name]
* during master initialization phase.
* @param failedServers A set of known failed servers
* @throws KeeperException
*/
void removeStaleRecoveringRegionsFromZK(final Set<ServerName> failedServers)
throws KeeperException {
if (!this.distributedLogReplay) {
// the function is only used in distributedLogReplay mode when master is in initialization
return;
}
Set<String> knownFailedServers = new HashSet<String>();
if (failedServers != null) {
for (ServerName tmpServerName : failedServers) {
knownFailedServers.add(tmpServerName.getServerName());
}
}
this.recoveringRegionLock.lock();
try {
List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
if (tasks != null) {
for (String t : tasks) {
byte[] data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t));
if (data != null) {
SplitLogTask slt = null;
try {
slt = SplitLogTask.parseFrom(data);
} catch (DeserializationException e) {
LOG.warn("Failed parse data for znode " + t, e);
}
if (slt != null && slt.isDone()) {
continue;
}
}
// decode the file name
t = ZKSplitLog.getFileName(t);
ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(new Path(t));
if (serverName != null) {
knownFailedServers.add(serverName.getServerName());
} else {
LOG.warn("Found invalid WAL log file name:" + t);
}
}
}
// remove recovering regions which doesn't have any RS associated with it
List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
if (regions != null) {
for (String region : regions) {
String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
if (regionFailedServers == null || regionFailedServers.isEmpty()) {
ZKUtil.deleteNode(watcher, nodePath);
continue;
}
boolean needMoreRecovery = false;
for (String tmpFailedServer : regionFailedServers) {
if (knownFailedServers.contains(tmpFailedServer)) {
needMoreRecovery = true;
break;
}
}
if (!needMoreRecovery) {
ZKUtil.deleteNode(watcher, nodePath);
}
}
}
} finally {
this.recoveringRegionLock.unlock();
}
}
private void deleteRecoveringRegionZNodes(List<String> regions) {
try {
if (regions == null) {
// remove all children under /home/recovering-regions
LOG.info("Garbage collecting all recovering regions.");
ZKUtil.deleteChildrenRecursively(watcher, watcher.recoveringRegionsZNode);
} else {
for (String curRegion : regions) {
String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, curRegion);
ZKUtil.deleteNodeRecursively(watcher, nodePath);
}
}
} catch (KeeperException e) {
LOG.warn("Cannot remove recovering regions from ZooKeeper", e);
}
}
private void setDone(String path, TerminationStatus status) { private void setDone(String path, TerminationStatus status) {
Task task = tasks.get(path); Task task = tasks.get(path);
if (task == null) { if (task == null) {
@ -859,9 +1063,131 @@ public class SplitLogManager extends ZooKeeperListener {
} }
/** /**
* Keeps track of the batch of tasks submitted together by a caller in * Create znodes /hbase/recovering-regions/[region_ids...]/[failed region server names ...] for
* splitLogDistributed(). Clients threads use this object to wait for all * all regions of the passed in region servers
* their tasks to be done. * @param serverName the name of a region server
* @param userRegions user regiones assigned on the region server
*/
void markRegionsRecoveringInZK(final ServerName serverName, Set<HRegionInfo> userRegions)
throws KeeperException {
if (userRegions == null || !this.distributedLogReplay) {
return;
}
try {
this.recoveringRegionLock.lock();
// mark that we're creating recovering znodes
this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTimeMillis();
for (HRegionInfo region : userRegions) {
String regionEncodeName = region.getEncodedName();
long retries = this.zkretries;
do {
String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regionEncodeName);
long lastRecordedFlushedSequenceId = -1;
try {
long lastSequenceId = this.master.getServerManager().getLastFlushedSequenceId(
regionEncodeName.getBytes());
/*
* znode layout: .../region_id[last known flushed sequence id]/failed server[last known
* flushed sequence id for the server]
*/
byte[] data = ZKUtil.getData(this.watcher, nodePath);
if (data == null) {
ZKUtil.createSetData(this.watcher, nodePath,
ZKUtil.positionToByteArray(lastSequenceId));
} else {
lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
if (lastRecordedFlushedSequenceId < lastSequenceId) {
// update last flushed sequence id in the region level
ZKUtil.setData(this.watcher, nodePath,
ZKUtil.positionToByteArray(lastSequenceId));
}
}
// go one level deeper with server name
nodePath = ZKUtil.joinZNode(nodePath, serverName.getServerName());
if (lastSequenceId <= lastRecordedFlushedSequenceId) {
// the newly assigned RS failed even before any flush to the region
lastSequenceId = lastRecordedFlushedSequenceId;
}
ZKUtil.createSetData(this.watcher, nodePath,
ZKUtil.positionToByteArray(lastSequenceId));
LOG.debug("Mark region " + regionEncodeName + " recovering from failed region server "
+ serverName);
// break retry loop
break;
} catch (KeeperException e) {
// ignore ZooKeeper exceptions inside retry loop
if (retries <= 1) {
throw e;
}
// wait a little bit for retry
try {
Thread.sleep(20);
} catch (Exception ignoreE) {
// ignore
}
}
} while ((--retries) > 0 && (!this.stopper.isStopped()));
}
} finally {
this.recoveringRegionLock.unlock();
}
}
/**
* @param bytes - Content of a failed region server or recovering region znode.
* @return long - The last flushed sequence Id for the region server
*/
public static long parseLastFlushedSequenceIdFrom(final byte[] bytes) {
long lastRecordedFlushedSequenceId = -1l;
try {
lastRecordedFlushedSequenceId = ZKUtil.parseHLogPositionFrom(bytes);
} catch (DeserializationException e) {
lastRecordedFlushedSequenceId = -1l;
LOG.warn("Can't parse last flushed sequence Id", e);
}
return lastRecordedFlushedSequenceId;
}
/**
* This function is used in distributedLogReplay to fetch last flushed sequence id from ZK
* @param zkw
* @param serverName
* @param encodedRegionName
* @return the last flushed sequence id recorded in ZK of the region for <code>serverName<code>
* @throws IOException
*/
public static long getLastFlushedSequenceId(ZooKeeperWatcher zkw, String serverName,
String encodedRegionName) throws IOException {
// when SplitLogWorker recovers a region by directly replaying unflushed WAL edits,
// last flushed sequence Id changes when newly assigned RS flushes writes to the region.
// If the newly assigned RS fails again(a chained RS failures scenario), the last flushed
// sequence Id name space (sequence Id only valid for a particular RS instance), changes
// when different newly assigned RS flushes the region.
// Therefore, in this mode we need to fetch last sequence Ids from ZK where we keep history of
// last flushed sequence Id for each failed RS instance.
long lastFlushedSequenceId = -1;
String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
nodePath = ZKUtil.joinZNode(nodePath, serverName);
try {
byte[] data = ZKUtil.getData(zkw, nodePath);
if (data != null) {
lastFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
}
} catch (KeeperException e) {
throw new IOException("Cannot get lastFlushedSequenceId from ZooKeeper for server="
+ serverName + "; region=" + encodedRegionName, e);
}
return lastFlushedSequenceId;
}
/**
* Keeps track of the batch of tasks submitted together by a caller in splitLogDistributed().
* Clients threads use this object to wait for all their tasks to be done.
* <p> * <p>
* All access is synchronized. * All access is synchronized.
*/ */
@ -944,18 +1270,14 @@ public class SplitLogManager extends ZooKeeperListener {
LOG.info("dead splitlog worker " + workerName); LOG.info("dead splitlog worker " + workerName);
} }
void handleDeadWorkers(List<ServerName> serverNames) { void handleDeadWorkers(Set<ServerName> serverNames) {
List<ServerName> workerNames = new ArrayList<ServerName>(serverNames.size());
for (ServerName serverName : serverNames) {
workerNames.add(serverName);
}
synchronized (deadWorkersLock) { synchronized (deadWorkersLock) {
if (deadWorkers == null) { if (deadWorkers == null) {
deadWorkers = new HashSet<ServerName>(100); deadWorkers = new HashSet<ServerName>(100);
} }
deadWorkers.addAll(workerNames); deadWorkers.addAll(serverNames);
} }
LOG.info("dead splitlog workers " + workerNames); LOG.info("dead splitlog workers " + serverNames);
} }
/** /**
@ -1052,6 +1374,20 @@ public class SplitLogManager extends ZooKeeperListener {
} }
failedDeletions.removeAll(tmpPaths); failedDeletions.removeAll(tmpPaths);
} }
// Garbage collect left-over /hbase/recovering-regions/... znode
long timeInterval = EnvironmentEdgeManager.currentTimeMillis()
- lastRecoveringNodeCreationTime;
if (!failedRecoveringRegionDeletions.isEmpty()
|| (tot == 0 && tasks.size() == 0 && (timeInterval > checkRecoveringTimeThreshold))) {
// inside the function there have more checks before GC anything
Set<ServerName> previouslyFailedDeletoins = null;
if (!failedRecoveringRegionDeletions.isEmpty()) {
previouslyFailedDeletoins = new HashSet<ServerName>(failedRecoveringRegionDeletions);
failedRecoveringRegionDeletions.removeAll(previouslyFailedDeletoins);
}
removeRecoveringRegionsFromZK(previouslyFailedDeletoins);
}
} }
} }

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.hbase.master.handler; package org.apache.hadoop.hbase.master.handler;
import java.io.IOException; import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -27,6 +29,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.DeadServer; import org.apache.hadoop.hbase.master.DeadServer;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -48,29 +51,54 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
public void process() throws IOException { public void process() throws IOException {
boolean gotException = true; boolean gotException = true;
try { try {
AssignmentManager am = this.services.getAssignmentManager();
try { try {
LOG.info("Splitting META logs for " + serverName);
if (this.shouldSplitHlog) { if (this.shouldSplitHlog) {
LOG.info("Splitting META logs for " + serverName);
if(this.distributedLogReplay) {
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
regions.add(HRegionInfo.FIRST_META_REGIONINFO);
this.services.getMasterFileSystem().prepareMetaLogReplay(serverName, regions);
} else {
this.services.getMasterFileSystem().splitMetaLog(serverName); this.services.getMasterFileSystem().splitMetaLog(serverName);
} }
}
} catch (IOException ioe) { } catch (IOException ioe) {
this.services.getExecutorService().submit(this); this.services.getExecutorService().submit(this);
this.deadServers.add(serverName); this.deadServers.add(serverName);
throw new IOException("failed log splitting for " + throw new IOException("failed log splitting for " + serverName + ", will retry", ioe);
serverName + ", will retry", ioe);
} }
// Assign meta if we were carrying it. // Assign meta if we were carrying it.
// Check again: region may be assigned to other where because of RIT // Check again: region may be assigned to other where because of RIT
// timeout // timeout
if (this.services.getAssignmentManager().isCarryingMeta(serverName)) { if (am.isCarryingMeta(serverName)) {
LOG.info("Server " + serverName + " was carrying META. Trying to assign."); LOG.info("Server " + serverName + " was carrying META. Trying to assign.");
this.services.getAssignmentManager().regionOffline(HRegionInfo.FIRST_META_REGIONINFO); am.regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
verifyAndAssignMetaWithRetries(); verifyAndAssignMetaWithRetries();
} else { } else {
LOG.info("META has been assigned to otherwhere, skip assigning."); LOG.info("META has been assigned to otherwhere, skip assigning.");
} }
try {
if (this.shouldSplitHlog && this.distributedLogReplay) {
if (!am.waitOnRegionToClearRegionsInTransition(HRegionInfo.FIRST_META_REGIONINFO,
regionAssignmentWaitTimeout)) {
throw new IOException("Region " + HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()
+ " didn't complete assignment in time");
}
this.services.getMasterFileSystem().splitMetaLog(serverName);
}
} catch (Exception ex) {
if (ex instanceof IOException) {
this.services.getExecutorService().submit(this);
this.deadServers.add(serverName);
throw new IOException("failed log splitting for " + serverName + ", will retry", ex);
} else {
throw new IOException(ex);
}
}
gotException = false; gotException = false;
} finally { } finally {
if (gotException){ if (gotException){
@ -78,9 +106,15 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
this.deadServers.finish(serverName); this.deadServers.finish(serverName);
} }
} }
super.process(); super.process();
} }
@Override
boolean isCarryingMeta() {
return true;
}
/** /**
* Before assign the META region, ensure it haven't * Before assign the META region, ensure it haven't
* been assigned by other place * been assigned by other place

View File

@ -20,13 +20,16 @@ package org.apache.hadoop.hbase.master.handler;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NavigableMap; import java.util.NavigableMap;
import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
@ -56,6 +59,8 @@ public class ServerShutdownHandler extends EventHandler {
protected final MasterServices services; protected final MasterServices services;
protected final DeadServer deadServers; protected final DeadServer deadServers;
protected final boolean shouldSplitHlog; // whether to split HLog or not protected final boolean shouldSplitHlog; // whether to split HLog or not
protected final boolean distributedLogReplay;
protected final int regionAssignmentWaitTimeout;
public ServerShutdownHandler(final Server server, final MasterServices services, public ServerShutdownHandler(final Server server, final MasterServices services,
final DeadServer deadServers, final ServerName serverName, final DeadServer deadServers, final ServerName serverName,
@ -76,6 +81,11 @@ public class ServerShutdownHandler extends EventHandler {
LOG.warn(this.serverName + " is NOT in deadservers; it should be!"); LOG.warn(this.serverName + " is NOT in deadservers; it should be!");
} }
this.shouldSplitHlog = shouldSplitHlog; this.shouldSplitHlog = shouldSplitHlog;
this.distributedLogReplay = server.getConfiguration().getBoolean(
HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
this.regionAssignmentWaitTimeout = server.getConfiguration().getInt(
HConstants.LOG_REPLAY_WAIT_REGION_TIMEOUT, 15000);
} }
@Override @Override
@ -107,21 +117,7 @@ public class ServerShutdownHandler extends EventHandler {
public void process() throws IOException { public void process() throws IOException {
final ServerName serverName = this.serverName; final ServerName serverName = this.serverName;
try { try {
try {
if (this.shouldSplitHlog) {
LOG.info("Splitting logs for " + serverName);
this.services.getMasterFileSystem().splitLog(serverName);
} else {
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.deadServers.add(serverName);
throw new IOException("failed log splitting for " +
serverName + ", will retry", ioe);
}
// We don't want worker thread in the MetaServerShutdownHandler // We don't want worker thread in the MetaServerShutdownHandler
// executor pool to block by waiting availability of .META. // executor pool to block by waiting availability of .META.
// Otherwise, it could run into the following issue: // Otherwise, it could run into the following issue:
@ -145,7 +141,7 @@ public class ServerShutdownHandler extends EventHandler {
// the dead server for further processing too. // the dead server for further processing too.
if (isCarryingMeta() // .META. if (isCarryingMeta() // .META.
|| !services.getAssignmentManager().isFailoverCleanupDone()) { || !services.getAssignmentManager().isFailoverCleanupDone()) {
this.services.getServerManager().processDeadServer(serverName); this.services.getServerManager().processDeadServer(serverName, this.shouldSplitHlog);
return; return;
} }
@ -183,6 +179,23 @@ public class ServerShutdownHandler extends EventHandler {
throw new IOException("Server is stopped"); throw new IOException("Server is stopped");
} }
try {
if (this.shouldSplitHlog) {
LOG.info("Splitting logs for " + serverName + " before assignment.");
if(this.distributedLogReplay){
Set<ServerName> serverNames = new HashSet<ServerName>();
serverNames.add(serverName);
this.services.getMasterFileSystem().prepareLogReplay(serverNames);
} else {
this.services.getMasterFileSystem().splitLog(serverName);
}
} else {
LOG.info("Skipping log splitting for " + serverName);
}
} catch (IOException ioe) {
resubmit(serverName, ioe);
}
// Clean out anything in regions in transition. Being conservative and // Clean out anything in regions in transition. Being conservative and
// doing after log splitting. Could do some states before -- OPENING? // doing after log splitting. Could do some states before -- OPENING?
// OFFLINE? -- and then others after like CLOSING that depend on log // OFFLINE? -- and then others after like CLOSING that depend on log
@ -258,18 +271,47 @@ public class ServerShutdownHandler extends EventHandler {
} }
} }
} }
try { try {
am.assign(toAssignRegions); am.assign(toAssignRegions);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
LOG.error("Caught " + ie + " during round-robin assignment"); LOG.error("Caught " + ie + " during round-robin assignment");
throw new IOException(ie); throw new IOException(ie);
} }
try {
if (this.shouldSplitHlog && this.distributedLogReplay) {
// wait for region assignment completes
for (HRegionInfo hri : toAssignRegions) {
if (!am.waitOnRegionToClearRegionsInTransition(hri, regionAssignmentWaitTimeout)) {
throw new IOException("Region " + hri.getEncodedName()
+ " didn't complete assignment in time");
}
}
this.services.getMasterFileSystem().splitLog(serverName);
}
} catch (Exception ex) {
if (ex instanceof IOException) {
resubmit(serverName, (IOException)ex);
} else {
throw new IOException(ex);
}
}
} finally { } finally {
this.deadServers.finish(serverName); this.deadServers.finish(serverName);
} }
LOG.info("Finished processing of shutdown of " + serverName); LOG.info("Finished processing of shutdown of " + serverName);
} }
private void resubmit(final ServerName serverName, IOException ex) throws IOException {
// 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.deadServers.add(serverName);
throw new IOException("failed log splitting for " + serverName + ", will retry", ex);
}
/** /**
* Process a dead region from a dead RS. Checks if the region is disabled or * Process a dead region from a dead RS. Checks if the region is disabled or
* disabling or if the region has a partially completed split. * disabling or if the region has a partially completed split.

View File

@ -95,6 +95,7 @@ import org.apache.hadoop.hbase.exceptions.DroppedSnapshotException;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.exceptions.NotServingRegionException; import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
import org.apache.hadoop.hbase.exceptions.RegionTooBusyException; import org.apache.hadoop.hbase.exceptions.RegionTooBusyException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.exceptions.UnknownScannerException; import org.apache.hadoop.hbase.exceptions.UnknownScannerException;
@ -200,6 +201,16 @@ public class HRegion implements HeapSize { // , Writable{
protected long completeSequenceId = -1L; protected long completeSequenceId = -1L;
/**
* Operation enum is used in {@link HRegion#startRegionOperation} to provide operation context for
* startRegionOperation to possibly invoke different checks before any region operations. Not all
* operations have to be defined here. It's only needed when a special check is need in
* startRegionOperation
*/
protected enum Operation {
ANY, GET, PUT, DELETE, SCAN, APPEND, INCREMENT, SPLIT_REGION, MERGE_REGION
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Members // Members
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
@ -280,6 +291,11 @@ public class HRegion implements HeapSize { // , Writable{
private final AtomicInteger majorInProgress = new AtomicInteger(0); private final AtomicInteger majorInProgress = new AtomicInteger(0);
private final AtomicInteger minorInProgress = new AtomicInteger(0); private final AtomicInteger minorInProgress = new AtomicInteger(0);
/**
* Min sequence id stored in store files of a region when opening the region
*/
private long minSeqIdForLogReplay = -1;
/** /**
* @return The smallest mvcc readPoint across all the scanners in this * @return The smallest mvcc readPoint across all the scanners in this
* region. Writes older than this readPoint, are included in every * region. Writes older than this readPoint, are included in every
@ -641,6 +657,9 @@ public class HRegion implements HeapSize { // , Writable{
long storeSeqIdForReplay = store.getMaxSequenceId(false); long storeSeqIdForReplay = store.getMaxSequenceId(false);
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
storeSeqIdForReplay); storeSeqIdForReplay);
if (this.minSeqIdForLogReplay == -1 || storeSeqIdForReplay < this.minSeqIdForLogReplay) {
this.minSeqIdForLogReplay = storeSeqIdForReplay;
}
// Include bulk loaded files when determining seqIdForAssignment // Include bulk loaded files when determining seqIdForAssignment
long storeSeqIdForAssignment = store.getMaxSequenceId(true); long storeSeqIdForAssignment = store.getMaxSequenceId(true);
if (maxSeqId == -1 || storeSeqIdForAssignment > maxSeqId) { if (maxSeqId == -1 || storeSeqIdForAssignment > maxSeqId) {
@ -778,6 +797,21 @@ public class HRegion implements HeapSize { // , Writable{
return this.closing.get(); return this.closing.get();
} }
/**
* Reset recovering state of current region
* @param newState
*/
public void setRecovering(boolean newState) {
this.getRegionInfo().setRecovering(newState);
}
/**
* @return True if current region is in recovering
*/
public boolean isRecovering() {
return this.getRegionInfo().isRecovering();
}
/** @return true if region is available (not closed and not closing) */ /** @return true if region is available (not closed and not closing) */
public boolean isAvailable() { public boolean isAvailable() {
return !isClosed() && !isClosing(); return !isClosed() && !isClosing();
@ -1608,7 +1642,7 @@ public class HRegion implements HeapSize { // , Writable{
// look across all the HStores for this region and determine what the // look across all the HStores for this region and determine what the
// closest key is across all column families, since the data may be sparse // closest key is across all column families, since the data may be sparse
checkRow(row, "getClosestRowBefore"); checkRow(row, "getClosestRowBefore");
startRegionOperation(); startRegionOperation(Operation.GET);
this.readRequestsCount.increment(); this.readRequestsCount.increment();
try { try {
Store store = getStore(family); Store store = getStore(family);
@ -1654,7 +1688,7 @@ public class HRegion implements HeapSize { // , Writable{
protected RegionScanner getScanner(Scan scan, protected RegionScanner getScanner(Scan scan,
List<KeyValueScanner> additionalScanners) throws IOException { List<KeyValueScanner> additionalScanners) throws IOException {
startRegionOperation(); startRegionOperation(Operation.SCAN);
try { try {
// Verify families are all valid // Verify families are all valid
prepareScanner(scan); prepareScanner(scan);
@ -1705,7 +1739,7 @@ public class HRegion implements HeapSize { // , Writable{
throws IOException { throws IOException {
checkReadOnly(); checkReadOnly();
checkResources(); checkResources();
startRegionOperation(); startRegionOperation(Operation.DELETE);
this.writeRequestsCount.increment(); this.writeRequestsCount.increment();
try { try {
delete.getRow(); delete.getRow();
@ -1804,7 +1838,7 @@ public class HRegion implements HeapSize { // , Writable{
// read lock, resources may run out. For now, the thought is that this // read lock, resources may run out. For now, the thought is that this
// will be extremely rare; we'll deal with it when it happens. // will be extremely rare; we'll deal with it when it happens.
checkResources(); checkResources();
startRegionOperation(); startRegionOperation(Operation.PUT);
this.writeRequestsCount.increment(); this.writeRequestsCount.increment();
try { try {
// All edits for the given row (across all column families) must happen atomically. // All edits for the given row (across all column families) must happen atomically.
@ -1862,13 +1896,29 @@ public class HRegion implements HeapSize { // , Writable{
*/ */
public OperationStatus[] batchMutate( public OperationStatus[] batchMutate(
Pair<Mutation, Integer>[] mutationsAndLocks) throws IOException { Pair<Mutation, Integer>[] mutationsAndLocks) throws IOException {
return batchMutate(mutationsAndLocks, false);
}
/**
* Perform a batch of mutations.
* It supports only Put and Delete mutations and will ignore other types passed.
* @param mutationsAndLocks
* the list of mutations paired with their requested lock IDs.
* @return an array of OperationStatus which internally contains the
* OperationStatusCode and the exceptionMessage if any.
* @throws IOException
*/
OperationStatus[] batchMutate(Pair<Mutation, Integer>[] mutationsAndLocks, boolean isReplay)
throws IOException {
BatchOperationInProgress<Pair<Mutation, Integer>> batchOp = BatchOperationInProgress<Pair<Mutation, Integer>> batchOp =
new BatchOperationInProgress<Pair<Mutation,Integer>>(mutationsAndLocks); new BatchOperationInProgress<Pair<Mutation,Integer>>(mutationsAndLocks);
boolean initialized = false; boolean initialized = false;
while (!batchOp.isDone()) { while (!batchOp.isDone()) {
if (!isReplay) {
checkReadOnly(); checkReadOnly();
}
checkResources(); checkResources();
long newSize; long newSize;
@ -1876,11 +1926,13 @@ public class HRegion implements HeapSize { // , Writable{
try { try {
if (!initialized) { if (!initialized) {
if (!isReplay) {
this.writeRequestsCount.increment(); this.writeRequestsCount.increment();
doPreMutationHook(batchOp); doPreMutationHook(batchOp);
}
initialized = true; initialized = true;
} }
long addedSize = doMiniBatchMutation(batchOp); long addedSize = doMiniBatchMutation(batchOp, isReplay);
newSize = this.addAndGetGlobalMemstoreSize(addedSize); newSize = this.addAndGetGlobalMemstoreSize(addedSize);
} finally { } finally {
closeRegionOperation(); closeRegionOperation();
@ -1892,6 +1944,7 @@ public class HRegion implements HeapSize { // , Writable{
return batchOp.retCodeDetails; return batchOp.retCodeDetails;
} }
private void doPreMutationHook(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp) private void doPreMutationHook(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp)
throws IOException { throws IOException {
/* Run coprocessor pre hook outside of locks to avoid deadlock */ /* Run coprocessor pre hook outside of locks to avoid deadlock */
@ -1927,10 +1980,9 @@ public class HRegion implements HeapSize { // , Writable{
} }
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private long doMiniBatchMutation( private long doMiniBatchMutation(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp,
BatchOperationInProgress<Pair<Mutation, Integer>> batchOp) throws IOException { boolean isInReplay) throws IOException {
// variable to note if all Put items are for the same CF -- metrics related // variable to note if all Put items are for the same CF -- metrics related
boolean putsCfSetConsistent = true; boolean putsCfSetConsistent = true;
@ -1941,7 +1993,7 @@ public class HRegion implements HeapSize { // , Writable{
//The set of columnFamilies first seen for Delete. //The set of columnFamilies first seen for Delete.
Set<byte[]> deletesCfSet = null; Set<byte[]> deletesCfSet = null;
WALEdit walEdit = new WALEdit(); WALEdit walEdit = new WALEdit(isInReplay);
MultiVersionConsistencyControl.WriteEntry w = null; MultiVersionConsistencyControl.WriteEntry w = null;
long txid = 0; long txid = 0;
boolean walSyncSuccessful = false; boolean walSyncSuccessful = false;
@ -1983,7 +2035,11 @@ public class HRegion implements HeapSize { // , Writable{
try { try {
if (isPutMutation) { if (isPutMutation) {
// Check the families in the put. If bad, skip this one. // Check the families in the put. If bad, skip this one.
if (isInReplay) {
removeNonExistentColumnFamilyForReplay(familyMap);
} else {
checkFamilies(familyMap.keySet()); checkFamilies(familyMap.keySet());
}
checkTimestamps(mutation.getFamilyMap(), now); checkTimestamps(mutation.getFamilyMap(), now);
} else { } else {
prepareDelete((Delete) mutation); prepareDelete((Delete) mutation);
@ -2081,7 +2137,7 @@ public class HRegion implements HeapSize { // , Writable{
w = mvcc.beginMemstoreInsert(); w = mvcc.beginMemstoreInsert();
// calling the pre CP hook for batch mutation // calling the pre CP hook for batch mutation
if (coprocessorHost != null) { if (!isInReplay && coprocessorHost != null) {
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp = MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations, new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive); batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
@ -2168,7 +2224,7 @@ public class HRegion implements HeapSize { // , Writable{
} }
walSyncSuccessful = true; walSyncSuccessful = true;
// calling the post CP hook for batch mutation // calling the post CP hook for batch mutation
if (coprocessorHost != null) { if (!isInReplay && coprocessorHost != null) {
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp = MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations, new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive); batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
@ -2187,7 +2243,7 @@ public class HRegion implements HeapSize { // , Writable{
// STEP 9. Run coprocessor post hooks. This should be done after the wal is // STEP 9. Run coprocessor post hooks. This should be done after the wal is
// synced so that the coprocessor contract is adhered to. // synced so that the coprocessor contract is adhered to.
// ------------------------------------ // ------------------------------------
if (coprocessorHost != null) { if (!isInReplay && coprocessorHost != null) {
for (int i = firstIndex; i < lastIndexExclusive; i++) { for (int i = firstIndex; i < lastIndexExclusive; i++) {
// only for successful puts // only for successful puts
if (batchOp.retCodeDetails[i].getOperationStatusCode() if (batchOp.retCodeDetails[i].getOperationStatusCode()
@ -2626,6 +2682,30 @@ public class HRegion implements HeapSize { // , Writable{
} }
} }
/**
* During replay, there could exist column families which are removed between region server
* failure and replay
*/
private void removeNonExistentColumnFamilyForReplay(
final Map<byte[], List<? extends Cell>> familyMap) {
List<byte[]> nonExistentList = null;
for (byte[] family : familyMap.keySet()) {
if (!this.htableDescriptor.hasFamily(family)) {
if (nonExistentList == null) {
nonExistentList = new ArrayList<byte[]>();
}
nonExistentList.add(family);
}
}
if (nonExistentList != null) {
for (byte[] family : nonExistentList) {
// Perhaps schema was changed between crash and replay
LOG.info("No family for " + Bytes.toString(family) + " omit from reply.");
familyMap.remove(family);
}
}
}
void checkTimestamps(final Map<byte[], List<? extends Cell>> familyMap, void checkTimestamps(final Map<byte[], List<? extends Cell>> familyMap,
long now) throws FailedSanityCheckException { long now) throws FailedSanityCheckException {
if (timestampSlop == HConstants.LATEST_TIMESTAMP) { if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
@ -3480,7 +3560,7 @@ public class HRegion implements HeapSize { // , Writable{
"after we renewed it. Could be caused by a very slow scanner " + "after we renewed it. Could be caused by a very slow scanner " +
"or a lengthy garbage collection"); "or a lengthy garbage collection");
} }
startRegionOperation(); startRegionOperation(Operation.SCAN);
readRequestsCount.increment(); readRequestsCount.increment();
try { try {
@ -4651,7 +4731,7 @@ public class HRegion implements HeapSize { // , Writable{
checkReadOnly(); checkReadOnly();
// Lock row // Lock row
startRegionOperation(); startRegionOperation(Operation.APPEND);
this.writeRequestsCount.increment(); this.writeRequestsCount.increment();
WriteEntry w = null; WriteEntry w = null;
try { try {
@ -4819,7 +4899,7 @@ public class HRegion implements HeapSize { // , Writable{
checkReadOnly(); checkReadOnly();
// Lock row // Lock row
startRegionOperation(); startRegionOperation(Operation.INCREMENT);
this.writeRequestsCount.increment(); this.writeRequestsCount.increment();
WriteEntry w = null; WriteEntry w = null;
try { try {
@ -4956,7 +5036,7 @@ public class HRegion implements HeapSize { // , Writable{
ClassSize.OBJECT + ClassSize.OBJECT +
ClassSize.ARRAY + ClassSize.ARRAY +
38 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT + 38 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
(11 * Bytes.SIZEOF_LONG) + (12 * Bytes.SIZEOF_LONG) +
Bytes.SIZEOF_BOOLEAN); Bytes.SIZEOF_BOOLEAN);
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD + public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
@ -5236,6 +5316,37 @@ public class HRegion implements HeapSize { // , Writable{
*/ */
public void startRegionOperation() public void startRegionOperation()
throws NotServingRegionException, RegionTooBusyException, InterruptedIOException { throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
startRegionOperation(Operation.ANY);
}
/**
* @param op The operation is about to be taken on the region
* @throws NotServingRegionException
* @throws RegionTooBusyException
* @throws InterruptedIOException
*/
protected void startRegionOperation(Operation op) throws NotServingRegionException,
RegionTooBusyException, InterruptedIOException {
switch (op) {
case INCREMENT:
case APPEND:
case GET:
case SCAN:
case SPLIT_REGION:
case MERGE_REGION:
// when a region is in recovering state, no read, split or merge is allowed
if (this.isRecovering()) {
throw new RegionInRecoveryException(this.getRegionNameAsString()
+ " is recovering");
}
break;
default:
break;
}
if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION) {
// split or merge region doesn't need to check the closing/closed state or lock the region
return;
}
if (this.closing.get()) { if (this.closing.get()) {
throw new NotServingRegionException(getRegionNameAsString() + " is closing"); throw new NotServingRegionException(getRegionNameAsString() + " is closing");
} }
@ -5449,6 +5560,14 @@ public class HRegion implements HeapSize { // , Writable{
return this.openSeqNum; return this.openSeqNum;
} }
/**
* Gets the min sequence number that was read from storage when this region was opened. WAL Edits
* with smaller sequence number will be skipped from replay.
*/
public long getMinSeqIdForLogReplay() {
return this.minSeqIdForLogReplay;
}
/** /**
* @return if a given region is in compaction now. * @return if a given region is in compaction now.
*/ */

View File

@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.exceptions.NotServingRegionException; import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.RegionAlreadyInTransitionException; import org.apache.hadoop.hbase.exceptions.RegionAlreadyInTransitionException;
import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
import org.apache.hadoop.hbase.exceptions.RegionServerRunningException; import org.apache.hadoop.hbase.exceptions.RegionServerRunningException;
@ -112,6 +113,7 @@ import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.SplitLogManager;
import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
@ -178,6 +180,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
@ -204,6 +207,7 @@ import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker; import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
@ -214,6 +218,7 @@ import org.apache.hadoop.net.DNS;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.cliffc.high_scale_lib.Counter; import org.cliffc.high_scale_lib.Counter;
import com.google.protobuf.BlockingRpcChannel; import com.google.protobuf.BlockingRpcChannel;
@ -258,6 +263,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
// catalog tracker // catalog tracker
protected CatalogTracker catalogTracker; protected CatalogTracker catalogTracker;
// Watch if a region is out of recovering state from ZooKeeper
@SuppressWarnings("unused")
private RecoveringRegionWatcher recoveringRegionWatcher;
/** /**
* Go here to get table descriptors. * Go here to get table descriptors.
*/ */
@ -292,6 +301,13 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap = protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap =
new ConcurrentHashMap<String, InetSocketAddress[]>(); new ConcurrentHashMap<String, InetSocketAddress[]>();
/**
* Set of regions currently being in recovering state which means it can accept writes(edits from
* previous failed region server) but not reads. A recovering region is also an online region.
*/
protected final Map<String, HRegion> recoveringRegions = Collections
.synchronizedMap(new HashMap<String, HRegion>());
// Leases // Leases
protected Leases leases; protected Leases leases;
@ -457,6 +473,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
/** Handle all the snapshot requests to this server */ /** Handle all the snapshot requests to this server */
RegionServerSnapshotManager snapshotManager; RegionServerSnapshotManager snapshotManager;
// configuration setting on if replay WAL edits directly to another RS
private final boolean distributedLogReplay;
// Table level lock manager for locking for region operations // Table level lock manager for locking for region operations
private TableLockManager tableLockManager; private TableLockManager tableLockManager;
@ -547,6 +566,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
} }
}; };
this.rsHost = new RegionServerCoprocessorHost(this, this.conf); this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
} }
/** /**
@ -671,6 +693,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
} }
this.tableLockManager = TableLockManager.createTableLockManager(conf, zooKeeper, this.tableLockManager = TableLockManager.createTableLockManager(conf, zooKeeper,
new ServerName(isa.getHostName(), isa.getPort(), startcode)); new ServerName(isa.getHostName(), isa.getPort(), startcode));
// register watcher for recovering regions
if(this.distributedLogReplay) {
this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);
}
} }
/** /**
@ -1515,8 +1542,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
this.rpcServer.start(); this.rpcServer.start();
// Create the log splitting worker and start it // Create the log splitting worker and start it
this.splitLogWorker = new SplitLogWorker(this.zooKeeper, this.splitLogWorker = new SplitLogWorker(this.zooKeeper, this.getConfiguration(), this, this);
this.getConfiguration(), this.getServerName(), this);
splitLogWorker.start(); splitLogWorker.start();
} }
@ -1641,6 +1667,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
LOG.error("No sequence number found when opening " + r.getRegionNameAsString()); LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
openSeqNum = 0; openSeqNum = 0;
} }
// Update flushed sequence id of a recovering region in ZK
updateRecoveringRegionLastFlushedSequenceId(r);
// Update ZK, or META // Update ZK, or META
if (r.getRegionInfo().isMetaRegion()) { if (r.getRegionInfo().isMetaRegion()) {
MetaRegionTracker.setMetaLocation(getZooKeeper(), MetaRegionTracker.setMetaLocation(getZooKeeper(),
@ -1884,14 +1914,13 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
public long getLastSequenceId(byte[] region) { public long getLastSequenceId(byte[] region) {
Long lastFlushedSequenceId = -1l; Long lastFlushedSequenceId = -1l;
try { try {
GetLastFlushedSequenceIdRequest req = GetLastFlushedSequenceIdRequest req = RequestConverter
RequestConverter.buildGetLastFlushedSequenceIdRequest(region); .buildGetLastFlushedSequenceIdRequest(region);
lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req) lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req)
.getLastFlushedSequenceId(); .getLastFlushedSequenceId();
} catch (ServiceException e) { } catch (ServiceException e) {
lastFlushedSequenceId = -1l; lastFlushedSequenceId = -1l;
LOG.warn("Unable to connect to the master to check " + LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id", e);
"the last flushed sequence id", e);
} }
return lastFlushedSequenceId; return lastFlushedSequenceId;
} }
@ -1965,6 +1994,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
return this.stopping; return this.stopping;
} }
public Map<String, HRegion> getRecoveringRegions() {
return this.recoveringRegions;
}
/** /**
* *
* @return the configuration * @return the configuration
@ -2651,10 +2684,12 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
try { try {
requestCount.increment(); requestCount.increment();
HRegion region = getRegion(request.getRegion()); HRegion region = getRegion(request.getRegion());
GetResponse.Builder builder = GetResponse.newBuilder(); GetResponse.Builder builder = GetResponse.newBuilder();
ClientProtos.Get get = request.getGet(); ClientProtos.Get get = request.getGet();
Boolean existence = null; Boolean existence = null;
Result r = null; Result r = null;
if (request.getClosestRowBefore()) { if (request.getClosestRowBefore()) {
if (get.getColumnCount() != 1) { if (get.getColumnCount() != 1) {
throw new DoNotRetryIOException( throw new DoNotRetryIOException(
@ -3006,7 +3041,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
} }
List<KeyValue> values = new ArrayList<KeyValue>(); List<KeyValue> values = new ArrayList<KeyValue>();
MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint()); MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
region.startRegionOperation(); region.startRegionOperation(Operation.SCAN);
try { try {
int i = 0; int i = 0;
synchronized(scanner) { synchronized(scanner) {
@ -3450,6 +3485,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
removeFromMovedRegions(region.getEncodedName()); removeFromMovedRegions(region.getEncodedName());
if (previous == null) { if (previous == null) {
// check if the region to be opened is marked in recovering state in ZK
if (isRegionMarkedRecoveringInZK(region.getEncodedName())) {
this.recoveringRegions.put(region.getEncodedName(), null);
}
// If there is no action in progress, we can submit a specific handler. // If there is no action in progress, we can submit a specific handler.
// Need to pass the expected version in the constructor. // Need to pass the expected version in the constructor.
if (region.isMetaRegion()) { if (region.isMetaRegion()) {
@ -3465,6 +3504,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
builder.addOpeningState(RegionOpeningState.OPENED); builder.addOpeningState(RegionOpeningState.OPENED);
} catch (KeeperException zooKeeperEx) {
LOG.error("Can't retrieve recovering state from zookeeper", zooKeeperEx);
throw new ServiceException(zooKeeperEx);
} catch (IOException ie) { } catch (IOException ie) {
LOG.warn("Failed opening region " + region.getRegionNameAsString(), ie); LOG.warn("Failed opening region " + region.getRegionNameAsString(), ie);
if (isBulkAssign) { if (isBulkAssign) {
@ -3589,6 +3631,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
checkOpen(); checkOpen();
requestCount.increment(); requestCount.increment();
HRegion region = getRegion(request.getRegion()); HRegion region = getRegion(request.getRegion());
region.startRegionOperation(Operation.SPLIT_REGION);
LOG.info("Splitting " + region.getRegionNameAsString()); LOG.info("Splitting " + region.getRegionNameAsString());
region.flushcache(); region.flushcache();
byte[] splitPoint = null; byte[] splitPoint = null;
@ -3621,6 +3664,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
HRegion regionA = getRegion(request.getRegionA()); HRegion regionA = getRegion(request.getRegionA());
HRegion regionB = getRegion(request.getRegionB()); HRegion regionB = getRegion(request.getRegionB());
boolean forcible = request.getForcible(); boolean forcible = request.getForcible();
regionA.startRegionOperation(Operation.MERGE_REGION);
regionB.startRegionOperation(Operation.MERGE_REGION);
LOG.info("Receiving merging request for " + regionA + ", " + regionB LOG.info("Receiving merging request for " + regionA + ", " + regionB
+ ",forcible=" + forcible); + ",forcible=" + forcible);
regionA.flushcache(); regionA.flushcache();
@ -3713,9 +3758,58 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
} }
} }
/**
* Replay the given changes when distributedLogReplay WAL edits from a failed RS. The guarantee is
* that the given mutations will be durable on the receiving RS if this method returns without any
* exception.
* @param rpcc the RPC controller
* @param request the request
* @throws ServiceException
*/
@Override
@QosPriority(priority = HConstants.REPLAY_QOS)
public MultiResponse replay(final RpcController rpcc, final MultiRequest request)
throws ServiceException {
long before = EnvironmentEdgeManager.currentTimeMillis();
PayloadCarryingRpcController controller = (PayloadCarryingRpcController) rpcc;
CellScanner cellScanner = controller != null ? controller.cellScanner() : null;
// Clear scanner so we are not holding on to reference across call.
controller.setCellScanner(null);
try {
checkOpen();
HRegion region = getRegion(request.getRegion());
MultiResponse.Builder builder = MultiResponse.newBuilder();
List<MutationProto> mutates = new ArrayList<MutationProto>();
for (ClientProtos.MultiAction actionUnion : request.getActionList()) {
if (actionUnion.hasMutation()) {
MutationProto mutate = actionUnion.getMutation();
MutationType type = mutate.getMutateType();
switch (type) {
case PUT:
case DELETE:
mutates.add(mutate);
break;
default:
throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
}
} else {
LOG.warn("Error: invalid action: " + actionUnion + ". " + "it must be a Mutation.");
throw new DoNotRetryIOException("Invalid action, " + "it must be a Mutation.");
}
}
if (!mutates.isEmpty()) {
doBatchOp(builder, region, mutates, cellScanner, true);
}
return builder.build();
} catch (IOException ie) {
throw new ServiceException(ie);
} finally {
metricsRegionServer.updateReplay(EnvironmentEdgeManager.currentTimeMillis() - before);
}
}
/** /**
* Roll the WAL writer of the region server. * Roll the WAL writer of the region server.
*
* @param controller the RPC controller * @param controller the RPC controller
* @param request the request * @param request the request
* @throws ServiceException * @throws ServiceException
@ -3841,6 +3935,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
return r; return r;
} }
/**
* Execute a list of Put/Delete mutations.
*/
protected void doBatchOp(final MultiResponse.Builder builder,
final HRegion region, final List<MutationProto> mutates, final CellScanner cells) {
doBatchOp(builder, region, mutates, cells, false);
}
/** /**
* Execute a list of Put/Delete mutations. * Execute a list of Put/Delete mutations.
* *
@ -3849,7 +3951,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
* @param mutations * @param mutations
*/ */
protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region, protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region,
final List<MutationProto> mutations, final CellScanner cells) { final List<MutationProto> mutations, final CellScanner cells, boolean isReplay) {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutations.size()]; Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutations.size()];
long before = EnvironmentEdgeManager.currentTimeMillis(); long before = EnvironmentEdgeManager.currentTimeMillis();
@ -3877,7 +3979,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
OperationStatus codes[] = region.batchMutate(mutationsWithLocks); OperationStatus codes[] = region.batchMutate(mutationsWithLocks, isReplay);
for (i = 0; i < codes.length; i++) { for (i = 0; i < codes.length; i++) {
switch (codes[i].getOperationStatusCode()) { switch (codes[i].getOperationStatusCode()) {
case BAD_FAMILY: case BAD_FAMILY:
@ -4097,4 +4199,91 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
public CompactSplitThread getCompactSplitThread() { public CompactSplitThread getCompactSplitThread() {
return this.compactSplitThread; return this.compactSplitThread;
} }
/**
* check if /hbase/recovering-regions/<current region encoded name> exists. Returns true if exists
* and set watcher as well.
* @param regionEncodedName region encode name
* @return true when /hbase/recovering-regions/<current region encoded name> exists
* @throws KeeperException
*/
private boolean isRegionMarkedRecoveringInZK(String regionEncodedName) throws KeeperException {
boolean result = false;
String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode, regionEncodedName);
byte[] node = ZKUtil.getDataAndWatch(this.zooKeeper, nodePath);
if (node != null) {
result = true;
}
return result;
}
/**
* A helper function to store the last flushed sequence Id with the previous failed RS for a
* recovering region. The Id is used to skip wal edits which are flushed. Since the flushed
* sequence id is only valid for each RS, we associate the Id with corresponding failed RS.
* @throws KeeperException
* @throws IOException
*/
private void updateRecoveringRegionLastFlushedSequenceId(HRegion r) throws KeeperException,
IOException {
if (!r.isRecovering()) {
// return immdiately for non-recovering regions
return;
}
HRegionInfo region = r.getRegionInfo();
ZooKeeperWatcher zkw = getZooKeeper();
String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName());
long minSeqIdForLogReplay = r.getMinSeqIdForLogReplay();
long lastRecordedFlushedSequenceId = -1;
String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
region.getEncodedName());
// recovering-region level
byte[] data = ZKUtil.getData(zkw, nodePath);
if (data != null) {
lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
}
if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
}
if (previousRSName != null) {
// one level deeper for failed RS
nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for "
+ previousRSName);
} else {
LOG.warn("Can't find failed region server for recovering region " + region.getEncodedName());
}
}
/**
* Return the last failed RS name under /hbase/recovering-regions/encodedRegionName
* @param encodedRegionName
* @return
* @throws IOException
* @throws KeeperException
*/
private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
String result = null;
long maxZxid = 0;
ZooKeeperWatcher zkw = this.getZooKeeper();
String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
List<String> failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath);
if (failedServers == null || failedServers.isEmpty()) {
return result;
}
for (String failedServer : failedServers) {
String rsPath = ZKUtil.joinZNode(nodePath, failedServer);
Stat stat = new Stat();
ZKUtil.getDataNoWatch(zkw, rsPath, stat);
if (maxZxid < stat.getCzxid()) {
maxZxid = stat.getCzxid();
result = failedServer;
}
}
return result;
}
} }

View File

@ -26,8 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public interface LastSequenceId { public interface LastSequenceId {
/** /**
* @param regionname * @param regionName Encoded region name
* @return Last flushed sequence Id for regionname * @return Last flushed sequence Id for regionName or -1 if it can't be determined
*/ */
public long getLastSequenceId(byte[] regionname); public long getLastSequenceId(byte[] regionName);
} }

View File

@ -87,4 +87,8 @@ public class MetricsRegionServer {
} }
serverSource.updateAppend(t); serverSource.updateAppend(t);
} }
public void updateReplay(long t){
serverSource.updateReplay(t);
}
} }

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -108,4 +109,9 @@ public interface RegionServerServices extends OnlineRegions {
* @return The RegionServer's CatalogTracker * @return The RegionServer's CatalogTracker
*/ */
public CatalogTracker getCatalogTracker(); public CatalogTracker getCatalogTracker();
/**
* @return set of recovering regions on the hosting region server
*/
public Map<String, HRegion> getRecoveringRegions();
} }

View File

@ -20,7 +20,11 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import java.net.ConnectException;
import java.net.SocketTimeoutException;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -29,10 +33,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters; import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.SplitLogTask; import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
import org.apache.hadoop.hbase.master.SplitLogManager; import org.apache.hadoop.hbase.master.SplitLogManager;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.CancelableProgressable;
@ -70,6 +76,7 @@ import org.apache.zookeeper.data.Stat;
@InterfaceAudience.Private @InterfaceAudience.Private
public class SplitLogWorker extends ZooKeeperListener implements Runnable { public class SplitLogWorker extends ZooKeeperListener implements Runnable {
private static final Log LOG = LogFactory.getLog(SplitLogWorker.class); private static final Log LOG = LogFactory.getLog(SplitLogWorker.class);
private static final int checkInterval = 5000; // 5 seconds
Thread worker; Thread worker;
private final ServerName serverName; private final ServerName serverName;
@ -83,20 +90,30 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
private final Object grabTaskLock = new Object(); private final Object grabTaskLock = new Object();
private boolean workerInGrabTask = false; private boolean workerInGrabTask = false;
private final int report_period; private final int report_period;
private RegionServerServices server = null;
public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf, public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf,
ServerName serverName, TaskExecutor splitTaskExecutor) { RegionServerServices server, TaskExecutor splitTaskExecutor) {
super(watcher);
this.server = server;
this.serverName = server.getServerName();
this.splitTaskExecutor = splitTaskExecutor;
report_period = conf.getInt("hbase.splitlog.report.period",
conf.getInt("hbase.splitlog.manager.timeout", SplitLogManager.DEFAULT_TIMEOUT) / 3);
}
public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf, ServerName serverName,
TaskExecutor splitTaskExecutor) {
super(watcher); super(watcher);
this.serverName = serverName; this.serverName = serverName;
this.splitTaskExecutor = splitTaskExecutor; this.splitTaskExecutor = splitTaskExecutor;
report_period = conf.getInt("hbase.splitlog.report.period", report_period = conf.getInt("hbase.splitlog.report.period",
conf.getInt("hbase.splitlog.manager.timeout", conf.getInt("hbase.splitlog.manager.timeout", SplitLogManager.DEFAULT_TIMEOUT) / 3);
SplitLogManager.DEFAULT_TIMEOUT) / 2);
} }
public SplitLogWorker(ZooKeeperWatcher watcher, final Configuration conf, public SplitLogWorker(final ZooKeeperWatcher watcher, final Configuration conf,
final ServerName serverName, final LastSequenceId sequenceIdChecker) { RegionServerServices server, final LastSequenceId sequenceIdChecker) {
this(watcher, conf, serverName, new TaskExecutor () { this(watcher, conf, server, new TaskExecutor() {
@Override @Override
public Status exec(String filename, CancelableProgressable p) { public Status exec(String filename, CancelableProgressable p) {
Path rootdir; Path rootdir;
@ -113,7 +130,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
// encountered a bad non-retry-able persistent error. // encountered a bad non-retry-able persistent error.
try { try {
if (!HLogSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)), if (!HLogSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)),
fs, conf, p, sequenceIdChecker)) { fs, conf, p, sequenceIdChecker, watcher)) {
return Status.PREEMPTED; return Status.PREEMPTED;
} }
} catch (InterruptedIOException iioe) { } catch (InterruptedIOException iioe) {
@ -121,9 +138,18 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
return Status.RESIGNED; return Status.RESIGNED;
} catch (IOException e) { } catch (IOException e) {
Throwable cause = e.getCause(); Throwable cause = e.getCause();
if (cause instanceof InterruptedException) { if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException
|| cause instanceof ConnectException
|| cause instanceof SocketTimeoutException)) {
LOG.warn("log replaying of " + filename + " can't connect to the target regionserver, "
+ "resigning", e);
return Status.RESIGNED;
} else if (cause instanceof InterruptedException) {
LOG.warn("log splitting of " + filename + " interrupted, resigning", e); LOG.warn("log splitting of " + filename + " interrupted, resigning", e);
return Status.RESIGNED; return Status.RESIGNED;
} else if(cause instanceof KeeperException) {
LOG.warn("log splitting of " + filename + " hit ZooKeeper issue, resigning", e);
return Status.RESIGNED;
} }
LOG.warn("log splitting of " + filename + " failed, returning error", e); LOG.warn("log splitting of " + filename + " failed, returning error", e);
return Status.ERR; return Status.ERR;
@ -204,7 +230,39 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
synchronized (taskReadyLock) { synchronized (taskReadyLock) {
while (seq_start == taskReadySeq) { while (seq_start == taskReadySeq) {
try { try {
taskReadyLock.wait(); taskReadyLock.wait(checkInterval);
if (this.server != null) {
// check to see if we have stale recovering regions in our internal memory state
Map<String, HRegion> recoveringRegions = this.server.getRecoveringRegions();
if (!recoveringRegions.isEmpty()) {
// Make a local copy to prevent ConcurrentModificationException when other threads
// modify recoveringRegions
List<String> tmpCopy = new ArrayList<String>(recoveringRegions.keySet());
for (String region : tmpCopy) {
String nodePath = ZKUtil.joinZNode(this.watcher.recoveringRegionsZNode, region);
try {
if (ZKUtil.checkExists(this.watcher, nodePath) == -1) {
HRegion r = recoveringRegions.remove(region);
if (r != null) {
r.setRecovering(false);
}
LOG.debug("Mark recovering region:" + region + " up.");
} else {
// current check is a defensive(or redundant) mechanism to prevent us from
// having stale recovering regions in our internal RS memory state while
// zookeeper(source of truth) says differently. We stop at the first good one
// because we should not have a single instance such as this in normal case so
// check the first one is good enough.
break;
}
} catch (KeeperException e) {
// ignore zookeeper error
LOG.debug("Got a zookeeper when trying to open a recovering region", e);
break;
}
}
}
}
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.info("SplitLogWorker interrupted while waiting for task," + LOG.info("SplitLogWorker interrupted while waiting for task," +
" exiting: " + e.toString() + (exitWorker ? "" : " exiting: " + e.toString() + (exitWorker ? "" :
@ -214,6 +272,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
} }
} }
} }
} }
} }
@ -463,9 +522,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
} }
} }
@Override @Override
public void nodeDataChanged(String path) { public void nodeDataChanged(String path) {
// there will be a self generated dataChanged event every time attemptToOwnTask() // there will be a self generated dataChanged event every time attemptToOwnTask()
@ -510,7 +566,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
return childrenPaths; return childrenPaths;
} }
@Override @Override
public void nodeChildrenChanged(String path) { public void nodeChildrenChanged(String path) {
if(path.equals(watcher.splitLogZNode)) { if(path.equals(watcher.splitLogZNode)) {

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.regionserver.handler; package org.apache.hadoop.hbase.regionserver.handler;
import java.io.IOException; import java.io.IOException;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -137,6 +138,16 @@ public class OpenRegionHandler extends EventHandler {
if (region == null) { if (region == null) {
return; return;
} }
// check if we need set current region in recovering state
region.setRecovering(false);
Map<String, HRegion> recoveringRegions = this.rsServices.getRecoveringRegions();
if (recoveringRegions != null && !recoveringRegions.isEmpty()
&& recoveringRegions.containsKey(region.getRegionInfo().getEncodedName())) {
region.setRecovering(true);
recoveringRegions.put(region.getRegionInfo().getEncodedName(), region);
}
boolean failed = true; boolean failed = true;
if (tickleOpening("post_region_open")) { if (tickleOpening("post_region_open")) {
if (updateMeta(region)) { if (updateMeta(region)) {

View File

@ -1200,6 +1200,10 @@ class FSHLog implements HLog, Syncable {
long now = EnvironmentEdgeManager.currentTimeMillis(); long now = EnvironmentEdgeManager.currentTimeMillis();
// coprocessor hook: // coprocessor hook:
if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) { if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) {
if (logEdit.isReplay()) {
// set replication scope null so that this won't be replicated
logKey.setScopes(null);
}
// write to our buffer for the Hlog file. // write to our buffer for the Hlog file.
logSyncer.append(new FSHLog.Entry(logKey, logEdit)); logSyncer.append(new FSHLog.Entry(logKey, logEdit));
} }

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Bytes;
public class HLogUtil { public class HLogUtil {
static final Log LOG = LogFactory.getLog(HLogUtil.class); static final Log LOG = LogFactory.getLog(HLogUtil.class);
@ -167,6 +168,37 @@ public class HLogUtil {
return ServerName.parseServerName(serverName); return ServerName.parseServerName(serverName);
} }
/**
* This function returns region server name from a log file name which is in either format:
* hdfs://<name node>/hbase/.logs/<server name>-splitting/... or hdfs://<name
* node>/hbase/.logs/<server name>/...
* @param logFile
* @return null if the passed in logFile isn't a valid HLog file path
*/
public static ServerName getServerNameFromHLogDirectoryName(Path logFile) {
Path logDir = logFile.getParent();
String logDirName = logDir.getName();
if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) {
logDir = logFile;
logDirName = logDir.getName();
}
ServerName serverName = null;
if (logDirName.endsWith(HLog.SPLITTING_EXT)) {
logDirName = logDirName.substring(0, logDirName.length() - HLog.SPLITTING_EXT.length());
}
try {
serverName = ServerName.parseServerName(logDirName);
} catch (IllegalArgumentException ex) {
serverName = null;
LOG.warn("Invalid log file path=" + logFile, ex);
}
if (serverName != null && serverName.getStartcode() < 0) {
LOG.warn("Invalid log file path=" + logFile);
return null;
}
return serverName;
}
/** /**
* Returns sorted set of edit files made by wal-log splitter, excluding files * Returns sorted set of edit files made by wal-log splitter, excluding files
* with '.temp' suffix. * with '.temp' suffix.

View File

@ -0,0 +1,60 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.util.StringUtils;
/**
* Class used to push numbers about WAL edits replay into the metrics subsystem. This will take a
* single function call and turn it into multiple manipulations of the hadoop metrics system.
*/
@InterfaceAudience.Private
public class MetricsWALEditsReplay {
static final Log LOG = LogFactory.getLog(MetricsWALEditsReplay.class);
private final MetricsEditsReplaySource source;
public MetricsWALEditsReplay() {
source = CompatibilitySingletonFactory.getInstance(MetricsEditsReplaySource.class);
}
/**
* Add the time a replay command took
*/
void updateReplayTime(long time) {
source.updateReplayTime(time);
}
/**
* Add the batch size of each replay
*/
void updateReplayBatchSize(long size) {
source.updateReplayDataSize(size);
}
/**
* Add the payload data size of each replay
*/
void updateReplayDataSize(long size) {
}
}

View File

@ -83,6 +83,7 @@ public class WALEdit implements Writable, HeapSize {
static final byte[] COMPLETE_CACHE_FLUSH = Bytes.toBytes("HBASE::CACHEFLUSH"); static final byte[] COMPLETE_CACHE_FLUSH = Bytes.toBytes("HBASE::CACHEFLUSH");
static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION"); static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
private final int VERSION_2 = -1; private final int VERSION_2 = -1;
private final boolean isReplay;
private final ArrayList<KeyValue> kvs = new ArrayList<KeyValue>(); private final ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
@ -93,6 +94,11 @@ public class WALEdit implements Writable, HeapSize {
private CompressionContext compressionContext; private CompressionContext compressionContext;
public WALEdit() { public WALEdit() {
this(false);
}
public WALEdit(boolean isReplay) {
this.isReplay = isReplay;
} }
/** /**
@ -103,6 +109,14 @@ public class WALEdit implements Writable, HeapSize {
return Bytes.equals(METAFAMILY, f); return Bytes.equals(METAFAMILY, f);
} }
/**
* @return True when current WALEdit is created by log replay. Replication skips WALEdits from
* replay.
*/
public boolean isReplay() {
return this.isReplay;
}
public void setCompressionContext(final CompressionContext compressionContext) { public void setCompressionContext(final CompressionContext compressionContext) {
this.compressionContext = compressionContext; this.compressionContext = compressionContext;
} }

View File

@ -0,0 +1,233 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.ServiceException;
/**
* This class is responsible for replaying the edits coming from a failed region server.
* <p/>
* This class uses the native HBase client in order to replay WAL entries.
* <p/>
*/
@InterfaceAudience.Private
public class WALEditsReplaySink {
private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class);
private final Configuration conf;
private final HConnection conn;
private final byte[] tableName;
private final MetricsWALEditsReplay metrics;
private final AtomicLong totalReplayedEdits = new AtomicLong();
private final boolean skipErrors;
/**
* Create a sink for WAL log entries replay
* @param conf
* @param tableName
* @param conn
* @throws IOException
*/
public WALEditsReplaySink(Configuration conf, byte[] tableName, HConnection conn)
throws IOException {
this.conf = conf;
this.metrics = new MetricsWALEditsReplay();
this.conn = conn;
this.tableName = tableName;
this.skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS);
}
/**
* Replay an array of actions of the same region directly into the newly assigned Region Server
* @param actions
* @throws IOException
*/
public void replayEntries(List<Pair<HRegionLocation, Row>> actions) throws IOException {
if (actions.size() == 0) {
return;
}
int batchSize = actions.size();
int dataSize = 0;
Map<HRegionInfo, List<Action<Row>>> actionsByRegion =
new HashMap<HRegionInfo, List<Action<Row>>>();
HRegionLocation loc = null;
Row row = null;
List<Action<Row>> regionActions = null;
// Build the action list.
for (int i = 0; i < batchSize; i++) {
loc = actions.get(i).getFirst();
row = actions.get(i).getSecond();
if (actionsByRegion.containsKey(loc.getRegionInfo())) {
regionActions = actionsByRegion.get(loc.getRegionInfo());
} else {
regionActions = new ArrayList<Action<Row>>();
actionsByRegion.put(loc.getRegionInfo(), regionActions);
}
Action<Row> action = new Action<Row>(row, i);
regionActions.add(action);
dataSize += row.getRow().length;
}
long startTime = EnvironmentEdgeManager.currentTimeMillis();
// replaying edits by region
for (HRegionInfo curRegion : actionsByRegion.keySet()) {
replayEdits(loc, curRegion, actionsByRegion.get(curRegion));
}
long endTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
LOG.debug("number of rows:" + actions.size() + " are sent by batch! spent " + endTime
+ "(ms)!");
metrics.updateReplayTime(endTime);
metrics.updateReplayBatchSize(batchSize);
metrics.updateReplayDataSize(dataSize);
this.totalReplayedEdits.addAndGet(batchSize);
}
/**
* Get a string representation of this sink's metrics
* @return string with the total replayed edits count
*/
public String getStats() {
return this.totalReplayedEdits.get() == 0 ? "" : "Sink: total replayed edits: "
+ this.totalReplayedEdits;
}
private void replayEdits(final HRegionLocation regionLoc, final HRegionInfo regionInfo,
final List<Action<Row>> actions) throws IOException {
try {
ReplayServerCallable<MultiResponse> callable = new ReplayServerCallable<MultiResponse>(
this.conn, this.tableName, regionLoc, regionInfo, actions);
callable.withRetries();
} catch (IOException ie) {
if (skipErrors) {
LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
+ "=true so continuing replayEdits with error:" + ie.getMessage());
} else {
throw ie;
}
}
}
/**
* Callable that handles the <code>replay</code> method call going against a single regionserver
* @param <R>
*/
class ReplayServerCallable<R> extends ServerCallable<MultiResponse> {
private HRegionInfo regionInfo;
private List<Action<Row>> actions;
private Map<HRegionLocation, Map<HRegionInfo, List<Action<Row>>>> retryActions = null;
ReplayServerCallable(final HConnection connection, final byte [] tableName,
final HRegionLocation regionLoc, final HRegionInfo regionInfo,
final List<Action<Row>> actions) {
super(connection, tableName, null);
this.actions = actions;
this.regionInfo = regionInfo;
this.location = regionLoc;
}
@Override
public MultiResponse call() throws IOException {
try {
replayToServer(this.regionInfo, this.actions);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
return null;
}
private void replayToServer(HRegionInfo regionInfo, List<Action<Row>> actions)
throws IOException, ServiceException {
AdminService.BlockingInterface remoteSvr = connection.getAdmin(location.getServerName());
MultiRequest request = RequestConverter.buildMultiRequest(regionInfo.getRegionName(),
actions);
MultiResponse protoResults = remoteSvr.replay(null, request);
// check if it's a partial success
List<ActionResult> resultList = protoResults.getResultList();
for (int i = 0, n = resultList.size(); i < n; i++) {
ActionResult result = resultList.get(i);
if (result.hasException()) {
Throwable t = ProtobufUtil.toException(result.getException());
if (!skipErrors) {
IOException ie = new IOException();
ie.initCause(t);
// retry
throw ie;
} else {
LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
+ "=true so continuing replayToServer with error:" + t.getMessage());
return;
}
}
}
}
@Override
public void prepare(boolean reload) throws IOException {
if (!reload) return;
// relocate regions in case we have a new dead server or network hiccup
// if not due to connection issue, the following code should run fast because it uses
// cached location
for (Action<Row> action : actions) {
// use first row to relocate region because all actions are for one region
this.location = this.connection.locateRegion(tableName, action.getAction().getRow());
break;
}
}
}
}

View File

@ -0,0 +1,95 @@
/**
*
* 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.zookeeper;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.zookeeper.KeeperException;
/**
* Watcher used to be notified of the recovering region coming out of recovering state
*/
@InterfaceAudience.Private
public class RecoveringRegionWatcher extends ZooKeeperListener {
private static final Log LOG = LogFactory.getLog(RecoveringRegionWatcher.class);
private HRegionServer server;
/**
* Construct a ZooKeeper event listener.
*/
public RecoveringRegionWatcher(ZooKeeperWatcher watcher, HRegionServer server) {
super(watcher);
watcher.registerListener(this);
this.server = server;
}
/**
* Called when a node has been deleted
* @param path full path of the deleted node
*/
public void nodeDeleted(String path) {
if (this.server.isStopped() || this.server.isStopping()) {
return;
}
String parentPath = path.substring(0, path.lastIndexOf('/'));
if (!this.watcher.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) {
return;
}
String regionName = path.substring(parentPath.length() + 1);
HRegion region = this.server.getRecoveringRegions().remove(regionName);
if (region != null) {
region.setRecovering(false);
}
LOG.info(path + " znode deleted. Region: " + regionName + " completes recovery.");
}
@Override
public void nodeDataChanged(String path) {
registerWatcher(path);
}
@Override
public void nodeChildrenChanged(String path) {
registerWatcher(path);
}
/**
* Reinstall watcher because watcher only fire once though we're only interested in nodeDeleted
* event we need to register the watcher in case other event happens
*/
private void registerWatcher(String path) {
String parentPath = path.substring(0, path.lastIndexOf('/'));
if (!this.watcher.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) {
return;
}
try {
ZKUtil.getDataAndWatch(watcher, path);
} catch (KeeperException e) {
LOG.warn("Can't register watcher on znode " + path, e);
}
}
}

View File

@ -217,6 +217,7 @@ public class TestIOFencing {
public void doTest(Class<?> regionClass) throws Exception { public void doTest(Class<?> regionClass) throws Exception {
Configuration c = TEST_UTIL.getConfiguration(); Configuration c = TEST_UTIL.getConfiguration();
c.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
// Insert our custom region // Insert our custom region
c.setClass(HConstants.REGION_IMPL, regionClass, HRegion.class); c.setClass(HConstants.REGION_IMPL, regionClass, HRegion.class);
c.setBoolean("dfs.support.append", true); c.setBoolean("dfs.support.append", true);

View File

@ -58,7 +58,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); // Let's fail fast. conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); // Let's fail fast.
conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, BuggyRegionObserver.class.getName()); conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, BuggyRegionObserver.class.getName());
conf.set("hbase.coprocessor.abortonerror", "true"); conf.set("hbase.coprocessor.abortonerror", "true");
TEST_UTIL.startMiniCluster(2); TEST_UTIL.startMiniCluster(3);
} }
@AfterClass @AfterClass

View File

@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
@ -520,4 +521,17 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
public ExecutorService getExecutorService() { public ExecutorService getExecutorService() {
return null; return null;
} }
@Override
public MultiResponse replay(RpcController controller, MultiRequest request)
throws ServiceException {
// TODO Auto-generated method stub
return null;
}
@Override
public Map<String, HRegion> getRecoveringRegions() {
// TODO Auto-generated method stub
return null;
}
} }

View File

@ -331,6 +331,12 @@ public class TestCatalogJanitor {
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
boolean forcible) throws IOException { boolean forcible) throws IOException {
} }
@Override
public boolean isInitialized() {
// Auto-generated method stub
return false;
}
} }
@Test @Test

View File

@ -18,15 +18,25 @@
*/ */
package org.apache.hadoop.hbase.master; package org.apache.hadoop.hbase.master;
import static org.apache.hadoop.hbase.SplitLogCounters.*; import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_wait_for_zk_delete;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_final_transition_failed;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_preempt_task;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_acquired;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_done;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_err;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_resigned;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.NavigableSet; import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
@ -42,7 +52,17 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch; import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
@ -56,9 +76,12 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
@ -89,9 +112,14 @@ public class TestDistributedLogSplitting {
HBaseTestingUtility TEST_UTIL; HBaseTestingUtility TEST_UTIL;
private void startCluster(int num_rs) throws Exception{ private void startCluster(int num_rs) throws Exception{
conf = HBaseConfiguration.create();
startCluster(num_rs, conf);
}
private void startCluster(int num_rs, Configuration inConf) throws Exception {
SplitLogCounters.resetCounters(); SplitLogCounters.resetCounters();
LOG.info("Starting cluster"); LOG.info("Starting cluster");
conf = HBaseConfiguration.create(); this.conf = inConf;
conf.getLong("hbase.splitlog.max.resubmit", 0); conf.getLong("hbase.splitlog.max.resubmit", 0);
// Make the failure test faster // Make the failure test faster
conf.setInt("zookeeper.recovery.retry", 0); conf.setInt("zookeeper.recovery.retry", 0);
@ -111,13 +139,20 @@ public class TestDistributedLogSplitting {
@After @After
public void after() throws Exception { public void after() throws Exception {
for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
mt.getMaster().abort("closing...", new Exception("Trace info"));
}
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }
@Test (timeout=300000) @Test (timeout=300000)
public void testRecoveredEdits() throws Exception { public void testRecoveredEdits() throws Exception {
LOG.info("testRecoveredEdits"); LOG.info("testRecoveredEdits");
startCluster(NUM_RS); Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
startCluster(NUM_RS, curConf);
final int NUM_LOG_LINES = 1000; final int NUM_LOG_LINES = 1000;
final SplitLogManager slm = master.getMasterFileSystem().splitLogManager; final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
// turn off load balancing to prevent regions from moving around otherwise // turn off load balancing to prevent regions from moving around otherwise
@ -150,8 +185,7 @@ public class TestDistributedLogSplitting {
it.remove(); it.remove();
} }
} }
makeHLog(hrs.getWAL(), regions, "table", makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
NUM_LOG_LINES, 100);
slm.splitLogDistributed(logDir); slm.splitLogDistributed(logDir);
@ -172,6 +206,597 @@ public class TestDistributedLogSplitting {
assertEquals(NUM_LOG_LINES, count); assertEquals(NUM_LOG_LINES, count);
} }
@Test(timeout = 300000)
public void testLogReplayWithNonMetaRSDown() throws Exception {
LOG.info("testLogReplayWithNonMetaRSDown");
Configuration curConf = HBaseConfiguration.create();
curConf.setLong("hbase.regionserver.hlog.blocksize", 100*1024);
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
startCluster(NUM_RS, curConf);
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_LOG_LINES = 1000;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
}
}
if (isCarryingMeta) {
continue;
}
break;
}
LOG.info("#regions = " + regions.size());
Iterator<HRegionInfo> it = regions.iterator();
while (it.hasNext()) {
HRegionInfo region = it.next();
if (region.isMetaTable()) {
it.remove();
}
}
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
// wait for abort completes
this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
ht.close();
}
@Test(timeout = 300000)
public void testLogReplayWithMetaRSDown() throws Exception {
LOG.info("testRecoveredEditsReplayWithMetaRSDown");
Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
startCluster(NUM_RS, curConf);
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_LOG_LINES = 1000;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
}
}
if (!isCarryingMeta) {
continue;
}
break;
}
LOG.info("#regions = " + regions.size());
Iterator<HRegionInfo> it = regions.iterator();
while (it.hasNext()) {
HRegionInfo region = it.next();
if (region.isMetaTable()) {
it.remove();
}
}
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
ht.close();
}
private void abortRSAndVerifyRecovery(HRegionServer hrs, HTable ht, final ZooKeeperWatcher zkw,
final int numRegions, final int numofLines) throws Exception {
abortRSAndWaitForRecovery(hrs, zkw, numRegions);
assertEquals(numofLines, TEST_UTIL.countRows(ht));
}
private void abortRSAndWaitForRecovery(HRegionServer hrs, final ZooKeeperWatcher zkw,
final int numRegions) throws Exception {
final MiniHBaseCluster tmpCluster = this.cluster;
// abort RS
LOG.info("Aborting region server: " + hrs.getServerName());
hrs.abort("testing");
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (tmpCluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
}
});
// wait for regions come online
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(tmpCluster).size() >= (numRegions + 1));
}
});
// wait for all regions are fully recovered
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
zkw.recoveringRegionsZNode, false);
return (recoveringRegions != null && recoveringRegions.size() == 0);
}
});
}
@Test(timeout = 300000)
public void testMasterStartsUpWithLogSplittingWork() throws Exception {
LOG.info("testMasterStartsUpWithLogSplittingWork");
Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
curConf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
startCluster(NUM_RS, curConf);
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_LOG_LINES = 1000;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
}
}
if (isCarryingMeta) {
continue;
}
break;
}
LOG.info("#regions = " + regions.size());
Iterator<HRegionInfo> it = regions.iterator();
while (it.hasNext()) {
HRegionInfo region = it.next();
if (region.isMetaTable()) {
it.remove();
}
}
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
// abort master
abortMaster(cluster);
// abort RS
int numRS = cluster.getLiveRegionServerThreads().size();
LOG.info("Aborting region server: " + hrs.getServerName());
hrs.abort("testing");
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
}
});
Thread.sleep(2000);
LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size());
startMasterAndWaitUntilLogSplit(cluster);
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
}
});
LOG.info("Current Open Regions After Master Node Starts Up:"
+ getAllOnlineRegions(cluster).size());
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
ht.close();
}
@Test(timeout = 300000)
public void testMasterStartsUpWithLogReplayWork() throws Exception {
LOG.info("testMasterStartsUpWithLogReplayWork");
Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
curConf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
startCluster(NUM_RS, curConf);
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_LOG_LINES = 1000;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
}
}
if (isCarryingMeta) {
continue;
}
break;
}
LOG.info("#regions = " + regions.size());
Iterator<HRegionInfo> it = regions.iterator();
while (it.hasNext()) {
HRegionInfo region = it.next();
if (region.isMetaTable()) {
it.remove();
}
}
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
// abort master
abortMaster(cluster);
// abort RS
int numRS = cluster.getLiveRegionServerThreads().size();
LOG.info("Aborting region server: " + hrs.getServerName());
hrs.abort("testing");
// wait for the RS dies
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
}
});
Thread.sleep(2000);
LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size());
startMasterAndWaitUntilLogSplit(cluster);
// wait for all regions are fully recovered
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
zkw.recoveringRegionsZNode, false);
return (recoveringRegions != null && recoveringRegions.size() == 0);
}
});
LOG.info("Current Open Regions After Master Node Starts Up:"
+ getAllOnlineRegions(cluster).size());
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
ht.close();
}
@Test(timeout = 300000)
public void testLogReplayTwoSequentialRSDown() throws Exception {
LOG.info("testRecoveredEditsReplayTwoSequentialRSDown");
Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
startCluster(NUM_RS, curConf);
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_LOG_LINES = 1000;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs1 = rsts.get(0).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs1);
makeHLog(hrs1.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
// abort RS1
LOG.info("Aborting region server: " + hrs1.getServerName());
hrs1.abort("testing");
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
}
});
// wait for regions come online
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
}
});
// sleep a little bit in order to interrupt recovering in the middle
Thread.sleep(300);
// abort second region server
rsts = cluster.getLiveRegionServerThreads();
HRegionServer hrs2 = rsts.get(0).getRegionServer();
LOG.info("Aborting one more region server: " + hrs2.getServerName());
hrs2.abort("testing");
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 2));
}
});
// wait for regions come online
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
}
});
// wait for all regions are fully recovered
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
zkw.recoveringRegionsZNode, false);
return (recoveringRegions != null && recoveringRegions.size() == 0);
}
});
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
ht.close();
}
@Test(timeout = 300000)
public void testMarkRegionsRecoveringInZK() throws Exception {
LOG.info("testMarkRegionsRecoveringInZK");
Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
startCluster(NUM_RS, curConf);
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = master.getZooKeeperWatcher();
HTable ht = installTable(zkw, "table", "family", 40);
final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
final HRegionServer hrs = rsts.get(0).getRegionServer();
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
HRegionInfo region = regions.get(0);
Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
regionSet.add(region);
slm.markRegionsRecoveringInZK(rsts.get(0).getRegionServer().getServerName(), regionSet);
slm.markRegionsRecoveringInZK(rsts.get(1).getRegionServer().getServerName(), regionSet);
List<String> recoveringRegions = ZKUtil.listChildrenNoWatch(zkw,
ZKUtil.joinZNode(zkw.recoveringRegionsZNode, region.getEncodedName()));
assertEquals(recoveringRegions.size(), 2);
// wait for splitLogWorker to mark them up because there is no WAL files recorded in ZK
TEST_UTIL.waitFor(60000, 1000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (hrs.getRecoveringRegions().size() == 0);
}
});
ht.close();
}
@Test(timeout = 300000)
public void testReplayCmd() throws Exception {
LOG.info("testReplayCmd");
Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
startCluster(NUM_RS, curConf);
final int NUM_REGIONS_TO_CREATE = 40;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
}
}
if (isCarryingMeta) {
continue;
}
break;
}
LOG.info("#regions = " + regions.size());
Iterator<HRegionInfo> it = regions.iterator();
while (it.hasNext()) {
HRegionInfo region = it.next();
if (region.isMetaTable()) {
it.remove();
}
}
this.prepareData(ht, Bytes.toBytes("family"), Bytes.toBytes("c1"));
String originalCheckSum = TEST_UTIL.checksumRows(ht);
// abort RA and trigger replay
abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
assertEquals("Data should remain after reopening of regions", originalCheckSum,
TEST_UTIL.checksumRows(ht));
ht.close();
}
@Test(timeout = 300000)
public void testLogReplayForDisablingTable() throws Exception {
LOG.info("testLogReplayWithNonMetaRSDown");
Configuration curConf = HBaseConfiguration.create();
curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
startCluster(NUM_RS, curConf);
final int NUM_REGIONS_TO_CREATE = 40;
final int NUM_LOG_LINES = 1000;
// turn off load balancing to prevent regions from moving around otherwise
// they will consume recovered.edits
master.balanceSwitch(false);
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable disablingHT = installTable(zkw, "disableTable", "family", NUM_REGIONS_TO_CREATE);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE, NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs = null;
for (int i = 0; i < NUM_RS; i++) {
boolean isCarryingMeta = false;
hrs = rsts.get(i).getRegionServer();
regions = ProtobufUtil.getOnlineRegions(hrs);
for (HRegionInfo region : regions) {
if (region.isMetaRegion()) {
isCarryingMeta = true;
break;
}
}
if (isCarryingMeta) {
continue;
}
break;
}
LOG.info("#regions = " + regions.size());
Iterator<HRegionInfo> it = regions.iterator();
while (it.hasNext()) {
HRegionInfo region = it.next();
if (region.isMetaTable()) {
it.remove();
}
}
makeHLog(hrs.getWAL(), regions, "disableTable", "family", NUM_LOG_LINES, 100, false);
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
LOG.info("Disabling table\n");
TEST_UTIL.getHBaseAdmin().disableTable(Bytes.toBytes("disableTable"));
// abort RS
LOG.info("Aborting region server: " + hrs.getServerName());
hrs.abort("testing");
// wait for abort completes
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
}
});
// wait for regions come online
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1));
}
});
// wait for all regions are fully recovered
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
zkw.recoveringRegionsZNode, false);
return (recoveringRegions != null && recoveringRegions.size() == 0);
}
});
int count = 0;
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path rootdir = FSUtils.getRootDir(conf);
Path tdir = HTableDescriptor.getTableDir(rootdir, Bytes.toBytes("disableTable"));
for (HRegionInfo hri : regions) {
@SuppressWarnings("deprecation")
Path editsdir =
HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
LOG.debug("checking edits dir " + editsdir);
FileStatus[] files = fs.listStatus(editsdir);
if(files != null) {
for(FileStatus file : files) {
int c = countHLog(file.getPath(), fs, conf);
count += c;
LOG.info(c + " edits in " + file.getPath());
}
}
}
LOG.info("Verify edits in recovered.edits files");
assertEquals(NUM_LOG_LINES, count);
LOG.info("Verify replayed edits");
assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
// clean up
for (HRegionInfo hri : regions) {
@SuppressWarnings("deprecation")
Path editsdir =
HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName()));
fs.delete(editsdir, true);
}
disablingHT.close();
ht.close();
}
/** /**
* The original intention of this test was to force an abort of a region * The original intention of this test was to force an abort of a region
* server and to make sure that the failure path in the region servers is * server and to make sure that the failure path in the region servers is
@ -197,8 +822,9 @@ public class TestDistributedLogSplitting {
installTable(new ZooKeeperWatcher(conf, "table-creation", null), installTable(new ZooKeeperWatcher(conf, "table-creation", null),
"table", "family", 40); "table", "family", 40);
makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs), "table",
NUM_LOG_LINES, 100); makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs), "table", "family", NUM_LOG_LINES,
100);
new Thread() { new Thread() {
public void run() { public void run() {
@ -244,7 +870,7 @@ public class TestDistributedLogSplitting {
startCluster(NUM_RS); // NUM_RS=6. startCluster(NUM_RS); // NUM_RS=6.
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
"distributed log splitting test", null); "distributed log splitting test", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
@ -273,6 +899,16 @@ public class TestDistributedLogSplitting {
Thread.sleep(200); Thread.sleep(200);
} }
// wait for all regions are fully recovered
TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
List<String> recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
zkw.recoveringRegionsZNode, false);
return (recoveringRegions != null && recoveringRegions.size() == 0);
}
});
assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION, assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
TEST_UTIL.countRows(ht)); TEST_UTIL.countRows(ht));
ht.close(); ht.close();
@ -344,8 +980,12 @@ public class TestDistributedLogSplitting {
} }
} }
HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
int nrs ) throws Exception { return installTable(zkw, tname, fname, nrs, 0);
}
HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
int existingRegions) throws Exception {
// Create a table with regions // Create a table with regions
byte [] table = Bytes.toBytes(tname); byte [] table = Bytes.toBytes(tname);
byte [] family = Bytes.toBytes(fname); byte [] family = Bytes.toBytes(fname);
@ -367,14 +1007,14 @@ public class TestDistributedLogSplitting {
for (String oregion : regions) for (String oregion : regions)
LOG.debug("Region still online: " + oregion); LOG.debug("Region still online: " + oregion);
} }
assertEquals(1, regions.size()); assertEquals(1 + existingRegions, regions.size());
LOG.debug("Enabling table\n"); LOG.debug("Enabling table\n");
TEST_UTIL.getHBaseAdmin().enableTable(table); TEST_UTIL.getHBaseAdmin().enableTable(table);
LOG.debug("Waiting for no more RIT\n"); LOG.debug("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master); blockUntilNoRIT(zkw, master);
LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n"); LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
regions = getAllOnlineRegions(cluster); regions = getAllOnlineRegions(cluster);
assertEquals(numRegions + 1, regions.size()); assertEquals(numRegions + 1 + existingRegions, regions.size());
return ht; return ht;
} }
@ -400,40 +1040,57 @@ public class TestDistributedLogSplitting {
} }
} }
public void makeHLog(HLog log, public void makeHLog(HLog log, List<HRegionInfo> regions, String tname, String fname,
List<HRegionInfo> hris, String tname,
int num_edits, int edit_size) throws IOException { int num_edits, int edit_size) throws IOException {
makeHLog(log, regions, tname, fname, num_edits, edit_size, true);
}
public void makeHLog(HLog log, List<HRegionInfo> regions, String tname, String fname,
int num_edits, int edit_size, boolean closeLog) throws IOException {
// remove root and meta region // remove root and meta region
hris.remove(HRegionInfo.ROOT_REGIONINFO); regions.remove(HRegionInfo.FIRST_META_REGIONINFO);
hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
byte[] table = Bytes.toBytes(tname); byte[] table = Bytes.toBytes(tname);
HTableDescriptor htd = new HTableDescriptor(tname); HTableDescriptor htd = new HTableDescriptor(tname);
byte[] value = new byte[edit_size]; byte[] value = new byte[edit_size];
List<HRegionInfo> hris = new ArrayList<HRegionInfo>();
for (HRegionInfo region : regions) {
if (!region.getTableNameAsString().equalsIgnoreCase(tname)) {
continue;
}
hris.add(region);
}
for (int i = 0; i < edit_size; i++) { for (int i = 0; i < edit_size; i++) {
value[i] = (byte) ('a' + (i % 26)); value[i] = (byte) ('a' + (i % 26));
} }
int n = hris.size(); int n = hris.size();
int[] counts = new int[n]; int[] counts = new int[n];
int j = 0;
if (n > 0) { if (n > 0) {
for (int i = 0; i < num_edits; i += 1) { for (int i = 0; i < num_edits; i += 1) {
WALEdit e = new WALEdit(); WALEdit e = new WALEdit();
byte [] row = Bytes.toBytes("r" + Integer.toString(i)); HRegionInfo curRegionInfo = hris.get(i % n);
byte [] family = Bytes.toBytes("f"); byte[] startRow = curRegionInfo.getStartKey();
if (startRow == null || startRow.length == 0) {
startRow = new byte[] { 0, 0, 0, 0, 1 };
}
byte[] row = Bytes.incrementBytes(startRow, counts[i % n]);
row = Arrays.copyOfRange(row, 3, 8); // use last 5 bytes because
// HBaseTestingUtility.createMultiRegions use 5 bytes
// key
byte[] family = Bytes.toBytes(fname);
byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i)); byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
e.add(new KeyValue(row, family, qualifier, e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
System.currentTimeMillis(), value)); log.append(curRegionInfo, table, e, System.currentTimeMillis(), htd);
j++; counts[i % n] += 1;
log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd);
counts[j % n] += 1;
} }
} }
log.sync(); log.sync();
if(closeLog) {
log.close(); log.close();
}
for (int i = 0; i < n; i++) { for (int i = 0; i < n; i++) {
LOG.info("region " + hris.get(i).getRegionNameAsString() + LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits");
" has " + counts[i] + " edits");
} }
return; return;
} }
@ -466,6 +1123,38 @@ public class TestDistributedLogSplitting {
} }
} }
/**
* Load table with puts and deletes with expected values so that we can verify later
*/
private void prepareData(final HTable t, final byte[] f, final byte[] column) throws IOException {
t.setAutoFlush(false);
byte[] k = new byte[3];
// add puts
for (byte b1 = 'a'; b1 <= 'z'; b1++) {
for (byte b2 = 'a'; b2 <= 'z'; b2++) {
for (byte b3 = 'a'; b3 <= 'z'; b3++) {
k[0] = b1;
k[1] = b2;
k[2] = b3;
Put put = new Put(k);
put.add(f, column, k);
t.put(put);
}
}
}
t.flushCommits();
// add deletes
for (byte b3 = 'a'; b3 <= 'z'; b3++) {
k[0] = 'a';
k[1] = 'a';
k[2] = b3;
Delete del = new Delete(k);
t.delete(del);
}
t.flushCommits();
}
private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster) private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
throws IOException { throws IOException {
NavigableSet<String> online = new TreeSet<String>(); NavigableSet<String> online = new TreeSet<String>();
@ -493,4 +1182,27 @@ public class TestDistributedLogSplitting {
assertTrue(false); assertTrue(false);
} }
private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException {
for (MasterThread mt : cluster.getLiveMasterThreads()) {
if (mt.getMaster().isActiveMaster()) {
mt.getMaster().abort("Aborting for tests", new Exception("Trace info"));
mt.join();
break;
}
}
LOG.debug("Master is aborted");
}
private void startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster)
throws IOException, InterruptedException {
cluster.startMaster();
HMaster master = cluster.getMaster();
while (!master.isInitialized()) {
Thread.sleep(100);
}
ServerManager serverManager = master.getServerManager();
while (serverManager.areDeadServersInProgress()) {
Thread.sleep(100);
}
}
} }

View File

@ -783,6 +783,7 @@ public class TestMasterFailover {
while (master.getServerManager().areDeadServersInProgress()) { while (master.getServerManager().areDeadServersInProgress()) {
Thread.sleep(10); Thread.sleep(10);
} }
// Failover should be completed, now wait for no RIT // Failover should be completed, now wait for no RIT
log("Waiting for no more RIT"); log("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw); ZKAssign.blockUntilNoRIT(zkw);

View File

@ -18,14 +18,28 @@
package org.apache.hadoop.hbase.master; package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.HashSet;
import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -63,4 +77,40 @@ public class TestMasterFileSystem {
assertEquals(masterRoot, rootDir); assertEquals(masterRoot, rootDir);
} }
@Test
public void testRemoveStaleRecoveringRegionsDuringMasterInitialization() throws Exception {
LOG.info("Starting testRemoveStaleRecoveringRegionsDuringMasterInitialization");
HMaster master = UTIL.getMiniHBaseCluster().getMaster();
MasterFileSystem fs = master.getMasterFileSystem();
String failedRegion = "failedRegoin1";
String staleRegion = "staleRegion";
ServerName inRecoveryServerName = new ServerName("mgr,1,1");
ServerName previouselyFaildServerName = new ServerName("previous,1,1");
String walPath = "/hbase/data/.logs/" + inRecoveryServerName.getServerName()
+ "-splitting/test";
// Create a ZKW to use in the test
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(UTIL);
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, walPath),
new SplitLogTask.Owned(inRecoveryServerName).toByteArray(), Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
String staleRegionPath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, staleRegion);
ZKUtil.createWithParents(zkw, staleRegionPath);
String inRecoveringRegionPath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, failedRegion);
inRecoveringRegionPath = ZKUtil.joinZNode(inRecoveringRegionPath,
inRecoveryServerName.getServerName());
ZKUtil.createWithParents(zkw, inRecoveringRegionPath);
Set<ServerName> servers = new HashSet<ServerName>();
servers.add(previouselyFaildServerName);
fs.removeStaleRecoveringRegionsFromZK(servers);
// verification
assertFalse(ZKUtil.checkExists(zkw, staleRegionPath) != -1);
assertTrue(ZKUtil.checkExists(zkw, inRecoveringRegionPath) != -1);
ZKUtil.deleteChildrenRecursively(zkw, zkw.recoveringRegionsZNode);
ZKUtil.deleteChildrenRecursively(zkw, zkw.splitLogZNode);
zkw.close();
}
} }

View File

@ -355,8 +355,8 @@ public class TestMasterNoCluster {
HMaster master = new HMaster(conf) { HMaster master = new HMaster(conf) {
@Override @Override
boolean assignMeta(MonitoredTask status) { void assignMeta(MonitoredTask status) {
return true; return;
} }
@Override @Override

View File

@ -101,23 +101,6 @@ public class TestRSKilledWhenMasterInitializing {
KeeperException, InterruptedException { KeeperException, InterruptedException {
super(conf); super(conf);
} }
@Override
protected void splitLogAfterStartup(MasterFileSystem mfs) {
super.splitLogAfterStartup(mfs);
logSplit = true;
// If "TestingMaster.sleep" is set, sleep after log split.
if (getConfiguration().getBoolean("TestingMaster.sleep", false)) {
int duration = getConfiguration().getInt(
"TestingMaster.sleep.duration", 0);
Threads.sleep(duration);
}
}
public boolean isLogSplitAfterStartup() {
return logSplit;
}
} }
@Test(timeout = 120000) @Test(timeout = 120000)
@ -163,7 +146,7 @@ public class TestRSKilledWhenMasterInitializing {
/* NO.1 .META. region correctness */ /* NO.1 .META. region correctness */
// First abort master // First abort master
abortMaster(cluster); abortMaster(cluster);
TestingMaster master = startMasterAndWaitUntilLogSplit(cluster); TestingMaster master = startMasterAndWaitTillMetaRegionAssignment(cluster);
// Second kill meta server // Second kill meta server
int metaServerNum = cluster.getServerWithMeta(); int metaServerNum = cluster.getServerWithMeta();
@ -216,14 +199,12 @@ public class TestRSKilledWhenMasterInitializing {
LOG.debug("Master is aborted"); LOG.debug("Master is aborted");
} }
private TestingMaster startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster) private TestingMaster startMasterAndWaitTillMetaRegionAssignment(MiniHBaseCluster cluster)
throws IOException, InterruptedException { throws IOException, InterruptedException {
TestingMaster master = (TestingMaster) cluster.startMaster().getMaster(); TestingMaster master = (TestingMaster) cluster.startMaster().getMaster();
while (!master.isLogSplitAfterStartup()) { while (!master.isInitializationStartsMetaRegionAssignment()) {
Thread.sleep(100); Thread.sleep(100);
} }
LOG.debug("splitted:" + master.isLogSplitAfterStartup() + ",initialized:"
+ master.isInitialized());
return master; return master;
} }
@ -232,7 +213,9 @@ public class TestRSKilledWhenMasterInitializing {
while (!master.isInitialized()) { while (!master.isInitialized()) {
Thread.sleep(100); Thread.sleep(100);
} }
while (master.getServerManager().areDeadServersInProgress()) {
Thread.sleep(100);
}
LOG.debug("master isInitialized"); LOG.debug("master isInitialized");
} }
} }

View File

@ -1221,7 +1221,7 @@ public class TestHLogSplit {
generateHLogs(1, 10, -1); generateHLogs(1, 10, -1);
FileStatus logfile = fs.listStatus(HLOGDIR)[0]; FileStatus logfile = fs.listStatus(HLOGDIR)[0];
fs.initialize(fs.getUri(), conf); fs.initialize(fs.getUri(), conf);
HLogSplitter.splitLogFile(HBASEDIR, logfile, fs, conf, reporter, null); HLogSplitter.splitLogFile(HBASEDIR, logfile, fs, conf, reporter);
HLogSplitter.finishSplitLogFile(HBASEDIR, OLDLOGDIR, logfile.getPath() HLogSplitter.finishSplitLogFile(HBASEDIR, OLDLOGDIR, logfile.getPath()
.toString(), conf); .toString(), conf);

View File

@ -191,4 +191,10 @@ public class MockRegionServerServices implements RegionServerServices {
public ExecutorService getExecutorService() { public ExecutorService getExecutorService() {
return null; return null;
} }
@Override
public Map<String, HRegion> getRecoveringRegions() {
// TODO Auto-generated method stub
return null;
}
} }