HBASE-19165 TODO Handle stuck in transition: rit=OPENING, location=ve0538....

This commit is contained in:
Michael Stack 2017-11-03 15:13:13 -07:00
parent c04d17c4a6
commit fd86de98e1
8 changed files with 45 additions and 18 deletions

View File

@ -295,12 +295,13 @@ public class RecoverableZooKeeper {
private void retryOrThrow(RetryCounter retryCounter, KeeperException e, private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
String opName) throws KeeperException { String opName) throws KeeperException {
LOG.debug("Possibly transient ZooKeeper, quorum=" + quorumServers + ", exception=" + e);
if (!retryCounter.shouldRetry()) { if (!retryCounter.shouldRetry()) {
LOG.error("ZooKeeper " + opName + " failed after " LOG.error("ZooKeeper " + opName + " failed after "
+ retryCounter.getMaxAttempts() + " attempts"); + retryCounter.getMaxAttempts() + " attempts");
throw e; throw e;
} }
LOG.debug("Retry, connectivity issue (JVM Pause?); quorum=" + quorumServers + "," +
"exception=" + e);
} }
/** /**

View File

@ -656,7 +656,7 @@ public class ServerManager {
} }
if (!master.getAssignmentManager().isFailoverCleanupDone()) { if (!master.getAssignmentManager().isFailoverCleanupDone()) {
LOG.info("AssignmentManager hasn't finished failover cleanup; waiting"); LOG.debug("AssignmentManager failover cleanup not done.");
} }
for (Map.Entry<ServerName, Boolean> entry : requeuedDeadServers.entrySet()) { for (Map.Entry<ServerName, Boolean> entry : requeuedDeadServers.entrySet()) {

View File

@ -935,7 +935,7 @@ public class AssignmentManager implements ServerListener {
wakeServerReportEvent(serverNode); wakeServerReportEvent(serverNode);
} }
public void checkOnlineRegionsReportForMeta(final ServerStateNode serverNode, void checkOnlineRegionsReportForMeta(final ServerStateNode serverNode,
final Set<byte[]> regionNames) { final Set<byte[]> regionNames) {
try { try {
for (byte[] regionName: regionNames) { for (byte[] regionName: regionNames) {
@ -951,7 +951,7 @@ public class AssignmentManager implements ServerListener {
final RegionStateNode regionNode = regionStates.getOrCreateRegionNode(hri); final RegionStateNode regionNode = regionStates.getOrCreateRegionNode(hri);
LOG.info("META REPORTED: " + regionNode); LOG.info("META REPORTED: " + regionNode);
if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) { if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) {
LOG.warn("META REPORTED but no procedure found"); LOG.warn("META REPORTED but no procedure found (complete?)");
regionNode.setRegionLocation(serverNode.getServerName()); regionNode.setRegionLocation(serverNode.getServerName());
} else if (LOG.isTraceEnabled()) { } else if (LOG.isTraceEnabled()) {
LOG.trace("META REPORTED: " + regionNode); LOG.trace("META REPORTED: " + regionNode);
@ -1183,17 +1183,26 @@ public class AssignmentManager implements ServerListener {
public void visitRegionState(final RegionInfo regionInfo, final State state, public void visitRegionState(final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) { final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) {
final RegionStateNode regionNode = regionStates.getOrCreateRegionNode(regionInfo); final RegionStateNode regionNode = regionStates.getOrCreateRegionNode(regionInfo);
State localState = state;
if (localState == null) {
// No region state column data in hbase:meta table! Are I doing a rolling upgrade from
// hbase1 to hbase2? Am I restoring a SNAPSHOT or otherwise adding a region to hbase:meta?
// In any of these cases, state is empty. For now, presume OFFLINE but there are probably
// cases where we need to probe more to be sure this correct; TODO informed by experience.
LOG.info(regionInfo.getEncodedName() + " state=null; presuming " + State.OFFLINE);
localState = State.OFFLINE;
}
synchronized (regionNode) { synchronized (regionNode) {
if (!regionNode.isInTransition()) { if (!regionNode.isInTransition()) {
regionNode.setState(state); regionNode.setState(localState);
regionNode.setLastHost(lastHost); regionNode.setLastHost(lastHost);
regionNode.setRegionLocation(regionLocation); regionNode.setRegionLocation(regionLocation);
regionNode.setOpenSeqNum(openSeqNum); regionNode.setOpenSeqNum(openSeqNum);
if (state == State.OPEN) { if (localState == State.OPEN) {
assert regionLocation != null : "found null region location for " + regionNode; assert regionLocation != null : "found null region location for " + regionNode;
regionStates.addRegionToServer(regionLocation, regionNode); regionStates.addRegionToServer(regionLocation, regionNode);
} else if (state == State.OFFLINE || regionInfo.isOffline()) { } else if (localState == State.OFFLINE || regionInfo.isOffline()) {
regionStates.addToOfflineRegions(regionNode); regionStates.addToOfflineRegions(regionNode);
} else { } else {
// These regions should have a procedure in replay // These regions should have a procedure in replay

View File

@ -90,10 +90,15 @@ public class RegionStateStore {
@Override @Override
public boolean visit(final Result r) throws IOException { public boolean visit(final Result r) throws IOException {
if (r != null && !r.isEmpty()) { if (r != null && !r.isEmpty()) {
long st = System.currentTimeMillis(); long st = 0;
if (LOG.isTraceEnabled()) {
st = System.currentTimeMillis();
}
visitMetaEntry(visitor, r); visitMetaEntry(visitor, r);
long et = System.currentTimeMillis(); if (LOG.isTraceEnabled()) {
LOG.info("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st)); long et = System.currentTimeMillis();
LOG.trace("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st));
}
} else if (isDebugEnabled) { } else if (isDebugEnabled) {
LOG.debug("NULL result from meta - ignoring but this is strange."); LOG.debug("NULL result from meta - ignoring but this is strange.");
} }
@ -310,11 +315,13 @@ public class RegionStateStore {
/** /**
* Pull the region state from a catalog table {@link Result}. * Pull the region state from a catalog table {@link Result}.
* @param r Result to pull the region state from * @param r Result to pull the region state from
* @return the region state, or OPEN if there's no value written. * @return the region state, or null if unknown.
*/ */
protected State getRegionState(final Result r, int replicaId) { protected State getRegionState(final Result r, int replicaId) {
Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(replicaId)); Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(replicaId));
if (cell == null || cell.getValueLength() == 0) return State.OPENING; if (cell == null || cell.getValueLength() == 0) {
return null;
}
return State.valueOf(Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())); return State.valueOf(Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
} }

