HBASE-7551 nodeChildrenChange event may happen after the transition to RS_ZK_REGION_SPLITTING in SplitTransaction causing the SPLIT event to be missed in the master side. (Ram, Ted, and Lars H)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1433696 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
larsh 2013-01-15 22:14:23 +00:00
parent 839dec12d2
commit 289c12dea3
1 changed files with 18 additions and 4 deletions

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -40,7 +39,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import com.google.common.collect.LinkedHashMultimap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -85,6 +83,8 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.data.Stat;
import com.google.common.collect.LinkedHashMultimap;
/**
* Manages and performs region assignment.
* <p>
@ -1061,13 +1061,27 @@ public class AssignmentManager extends ZooKeeperListener {
ZKUtil.listChildrenAndWatchForNewChildren(
watcher, watcher.assignmentZNode);
if (children != null) {
Stat stat = new Stat();
for (String child : children) {
// if region is in transition, we already have a watch
// on it, so no need to watch it again. So, as I know for now,
// this is needed to watch splitting nodes only.
if (!regionStates.isRegionInTransition(child)) {
ZKUtil.watchAndCheckExists(watcher,
ZKUtil.joinZNode(watcher.assignmentZNode, child));
stat.setVersion(0);
byte[] data = ZKAssign.getDataAndWatch(watcher,
ZKUtil.joinZNode(watcher.assignmentZNode, child), stat);
if (data != null && stat.getVersion() > 0) {
try {
RegionTransition rt = RegionTransition.parseFrom(data);
//See HBASE-7551, handle splitting too, in case we miss the node change event
if (rt.getEventType() == EventType.RS_ZK_REGION_SPLITTING) {
handleRegion(rt, stat.getVersion());
}
} catch (DeserializationException de) {
LOG.error("error getting data for " + child, de);
}
}
}
}
}