HBASE-11594 Unhandled NoNodeException in distributed log replay mode (Jeffrey Zhong)

This commit is contained in:
Andrew Purtell 2014-08-01 09:37:56 -07:00
parent 19e9b8aa52
commit b727d1e181
1 changed files with 32 additions and 24 deletions

View File

@ -145,6 +145,7 @@ import org.apache.hadoop.metrics.util.MBeanUtil;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import com.google.common.annotations.VisibleForTesting;
@ -2852,31 +2853,38 @@ public class HRegionServer extends HasThread implements
minSeqIdForLogReplay = storeSeqIdForReplay;
}
}
long lastRecordedFlushedSequenceId = -1;
String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
region.getEncodedName());
// recovering-region level
byte[] data;
try {
data = ZKUtil.getData(zkw, nodePath);
} catch (InterruptedException e) {
throw new InterruptedIOException();
}
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 the failed RS
nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
ZKUtil.setData(zkw, nodePath,
ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
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());
long lastRecordedFlushedSequenceId = -1;
String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
region.getEncodedName());
// recovering-region level
byte[] data;
try {
data = ZKUtil.getData(zkw, nodePath);
} catch (InterruptedException e) {
throw new InterruptedIOException();
}
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 the failed RS
nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
ZKUtil.setData(zkw, nodePath,
ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
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());
}
} catch (NoNodeException ignore) {
LOG.debug("Region " + region.getEncodedName() +
" must have completed recovery because its recovery znode has been removed", ignore);
}
}