View File

@ -542,11 +542,19 @@ public class RegionStates {
return !getTableRegionStates(tableName).isEmpty(); return !getTableRegionStates(tableName).isEmpty();
} }
/**
* @return Return online regions of table; does not include OFFLINE or SPLITTING regions.
*/
public List<RegionInfo> getRegionsOfTable(final TableName table) { public List<RegionInfo> getRegionsOfTable(final TableName table) {
return getRegionsOfTable(table, false); return getRegionsOfTable(table, false);
} }
List<RegionInfo> getRegionsOfTable(final TableName table, final boolean offline) { /**
* @return Return the regions of the table; does not include OFFLINE unless you set
* <code>offline</code> to true. Does not include regions that are in the
* {@link State#SPLIT} state.
*/
public List<RegionInfo> getRegionsOfTable(final TableName table, final boolean offline) {
final ArrayList<RegionStateNode> nodes = getTableRegionStateNodes(table); final ArrayList<RegionStateNode> nodes = getTableRegionStateNodes(table);
final ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(nodes.size()); final ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(nodes.size());
for (RegionStateNode node: nodes) { for (RegionStateNode node: nodes) {

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -120,9 +120,9 @@ public class EnableTableProcedure
// Get the replica count // Get the replica count
int regionReplicaCount = hTableDescriptor.getRegionReplication(); int regionReplicaCount = hTableDescriptor.getRegionReplication();
// Get the regions for the table from the memory // Get the regions for the table from memory; get both online and offline regions ('true').
List<RegionInfo> regionsOfTable = List<RegionInfo> regionsOfTable =
env.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName); env.getAssignmentManager().getRegionStates().getRegionsOfTable(tableName, true);
if (regionReplicaCount > 1) { if (regionReplicaCount > 1) {
int currentMaxReplica = 0; int currentMaxReplica = 0;

View File

@ -56,7 +56,9 @@ public class ZooKeeperMainServer {
while (!this.zk.getState().isConnected()) { while (!this.zk.getState().isConnected()) {
Thread.sleep(1); Thread.sleep(1);
if (stopWatch.elapsed(TimeUnit.SECONDS) > 10) { if (stopWatch.elapsed(TimeUnit.SECONDS) > 10) {
throw new InterruptedException("Failed connect " + this.zk); throw new InterruptedException("Failed connect after waiting " +
stopWatch.elapsed(TimeUnit.SECONDS) + "seconds; state=" + this.zk.getState() +
"; " + this.zk);
} }
} }
} }

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information