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:
parent
6bf0abbec9
commit
cb53db35f3
|
@ -185,6 +185,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
private byte[] tableName = 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 */
|
||||
public static final HRegionInfo ROOT_REGIONINFO =
|
||||
new HRegionInfo(0L, Bytes.toBytes("-ROOT-"));
|
||||
|
@ -293,6 +296,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
this.startKey = startKey == null?
|
||||
HConstants.EMPTY_START_ROW: startKey.clone();
|
||||
this.tableName = tableName.clone();
|
||||
this.recovering = false;
|
||||
setHashCode();
|
||||
}
|
||||
|
||||
|
@ -313,6 +317,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
this.hashCode = other.hashCode();
|
||||
this.encodedName = other.getEncodedName();
|
||||
this.tableName = other.tableName;
|
||||
this.recovering = other.isRecovering();
|
||||
}
|
||||
|
||||
|
||||
|
@ -596,6 +601,20 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
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.
|
||||
*/
|
||||
|
@ -833,6 +852,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
}
|
||||
builder.setOffline(info.isOffline());
|
||||
builder.setSplit(info.isSplit());
|
||||
builder.setRecovering(info.isRecovering());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -865,6 +885,9 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
if (proto.hasOffline()) {
|
||||
hri.setOffline(proto.getOffline());
|
||||
}
|
||||
if (proto.hasRecovering()) {
|
||||
hri.setRecovering(proto.getRecovering());
|
||||
}
|
||||
return hri;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,14 @@
|
|||
|
||||
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.InterfaceStability;
|
||||
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.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
|
||||
@InterfaceStability.Evolving
|
||||
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 long ts = HConstants.LATEST_TIMESTAMP;
|
||||
protected Durability durability = Durability.USE_DEFAULT;
|
||||
|
||||
// A Map sorted by column family.
|
||||
protected NavigableMap<byte [], List<? extends Cell>> familyMap =
|
||||
new TreeMap<byte [], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
|
||||
|
|
|
@ -178,7 +178,7 @@ public abstract class ServerCallable<T> implements Callable<T> {
|
|||
if (t instanceof SocketTimeoutException ||
|
||||
t instanceof ConnectException ||
|
||||
t instanceof RetriesExhaustedException ||
|
||||
getConnection().isDeadServer(location.getServerName())) {
|
||||
(location != null && getConnection().isDeadServer(location.getServerName()))) {
|
||||
// if thrown these exceptions, we clear all the cache entries that
|
||||
// map to that slow/dead server; otherwise, let cache miss and ask
|
||||
// .META. again to find the new location
|
||||
|
@ -261,7 +261,9 @@ public abstract class ServerCallable<T> implements Callable<T> {
|
|||
*/
|
||||
protected static Throwable translateException(Throwable t) throws DoNotRetryIOException {
|
||||
if (t instanceof UndeclaredThrowableException) {
|
||||
t = t.getCause();
|
||||
if(t.getCause() != null) {
|
||||
t = t.getCause();
|
||||
}
|
||||
}
|
||||
if (t instanceof RemoteException) {
|
||||
t = ((RemoteException)t).unwrapRemoteException();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -431,6 +431,30 @@ public class ReplicationZookeeper implements Closeable {
|
|||
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.
|
||||
* @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 {
|
||||
return ZKUtil.checkExists(this.zookeeper,
|
||||
ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
|
||||
|
|
|
@ -376,6 +376,7 @@ public class ZKTable {
|
|||
Set<String> allTables = new HashSet<String>();
|
||||
List<String> children =
|
||||
ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
|
||||
if(children == null) return allTables;
|
||||
for (String child: children) {
|
||||
ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(zkw, child);
|
||||
for (ZooKeeperProtos.Table.State expectedState: states) {
|
||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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.Threads;
|
||||
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.server.ZooKeeperSaslServer;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
import javax.security.auth.login.AppConfigurationEntry;
|
||||
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -107,6 +107,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
public String balancerZNode;
|
||||
// znode containing the lock for the tables
|
||||
public String tableLockZNode;
|
||||
// znode containing the state of recovering regions
|
||||
public String recoveringRegionsZNode;
|
||||
|
||||
// Certain ZooKeeper nodes need to be world-readable
|
||||
public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
|
||||
|
@ -133,11 +135,11 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
|
||||
/**
|
||||
* 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 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
|
||||
* context.
|
||||
* context.
|
||||
* @param canCreateBaseZNode
|
||||
* @throws IOException
|
||||
* @throws ZooKeeperConnectionException
|
||||
|
@ -176,6 +178,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
ZKUtil.createAndFailSilent(this, splitLogZNode);
|
||||
ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode);
|
||||
ZKUtil.createAndFailSilent(this, tableLockZNode);
|
||||
ZKUtil.createAndFailSilent(this, recoveringRegionsZNode);
|
||||
} catch (KeeperException e) {
|
||||
throw new ZooKeeperConnectionException(
|
||||
prefix("Unexpected KeeperException creating base node"), e);
|
||||
|
@ -227,6 +230,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
conf.get("zookeeper.znode.balancer", "balancer"));
|
||||
tableLockZNode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.tableLock", "table-lock"));
|
||||
recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.recovering.regions", "recovering-regions"));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -717,6 +717,14 @@ public final class HConstants {
|
|||
public static final String DISTRIBUTED_LOG_SPLITTING_KEY =
|
||||
"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 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 HIGH_QOS = 100;
|
||||
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 */
|
||||
public static final String HFILE_ARCHIVE_DIRECTORY = ".archive";
|
||||
|
|
|
@ -60,6 +60,8 @@ public interface MetricsMasterSource extends BaseSource {
|
|||
static final String SNAPSHOT_TIME_NAME = "snapshotTime";
|
||||
static final String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime";
|
||||
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 RIT_COUNT_NAME = "ritCount";
|
||||
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_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 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.
|
||||
|
@ -117,4 +120,9 @@ public interface MetricsMasterSource extends BaseSource {
|
|||
void updateSnapshotCloneTime(long time);
|
||||
|
||||
void updateSnapshotRestoreTime(long time);
|
||||
|
||||
void updateMetaWALSplitTime(long time);
|
||||
|
||||
void updateMetaWALSplitSize(long size);
|
||||
|
||||
}
|
||||
|
|
|
@ -80,6 +80,13 @@ public interface MetricsRegionServerSource extends BaseSource {
|
|||
*/
|
||||
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.
|
||||
*/
|
||||
|
@ -188,6 +195,7 @@ public interface MetricsRegionServerSource extends BaseSource {
|
|||
static final String INCREMENT_KEY = "increment";
|
||||
static final String MUTATE_KEY = "mutate";
|
||||
static final String APPEND_KEY = "append";
|
||||
static final String REPLAY_KEY = "replay";
|
||||
static final String SCAN_NEXT_KEY = "scanNext";
|
||||
static final String SLOW_MUTATE_KEY = "slowPutCount";
|
||||
static final String SLOW_GET_KEY = "slowGetCount";
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -48,6 +48,8 @@ public class MetricsMasterSourceImpl
|
|||
private MetricMutableStat snapshotTimeHisto;
|
||||
private MetricMutableStat snapshotCloneTimeHisto;
|
||||
private MetricMutableStat snapshotRestoreTimeHisto;
|
||||
private MetricMutableHistogram metaSplitTimeHisto;
|
||||
private MetricMutableHistogram metaSplitSizeHisto;
|
||||
|
||||
public MetricsMasterSourceImpl(MetricsMasterWrapper 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);
|
||||
snapshotRestoreTimeHisto = metricsRegistry.newStat(
|
||||
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) {
|
||||
|
@ -120,6 +124,16 @@ public class MetricsMasterSourceImpl
|
|||
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.
|
||||
*
|
||||
|
|
|
@ -38,6 +38,7 @@ public class MetricsRegionServerSourceImpl
|
|||
private final MetricHistogram getHisto;
|
||||
private final MetricHistogram incrementHisto;
|
||||
private final MetricHistogram appendHisto;
|
||||
private final MetricHistogram replayHisto;
|
||||
private final MetricMutableCounterLong slowPut;
|
||||
private final MetricMutableCounterLong slowDelete;
|
||||
private final MetricMutableCounterLong slowGet;
|
||||
|
@ -70,6 +71,8 @@ public class MetricsRegionServerSourceImpl
|
|||
|
||||
appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY);
|
||||
slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l);
|
||||
|
||||
replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -97,6 +100,11 @@ public class MetricsRegionServerSourceImpl
|
|||
appendHisto.add(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateReplay(long t) {
|
||||
replayHisto.add(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrSlowPut() {
|
||||
slowPut.incr();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1 @@
|
|||
org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySourceImpl
|
|
@ -45,6 +45,8 @@ public class MetricsMasterSourceImpl
|
|||
private MutableStat snapshotTimeHisto;
|
||||
private MutableStat snapshotCloneTimeHisto;
|
||||
private MutableStat snapshotRestoreTimeHisto;
|
||||
private MutableHistogram metaSplitTimeHisto;
|
||||
private MutableHistogram metaSplitSizeHisto;
|
||||
|
||||
public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
|
||||
this(METRICS_NAME,
|
||||
|
@ -79,6 +81,8 @@ public class MetricsMasterSourceImpl
|
|||
SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true);
|
||||
snapshotRestoreTimeHisto = metricsRegistry.newStat(
|
||||
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) {
|
||||
|
@ -122,6 +126,16 @@ public class MetricsMasterSourceImpl
|
|||
snapshotRestoreTimeHisto.add(time);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateMetaWALSplitTime(long time) {
|
||||
metaSplitTimeHisto.add(time);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateMetaWALSplitSize(long size) {
|
||||
metaSplitSizeHisto.add(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getMetrics(MetricsCollector metricsCollector, boolean all) {
|
||||
|
||||
|
|
|
@ -41,6 +41,7 @@ public class MetricsRegionServerSourceImpl
|
|||
private final MetricHistogram getHisto;
|
||||
private final MetricHistogram incrementHisto;
|
||||
private final MetricHistogram appendHisto;
|
||||
private final MetricHistogram replayHisto;
|
||||
|
||||
private final MutableCounterLong slowPut;
|
||||
private final MutableCounterLong slowDelete;
|
||||
|
@ -75,6 +76,8 @@ public class MetricsRegionServerSourceImpl
|
|||
|
||||
appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY);
|
||||
slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l);
|
||||
|
||||
replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -102,6 +105,11 @@ public class MetricsRegionServerSourceImpl
|
|||
appendHisto.add(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateReplay(long t) {
|
||||
replayHisto.add(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrSlowPut() {
|
||||
slowPut.incr();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1 @@
|
|||
org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySourceImpl
|
|
@ -14155,6 +14155,11 @@ public final class AdminProtos {
|
|||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request,
|
||||
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(
|
||||
com.google.protobuf.RpcController controller,
|
||||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request,
|
||||
|
@ -14255,6 +14260,14 @@ public final class AdminProtos {
|
|||
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
|
||||
public void rollWALWriter(
|
||||
com.google.protobuf.RpcController controller,
|
||||
|
@ -14322,10 +14335,12 @@ public final class AdminProtos {
|
|||
case 9:
|
||||
return impl.replicateWALEntry(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request);
|
||||
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:
|
||||
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:
|
||||
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);
|
||||
default:
|
||||
throw new java.lang.AssertionError("Can't get here.");
|
||||
|
@ -14362,10 +14377,12 @@ public final class AdminProtos {
|
|||
case 9:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance();
|
||||
case 10:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance();
|
||||
case 11:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance();
|
||||
case 12:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance();
|
||||
case 13:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance();
|
||||
default:
|
||||
throw new java.lang.AssertionError("Can't get here.");
|
||||
|
@ -14402,10 +14419,12 @@ public final class AdminProtos {
|
|||
case 9:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance();
|
||||
case 10:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance();
|
||||
case 11:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance();
|
||||
case 12:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance();
|
||||
case 13:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance();
|
||||
default:
|
||||
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,
|
||||
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(
|
||||
com.google.protobuf.RpcController controller,
|
||||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request,
|
||||
|
@ -14553,16 +14577,21 @@ public final class AdminProtos {
|
|||
done));
|
||||
return;
|
||||
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,
|
||||
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse>specializeCallback(
|
||||
done));
|
||||
return;
|
||||
case 11:
|
||||
case 12:
|
||||
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(
|
||||
done));
|
||||
return;
|
||||
case 12:
|
||||
case 13:
|
||||
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(
|
||||
done));
|
||||
|
@ -14602,10 +14631,12 @@ public final class AdminProtos {
|
|||
case 9:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance();
|
||||
case 10:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance();
|
||||
case 11:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance();
|
||||
case 12:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance();
|
||||
case 13:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance();
|
||||
default:
|
||||
throw new java.lang.AssertionError("Can't get here.");
|
||||
|
@ -14642,10 +14673,12 @@ public final class AdminProtos {
|
|||
case 9:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance();
|
||||
case 10:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance();
|
||||
case 11:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance();
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance();
|
||||
case 12:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance();
|
||||
case 13:
|
||||
return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance();
|
||||
default:
|
||||
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()));
|
||||
}
|
||||
|
||||
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(
|
||||
com.google.protobuf.RpcController controller,
|
||||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request,
|
||||
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse> done) {
|
||||
channel.callMethod(
|
||||
getDescriptor().getMethods().get(10),
|
||||
getDescriptor().getMethods().get(11),
|
||||
controller,
|
||||
request,
|
||||
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,
|
||||
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse> done) {
|
||||
channel.callMethod(
|
||||
getDescriptor().getMethods().get(11),
|
||||
getDescriptor().getMethods().get(12),
|
||||
controller,
|
||||
request,
|
||||
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,
|
||||
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse> done) {
|
||||
channel.callMethod(
|
||||
getDescriptor().getMethods().get(12),
|
||||
getDescriptor().getMethods().get(13),
|
||||
controller,
|
||||
request,
|
||||
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)
|
||||
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(
|
||||
com.google.protobuf.RpcController controller,
|
||||
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(
|
||||
com.google.protobuf.RpcController controller,
|
||||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request)
|
||||
throws com.google.protobuf.ServiceException {
|
||||
return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse) channel.callBlockingMethod(
|
||||
getDescriptor().getMethods().get(10),
|
||||
getDescriptor().getMethods().get(11),
|
||||
controller,
|
||||
request,
|
||||
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)
|
||||
throws com.google.protobuf.ServiceException {
|
||||
return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse) channel.callBlockingMethod(
|
||||
getDescriptor().getMethods().get(11),
|
||||
getDescriptor().getMethods().get(12),
|
||||
controller,
|
||||
request,
|
||||
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)
|
||||
throws com.google.protobuf.ServiceException {
|
||||
return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse) channel.callBlockingMethod(
|
||||
getDescriptor().getMethods().get(12),
|
||||
getDescriptor().getMethods().get(13),
|
||||
controller,
|
||||
request,
|
||||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance());
|
||||
|
@ -15255,78 +15320,80 @@ public final class AdminProtos {
|
|||
descriptor;
|
||||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\013Admin.proto\032\013hbase.proto\032\tWAL.proto\"Q\n" +
|
||||
"\024GetRegionInfoRequest\022 \n\006region\030\001 \002(\0132\020." +
|
||||
"RegionSpecifier\022\027\n\017compactionState\030\002 \001(\010" +
|
||||
"\"\301\001\n\025GetRegionInfoResponse\022\037\n\nregionInfo" +
|
||||
"\030\001 \002(\0132\013.RegionInfo\022?\n\017compactionState\030\002" +
|
||||
" \001(\0162&.GetRegionInfoResponse.CompactionS" +
|
||||
"tate\"F\n\017CompactionState\022\010\n\004NONE\020\000\022\t\n\005MIN" +
|
||||
"OR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_AND_MINOR\020\003\"G\n\023" +
|
||||
"GetStoreFileRequest\022 \n\006region\030\001 \002(\0132\020.Re" +
|
||||
"gionSpecifier\022\016\n\006family\030\002 \003(\014\")\n\024GetStor",
|
||||
"eFileResponse\022\021\n\tstoreFile\030\001 \003(\t\"\030\n\026GetO" +
|
||||
"nlineRegionRequest\":\n\027GetOnlineRegionRes" +
|
||||
"ponse\022\037\n\nregionInfo\030\001 \003(\0132\013.RegionInfo\"\270" +
|
||||
"\001\n\021OpenRegionRequest\0223\n\010openInfo\030\001 \003(\0132!" +
|
||||
".OpenRegionRequest.RegionOpenInfo\032n\n\016Reg" +
|
||||
"ionOpenInfo\022\033\n\006region\030\001 \002(\0132\013.RegionInfo" +
|
||||
"\022\034\n\024versionOfOfflineNode\030\002 \001(\r\022!\n\014favore" +
|
||||
"dNodes\030\003 \003(\0132\013.ServerName\"\234\001\n\022OpenRegion" +
|
||||
"Response\022<\n\014openingState\030\001 \003(\0162&.OpenReg" +
|
||||
"ionResponse.RegionOpeningState\"H\n\022Region",
|
||||
"OpeningState\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPEN" +
|
||||
"ED\020\001\022\022\n\016FAILED_OPENING\020\002\"\232\001\n\022CloseRegion" +
|
||||
"Request\022 \n\006region\030\001 \002(\0132\020.RegionSpecifie" +
|
||||
"r\022\034\n\024versionOfClosingNode\030\002 \001(\r\022\034\n\016trans" +
|
||||
"itionInZK\030\003 \001(\010:\004true\022&\n\021destinationServ" +
|
||||
"er\030\004 \001(\0132\013.ServerName\"%\n\023CloseRegionResp" +
|
||||
"onse\022\016\n\006closed\030\001 \002(\010\"M\n\022FlushRegionReque" +
|
||||
"st\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\025\n\r" +
|
||||
"ifOlderThanTs\030\002 \001(\004\"=\n\023FlushRegionRespon" +
|
||||
"se\022\025\n\rlastFlushTime\030\001 \002(\004\022\017\n\007flushed\030\002 \001",
|
||||
"(\010\"J\n\022SplitRegionRequest\022 \n\006region\030\001 \002(\013" +
|
||||
"2\020.RegionSpecifier\022\022\n\nsplitPoint\030\002 \001(\014\"\025" +
|
||||
"\n\023SplitRegionResponse\"W\n\024CompactRegionRe" +
|
||||
"quest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022" +
|
||||
"\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025Compac" +
|
||||
"tRegionResponse\"t\n\023MergeRegionsRequest\022!" +
|
||||
"\n\007regionA\030\001 \002(\0132\020.RegionSpecifier\022!\n\007reg" +
|
||||
"ionB\030\002 \002(\0132\020.RegionSpecifier\022\027\n\010forcible" +
|
||||
"\030\003 \001(\010:\005false\"\026\n\024MergeRegionsResponse\"7\n" +
|
||||
"\010WALEntry\022\024\n\003key\030\001 \002(\0132\007.WALKey\022\025\n\rkeyVa",
|
||||
"lueBytes\030\002 \003(\014\"4\n\030ReplicateWALEntryReque" +
|
||||
"st\022\030\n\005entry\030\001 \003(\0132\t.WALEntry\"\033\n\031Replicat" +
|
||||
"eWALEntryResponse\"\026\n\024RollWALWriterReques" +
|
||||
"t\".\n\025RollWALWriterResponse\022\025\n\rregionToFl" +
|
||||
"ush\030\001 \003(\014\"#\n\021StopServerRequest\022\016\n\006reason" +
|
||||
"\030\001 \002(\t\"\024\n\022StopServerResponse\"\026\n\024GetServe" +
|
||||
"rInfoRequest\"@\n\nServerInfo\022\037\n\nserverName" +
|
||||
"\030\001 \002(\0132\013.ServerName\022\021\n\twebuiPort\030\002 \001(\r\"8" +
|
||||
"\n\025GetServerInfoResponse\022\037\n\nserverInfo\030\001 " +
|
||||
"\002(\0132\013.ServerInfo2\266\006\n\014AdminService\022>\n\rget",
|
||||
"RegionInfo\022\025.GetRegionInfoRequest\032\026.GetR" +
|
||||
"egionInfoResponse\022;\n\014getStoreFile\022\024.GetS" +
|
||||
"toreFileRequest\032\025.GetStoreFileResponse\022D" +
|
||||
"\n\017getOnlineRegion\022\027.GetOnlineRegionReque" +
|
||||
"st\032\030.GetOnlineRegionResponse\0225\n\nopenRegi" +
|
||||
"on\022\022.OpenRegionRequest\032\023.OpenRegionRespo" +
|
||||
"nse\0228\n\013closeRegion\022\023.CloseRegionRequest\032" +
|
||||
"\024.CloseRegionResponse\0228\n\013flushRegion\022\023.F" +
|
||||
"lushRegionRequest\032\024.FlushRegionResponse\022" +
|
||||
"8\n\013splitRegion\022\023.SplitRegionRequest\032\024.Sp",
|
||||
"litRegionResponse\022>\n\rcompactRegion\022\025.Com" +
|
||||
"pactRegionRequest\032\026.CompactRegionRespons" +
|
||||
"e\022;\n\014mergeRegions\022\024.MergeRegionsRequest\032" +
|
||||
"\025.MergeRegionsResponse\022J\n\021replicateWALEn" +
|
||||
"try\022\031.ReplicateWALEntryRequest\032\032.Replica" +
|
||||
"teWALEntryResponse\022>\n\rrollWALWriter\022\025.Ro" +
|
||||
"llWALWriterRequest\032\026.RollWALWriterRespon" +
|
||||
"se\022>\n\rgetServerInfo\022\025.GetServerInfoReque" +
|
||||
"st\032\026.GetServerInfoResponse\0225\n\nstopServer" +
|
||||
"\022\022.StopServerRequest\032\023.StopServerRespons",
|
||||
"eBA\n*org.apache.hadoop.hbase.protobuf.ge" +
|
||||
"neratedB\013AdminProtosH\001\210\001\001\240\001\001"
|
||||
"\n\013Admin.proto\032\014Client.proto\032\013hbase.proto" +
|
||||
"\032\tWAL.proto\"Q\n\024GetRegionInfoRequest\022 \n\006r" +
|
||||
"egion\030\001 \002(\0132\020.RegionSpecifier\022\027\n\017compact" +
|
||||
"ionState\030\002 \001(\010\"\301\001\n\025GetRegionInfoResponse" +
|
||||
"\022\037\n\nregionInfo\030\001 \002(\0132\013.RegionInfo\022?\n\017com" +
|
||||
"pactionState\030\002 \001(\0162&.GetRegionInfoRespon" +
|
||||
"se.CompactionState\"F\n\017CompactionState\022\010\n" +
|
||||
"\004NONE\020\000\022\t\n\005MINOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_A" +
|
||||
"ND_MINOR\020\003\"G\n\023GetStoreFileRequest\022 \n\006reg" +
|
||||
"ion\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006family\030\002 ",
|
||||
"\003(\014\")\n\024GetStoreFileResponse\022\021\n\tstoreFile" +
|
||||
"\030\001 \003(\t\"\030\n\026GetOnlineRegionRequest\":\n\027GetO" +
|
||||
"nlineRegionResponse\022\037\n\nregionInfo\030\001 \003(\0132" +
|
||||
"\013.RegionInfo\"\270\001\n\021OpenRegionRequest\0223\n\010op" +
|
||||
"enInfo\030\001 \003(\0132!.OpenRegionRequest.RegionO" +
|
||||
"penInfo\032n\n\016RegionOpenInfo\022\033\n\006region\030\001 \002(" +
|
||||
"\0132\013.RegionInfo\022\034\n\024versionOfOfflineNode\030\002" +
|
||||
" \001(\r\022!\n\014favoredNodes\030\003 \003(\0132\013.ServerName\"" +
|
||||
"\234\001\n\022OpenRegionResponse\022<\n\014openingState\030\001" +
|
||||
" \003(\0162&.OpenRegionResponse.RegionOpeningS",
|
||||
"tate\"H\n\022RegionOpeningState\022\n\n\006OPENED\020\000\022\022" +
|
||||
"\n\016ALREADY_OPENED\020\001\022\022\n\016FAILED_OPENING\020\002\"\232" +
|
||||
"\001\n\022CloseRegionRequest\022 \n\006region\030\001 \002(\0132\020." +
|
||||
"RegionSpecifier\022\034\n\024versionOfClosingNode\030" +
|
||||
"\002 \001(\r\022\034\n\016transitionInZK\030\003 \001(\010:\004true\022&\n\021d" +
|
||||
"estinationServer\030\004 \001(\0132\013.ServerName\"%\n\023C" +
|
||||
"loseRegionResponse\022\016\n\006closed\030\001 \002(\010\"M\n\022Fl" +
|
||||
"ushRegionRequest\022 \n\006region\030\001 \002(\0132\020.Regio" +
|
||||
"nSpecifier\022\025\n\rifOlderThanTs\030\002 \001(\004\"=\n\023Flu" +
|
||||
"shRegionResponse\022\025\n\rlastFlushTime\030\001 \002(\004\022",
|
||||
"\017\n\007flushed\030\002 \001(\010\"J\n\022SplitRegionRequest\022 " +
|
||||
"\n\006region\030\001 \002(\0132\020.RegionSpecifier\022\022\n\nspli" +
|
||||
"tPoint\030\002 \001(\014\"\025\n\023SplitRegionResponse\"W\n\024C" +
|
||||
"ompactRegionRequest\022 \n\006region\030\001 \002(\0132\020.Re" +
|
||||
"gionSpecifier\022\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003" +
|
||||
" \001(\014\"\027\n\025CompactRegionResponse\"t\n\023MergeRe" +
|
||||
"gionsRequest\022!\n\007regionA\030\001 \002(\0132\020.RegionSp" +
|
||||
"ecifier\022!\n\007regionB\030\002 \002(\0132\020.RegionSpecifi" +
|
||||
"er\022\027\n\010forcible\030\003 \001(\010:\005false\"\026\n\024MergeRegi" +
|
||||
"onsResponse\"7\n\010WALEntry\022\024\n\003key\030\001 \002(\0132\007.W",
|
||||
"ALKey\022\025\n\rkeyValueBytes\030\002 \003(\014\"4\n\030Replicat" +
|
||||
"eWALEntryRequest\022\030\n\005entry\030\001 \003(\0132\t.WALEnt" +
|
||||
"ry\"\033\n\031ReplicateWALEntryResponse\"\026\n\024RollW" +
|
||||
"ALWriterRequest\".\n\025RollWALWriterResponse" +
|
||||
"\022\025\n\rregionToFlush\030\001 \003(\014\"#\n\021StopServerReq" +
|
||||
"uest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServerRespon" +
|
||||
"se\"\026\n\024GetServerInfoRequest\"@\n\nServerInfo" +
|
||||
"\022\037\n\nserverName\030\001 \002(\0132\013.ServerName\022\021\n\tweb" +
|
||||
"uiPort\030\002 \001(\r\"8\n\025GetServerInfoResponse\022\037\n" +
|
||||
"\nserverInfo\030\001 \002(\0132\013.ServerInfo2\337\006\n\014Admin",
|
||||
"Service\022>\n\rgetRegionInfo\022\025.GetRegionInfo" +
|
||||
"Request\032\026.GetRegionInfoResponse\022;\n\014getSt" +
|
||||
"oreFile\022\024.GetStoreFileRequest\032\025.GetStore" +
|
||||
"FileResponse\022D\n\017getOnlineRegion\022\027.GetOnl" +
|
||||
"ineRegionRequest\032\030.GetOnlineRegionRespon" +
|
||||
"se\0225\n\nopenRegion\022\022.OpenRegionRequest\032\023.O" +
|
||||
"penRegionResponse\0228\n\013closeRegion\022\023.Close" +
|
||||
"RegionRequest\032\024.CloseRegionResponse\0228\n\013f" +
|
||||
"lushRegion\022\023.FlushRegionRequest\032\024.FlushR" +
|
||||
"egionResponse\0228\n\013splitRegion\022\023.SplitRegi",
|
||||
"onRequest\032\024.SplitRegionResponse\022>\n\rcompa" +
|
||||
"ctRegion\022\025.CompactRegionRequest\032\026.Compac" +
|
||||
"tRegionResponse\022;\n\014mergeRegions\022\024.MergeR" +
|
||||
"egionsRequest\032\025.MergeRegionsResponse\022J\n\021" +
|
||||
"replicateWALEntry\022\031.ReplicateWALEntryReq" +
|
||||
"uest\032\032.ReplicateWALEntryResponse\022\'\n\006repl" +
|
||||
"ay\022\r.MultiRequest\032\016.MultiResponse\022>\n\rrol" +
|
||||
"lWALWriter\022\025.RollWALWriterRequest\032\026.Roll" +
|
||||
"WALWriterResponse\022>\n\rgetServerInfo\022\025.Get" +
|
||||
"ServerInfoRequest\032\026.GetServerInfoRespons",
|
||||
"e\0225\n\nstopServer\022\022.StopServerRequest\032\023.St" +
|
||||
"opServerResponseBA\n*org.apache.hadoop.hb" +
|
||||
"ase.protobuf.generatedB\013AdminProtosH\001\210\001\001" +
|
||||
"\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -15571,6 +15638,7 @@ public final class AdminProtos {
|
|||
com.google.protobuf.Descriptors.FileDescriptor
|
||||
.internalBuildGeneratedFileFrom(descriptorData,
|
||||
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.WALProtos.getDescriptor(),
|
||||
}, assigner);
|
||||
|
|
|
@ -3132,6 +3132,10 @@ public final class HBaseProtos {
|
|||
// optional bool split = 6;
|
||||
boolean hasSplit();
|
||||
boolean getSplit();
|
||||
|
||||
// optional bool recovering = 7;
|
||||
boolean hasRecovering();
|
||||
boolean getRecovering();
|
||||
}
|
||||
public static final class RegionInfo extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
|
@ -3222,6 +3226,16 @@ public final class HBaseProtos {
|
|||
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() {
|
||||
regionId_ = 0L;
|
||||
tableName_ = com.google.protobuf.ByteString.EMPTY;
|
||||
|
@ -3229,6 +3243,7 @@ public final class HBaseProtos {
|
|||
endKey_ = com.google.protobuf.ByteString.EMPTY;
|
||||
offline_ = false;
|
||||
split_ = false;
|
||||
recovering_ = false;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -3268,6 +3283,9 @@ public final class HBaseProtos {
|
|||
if (((bitField0_ & 0x00000020) == 0x00000020)) {
|
||||
output.writeBool(6, split_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
||||
output.writeBool(7, recovering_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -3301,6 +3319,10 @@ public final class HBaseProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBoolSize(6, split_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBoolSize(7, recovering_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -3354,6 +3376,11 @@ public final class HBaseProtos {
|
|||
result = result && (getSplit()
|
||||
== other.getSplit());
|
||||
}
|
||||
result = result && (hasRecovering() == other.hasRecovering());
|
||||
if (hasRecovering()) {
|
||||
result = result && (getRecovering()
|
||||
== other.getRecovering());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -3387,6 +3414,10 @@ public final class HBaseProtos {
|
|||
hash = (37 * hash) + SPLIT_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashBoolean(getSplit());
|
||||
}
|
||||
if (hasRecovering()) {
|
||||
hash = (37 * hash) + RECOVERING_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashBoolean(getRecovering());
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
return hash;
|
||||
}
|
||||
|
@ -3515,6 +3546,8 @@ public final class HBaseProtos {
|
|||
bitField0_ = (bitField0_ & ~0x00000010);
|
||||
split_ = false;
|
||||
bitField0_ = (bitField0_ & ~0x00000020);
|
||||
recovering_ = false;
|
||||
bitField0_ = (bitField0_ & ~0x00000040);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -3577,6 +3610,10 @@ public final class HBaseProtos {
|
|||
to_bitField0_ |= 0x00000020;
|
||||
}
|
||||
result.split_ = split_;
|
||||
if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
|
||||
to_bitField0_ |= 0x00000040;
|
||||
}
|
||||
result.recovering_ = recovering_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -3611,6 +3648,9 @@ public final class HBaseProtos {
|
|||
if (other.hasSplit()) {
|
||||
setSplit(other.getSplit());
|
||||
}
|
||||
if (other.hasRecovering()) {
|
||||
setRecovering(other.getRecovering());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -3680,6 +3720,11 @@ public final class HBaseProtos {
|
|||
split_ = input.readBool();
|
||||
break;
|
||||
}
|
||||
case 56: {
|
||||
bitField0_ |= 0x00000040;
|
||||
recovering_ = input.readBool();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -3821,6 +3866,27 @@ public final class HBaseProtos {
|
|||
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)
|
||||
}
|
||||
|
||||
|
@ -14719,60 +14785,60 @@ public final class HBaseProtos {
|
|||
"hema\022&\n\rconfiguration\030\004 \003(\0132\017.NameString" +
|
||||
"Pair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(\014" +
|
||||
"\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022&\n" +
|
||||
"\rconfiguration\030\003 \003(\0132\017.NameStringPair\"s\n",
|
||||
"\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableNa" +
|
||||
"me\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 \001" +
|
||||
"(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\"0\n\014Fa" +
|
||||
"voredNodes\022 \n\013favoredNode\030\001 \003(\0132\013.Server" +
|
||||
"Name\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001 \002(\0162$" +
|
||||
".RegionSpecifier.RegionSpecifierType\022\r\n\005" +
|
||||
"value\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017\n\013RE" +
|
||||
"GION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"\260\003\n" +
|
||||
"\nRegionLoad\022)\n\017regionSpecifier\030\001 \002(\0132\020.R" +
|
||||
"egionSpecifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstoref",
|
||||
"iles\030\003 \001(\r\022\037\n\027storeUncompressedSizeMB\030\004 " +
|
||||
"\001(\r\022\027\n\017storefileSizeMB\030\005 \001(\r\022\026\n\016memstore" +
|
||||
"SizeMB\030\006 \001(\r\022\034\n\024storefileIndexSizeMB\030\007 \001" +
|
||||
"(\r\022\031\n\021readRequestsCount\030\010 \001(\004\022\032\n\022writeRe" +
|
||||
"questsCount\030\t \001(\004\022\032\n\022totalCompactingKVs\030" +
|
||||
"\n \001(\004\022\033\n\023currentCompactedKVs\030\013 \001(\004\022\027\n\017ro" +
|
||||
"otIndexSizeKB\030\014 \001(\r\022\036\n\026totalStaticIndexS" +
|
||||
"izeKB\030\r \001(\r\022\036\n\026totalStaticBloomSizeKB\030\016 " +
|
||||
"\001(\r\022\032\n\022completeSequenceId\030\017 \001(\004\"\372\001\n\nServ" +
|
||||
"erLoad\022\030\n\020numberOfRequests\030\001 \001(\r\022\035\n\025tota",
|
||||
"lNumberOfRequests\030\002 \001(\r\022\022\n\nusedHeapMB\030\003 " +
|
||||
"\001(\r\022\021\n\tmaxHeapMB\030\004 \001(\r\022 \n\013regionLoads\030\005 " +
|
||||
"\003(\0132\013.RegionLoad\022\"\n\014coprocessors\030\006 \003(\0132\014" +
|
||||
".Coprocessor\022\027\n\017reportStartTime\030\007 \001(\004\022\025\n" +
|
||||
"\rreportEndTime\030\010 \001(\004\022\026\n\016infoServerPort\030\t" +
|
||||
" \001(\r\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 " +
|
||||
"\001(\004\"0\n\006Filter\022\014\n\004name\030\001 \002(\t\022\030\n\020serialize" +
|
||||
"dFilter\030\002 \001(\014\"x\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016" +
|
||||
"\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttim" +
|
||||
"estamp\030\004 \001(\004\022\032\n\007keyType\030\005 \001(\0162\t.CellType",
|
||||
"\022\r\n\005value\030\006 \001(\014\"?\n\nServerName\022\020\n\010hostNam" +
|
||||
"e\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004" +
|
||||
"\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStr" +
|
||||
"ingPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\r" +
|
||||
"NameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001" +
|
||||
"(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006s" +
|
||||
"econd\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001" +
|
||||
"(\t\022\r\n\005value\030\002 \001(\003\"\255\001\n\023SnapshotDescriptio" +
|
||||
"n\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\027\n\014creati" +
|
||||
"onTime\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snapshot",
|
||||
"Description.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005" +
|
||||
"\"\037\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\"\n\n\010Emp" +
|
||||
"tyMsg\"\032\n\007LongMsg\022\017\n\007longMsg\030\001 \002(\003\"&\n\rBig" +
|
||||
"DecimalMsg\022\025\n\rbigdecimalMsg\030\001 \002(\014\"1\n\004UUI" +
|
||||
"D\022\024\n\014leastSigBits\030\001 \002(\004\022\023\n\013mostSigBits\030\002" +
|
||||
" \002(\004*`\n\010CellType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n" +
|
||||
"\n\006DELETE\020\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_" +
|
||||
"FAMILY\020\016\022\014\n\007MAXIMUM\020\377\001*r\n\013CompareType\022\010\n" +
|
||||
"\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n" +
|
||||
"\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GR",
|
||||
"EATER\020\005\022\t\n\005NO_OP\020\006B>\n*org.apache.hadoop." +
|
||||
"hbase.protobuf.generatedB\013HBaseProtosH\001\240" +
|
||||
"\001\001"
|
||||
"\rconfiguration\030\003 \003(\0132\017.NameStringPair\"\207\001",
|
||||
"\n\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableN" +
|
||||
"ame\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 " +
|
||||
"\001(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\022\n\nr" +
|
||||
"ecovering\030\007 \001(\010\"0\n\014FavoredNodes\022 \n\013favor" +
|
||||
"edNode\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpec" +
|
||||
"ifier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Re" +
|
||||
"gionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regi" +
|
||||
"onSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCO" +
|
||||
"DED_REGION_NAME\020\002\"\260\003\n\nRegionLoad\022)\n\017regi" +
|
||||
"onSpecifier\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006s",
|
||||
"tores\030\002 \001(\r\022\022\n\nstorefiles\030\003 \001(\r\022\037\n\027store" +
|
||||
"UncompressedSizeMB\030\004 \001(\r\022\027\n\017storefileSiz" +
|
||||
"eMB\030\005 \001(\r\022\026\n\016memstoreSizeMB\030\006 \001(\r\022\034\n\024sto" +
|
||||
"refileIndexSizeMB\030\007 \001(\r\022\031\n\021readRequestsC" +
|
||||
"ount\030\010 \001(\004\022\032\n\022writeRequestsCount\030\t \001(\004\022\032" +
|
||||
"\n\022totalCompactingKVs\030\n \001(\004\022\033\n\023currentCom" +
|
||||
"pactedKVs\030\013 \001(\004\022\027\n\017rootIndexSizeKB\030\014 \001(\r" +
|
||||
"\022\036\n\026totalStaticIndexSizeKB\030\r \001(\r\022\036\n\026tota" +
|
||||
"lStaticBloomSizeKB\030\016 \001(\r\022\032\n\022completeSequ" +
|
||||
"enceId\030\017 \001(\004\"\372\001\n\nServerLoad\022\030\n\020numberOfR",
|
||||
"equests\030\001 \001(\r\022\035\n\025totalNumberOfRequests\030\002" +
|
||||
" \001(\r\022\022\n\nusedHeapMB\030\003 \001(\r\022\021\n\tmaxHeapMB\030\004 " +
|
||||
"\001(\r\022 \n\013regionLoads\030\005 \003(\0132\013.RegionLoad\022\"\n" +
|
||||
"\014coprocessors\030\006 \003(\0132\014.Coprocessor\022\027\n\017rep" +
|
||||
"ortStartTime\030\007 \001(\004\022\025\n\rreportEndTime\030\010 \001(" +
|
||||
"\004\022\026\n\016infoServerPort\030\t \001(\r\"%\n\tTimeRange\022\014" +
|
||||
"\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"0\n\006Filter\022\014\n\004na" +
|
||||
"me\030\001 \002(\t\022\030\n\020serializedFilter\030\002 \001(\014\"x\n\010Ke" +
|
||||
"yValue\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tq" +
|
||||
"ualifier\030\003 \002(\014\022\021\n\ttimestamp\030\004 \001(\004\022\032\n\007key",
|
||||
"Type\030\005 \001(\0162\t.CellType\022\r\n\005value\030\006 \001(\014\"?\n\n" +
|
||||
"ServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n\004port\030\002 \001" +
|
||||
"(\r\022\021\n\tstartCode\030\003 \001(\004\"\033\n\013Coprocessor\022\014\n\004" +
|
||||
"name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004name\030\001 \002" +
|
||||
"(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair\022\014\n\004na" +
|
||||
"me\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBytesPai" +
|
||||
"r\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n\rName" +
|
||||
"Int64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\255" +
|
||||
"\001\n\023SnapshotDescription\022\014\n\004name\030\001 \002(\t\022\r\n\005" +
|
||||
"table\030\002 \001(\t\022\027\n\014creationTime\030\003 \001(\003:\0010\022.\n\004",
|
||||
"type\030\004 \001(\0162\031.SnapshotDescription.Type:\005F" +
|
||||
"LUSH\022\017\n\007version\030\005 \001(\005\"\037\n\004Type\022\014\n\010DISABLE" +
|
||||
"D\020\000\022\t\n\005FLUSH\020\001\"\n\n\010EmptyMsg\"\032\n\007LongMsg\022\017\n" +
|
||||
"\007longMsg\030\001 \002(\003\"&\n\rBigDecimalMsg\022\025\n\rbigde" +
|
||||
"cimalMsg\030\001 \002(\014\"1\n\004UUID\022\024\n\014leastSigBits\030\001" +
|
||||
" \002(\004\022\023\n\013mostSigBits\030\002 \002(\004*`\n\010CellType\022\013\n" +
|
||||
"\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELET" +
|
||||
"E_COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM" +
|
||||
"\020\377\001*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_" +
|
||||
"EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GRE",
|
||||
"ATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006B" +
|
||||
">\n*org.apache.hadoop.hbase.protobuf.gene" +
|
||||
"ratedB\013HBaseProtosH\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -14808,7 +14874,7 @@ public final class HBaseProtos {
|
|||
internal_static_RegionInfo_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
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.Builder.class);
|
||||
internal_static_FavoredNodes_descriptor =
|
||||
|
|
|
@ -24,6 +24,7 @@ option java_generic_services = true;
|
|||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
import "Client.proto";
|
||||
import "hbase.proto";
|
||||
import "WAL.proto";
|
||||
|
||||
|
@ -233,6 +234,9 @@ service AdminService {
|
|||
|
||||
rpc replicateWALEntry(ReplicateWALEntryRequest)
|
||||
returns(ReplicateWALEntryResponse);
|
||||
|
||||
rpc replay(MultiRequest)
|
||||
returns(MultiResponse);
|
||||
|
||||
rpc rollWALWriter(RollWALWriterRequest)
|
||||
returns(RollWALWriterResponse);
|
||||
|
|
|
@ -81,6 +81,7 @@ message RegionInfo {
|
|||
optional bytes endKey = 4;
|
||||
optional bool offline = 5;
|
||||
optional bool split = 6;
|
||||
optional bool recovering = 7;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -422,7 +422,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
return;
|
||||
}
|
||||
|
||||
boolean failover = !serverManager.getDeadServers().isEmpty();
|
||||
boolean failover = (!serverManager.getDeadServers().isEmpty() || !serverManager
|
||||
.getRequeuedDeadServers().isEmpty());
|
||||
|
||||
if (!failover) {
|
||||
// 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)
|
||||
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;
|
||||
long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTimeMillis()
|
||||
+ timeOut;
|
||||
// There is already a timeout monitor on regions in transition so I
|
||||
// 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);
|
||||
if (EnvironmentEdgeManager.currentTimeMillis() > end) {
|
||||
LOG.info("Timed out on waiting for region:" + hri.getEncodedName() + " to be assigned.");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
if (this.server.isStopped()) {
|
||||
LOG.info("Giving up wait on regions in " +
|
||||
"transition because stoppable.isStopped is set");
|
||||
LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.net.UnknownHostException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -347,7 +348,15 @@ MasterServices, Server {
|
|||
|
||||
/** The health check chore. */
|
||||
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:
|
||||
|
@ -451,6 +460,9 @@ MasterServices, Server {
|
|||
clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
|
||||
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();
|
||||
}
|
||||
|
||||
// TODO: Should do this in background rather than block master startup
|
||||
status.setStatus("Splitting logs after master startup");
|
||||
splitLogAfterStartup(this.fileSystemManager);
|
||||
// get a list for previously failed RS which need log splitting work
|
||||
// we recover .META. region servers inside master initialization and
|
||||
// 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.
|
||||
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();
|
||||
|
||||
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
|
||||
// in meta. This must happen before we assign all user regions or else the assignment will
|
||||
// fail.
|
||||
// TODO: Remove this after 0.96, when we do 0.98.
|
||||
org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB
|
||||
.updateMetaIfNecessary(this);
|
||||
|
||||
|
@ -829,14 +870,6 @@ MasterServices, Server {
|
|||
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.
|
||||
* @param master
|
||||
|
@ -865,52 +898,66 @@ MasterServices, Server {
|
|||
}
|
||||
|
||||
/**
|
||||
* Check <code>.META.</code> are assigned. If not,
|
||||
* assign them.
|
||||
* Check <code>.META.</code> is assigned. If not, assign it.
|
||||
* @param status MonitoredTask
|
||||
* @throws InterruptedException
|
||||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
* @return True if meta is healthy, assigned
|
||||
*/
|
||||
boolean assignMeta(MonitoredTask status)
|
||||
throws InterruptedException, IOException, KeeperException {
|
||||
void assignMeta(MonitoredTask status)
|
||||
throws InterruptedException, IOException, KeeperException {
|
||||
// Work on meta region
|
||||
int assigned = 0;
|
||||
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");
|
||||
assignmentManager.getRegionStates().createRegionState(
|
||||
HRegionInfo.FIRST_META_REGIONINFO);
|
||||
boolean rit = this.assignmentManager.
|
||||
processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
ServerName currentMetaServer = null;
|
||||
boolean metaRegionLocation = catalogTracker.verifyMetaRegionLocation(timeout);
|
||||
|
||||
assignmentManager.getRegionStates().createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
boolean rit = this.assignmentManager
|
||||
.processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
|
||||
if (!rit && !metaRegionLocation) {
|
||||
currentMetaServer = this.catalogTracker.getMetaLocation();
|
||||
splitLogAndExpireIfOnline(currentMetaServer);
|
||||
this.assignmentManager.assignMeta();
|
||||
enableSSHandWaitForMeta();
|
||||
ServerName currentMetaServer = this.catalogTracker.getMetaLocation();
|
||||
if (currentMetaServer != null) {
|
||||
beingExpired = expireIfOnline(currentMetaServer);
|
||||
}
|
||||
if (beingExpired) {
|
||||
splitMetaLogBeforeAssignment(currentMetaServer);
|
||||
}
|
||||
assignmentManager.assignMeta();
|
||||
// Make sure a .META. location is set.
|
||||
if (!isMetaLocation()) return false;
|
||||
// This guarantees that the transition assigning .META. has completed
|
||||
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
enableSSHandWaitForMeta();
|
||||
assigned++;
|
||||
if (beingExpired && this.distributedLogReplay) {
|
||||
// In Replay WAL Mode, we need the new .META. server online
|
||||
this.fileSystemManager.splitMetaLog(currentMetaServer);
|
||||
}
|
||||
} else if (rit && !metaRegionLocation) {
|
||||
// Make sure a .META. location is set.
|
||||
if (!isMetaLocation()) return false;
|
||||
// This guarantees that the transition assigning .META. has completed
|
||||
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
enableSSHandWaitForMeta();
|
||||
assigned++;
|
||||
} else if (metaRegionLocation) {
|
||||
// Region already assigned. We didn't assign it. Add to in-memory state.
|
||||
} else {
|
||||
// Region already assigned. We didn't assign it. Add to in-memory state.
|
||||
this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
|
||||
this.catalogTracker.getMetaLocation());
|
||||
}
|
||||
enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME));
|
||||
LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
|
||||
", location=" + catalogTracker.getMetaLocation());
|
||||
LOG.info(".META. assigned=" + assigned + ", rit=" + rit + ", location="
|
||||
+ catalogTracker.getMetaLocation());
|
||||
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 {
|
||||
|
@ -921,24 +968,6 @@ MasterServices, Server {
|
|||
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) {
|
||||
if (!this.assignmentManager.getZKTable().isEnabledTable(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
|
||||
* servers.
|
||||
* Expire a server if we find it is one of the online servers.
|
||||
* @param sn ServerName to check.
|
||||
* @return true when server <code>sn<code> is being expired by the function.
|
||||
* @throws IOException
|
||||
*/
|
||||
private void splitLogAndExpireIfOnline(final ServerName sn)
|
||||
private boolean expireIfOnline(final ServerName sn)
|
||||
throws IOException {
|
||||
if (sn == null || !serverManager.isServerOnline(sn)) {
|
||||
return;
|
||||
return false;
|
||||
}
|
||||
LOG.info("Forcing splitLog and expire of " + sn);
|
||||
fileSystemManager.splitMetaLog(sn);
|
||||
fileSystemManager.splitLog(sn);
|
||||
LOG.info("Forcing expire of " + sn);
|
||||
serverManager.expireServer(sn);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2235,6 +2263,14 @@ MasterServices, Server {
|
|||
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
|
||||
public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest req)
|
||||
throws ServiceException {
|
||||
|
@ -2678,4 +2714,5 @@ MasterServices, Server {
|
|||
String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
|
||||
return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -20,7 +20,9 @@ package org.apache.hadoop.hbase.master;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
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.PathFilter;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.exceptions.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
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.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
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
|
||||
|
@ -83,6 +88,7 @@ public class MasterFileSystem {
|
|||
private final Path tempdir;
|
||||
// create the split log lock
|
||||
final Lock splitLogLock = new ReentrantLock();
|
||||
final boolean distributedLogReplay;
|
||||
final boolean distributedLogSplitting;
|
||||
final SplitLogManager splitLogManager;
|
||||
private final MasterServices services;
|
||||
|
@ -118,15 +124,14 @@ public class MasterFileSystem {
|
|||
FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
|
||||
// make sure the fs has the same conf
|
||||
fs.setConf(conf);
|
||||
this.distributedLogSplitting =
|
||||
conf.getBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, true);
|
||||
this.splitLogManager = new SplitLogManager(master.getZooKeeper(), master.getConfiguration(),
|
||||
master, services, master.getServerName());
|
||||
this.distributedLogSplitting = conf.getBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, true);
|
||||
if (this.distributedLogSplitting) {
|
||||
this.splitLogManager = new SplitLogManager(master.getZooKeeper(),
|
||||
master.getConfiguration(), master, services, master.getServerName());
|
||||
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
|
||||
// set up the archived logs path
|
||||
this.oldLogDir = createInitialFileSystemLayout();
|
||||
|
@ -212,21 +217,23 @@ public class MasterFileSystem {
|
|||
}
|
||||
|
||||
/**
|
||||
* Inspect the log directory to recover any log file without
|
||||
* an active region server.
|
||||
* Inspect the log directory to find dead servers which need recovery work
|
||||
* @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",
|
||||
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);
|
||||
|
||||
do {
|
||||
if (master.isStopped()) {
|
||||
LOG.warn("Master stopped while splitting logs");
|
||||
LOG.warn("Master stopped while trying to get failed servers.");
|
||||
break;
|
||||
}
|
||||
List<ServerName> serverNames = new ArrayList<ServerName>();
|
||||
try {
|
||||
if (!this.fs.exists(logsDirPath)) return;
|
||||
if (!this.fs.exists(logsDirPath)) return serverNames;
|
||||
FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
|
||||
// Get online servers after getting log folders to avoid log folder deletion of newly
|
||||
// checked in region servers . see HBASE-5916
|
||||
|
@ -235,7 +242,7 @@ public class MasterFileSystem {
|
|||
|
||||
if (logFolders == null || logFolders.length == 0) {
|
||||
LOG.debug("No log files to split, proceeding...");
|
||||
return;
|
||||
return serverNames;
|
||||
}
|
||||
for (FileStatus status : logFolders) {
|
||||
String sn = status.getPath().getName();
|
||||
|
@ -249,23 +256,19 @@ public class MasterFileSystem {
|
|||
+ "to a known region server, splitting");
|
||||
serverNames.add(serverName);
|
||||
} else {
|
||||
LOG.info("Log folder " + status.getPath()
|
||||
+ " belongs to an existing region server");
|
||||
LOG.info("Log folder " + status.getPath() + " belongs to an existing region server");
|
||||
}
|
||||
}
|
||||
splitLog(serverNames, META_FILTER);
|
||||
splitLog(serverNames, NON_META_FILTER);
|
||||
retrySplitting = false;
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Failed splitting of " + serverNames, ioe);
|
||||
LOG.warn("Failed getting failed servers to be recovered.", ioe);
|
||||
if (!checkFileSystem()) {
|
||||
LOG.warn("Bad Filesystem, exiting");
|
||||
Runtime.getRuntime().halt(1);
|
||||
}
|
||||
try {
|
||||
if (retrySplitting) {
|
||||
Thread.sleep(conf.getInt(
|
||||
"hbase.hlog.split.failure.retry.interval", 30 * 1000));
|
||||
Thread.sleep(conf.getInt("hbase.hlog.split.failure.retry.interval", 30 * 1000));
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("Interrupted, aborting since cannot return w/o splitting");
|
||||
|
@ -275,10 +278,12 @@ public class MasterFileSystem {
|
|||
}
|
||||
}
|
||||
} while (retrySplitting);
|
||||
|
||||
return serverNames;
|
||||
}
|
||||
|
||||
public void splitLog(final ServerName serverName) throws IOException {
|
||||
List<ServerName> serverNames = new ArrayList<ServerName>();
|
||||
Set<ServerName> serverNames = new HashSet<ServerName>();
|
||||
serverNames.add(serverName);
|
||||
splitLog(serverNames);
|
||||
}
|
||||
|
@ -290,23 +295,20 @@ public class MasterFileSystem {
|
|||
*/
|
||||
public void splitMetaLog(final ServerName serverName) throws IOException {
|
||||
long splitTime = 0, splitLogSize = 0;
|
||||
List<ServerName> serverNames = new ArrayList<ServerName>();
|
||||
Set<ServerName> serverNames = new HashSet<ServerName>();
|
||||
serverNames.add(serverName);
|
||||
List<Path> logDirs = getLogDirs(serverNames);
|
||||
if (logDirs.isEmpty()) {
|
||||
LOG.info("No meta logs to split");
|
||||
return;
|
||||
}
|
||||
|
||||
splitLogManager.handleDeadWorkers(serverNames);
|
||||
splitTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
splitLogSize = splitLogManager.splitLogDistributed(logDirs, META_FILTER);
|
||||
splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, META_FILTER);
|
||||
splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime;
|
||||
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>();
|
||||
for (ServerName serverName: serverNames) {
|
||||
Path logDir = new Path(this.rootdir, HLogUtil.getHLogDirectoryName(serverName.toString()));
|
||||
|
@ -327,30 +329,79 @@ public class MasterFileSystem {
|
|||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
* Wrapper function on {@link SplitLogManager#removeStaleRecoveringRegionsFromZK(Set)}
|
||||
* @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 filter
|
||||
* @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;
|
||||
List<Path> logDirs = getLogDirs(serverNames);
|
||||
|
||||
if (logDirs.isEmpty()) {
|
||||
LOG.info("No logs to split");
|
||||
return;
|
||||
}
|
||||
|
||||
if (distributedLogSplitting) {
|
||||
splitLogManager.handleDeadWorkers(serverNames);
|
||||
splitTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
splitLogSize = splitLogManager.splitLogDistributed(logDirs,filter);
|
||||
splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, filter);
|
||||
splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime;
|
||||
} else {
|
||||
for(Path logDir: logDirs){
|
||||
|
@ -358,8 +409,8 @@ public class MasterFileSystem {
|
|||
// one at a time
|
||||
this.splitLogLock.lock();
|
||||
try {
|
||||
HLogSplitter splitter = HLogSplitter.createLogSplitter(
|
||||
conf, rootdir, logDir, oldLogDir, this.fs);
|
||||
HLogSplitter splitter = HLogSplitter.createLogSplitter(conf, rootdir, logDir, oldLogDir,
|
||||
this.fs);
|
||||
try {
|
||||
// If FS is in safe mode, just wait till out of it.
|
||||
FSUtils.waitOnSafeMode(conf, conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 1000));
|
||||
|
@ -380,7 +431,11 @@ public class MasterFileSystem {
|
|||
}
|
||||
|
||||
if (this.metricsMaster != null) {
|
||||
this.metricsMaster.addSplit(splitTime, splitLogSize);
|
||||
if (filter == this.META_FILTER) {
|
||||
this.metricsMaster.addMetaWALSplit(splitTime, splitLogSize);
|
||||
} else {
|
||||
this.metricsMaster.addSplit(splitTime, splitLogSize);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -648,4 +703,18 @@ public class MasterFileSystem {
|
|||
this.services.getTableDescriptors().add(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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -181,4 +181,9 @@ public interface MasterServices extends Server {
|
|||
public void dispatchMergingRegions(final HRegionInfo region_a,
|
||||
final HRegionInfo region_b, final boolean forcible) throws IOException;
|
||||
|
||||
/**
|
||||
* @return true if master is initialized
|
||||
*/
|
||||
public boolean isInitialized();
|
||||
|
||||
}
|
||||
|
|
|
@ -58,6 +58,16 @@ public class MetricsMaster {
|
|||
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.
|
||||
*/
|
||||
|
|
|
@ -154,21 +154,21 @@ public class ServerManager {
|
|||
private Set<ServerName> queuedDeadServers = new HashSet<ServerName>();
|
||||
|
||||
/**
|
||||
* Set of region servers which are dead and submitted to ServerShutdownHandler to
|
||||
* process but not fully processed immediately.
|
||||
* Set of region servers which are dead and submitted to ServerShutdownHandler to process but not
|
||||
* fully processed immediately.
|
||||
* <p>
|
||||
* If one server died before assignment manager finished the failover cleanup, the server
|
||||
* will be added to this set and will be processed through calling
|
||||
* If one server died before assignment manager finished the failover cleanup, the server will be
|
||||
* added to this set and will be processed through calling
|
||||
* {@link ServerManager#processQueuedDeadServers()} by assignment manager.
|
||||
* <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>
|
||||
* ServerShutdownHandler processes a dead server submitted to the handler after
|
||||
* the handler is enabled. It may not be able to complete the processing because meta
|
||||
* is not yet online or master is currently in startup mode. In this case, the dead
|
||||
* server will be parked in this set temporarily.
|
||||
* ServerShutdownHandler processes a dead server submitted to the handler after the handler is
|
||||
* enabled. It may not be able to complete the processing because meta is not yet online or master
|
||||
* is currently in startup mode. In this case, the dead server will be parked in this set
|
||||
* temporarily.
|
||||
*/
|
||||
private Set<ServerName> requeuedDeadServers = new HashSet<ServerName>();
|
||||
private Map<ServerName, Boolean> requeuedDeadServers = new HashMap<ServerName, Boolean>();
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
|
@ -513,6 +513,10 @@ public class ServerManager {
|
|||
}
|
||||
|
||||
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
|
||||
// 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
|
||||
|
@ -522,13 +526,14 @@ public class ServerManager {
|
|||
// 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.
|
||||
if (!services.getAssignmentManager().isFailoverCleanupDone()) {
|
||||
requeuedDeadServers.add(serverName);
|
||||
requeuedDeadServers.put(serverName, shouldSplitHlog);
|
||||
return;
|
||||
}
|
||||
|
||||
this.deadservers.add(serverName);
|
||||
this.services.getExecutorService().submit(new ServerShutdownHandler(
|
||||
this.master, this.services, this.deadservers, serverName, false));
|
||||
this.services.getExecutorService().submit(
|
||||
new ServerShutdownHandler(this.master, this.services, this.deadservers, serverName,
|
||||
shouldSplitHlog));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -541,18 +546,20 @@ public class ServerManager {
|
|||
}
|
||||
Iterator<ServerName> serverIterator = queuedDeadServers.iterator();
|
||||
while (serverIterator.hasNext()) {
|
||||
expireServer(serverIterator.next());
|
||||
ServerName tmpServerName = serverIterator.next();
|
||||
expireServer(tmpServerName);
|
||||
serverIterator.remove();
|
||||
requeuedDeadServers.remove(tmpServerName);
|
||||
}
|
||||
|
||||
if (!services.getAssignmentManager().isFailoverCleanupDone()) {
|
||||
LOG.info("AssignmentManager hasn't finished failover cleanup");
|
||||
}
|
||||
serverIterator = requeuedDeadServers.iterator();
|
||||
while (serverIterator.hasNext()) {
|
||||
processDeadServer(serverIterator.next());
|
||||
serverIterator.remove();
|
||||
|
||||
for(ServerName tmpServerName : requeuedDeadServers.keySet()){
|
||||
processDeadServer(tmpServerName, requeuedDeadServers.get(tmpServerName));
|
||||
}
|
||||
requeuedDeadServers.clear();
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -838,6 +845,14 @@ public class ServerManager {
|
|||
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) {
|
||||
return serverName != null && onlineServers.containsKey(serverName);
|
||||
}
|
||||
|
@ -851,7 +866,7 @@ public class ServerManager {
|
|||
public synchronized boolean isServerDead(ServerName serverName) {
|
||||
return serverName == null || deadservers.isDeadServer(serverName)
|
||||
|| queuedDeadServers.contains(serverName)
|
||||
|| requeuedDeadServers.contains(serverName);
|
||||
|| requeuedDeadServers.containsKey(serverName);
|
||||
}
|
||||
|
||||
public void shutdownCluster() {
|
||||
|
|
|
@ -33,6 +33,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
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.PathFilter;
|
||||
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.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
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.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -118,6 +122,20 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
private long unassignedTimeout;
|
||||
private long lastNodeCreateTime = Long.MAX_VALUE;
|
||||
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 TimeoutMonitor timeoutMonitor;
|
||||
|
@ -181,10 +199,13 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
LOG.info("timeout=" + timeout + ", unassigned timeout=" + unassignedTimeout);
|
||||
|
||||
this.serverName = serverName;
|
||||
this.timeoutMonitor =
|
||||
new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper);
|
||||
this.timeoutMonitor = new TimeoutMonitor(
|
||||
conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper);
|
||||
|
||||
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) {
|
||||
|
@ -244,7 +265,22 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
* @return cumulative size of the logfiles split
|
||||
*/
|
||||
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
|
||||
* @return cumulative size of the logfiles split
|
||||
*/
|
||||
public long splitLogDistributed(final List<Path> logDirs, PathFilter filter)
|
||||
throws IOException {
|
||||
public long splitLogDistributed(final Set<ServerName> serverNames, final List<Path> logDirs,
|
||||
PathFilter filter) throws IOException {
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(
|
||||
"Doing distributed log split in " + logDirs);
|
||||
FileStatus[] logfiles = getFileList(logDirs, filter);
|
||||
status.setStatus("Checking directory contents...");
|
||||
LOG.debug("Scheduling batch of logs to split");
|
||||
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 totalSize = 0;
|
||||
TaskBatch batch = new TaskBatch();
|
||||
|
@ -283,6 +319,9 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
}
|
||||
waitForSplittingCompletion(batch, status);
|
||||
// remove recovering regions from ZK
|
||||
this.removeRecoveringRegionsFromZK(serverNames);
|
||||
|
||||
if (batch.done != batch.installed) {
|
||||
batch.isDead = true;
|
||||
SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet();
|
||||
|
@ -409,6 +448,171 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
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) {
|
||||
Task task = tasks.get(path);
|
||||
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
|
||||
* splitLogDistributed(). Clients threads use this object to wait for all
|
||||
* their tasks to be done.
|
||||
* Create znodes /hbase/recovering-regions/[region_ids...]/[failed region server names ...] for
|
||||
* all regions of the passed in region servers
|
||||
* @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>
|
||||
* All access is synchronized.
|
||||
*/
|
||||
|
@ -944,18 +1270,14 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
LOG.info("dead splitlog worker " + workerName);
|
||||
}
|
||||
|
||||
void handleDeadWorkers(List<ServerName> serverNames) {
|
||||
List<ServerName> workerNames = new ArrayList<ServerName>(serverNames.size());
|
||||
for (ServerName serverName : serverNames) {
|
||||
workerNames.add(serverName);
|
||||
}
|
||||
void handleDeadWorkers(Set<ServerName> serverNames) {
|
||||
synchronized (deadWorkersLock) {
|
||||
if (deadWorkers == null) {
|
||||
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);
|
||||
}
|
||||
|
||||
// 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
package org.apache.hadoop.hbase.master.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
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.ServerName;
|
||||
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.MasterServices;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -47,30 +50,55 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
|
|||
@Override
|
||||
public void process() throws IOException {
|
||||
boolean gotException = true;
|
||||
try{
|
||||
try {
|
||||
AssignmentManager am = this.services.getAssignmentManager();
|
||||
try {
|
||||
LOG.info("Splitting META logs for " + serverName);
|
||||
if (this.shouldSplitHlog) {
|
||||
this.services.getMasterFileSystem().splitMetaLog(serverName);
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
this.services.getExecutorService().submit(this);
|
||||
this.deadServers.add(serverName);
|
||||
throw new IOException("failed log splitting for " +
|
||||
serverName + ", will retry", ioe);
|
||||
throw new IOException("failed log splitting for " + serverName + ", will retry", ioe);
|
||||
}
|
||||
|
||||
// Assign meta if we were carrying it.
|
||||
// Check again: region may be assigned to other where because of RIT
|
||||
// timeout
|
||||
if (this.services.getAssignmentManager().isCarryingMeta(serverName)) {
|
||||
if (am.isCarryingMeta(serverName)) {
|
||||
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();
|
||||
} else {
|
||||
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;
|
||||
} finally {
|
||||
if (gotException){
|
||||
|
@ -78,9 +106,15 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
|
|||
this.deadServers.finish(serverName);
|
||||
}
|
||||
}
|
||||
|
||||
super.process();
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean isCarryingMeta() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Before assign the META region, ensure it haven't
|
||||
* been assigned by other place
|
||||
|
|
|
@ -20,13 +20,16 @@ package org.apache.hadoop.hbase.master.handler;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -56,6 +59,8 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
protected final MasterServices services;
|
||||
protected final DeadServer deadServers;
|
||||
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,
|
||||
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!");
|
||||
}
|
||||
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
|
||||
|
@ -107,21 +117,7 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
public void process() throws IOException {
|
||||
final ServerName serverName = this.serverName;
|
||||
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
|
||||
// executor pool to block by waiting availability of .META.
|
||||
// Otherwise, it could run into the following issue:
|
||||
|
@ -145,7 +141,7 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
// the dead server for further processing too.
|
||||
if (isCarryingMeta() // .META.
|
||||
|| !services.getAssignmentManager().isFailoverCleanupDone()) {
|
||||
this.services.getServerManager().processDeadServer(serverName);
|
||||
this.services.getServerManager().processDeadServer(serverName, this.shouldSplitHlog);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -183,6 +179,23 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
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
|
||||
// doing after log splitting. Could do some states before -- OPENING?
|
||||
// OFFLINE? -- and then others after like CLOSING that depend on log
|
||||
|
@ -258,18 +271,47 @@ public class ServerShutdownHandler extends EventHandler {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
am.assign(toAssignRegions);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Caught " + ie + " during round-robin assignment");
|
||||
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 {
|
||||
this.deadServers.finish(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
|
||||
* disabling or if the region has a partially completed split.
|
||||
|
|
|
@ -95,6 +95,7 @@ import org.apache.hadoop.hbase.exceptions.DroppedSnapshotException;
|
|||
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
||||
import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException;
|
||||
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.UnknownProtocolException;
|
||||
import org.apache.hadoop.hbase.exceptions.UnknownScannerException;
|
||||
|
@ -200,6 +201,16 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
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
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
@ -280,6 +291,11 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
private final AtomicInteger majorInProgress = 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
|
||||
* 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);
|
||||
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
|
||||
storeSeqIdForReplay);
|
||||
if (this.minSeqIdForLogReplay == -1 || storeSeqIdForReplay < this.minSeqIdForLogReplay) {
|
||||
this.minSeqIdForLogReplay = storeSeqIdForReplay;
|
||||
}
|
||||
// Include bulk loaded files when determining seqIdForAssignment
|
||||
long storeSeqIdForAssignment = store.getMaxSequenceId(true);
|
||||
if (maxSeqId == -1 || storeSeqIdForAssignment > maxSeqId) {
|
||||
|
@ -778,6 +797,21 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
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) */
|
||||
public boolean isAvailable() {
|
||||
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
|
||||
// closest key is across all column families, since the data may be sparse
|
||||
checkRow(row, "getClosestRowBefore");
|
||||
startRegionOperation();
|
||||
startRegionOperation(Operation.GET);
|
||||
this.readRequestsCount.increment();
|
||||
try {
|
||||
Store store = getStore(family);
|
||||
|
@ -1654,7 +1688,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
protected RegionScanner getScanner(Scan scan,
|
||||
List<KeyValueScanner> additionalScanners) throws IOException {
|
||||
startRegionOperation();
|
||||
startRegionOperation(Operation.SCAN);
|
||||
try {
|
||||
// Verify families are all valid
|
||||
prepareScanner(scan);
|
||||
|
@ -1705,7 +1739,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
throws IOException {
|
||||
checkReadOnly();
|
||||
checkResources();
|
||||
startRegionOperation();
|
||||
startRegionOperation(Operation.DELETE);
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
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
|
||||
// will be extremely rare; we'll deal with it when it happens.
|
||||
checkResources();
|
||||
startRegionOperation();
|
||||
startRegionOperation(Operation.PUT);
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
// 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(
|
||||
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 =
|
||||
new BatchOperationInProgress<Pair<Mutation,Integer>>(mutationsAndLocks);
|
||||
|
||||
boolean initialized = false;
|
||||
|
||||
while (!batchOp.isDone()) {
|
||||
checkReadOnly();
|
||||
if (!isReplay) {
|
||||
checkReadOnly();
|
||||
}
|
||||
checkResources();
|
||||
|
||||
long newSize;
|
||||
|
@ -1876,11 +1926,13 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
try {
|
||||
if (!initialized) {
|
||||
this.writeRequestsCount.increment();
|
||||
doPreMutationHook(batchOp);
|
||||
if (!isReplay) {
|
||||
this.writeRequestsCount.increment();
|
||||
doPreMutationHook(batchOp);
|
||||
}
|
||||
initialized = true;
|
||||
}
|
||||
long addedSize = doMiniBatchMutation(batchOp);
|
||||
long addedSize = doMiniBatchMutation(batchOp, isReplay);
|
||||
newSize = this.addAndGetGlobalMemstoreSize(addedSize);
|
||||
} finally {
|
||||
closeRegionOperation();
|
||||
|
@ -1891,6 +1943,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
return batchOp.retCodeDetails;
|
||||
}
|
||||
|
||||
|
||||
private void doPreMutationHook(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp)
|
||||
throws IOException {
|
||||
|
@ -1927,10 +1980,9 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private long doMiniBatchMutation(
|
||||
BatchOperationInProgress<Pair<Mutation, Integer>> batchOp) throws IOException {
|
||||
private long doMiniBatchMutation(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp,
|
||||
boolean isInReplay) throws IOException {
|
||||
|
||||
// variable to note if all Put items are for the same CF -- metrics related
|
||||
boolean putsCfSetConsistent = true;
|
||||
|
@ -1941,7 +1993,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
//The set of columnFamilies first seen for Delete.
|
||||
Set<byte[]> deletesCfSet = null;
|
||||
|
||||
WALEdit walEdit = new WALEdit();
|
||||
WALEdit walEdit = new WALEdit(isInReplay);
|
||||
MultiVersionConsistencyControl.WriteEntry w = null;
|
||||
long txid = 0;
|
||||
boolean walSyncSuccessful = false;
|
||||
|
@ -1983,7 +2035,11 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
try {
|
||||
if (isPutMutation) {
|
||||
// Check the families in the put. If bad, skip this one.
|
||||
checkFamilies(familyMap.keySet());
|
||||
if (isInReplay) {
|
||||
removeNonExistentColumnFamilyForReplay(familyMap);
|
||||
} else {
|
||||
checkFamilies(familyMap.keySet());
|
||||
}
|
||||
checkTimestamps(mutation.getFamilyMap(), now);
|
||||
} else {
|
||||
prepareDelete((Delete) mutation);
|
||||
|
@ -2042,7 +2098,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// we should record the timestamp only after we have acquired the rowLock,
|
||||
// otherwise, newer puts/deletes are not guaranteed to have a newer timestamp
|
||||
now = EnvironmentEdgeManager.currentTimeMillis();
|
||||
|
@ -2081,9 +2137,9 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
w = mvcc.beginMemstoreInsert();
|
||||
|
||||
// calling the pre CP hook for batch mutation
|
||||
if (coprocessorHost != null) {
|
||||
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
|
||||
new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
|
||||
if (!isInReplay && coprocessorHost != null) {
|
||||
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
|
||||
new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
|
||||
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
|
||||
if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
|
||||
}
|
||||
|
@ -2168,9 +2224,9 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
walSyncSuccessful = true;
|
||||
// calling the post CP hook for batch mutation
|
||||
if (coprocessorHost != null) {
|
||||
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
|
||||
new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
|
||||
if (!isInReplay && coprocessorHost != null) {
|
||||
MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
|
||||
new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
|
||||
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
|
||||
coprocessorHost.postBatchMutate(miniBatchOp);
|
||||
}
|
||||
|
@ -2187,7 +2243,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// STEP 9. Run coprocessor post hooks. This should be done after the wal is
|
||||
// synced so that the coprocessor contract is adhered to.
|
||||
// ------------------------------------
|
||||
if (coprocessorHost != null) {
|
||||
if (!isInReplay && coprocessorHost != null) {
|
||||
for (int i = firstIndex; i < lastIndexExclusive; i++) {
|
||||
// only for successful puts
|
||||
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,
|
||||
long now) throws FailedSanityCheckException {
|
||||
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 " +
|
||||
"or a lengthy garbage collection");
|
||||
}
|
||||
startRegionOperation();
|
||||
startRegionOperation(Operation.SCAN);
|
||||
readRequestsCount.increment();
|
||||
try {
|
||||
|
||||
|
@ -4651,7 +4731,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
checkReadOnly();
|
||||
// Lock row
|
||||
startRegionOperation();
|
||||
startRegionOperation(Operation.APPEND);
|
||||
this.writeRequestsCount.increment();
|
||||
WriteEntry w = null;
|
||||
try {
|
||||
|
@ -4819,7 +4899,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
checkReadOnly();
|
||||
// Lock row
|
||||
startRegionOperation();
|
||||
startRegionOperation(Operation.INCREMENT);
|
||||
this.writeRequestsCount.increment();
|
||||
WriteEntry w = null;
|
||||
try {
|
||||
|
@ -4956,7 +5036,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
ClassSize.OBJECT +
|
||||
ClassSize.ARRAY +
|
||||
38 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
|
||||
(11 * Bytes.SIZEOF_LONG) +
|
||||
(12 * Bytes.SIZEOF_LONG) +
|
||||
Bytes.SIZEOF_BOOLEAN);
|
||||
|
||||
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
|
||||
|
@ -5236,6 +5316,37 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
*/
|
||||
public void startRegionOperation()
|
||||
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()) {
|
||||
throw new NotServingRegionException(getRegionNameAsString() + " is closing");
|
||||
}
|
||||
|
@ -5449,6 +5560,14 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
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.
|
||||
*/
|
||||
|
|
|
@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException;
|
|||
import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
||||
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.RegionOpeningException;
|
||||
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.RpcServer.BlockingServiceAndInterface;
|
||||
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.protobuf.ProtobufUtil;
|
||||
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.RegionServerStatusService;
|
||||
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.compactions.CompactionProgress;
|
||||
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.MasterAddressTracker;
|
||||
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.ZKUtil;
|
||||
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.StringUtils;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.cliffc.high_scale_lib.Counter;
|
||||
|
||||
import com.google.protobuf.BlockingRpcChannel;
|
||||
|
@ -258,6 +263,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
// catalog tracker
|
||||
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.
|
||||
*/
|
||||
|
@ -291,6 +300,13 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
*/
|
||||
protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap =
|
||||
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
|
||||
protected Leases leases;
|
||||
|
@ -456,6 +472,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
|
||||
/** Handle all the snapshot requests to this server */
|
||||
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
|
||||
private TableLockManager tableLockManager;
|
||||
|
@ -547,6 +566,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
}
|
||||
};
|
||||
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,
|
||||
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();
|
||||
|
||||
// Create the log splitting worker and start it
|
||||
this.splitLogWorker = new SplitLogWorker(this.zooKeeper,
|
||||
this.getConfiguration(), this.getServerName(), this);
|
||||
this.splitLogWorker = new SplitLogWorker(this.zooKeeper, this.getConfiguration(), this, this);
|
||||
splitLogWorker.start();
|
||||
}
|
||||
|
||||
|
@ -1641,6 +1667,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
|
||||
openSeqNum = 0;
|
||||
}
|
||||
|
||||
// Update flushed sequence id of a recovering region in ZK
|
||||
updateRecoveringRegionLastFlushedSequenceId(r);
|
||||
|
||||
// Update ZK, or META
|
||||
if (r.getRegionInfo().isMetaRegion()) {
|
||||
MetaRegionTracker.setMetaLocation(getZooKeeper(),
|
||||
|
@ -1884,14 +1914,13 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
public long getLastSequenceId(byte[] region) {
|
||||
Long lastFlushedSequenceId = -1l;
|
||||
try {
|
||||
GetLastFlushedSequenceIdRequest req =
|
||||
RequestConverter.buildGetLastFlushedSequenceIdRequest(region);
|
||||
GetLastFlushedSequenceIdRequest req = RequestConverter
|
||||
.buildGetLastFlushedSequenceIdRequest(region);
|
||||
lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req)
|
||||
.getLastFlushedSequenceId();
|
||||
.getLastFlushedSequenceId();
|
||||
} catch (ServiceException e) {
|
||||
lastFlushedSequenceId = -1l;
|
||||
LOG.warn("Unable to connect to the master to check " +
|
||||
"the last flushed sequence id", e);
|
||||
LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id", e);
|
||||
}
|
||||
return lastFlushedSequenceId;
|
||||
}
|
||||
|
@ -1965,6 +1994,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
return this.stopping;
|
||||
}
|
||||
|
||||
public Map<String, HRegion> getRecoveringRegions() {
|
||||
return this.recoveringRegions;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the configuration
|
||||
|
@ -2651,10 +2684,12 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
try {
|
||||
requestCount.increment();
|
||||
HRegion region = getRegion(request.getRegion());
|
||||
|
||||
GetResponse.Builder builder = GetResponse.newBuilder();
|
||||
ClientProtos.Get get = request.getGet();
|
||||
Boolean existence = null;
|
||||
Result r = null;
|
||||
|
||||
if (request.getClosestRowBefore()) {
|
||||
if (get.getColumnCount() != 1) {
|
||||
throw new DoNotRetryIOException(
|
||||
|
@ -3006,7 +3041,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
}
|
||||
List<KeyValue> values = new ArrayList<KeyValue>();
|
||||
MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
|
||||
region.startRegionOperation();
|
||||
region.startRegionOperation(Operation.SCAN);
|
||||
try {
|
||||
int i = 0;
|
||||
synchronized(scanner) {
|
||||
|
@ -3450,6 +3485,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
removeFromMovedRegions(region.getEncodedName());
|
||||
|
||||
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.
|
||||
// Need to pass the expected version in the constructor.
|
||||
if (region.isMetaRegion()) {
|
||||
|
@ -3465,6 +3504,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
|
||||
builder.addOpeningState(RegionOpeningState.OPENED);
|
||||
|
||||
} catch (KeeperException zooKeeperEx) {
|
||||
LOG.error("Can't retrieve recovering state from zookeeper", zooKeeperEx);
|
||||
throw new ServiceException(zooKeeperEx);
|
||||
} catch (IOException ie) {
|
||||
LOG.warn("Failed opening region " + region.getRegionNameAsString(), ie);
|
||||
if (isBulkAssign) {
|
||||
|
@ -3589,6 +3631,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
checkOpen();
|
||||
requestCount.increment();
|
||||
HRegion region = getRegion(request.getRegion());
|
||||
region.startRegionOperation(Operation.SPLIT_REGION);
|
||||
LOG.info("Splitting " + region.getRegionNameAsString());
|
||||
region.flushcache();
|
||||
byte[] splitPoint = null;
|
||||
|
@ -3621,6 +3664,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
HRegion regionA = getRegion(request.getRegionA());
|
||||
HRegion regionB = getRegion(request.getRegionB());
|
||||
boolean forcible = request.getForcible();
|
||||
regionA.startRegionOperation(Operation.MERGE_REGION);
|
||||
regionB.startRegionOperation(Operation.MERGE_REGION);
|
||||
LOG.info("Receiving merging request for " + regionA + ", " + regionB
|
||||
+ ",forcible=" + forcible);
|
||||
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.
|
||||
*
|
||||
* @param controller the RPC controller
|
||||
* @param request the request
|
||||
* @throws ServiceException
|
||||
|
@ -3841,6 +3935,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
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.
|
||||
*
|
||||
|
@ -3849,7 +3951,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* @param mutations
|
||||
*/
|
||||
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")
|
||||
Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutations.size()];
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
|
@ -3877,7 +3979,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
|
||||
OperationStatus codes[] = region.batchMutate(mutationsWithLocks);
|
||||
OperationStatus codes[] = region.batchMutate(mutationsWithLocks, isReplay);
|
||||
for (i = 0; i < codes.length; i++) {
|
||||
switch (codes[i].getOperationStatusCode()) {
|
||||
case BAD_FAMILY:
|
||||
|
@ -4097,4 +4199,91 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
public CompactSplitThread getCompactSplitThread() {
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,8 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
@InterfaceAudience.Private
|
||||
public interface LastSequenceId {
|
||||
/**
|
||||
* @param regionname
|
||||
* @return Last flushed sequence Id for regionname
|
||||
* @param regionName Encoded region name
|
||||
* @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);
|
||||
}
|
||||
|
|
|
@ -87,4 +87,8 @@ public class MetricsRegionServer {
|
|||
}
|
||||
serverSource.updateAppend(t);
|
||||
}
|
||||
|
||||
public void updateReplay(long t){
|
||||
serverSource.updateReplay(t);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -108,4 +109,9 @@ public interface RegionServerServices extends OnlineRegions {
|
|||
* @return The RegionServer's CatalogTracker
|
||||
*/
|
||||
public CatalogTracker getCatalogTracker();
|
||||
|
||||
/**
|
||||
* @return set of recovering regions on the hosting region server
|
||||
*/
|
||||
public Map<String, HRegion> getRecoveringRegions();
|
||||
}
|
||||
|
|
|
@ -20,7 +20,11 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
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.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
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.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
|
@ -70,6 +76,7 @@ import org.apache.zookeeper.data.Stat;
|
|||
@InterfaceAudience.Private
|
||||
public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
||||
private static final Log LOG = LogFactory.getLog(SplitLogWorker.class);
|
||||
private static final int checkInterval = 5000; // 5 seconds
|
||||
|
||||
Thread worker;
|
||||
private final ServerName serverName;
|
||||
|
@ -83,20 +90,30 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
private final Object grabTaskLock = new Object();
|
||||
private boolean workerInGrabTask = false;
|
||||
private final int report_period;
|
||||
private RegionServerServices server = null;
|
||||
|
||||
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);
|
||||
this.serverName = serverName;
|
||||
this.splitTaskExecutor = splitTaskExecutor;
|
||||
report_period = conf.getInt("hbase.splitlog.report.period",
|
||||
conf.getInt("hbase.splitlog.manager.timeout",
|
||||
SplitLogManager.DEFAULT_TIMEOUT) / 2);
|
||||
conf.getInt("hbase.splitlog.manager.timeout", SplitLogManager.DEFAULT_TIMEOUT) / 3);
|
||||
}
|
||||
|
||||
public SplitLogWorker(ZooKeeperWatcher watcher, final Configuration conf,
|
||||
final ServerName serverName, final LastSequenceId sequenceIdChecker) {
|
||||
this(watcher, conf, serverName, new TaskExecutor () {
|
||||
public SplitLogWorker(final ZooKeeperWatcher watcher, final Configuration conf,
|
||||
RegionServerServices server, final LastSequenceId sequenceIdChecker) {
|
||||
this(watcher, conf, server, new TaskExecutor() {
|
||||
@Override
|
||||
public Status exec(String filename, CancelableProgressable p) {
|
||||
Path rootdir;
|
||||
|
@ -113,7 +130,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
// encountered a bad non-retry-able persistent error.
|
||||
try {
|
||||
if (!HLogSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)),
|
||||
fs, conf, p, sequenceIdChecker)) {
|
||||
fs, conf, p, sequenceIdChecker, watcher)) {
|
||||
return Status.PREEMPTED;
|
||||
}
|
||||
} catch (InterruptedIOException iioe) {
|
||||
|
@ -121,9 +138,18 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
return Status.RESIGNED;
|
||||
} catch (IOException e) {
|
||||
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);
|
||||
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);
|
||||
return Status.ERR;
|
||||
|
@ -204,7 +230,39 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
synchronized (taskReadyLock) {
|
||||
while (seq_start == taskReadySeq) {
|
||||
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) {
|
||||
LOG.info("SplitLogWorker interrupted while waiting for task," +
|
||||
" 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
|
||||
public void nodeDataChanged(String path) {
|
||||
// there will be a self generated dataChanged event every time attemptToOwnTask()
|
||||
|
@ -510,7 +566,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
|
|||
return childrenPaths;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void nodeChildrenChanged(String path) {
|
||||
if(path.equals(watcher.splitLogZNode)) {
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.regionserver.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -137,6 +138,16 @@ public class OpenRegionHandler extends EventHandler {
|
|||
if (region == null) {
|
||||
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;
|
||||
if (tickleOpening("post_region_open")) {
|
||||
if (updateMeta(region)) {
|
||||
|
|
|
@ -1200,6 +1200,10 @@ class FSHLog implements HLog, Syncable {
|
|||
long now = EnvironmentEdgeManager.currentTimeMillis();
|
||||
// coprocessor hook:
|
||||
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.
|
||||
logSyncer.append(new FSHLog.Entry(logKey, logEdit));
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
public class HLogUtil {
|
||||
static final Log LOG = LogFactory.getLog(HLogUtil.class);
|
||||
|
@ -167,6 +168,37 @@ public class HLogUtil {
|
|||
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
|
||||
* with '.temp' suffix.
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
}
|
|
@ -83,6 +83,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
static final byte[] COMPLETE_CACHE_FLUSH = Bytes.toBytes("HBASE::CACHEFLUSH");
|
||||
static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
|
||||
private final int VERSION_2 = -1;
|
||||
private final boolean isReplay;
|
||||
|
||||
private final ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
|
||||
|
||||
|
@ -93,6 +94,11 @@ public class WALEdit implements Writable, HeapSize {
|
|||
private CompressionContext compressionContext;
|
||||
|
||||
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 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) {
|
||||
this.compressionContext = compressionContext;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -217,6 +217,7 @@ public class TestIOFencing {
|
|||
|
||||
public void doTest(Class<?> regionClass) throws Exception {
|
||||
Configuration c = TEST_UTIL.getConfiguration();
|
||||
c.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
// Insert our custom region
|
||||
c.setClass(HConstants.REGION_IMPL, regionClass, HRegion.class);
|
||||
c.setBoolean("dfs.support.append", true);
|
||||
|
|
|
@ -58,7 +58,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
|
|||
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); // Let's fail fast.
|
||||
conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, BuggyRegionObserver.class.getName());
|
||||
conf.set("hbase.coprocessor.abortonerror", "true");
|
||||
TEST_UTIL.startMiniCluster(2);
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
|
|
@ -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.MultiGetResponse;
|
||||
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.MutateResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||
|
@ -520,4 +521,17 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
public ExecutorService getExecutorService() {
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -331,6 +331,12 @@ public class TestCatalogJanitor {
|
|||
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
|
||||
boolean forcible) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInitialized() {
|
||||
// Auto-generated method stub
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -18,15 +18,25 @@
|
|||
*/
|
||||
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.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
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.FileSystem;
|
||||
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.Put;
|
||||
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.EnvironmentEdgeManager;
|
||||
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.Threads;
|
||||
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.log4j.Level;
|
||||
import org.apache.log4j.Logger;
|
||||
|
@ -89,9 +112,14 @@ public class TestDistributedLogSplitting {
|
|||
HBaseTestingUtility TEST_UTIL;
|
||||
|
||||
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();
|
||||
LOG.info("Starting cluster");
|
||||
conf = HBaseConfiguration.create();
|
||||
this.conf = inConf;
|
||||
conf.getLong("hbase.splitlog.max.resubmit", 0);
|
||||
// Make the failure test faster
|
||||
conf.setInt("zookeeper.recovery.retry", 0);
|
||||
|
@ -111,13 +139,20 @@ public class TestDistributedLogSplitting {
|
|||
|
||||
@After
|
||||
public void after() throws Exception {
|
||||
for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
|
||||
mt.getMaster().abort("closing...", new Exception("Trace info"));
|
||||
}
|
||||
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test (timeout=300000)
|
||||
public void testRecoveredEdits() throws Exception {
|
||||
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 SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
|
||||
// turn off load balancing to prevent regions from moving around otherwise
|
||||
|
@ -150,8 +185,7 @@ public class TestDistributedLogSplitting {
|
|||
it.remove();
|
||||
}
|
||||
}
|
||||
makeHLog(hrs.getWAL(), regions, "table",
|
||||
NUM_LOG_LINES, 100);
|
||||
makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100);
|
||||
|
||||
slm.splitLogDistributed(logDir);
|
||||
|
||||
|
@ -172,6 +206,597 @@ public class TestDistributedLogSplitting {
|
|||
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
|
||||
* 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),
|
||||
"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() {
|
||||
public void run() {
|
||||
|
@ -244,7 +870,7 @@ public class TestDistributedLogSplitting {
|
|||
|
||||
startCluster(NUM_RS); // NUM_RS=6.
|
||||
|
||||
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
|
||||
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
|
||||
"distributed log splitting test", null);
|
||||
|
||||
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
|
||||
|
@ -273,6 +899,16 @@ public class TestDistributedLogSplitting {
|
|||
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,
|
||||
TEST_UTIL.countRows(ht));
|
||||
ht.close();
|
||||
|
@ -344,8 +980,12 @@ public class TestDistributedLogSplitting {
|
|||
}
|
||||
}
|
||||
|
||||
HTable installTable(ZooKeeperWatcher zkw, String tname, String fname,
|
||||
int nrs ) throws Exception {
|
||||
HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, 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
|
||||
byte [] table = Bytes.toBytes(tname);
|
||||
byte [] family = Bytes.toBytes(fname);
|
||||
|
@ -367,14 +1007,14 @@ public class TestDistributedLogSplitting {
|
|||
for (String oregion : regions)
|
||||
LOG.debug("Region still online: " + oregion);
|
||||
}
|
||||
assertEquals(1, regions.size());
|
||||
assertEquals(1 + existingRegions, regions.size());
|
||||
LOG.debug("Enabling table\n");
|
||||
TEST_UTIL.getHBaseAdmin().enableTable(table);
|
||||
LOG.debug("Waiting for no more RIT\n");
|
||||
blockUntilNoRIT(zkw, master);
|
||||
LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
|
||||
regions = getAllOnlineRegions(cluster);
|
||||
assertEquals(numRegions + 1, regions.size());
|
||||
assertEquals(numRegions + 1 + existingRegions, regions.size());
|
||||
return ht;
|
||||
}
|
||||
|
||||
|
@ -400,40 +1040,57 @@ public class TestDistributedLogSplitting {
|
|||
}
|
||||
}
|
||||
|
||||
public void makeHLog(HLog log,
|
||||
List<HRegionInfo> hris, String tname,
|
||||
public void makeHLog(HLog log, List<HRegionInfo> regions, String tname, String fname,
|
||||
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
|
||||
hris.remove(HRegionInfo.ROOT_REGIONINFO);
|
||||
hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
regions.remove(HRegionInfo.FIRST_META_REGIONINFO);
|
||||
byte[] table = Bytes.toBytes(tname);
|
||||
HTableDescriptor htd = new HTableDescriptor(tname);
|
||||
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++) {
|
||||
value[i] = (byte)('a' + (i % 26));
|
||||
value[i] = (byte) ('a' + (i % 26));
|
||||
}
|
||||
int n = hris.size();
|
||||
int[] counts = new int[n];
|
||||
int j = 0;
|
||||
if (n > 0) {
|
||||
for (int i = 0; i < num_edits; i += 1) {
|
||||
WALEdit e = new WALEdit();
|
||||
byte [] row = Bytes.toBytes("r" + Integer.toString(i));
|
||||
byte [] family = Bytes.toBytes("f");
|
||||
byte [] qualifier = Bytes.toBytes("c" + Integer.toString(i));
|
||||
e.add(new KeyValue(row, family, qualifier,
|
||||
System.currentTimeMillis(), value));
|
||||
j++;
|
||||
log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd);
|
||||
counts[j % n] += 1;
|
||||
HRegionInfo curRegionInfo = hris.get(i % n);
|
||||
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));
|
||||
e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
|
||||
log.append(curRegionInfo, table, e, System.currentTimeMillis(), htd);
|
||||
counts[i % n] += 1;
|
||||
}
|
||||
}
|
||||
log.sync();
|
||||
log.close();
|
||||
if(closeLog) {
|
||||
log.close();
|
||||
}
|
||||
for (int i = 0; i < n; i++) {
|
||||
LOG.info("region " + hris.get(i).getRegionNameAsString() +
|
||||
" has " + counts[i] + " edits");
|
||||
LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits");
|
||||
}
|
||||
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)
|
||||
throws IOException {
|
||||
NavigableSet<String> online = new TreeSet<String>();
|
||||
|
@ -493,4 +1182,27 @@ public class TestDistributedLogSplitting {
|
|||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -783,6 +783,7 @@ public class TestMasterFailover {
|
|||
while (master.getServerManager().areDeadServersInProgress()) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
|
||||
// Failover should be completed, now wait for no RIT
|
||||
log("Waiting for no more RIT");
|
||||
ZKAssign.blockUntilNoRIT(zkw);
|
||||
|
|
|
@ -18,14 +18,28 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
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.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.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.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.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
@ -63,4 +77,40 @@ public class TestMasterFileSystem {
|
|||
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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -355,8 +355,8 @@ public class TestMasterNoCluster {
|
|||
|
||||
HMaster master = new HMaster(conf) {
|
||||
@Override
|
||||
boolean assignMeta(MonitoredTask status) {
|
||||
return true;
|
||||
void assignMeta(MonitoredTask status) {
|
||||
return;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -101,23 +101,6 @@ public class TestRSKilledWhenMasterInitializing {
|
|||
KeeperException, InterruptedException {
|
||||
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)
|
||||
|
@ -163,7 +146,7 @@ public class TestRSKilledWhenMasterInitializing {
|
|||
/* NO.1 .META. region correctness */
|
||||
// First abort master
|
||||
abortMaster(cluster);
|
||||
TestingMaster master = startMasterAndWaitUntilLogSplit(cluster);
|
||||
TestingMaster master = startMasterAndWaitTillMetaRegionAssignment(cluster);
|
||||
|
||||
// Second kill meta server
|
||||
int metaServerNum = cluster.getServerWithMeta();
|
||||
|
@ -216,14 +199,12 @@ public class TestRSKilledWhenMasterInitializing {
|
|||
LOG.debug("Master is aborted");
|
||||
}
|
||||
|
||||
private TestingMaster startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster)
|
||||
private TestingMaster startMasterAndWaitTillMetaRegionAssignment(MiniHBaseCluster cluster)
|
||||
throws IOException, InterruptedException {
|
||||
TestingMaster master = (TestingMaster) cluster.startMaster().getMaster();
|
||||
while (!master.isLogSplitAfterStartup()) {
|
||||
while (!master.isInitializationStartsMetaRegionAssignment()) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
LOG.debug("splitted:" + master.isLogSplitAfterStartup() + ",initialized:"
|
||||
+ master.isInitialized());
|
||||
return master;
|
||||
}
|
||||
|
||||
|
@ -232,7 +213,9 @@ public class TestRSKilledWhenMasterInitializing {
|
|||
while (!master.isInitialized()) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
while (master.getServerManager().areDeadServersInProgress()) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
LOG.debug("master isInitialized");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1221,7 +1221,7 @@ public class TestHLogSplit {
|
|||
generateHLogs(1, 10, -1);
|
||||
FileStatus logfile = fs.listStatus(HLOGDIR)[0];
|
||||
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()
|
||||
.toString(), conf);
|
||||
|
||||
|
|
|
@ -191,4 +191,10 @@ public class MockRegionServerServices implements RegionServerServices {
|
|||
public ExecutorService getExecutorService() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HRegion> getRecoveringRegions() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue