diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon index 2e4c85d78f0..80f1d6d53d0 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon @@ -20,7 +20,7 @@ limitations under the License. <%import> org.apache.hadoop.hbase.HRegionInfo; org.apache.hadoop.hbase.master.AssignmentManager; -org.apache.hadoop.hbase.master.AssignmentManager.RegionState; +org.apache.hadoop.hbase.master.RegionState; org.apache.hadoop.conf.Configuration; org.apache.hadoop.hbase.HBaseConfiguration; org.apache.hadoop.hbase.HConstants; @@ -32,7 +32,8 @@ AssignmentManager assignmentManager; int limit = 100; <%java> -Map rit = assignmentManager.copyRegionsInTransition(); +Map rit = assignmentManager + .getRegionStates().getRegionsInTransition(); // process the map to find region in transition details Configuration conf = HBaseConfiguration.create(); int ritThreshold = conf.getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java index f14fcab17dd..478a15e49f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java @@ -20,19 +20,16 @@ package org.apache.hadoop.hbase; -import java.io.DataInput; -import java.io.DataOutput; -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.LinkedList; import java.util.Map; -import java.util.Set; -import java.util.TreeMap; -import java.util.HashSet; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.LiveServerInfo; @@ -41,13 +38,7 @@ import org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileConte import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; -import org.apache.hadoop.hbase.ServerLoad; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.hbase.RegionLoad; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.VersionMismatchException; import org.apache.hadoop.io.VersionedWritable; import com.google.protobuf.ByteString; @@ -83,7 +74,6 @@ public class ClusterStatus extends VersionedWritable { *
3
Added master and backupMasters
* */ - private static final byte VERSION_MASTER_BACKUPMASTERS = 2; private static final byte VERSION = 2; private String hbaseVersion; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java index 1be340e6c70..9450bcc9f80 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/UnknownRegionException.java @@ -19,8 +19,6 @@ */ package org.apache.hadoop.hbase; -import java.io.IOException; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -29,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability; */ @InterfaceAudience.Public @InterfaceStability.Stable -public class UnknownRegionException extends IOException { +public class UnknownRegionException extends RegionException { private static final long serialVersionUID = 1968858760475205392L; public UnknownRegionException(String regionName) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index b567ec7d04f..bb778593678 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -1177,6 +1177,16 @@ public class HBaseAdmin implements Abortable, Closeable { ProtobufUtil.closeRegion(admin, hri.getRegionName(), false); } + /** + * Get all the online regions on a region server. + */ + public List getOnlineRegions( + final ServerName sn) throws IOException { + AdminProtocol admin = + this.connection.getAdmin(sn.getHostname(), sn.getPort()); + return ProtobufUtil.getOnlineRegions(admin); + } + /** * Flush a table or an individual region. * Asynchronous operation. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 300cd89b5b4..d6b41ecbce5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -19,14 +19,11 @@ */ package org.apache.hadoop.hbase.master; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -34,13 +31,10 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.Set; -import java.util.SortedMap; import java.util.TreeMap; -import java.util.TreeSet; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -65,7 +59,6 @@ import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState.State; import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler; import org.apache.hadoop.hbase.master.handler.DisableTableHandler; import org.apache.hadoop.hbase.master.handler.EnableTableHandler; @@ -73,11 +66,9 @@ import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler; import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler; import org.apache.hadoop.hbase.master.handler.SplitRegionHandler; import org.apache.hadoop.hbase.master.metrics.MasterMetrics; -import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.KeyLocker; import org.apache.hadoop.hbase.util.Pair; @@ -124,6 +115,11 @@ public class AssignmentManager extends ZooKeeperListener { final private KeyLocker locker = new KeyLocker(); + /** + * Used for assignment only. TODO: revisit the assign lock scheme + */ + final private KeyLocker assignLocker = new KeyLocker(); + /** * Map of regions to reopen after the schema of a table is changed. Key - * encoded region name, value - HRegionInfo @@ -135,13 +131,6 @@ public class AssignmentManager extends ZooKeeperListener { */ private final int maximumAssignmentAttempts; - /** - * Regions currently in transition. Map of encoded region names to the master - * in-memory state for that region. - */ - final NotifiableConcurrentSkipListMap regionsInTransition = - new NotifiableConcurrentSkipListMap(); - /** Plans for region movement. Key is the encoded version of a region name*/ // TODO: When do plans get cleaned out? Ever? In server open and in server // shutdown processing -- St.Ack @@ -156,16 +145,6 @@ public class AssignmentManager extends ZooKeeperListener { // store all the enabling state tablenames. Set enablingTables = new HashSet(); - /** - * Server to regions assignment map. - * Contains the set of regions currently assigned to a given server. - * This Map and {@link #regions} are tied. Always update this in tandem - * with the other under a lock on {@link #regions}. - * @see #regions - */ - private final NavigableMap> servers = - new TreeMap>(); - /** * Contains the server which need to update timer, these servers will be * handled by {@link TimerUpdater} @@ -173,16 +152,6 @@ public class AssignmentManager extends ZooKeeperListener { private final ConcurrentSkipListSet serversInUpdatingTimer = new ConcurrentSkipListSet(); - /** - * Region to server assignment map. - * Contains the server a given region is currently assigned to. - * This Map and {@link #servers} are tied. Always update this in tandem - * with the other under a lock on {@link #regions}. - * @see #servers - */ - private final SortedMap regions = - new TreeMap(); - private final ExecutorService executorService; //Thread pool executor service for timeout monitor @@ -205,6 +174,8 @@ public class AssignmentManager extends ZooKeeperListener { // metrics instance to send metrics for RITs MasterMetrics masterMetrics; + private final RegionStates regionStates; + /** * Constructs a new assignment manager. * @@ -217,8 +188,7 @@ public class AssignmentManager extends ZooKeeperListener { */ public AssignmentManager(Server master, ServerManager serverManager, CatalogTracker catalogTracker, final LoadBalancer balancer, - final ExecutorService service, MasterMetrics metrics) - throws KeeperException, IOException { + final ExecutorService service, MasterMetrics metrics) throws KeeperException, IOException { super(master.getZooKeeper()); this.master = master; this.serverManager = serverManager; @@ -241,6 +211,7 @@ public class AssignmentManager extends ZooKeeperListener { this.balancer = balancer; this.threadPoolExecutorService = Executors.newCachedThreadPool(); this.masterMetrics = metrics;// can be null only with tests. + this.regionStates = new RegionStates(master, serverManager); } void startTimeOutMonitor() { @@ -248,26 +219,6 @@ public class AssignmentManager extends ZooKeeperListener { + ".timeoutMonitor"); } - /** - * Compute the average load across all region servers. - * Currently, this uses a very naive computation - just uses the number of - * regions being served, ignoring stats about number of requests. - * @return the average load - */ - double getAverageLoad() { - int totalLoad = 0; - int numServers = 0; - // Sync on this.regions because access to this.servers always synchronizes - // in this order. - synchronized (this.regions) { - for (Map.Entry> e: servers.entrySet()) { - numServers++; - totalLoad += e.getValue().size(); - } - } - return (double)totalLoad / (double)numServers; - } - /** * @return Instance of ZKTable. */ @@ -276,17 +227,19 @@ public class AssignmentManager extends ZooKeeperListener { // sharing. return this.zkTable; } + /** - * Returns the RegionServer to which hri is assigned. + * This SHOULD not be public. It is public now + * because of some unit tests. * - * @param hri - * HRegion for which this function returns the region server - * @return HServerInfo The region server to which hri belongs + * TODO: make it package private and keep RegionStates in the master package */ - public ServerName getRegionServerOfRegion(HRegionInfo hri) { - synchronized (this.regions ) { - return regions.get(hri); - } + public RegionStates getRegionStates() { + return regionStates; + } + + public RegionPlan getRegionReopenPlan(HRegionInfo hri) { + return new RegionPlan(hri, null, regionStates.getRegionServerOfRegion(hri)); } /** @@ -337,7 +290,8 @@ public class AssignmentManager extends ZooKeeperListener { for (HRegionInfo hri : hris) { String name = hri.getEncodedName(); // no lock concurrent access ok: sequential consistency respected. - if (regionsToReopen.containsKey(name) || regionsInTransition.containsKey(name)) { + if (regionsToReopen.containsKey(name) + || regionStates.isRegionInTransition(name)) { pending++; } } @@ -388,19 +342,6 @@ public class AssignmentManager extends ZooKeeperListener { recoverTableInEnablingState(this.enablingTables, isWatcherCreated); } - /** - * Process all regions that are in transition up in zookeeper. Used by - * master joining an already running cluster. - * @throws KeeperException - * @throws IOException - * @throws InterruptedException - */ - void processDeadServersAndRegionsInTransition() - throws KeeperException, IOException, InterruptedException { - // Pass null to signify no dead servers in this context. - processDeadServersAndRegionsInTransition(null); - } - /** * Process all regions that are in transition in zookeeper and also * processes the list of dead servers by scanning the META. @@ -417,35 +358,34 @@ public class AssignmentManager extends ZooKeeperListener { throws KeeperException, IOException, InterruptedException { List nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.assignmentZNode); - + if (nodes == null) { String errorMessage = "Failed to get the children from ZK"; master.abort(errorMessage, new IOException(errorMessage)); return; } - + // Run through all regions. If they are not assigned and not in RIT, then // its a clean cluster startup, else its a failover. - synchronized (this.regions) { - for (Map.Entry e: this.regions.entrySet()) { - if (!e.getKey().isMetaTable() && e.getValue() != null) { - LOG.debug("Found " + e + " out on cluster"); - this.failover = true; - break; - } - if (nodes.contains(e.getKey().getEncodedName())) { - LOG.debug("Found " + e.getKey().getRegionNameAsString() + " in RITs"); - // Could be a meta region. - this.failover = true; - break; - } + Map regions = regionStates.getRegionAssignments(); + for (Map.Entry e: regions.entrySet()) { + if (!e.getKey().isMetaTable() && e.getValue() != null) { + LOG.debug("Found " + e + " out on cluster"); + this.failover = true; + break; + } + if (nodes.contains(e.getKey().getEncodedName())) { + LOG.debug("Found " + e.getKey().getRegionNameAsString() + " in RITs"); + // Could be a meta region. + this.failover = true; + break; } } // Remove regions in RIT, they are possibly being processed by // ServerShutdownHandler. // no lock concurrent access ok: some threads may be adding/removing items but its java-valid - nodes.removeAll(regionsInTransition.keySet()); + nodes.removeAll(regionStates.getRegionsInTransition().keySet()); // If some dead servers are processed by ServerShutdownHandler, we shouldn't // assign all user regions( some would be assigned by @@ -488,11 +428,10 @@ public class AssignmentManager extends ZooKeeperListener { if (!intransistion) return intransistion; LOG.debug("Waiting on " + HRegionInfo.prettyPrint(hri.getEncodedName())); while (!this.master.isStopped() && - // no lock concurrent access ok: sequentially consistent - this.regionsInTransition.containsKey(hri.getEncodedName())) { + this.regionStates.isRegionInTransition(hri.getEncodedName())) { // We put a timeout because we may have the region getting in just between the test // and the waitForUpdate - this.regionsInTransition.waitForUpdate(100); + this.regionStates.waitForUpdate(100); } return intransistion; } @@ -543,8 +482,8 @@ public class AssignmentManager extends ZooKeeperListener { // is that we don't have two threads working on the same region. Lock lock = locker.acquireLock(encodedRegionName); try { - RegionState regionState = regionsInTransition.get(encodedRegionName); - if (regionState != null || failoverProcessedRegions.containsKey(encodedRegionName)) { + if (regionStates.isRegionInTransition(encodedRegionName) + || failoverProcessedRegions.containsKey(encodedRegionName)) { // Just return return; } @@ -552,15 +491,14 @@ public class AssignmentManager extends ZooKeeperListener { case M_ZK_REGION_CLOSING: // If zk node of the region was updated by a live server skip this // region and just add it into RIT. - if (isOnDeadServer(regionInfo, deadServers) && (sn == null || !isServerOnline(sn))) { + if (isOnDeadServer(regionInfo, deadServers) && !isServerOnline(sn)) { // If was on dead server, its closed now. Force to OFFLINE and this // will get it reassigned if appropriate forceOffline(regionInfo, rt); } else { // Just insert region into RIT. // If this never updates the timeout will trigger new assignment - regionsInTransition.put(encodedRegionName, - getRegionState(regionInfo, RegionState.State.CLOSING, rt)); + regionStates.updateRegionState(rt, RegionState.State.CLOSING); } failoverProcessedRegions.put(encodedRegionName, regionInfo); break; @@ -583,8 +521,7 @@ public class AssignmentManager extends ZooKeeperListener { // RPC is not yet sent addToRITandCallClose(regionInfo, RegionState.State.OFFLINE, rt); } else { - regionsInTransition.put(encodedRegionName, - getRegionState(regionInfo, RegionState.State.PENDING_OPEN, rt)); + regionStates.updateRegionState(rt, RegionState.State.PENDING_OPEN); } failoverProcessedRegions.put(encodedRegionName, regionInfo); break; @@ -595,8 +532,7 @@ public class AssignmentManager extends ZooKeeperListener { // Just insert region into RIT // If this never updates the timeout will trigger new assignment if (regionInfo.isMetaTable()) { - regionsInTransition.put(encodedRegionName, - getRegionState(regionInfo, RegionState.State.OPENING, rt)); + regionStates.updateRegionState(rt, RegionState.State.OPENING); // If ROOT or .META. table is waiting for timeout monitor to assign // it may take lot of time when the assignment.timeout.period is // the default value which may be very long. We will not be able @@ -609,16 +545,14 @@ public class AssignmentManager extends ZooKeeperListener { // it to a new RS. (HBASE-5882) processOpeningState(regionInfo); break; - } - regionsInTransition.put(encodedRegionName, - getRegionState(regionInfo, RegionState.State.OPENING, rt)); + } + regionStates.updateRegionState(rt, RegionState.State.OPENING); failoverProcessedRegions.put(encodedRegionName, regionInfo); break; case RS_ZK_REGION_OPENED: // Region is opened, insert into RIT and handle it - regionsInTransition.put(encodedRegionName, - getRegionState(regionInfo, RegionState.State.OPEN, rt)); + regionStates.updateRegionState(rt, RegionState.State.OPEN); // sn could be null if this server is no longer online. If // that is the case, just let this RIT timeout; it'll be assigned // to new server then. @@ -679,24 +613,10 @@ public class AssignmentManager extends ZooKeeperListener { */ private void addToRITandCallClose(final HRegionInfo hri, final RegionState.State state, final RegionTransition oldData) { - // No lock concurrency: adding a synchronized here would not prevent to have two - // entries as we don't check if the region is already there. This must be ensured by the - // method callers. - this.regionsInTransition.put(hri.getEncodedName(), getRegionState(hri, state, oldData)); + regionStates.updateRegionState(oldData, state); new ClosedRegionHandler(this.master, this, hri).process(); } - /** - * @param hri - * @param state - * @param rt - * @return A new {@link RegionState} instance made of the passed arguments - */ - RegionState getRegionState(final HRegionInfo hri, final RegionState.State state, - final RegionTransition rt) { - return new RegionState(hri, state, rt.getCreateTime(), rt.getServerName()); - } - /** * When a region is closed, it should be removed from the regionsToReopen * @param hri HRegionInfo of the region which was closed @@ -769,12 +689,12 @@ public class AssignmentManager extends ZooKeeperListener { try { // Printing if the event was created a long time ago helps debugging boolean lateEvent = createTime < (System.currentTimeMillis() - 15000); - RegionState regionState = regionsInTransition.get(encodedName); + RegionState regionState = regionStates.getRegionTransitionState(encodedName); LOG.debug("Handling transition=" + rt.getEventType() + ", server=" + sn + ", region=" + (prettyPrintedRegionName == null ? "null" : prettyPrintedRegionName) + (lateEvent ? ", which is more than 15 seconds late" : "") + - ", current state from RIT=" + regionState); + ", current state from region state map =" + regionState); switch (rt.getEventType()) { case M_ZK_REGION_OFFLINE: // Nothing to do. @@ -782,7 +702,7 @@ public class AssignmentManager extends ZooKeeperListener { case RS_ZK_REGION_SPLITTING: if (!isInStateForSplitting(regionState)) break; - addSplittingToRIT(sn, encodedName); + regionStates.updateRegionState(rt, RegionState.State.SPLITTING); break; case RS_ZK_REGION_SPLIT: @@ -790,7 +710,9 @@ public class AssignmentManager extends ZooKeeperListener { if (!isInStateForSplitting(regionState)) break; // If null, add SPLITTING state before going to SPLIT if (regionState == null) { - regionState = addSplittingToRIT(sn, encodedName); + regionState = regionStates.updateRegionState(rt, + RegionState.State.SPLITTING); + String message = "Received SPLIT for region " + prettyPrintedRegionName + " from server " + sn; // If still null, it means we cannot find it and it was already processed @@ -826,8 +748,8 @@ public class AssignmentManager extends ZooKeeperListener { case M_ZK_REGION_CLOSING: hri = checkIfInFailover(regionState, encodedName, regionName); if (hri != null) { - regionState = new RegionState(hri, RegionState.State.CLOSING, createTime, sn); - regionsInTransition.put(encodedName, regionState); + regionState = regionStates.updateRegionState( + hri, RegionState.State.CLOSING, createTime, sn); failoverProcessedRegions.put(encodedName, hri); break; } @@ -842,14 +764,14 @@ public class AssignmentManager extends ZooKeeperListener { return; } // Transition to CLOSING (or update stamp if already CLOSING) - regionState.update(RegionState.State.CLOSING, createTime, sn); + regionStates.updateRegionState(rt, RegionState.State.CLOSING); break; case RS_ZK_REGION_CLOSED: hri = checkIfInFailover(regionState, encodedName, regionName); if (hri != null) { - regionState = new RegionState(hri, RegionState.State.CLOSED, createTime, sn); - regionsInTransition.put(encodedName, regionState); + regionState = regionStates.updateRegionState( + hri, RegionState.State.CLOSED, createTime, sn); removeClosedRegion(regionState.getRegion()); new ClosedRegionHandler(master, this, regionState.getRegion()) .process(); @@ -868,7 +790,7 @@ public class AssignmentManager extends ZooKeeperListener { // Handle CLOSED by assigning elsewhere or stopping if a disable // If we got here all is good. Need to update RegionState -- else // what follows will fail because not in expected state. - regionState.update(RegionState.State.CLOSED, createTime, sn); + regionStates.updateRegionState(rt, RegionState.State.CLOSED); removeClosedRegion(regionState.getRegion()); this.executorService.submit(new ClosedRegionHandler(master, this, regionState.getRegion())); @@ -877,8 +799,8 @@ public class AssignmentManager extends ZooKeeperListener { case RS_ZK_REGION_FAILED_OPEN: hri = checkIfInFailover(regionState, encodedName, regionName); if (hri != null) { - regionState = new RegionState(hri, RegionState.State.CLOSED, createTime, sn); - regionsInTransition.put(encodedName, regionState); + regionState = regionStates.updateRegionState( + hri, RegionState.State.CLOSED, createTime, sn); new ClosedRegionHandler(master, this, regionState.getRegion()) .process(); failoverProcessedRegions.put(encodedName, hri); @@ -892,8 +814,8 @@ public class AssignmentManager extends ZooKeeperListener { return; } // Handle this the same as if it were opened and then closed. - regionState.update(RegionState.State.CLOSED, createTime, sn); - // When there are more than one region server a new RS is selected as the + regionStates.updateRegionState(rt, RegionState.State.CLOSED); + // When there are more than one region server a new RS is selected as the // destination and the same is updated in the regionplan. (HBASE-5546) getRegionPlan(regionState, sn, true); this.executorService.submit(new ClosedRegionHandler(master, @@ -903,8 +825,8 @@ public class AssignmentManager extends ZooKeeperListener { case RS_ZK_REGION_OPENING: hri = checkIfInFailover(regionState, encodedName, regionName); if (hri != null) { - regionState = new RegionState(hri, RegionState.State.OPENING, createTime, sn); - regionsInTransition.put(encodedName, regionState); + regionState = regionStates.updateRegionState( + hri, RegionState.State.OPENING, createTime, sn); failoverProcessedRegions.put(encodedName, hri); break; } @@ -920,14 +842,14 @@ public class AssignmentManager extends ZooKeeperListener { return; } // Transition to OPENING (or update stamp if already OPENING) - regionState.update(RegionState.State.OPENING, createTime, sn); + regionStates.updateRegionState(rt, RegionState.State.OPENING); break; case RS_ZK_REGION_OPENED: hri = checkIfInFailover(regionState, encodedName, regionName); if (hri != null) { - regionState = new RegionState(hri, RegionState.State.OPEN, createTime, sn); - regionsInTransition.put(encodedName, regionState); + regionState = regionStates.updateRegionState( + hri, RegionState.State.OPEN, createTime, sn); new OpenedRegionHandler(master, this, regionState.getRegion(), sn, expectedVersion).process(); failoverProcessedRegions.put(encodedName, hri); break; @@ -943,7 +865,7 @@ public class AssignmentManager extends ZooKeeperListener { return; } // Handle OPENED by removing from transition and deleted zk node - regionState.update(RegionState.State.OPEN, createTime, sn); + regionStates.updateRegionState(rt, RegionState.State.OPEN); this.executorService.submit( new OpenedRegionHandler(master, this, regionState.getRegion(), sn, expectedVersion)); break; @@ -975,7 +897,7 @@ public class AssignmentManager extends ZooKeeperListener { } return null; } - + /** * Gets the HRegionInfo from the META table * @param regionName @@ -1019,70 +941,14 @@ public class AssignmentManager extends ZooKeeperListener { private boolean convertPendingCloseToSplitting(final RegionState rs) { if (!rs.isPendingClose()) return false; LOG.debug("Converting PENDING_CLOSE to SPLITING; rs=" + rs); - rs.update(RegionState.State.SPLITTING); + regionStates.updateRegionState( + rs.getRegion(), RegionState.State.SPLITTING); // Clean up existing state. Clear from region plans seems all we // have to do here by way of clean up of PENDING_CLOSE. clearRegionPlan(rs.getRegion()); return true; } - /** - * @param serverName - * @param encodedName - * @return The SPLITTING RegionState we added to RIT for the passed region - * encodedName - */ - private RegionState addSplittingToRIT(final ServerName serverName, - final String encodedName) { - RegionState regionState = null; - synchronized (this.regions) { - regionState = findHRegionInfoThenAddToRIT(serverName, encodedName); - if (regionState != null) { - regionState.update(RegionState.State.SPLITTING, - System.currentTimeMillis(), serverName); - } - } - return regionState; - } - - /** - * Caller must hold lock on this.regions. - * @param serverName - * @param encodedName - * @return The instance of RegionState that was added to RIT or null if error. - */ - private RegionState findHRegionInfoThenAddToRIT(final ServerName serverName, - final String encodedName) { - HRegionInfo hri = findHRegionInfo(serverName, encodedName); - if (hri == null) { - LOG.warn("Region " + encodedName + " not found on server " + serverName + - "; failed processing"); - return null; - } - // Add to regions in transition, then update state to SPLITTING. - return addToRegionsInTransition(hri); - } - - /** - * Caller must hold lock on this.regions. - * @param serverName - * @param encodedName - * @return Found HRegionInfo or null. - */ - private HRegionInfo findHRegionInfo(final ServerName sn, - final String encodedName) { - if (!this.serverManager.isServerOnline(sn)) return null; - Set hris = this.servers.get(sn); - HRegionInfo foundHri = null; - for (HRegionInfo hri: hris) { - if (hri.getEncodedName().equals(encodedName)) { - foundHri = hri; - break; - } - } - return foundHri; - } - /** * Handle a ZK unassigned node transition triggered by HBCK repair tool. *

@@ -1094,7 +960,7 @@ public class AssignmentManager extends ZooKeeperListener { LOG.info("Handling HBCK triggered transition=" + rt.getEventType() + ", server=" + rt.getServerName() + ", region=" + HRegionInfo.prettyPrint(encodedName)); - RegionState regionState = regionsInTransition.get(encodedName); + RegionState regionState = regionStates.getRegionTransitionState(encodedName); switch (rt.getEventType()) { case M_ZK_REGION_OFFLINE: HRegionInfo regionInfo = null; @@ -1178,9 +1044,7 @@ public class AssignmentManager extends ZooKeeperListener { public void nodeDeleted(final String path) { if (path.startsWith(this.watcher.assignmentZNode)) { String regionName = ZKAssign.getRegionName(this.master.getZooKeeper(), path); - // no lock concurrency ok: sequentially consistent if someone adds/removes the region in - // the same time - RegionState rs = this.regionsInTransition.get(regionName); + RegionState rs = regionStates.getRegionTransitionState(regionName); if (rs != null) { HRegionInfo regionInfo = rs.getRegion(); if (rs.isSplit()) { @@ -1191,26 +1055,22 @@ public class AssignmentManager extends ZooKeeperListener { LOG.debug("The znode of region " + regionInfo.getRegionNameAsString() + " has been deleted."); if (rs.isOpened()) { - makeRegionOnline(rs, regionInfo); + ServerName serverName = rs.getServerName(); + regionOnline(regionInfo, serverName); + LOG.info("The master has opened the region " + + regionInfo.getRegionNameAsString() + " that was online on " + + serverName); + if (this.getZKTable().isDisablingOrDisabledTable( + regionInfo.getTableNameAsString())) { + LOG.debug("Opened region " + + regionInfo.getRegionNameAsString() + " but " + + "this table is disabled, triggering close of region"); + unassign(regionInfo); + } } } } } - - } - - private void makeRegionOnline(RegionState rs, HRegionInfo regionInfo) { - regionOnline(regionInfo, rs.serverName); - LOG.info("The master has opened the region " - + regionInfo.getRegionNameAsString() + " that was online on " - + rs.serverName); - if (this.getZKTable().isDisablingOrDisabledTable( - regionInfo.getTableNameAsString())) { - LOG.debug("Opened region " - + regionInfo.getRegionNameAsString() + " but " - + "this table is disabled, triggering close of region"); - unassign(regionInfo); - } } /** @@ -1247,24 +1107,13 @@ public class AssignmentManager extends ZooKeeperListener { * @param sn */ void regionOnline(HRegionInfo regionInfo, ServerName sn) { - // no lock concurrency ok. - this.regionsInTransition.remove(regionInfo.getEncodedName()); - - synchronized (this.regions) { - // Add check - ServerName oldSn = this.regions.get(regionInfo); - if (oldSn != null) LOG.warn("Overwriting " + regionInfo.getEncodedName() + - " on " + oldSn + " with " + sn); - - if (isServerOnline(sn)) { - this.regions.put(regionInfo, sn); - addToServers(sn, regionInfo); - this.regions.notifyAll(); - } else { - LOG.info("The server is not in online servers, ServerName=" + - sn.getServerName() + ", region=" + regionInfo.getEncodedName()); - } + if (!isServerOnline(sn)) { + LOG.warn("A region was opened on a dead server, ServerName=" + + sn.getServerName() + ", region=" + regionInfo.getEncodedName()); } + + regionStates.regionOnline(regionInfo, sn); + // Remove plan if one. clearRegionPlan(regionInfo); // Add the server to serversInUpdatingTimer @@ -1307,11 +1156,9 @@ public class AssignmentManager extends ZooKeeperListener { for (Map.Entry e : rps) { if (e.getValue() != null && e.getKey() != null && sn.equals(e.getValue().getDestination())) { - RegionState rs = this.regionsInTransition.get(e.getKey()); - if (rs != null) { - // no lock concurrency ok: there is a write when we update the timestamp but it's ok - // as it's an AtomicLong - rs.updateTimestampToNow(); + RegionState regionState = regionStates.getRegionTransitionState(e.getKey()); + if (regionState != null) { + regionState.updateTimestampToNow(); } } } @@ -1325,30 +1172,10 @@ public class AssignmentManager extends ZooKeeperListener { * @param regionInfo */ public void regionOffline(final HRegionInfo regionInfo) { - // no lock concurrency ok - this.regionsInTransition.remove(regionInfo.getEncodedName()); + regionStates.regionOffline(regionInfo); // remove the region plan as well just in case. clearRegionPlan(regionInfo); - setOffline(regionInfo); - } - - /** - * Sets the region as offline by removing in-memory assignment information but - * retaining transition information. - *

- * Used when a region has been closed but should be reassigned. - * @param regionInfo - */ - public void setOffline(HRegionInfo regionInfo) { - synchronized (this.regions) { - ServerName sn = this.regions.remove(regionInfo); - if (sn == null) return; - Set serverRegions = this.servers.get(sn); - if (!serverRegions.remove(regionInfo)) { - LOG.warn("No " + regionInfo + " on " + sn); - } - } } public void offlineDisabledRegion(HRegionInfo regionInfo) { @@ -1417,10 +1244,18 @@ public class AssignmentManager extends ZooKeeperListener { region.getRegionNameAsString()); return; } - RegionState state = addToRegionsInTransition(region, + RegionState state = forceRegionStateToOffline(region, hijack); - synchronized (state) { + // TODO: we can't synchronized on state any more since it could + // be an new instance. We need to reconsider how to avoid + // double/multiple assignments. + // This is to prevent double assignments? Does it work? + String encodedName = region.getEncodedName(); + Lock lock = assignLocker.acquireLock(encodedName); + try { assign(region, state, setOfflineInZK, forceNewPlan, hijack); + } finally { + lock.unlock(); } } @@ -1455,7 +1290,7 @@ public class AssignmentManager extends ZooKeeperListener { // time; i.e. handlers on backend won't be trying to set it to OPEN, etc. AtomicInteger counter = new AtomicInteger(0); CreateUnassignedAsyncCallback cb = - new CreateUnassignedAsyncCallback(this.watcher, destination, counter); + new CreateUnassignedAsyncCallback(regionStates, this.watcher, destination, counter); for (RegionState state: states) { if (!asyncSetOfflineInZooKeeper(state, cb, state)) { return false; @@ -1579,12 +1414,15 @@ public class AssignmentManager extends ZooKeeperListener { */ static class CreateUnassignedAsyncCallback implements AsyncCallback.StringCallback { private final Log LOG = LogFactory.getLog(CreateUnassignedAsyncCallback.class); + private final RegionStates regionStates; private final ZooKeeperWatcher zkw; private final ServerName destination; private final AtomicInteger counter; - CreateUnassignedAsyncCallback(final ZooKeeperWatcher zkw, - final ServerName destination, final AtomicInteger counter) { + CreateUnassignedAsyncCallback(final RegionStates regionStates, + final ZooKeeperWatcher zkw, final ServerName destination, + final AtomicInteger counter) { + this.regionStates = regionStates; this.zkw = zkw; this.destination = destination; this.counter = counter; @@ -1604,7 +1442,7 @@ public class AssignmentManager extends ZooKeeperListener { // Async exists to set a watcher so we'll get triggered when // unassigned node changes. this.zkw.getRecoverableZooKeeper().getZooKeeper().exists(path, this.zkw, - new ExistsUnassignedAsyncCallback(this.counter, destination), ctx); + new ExistsUnassignedAsyncCallback(regionStates, counter, destination), ctx); } } @@ -1614,10 +1452,13 @@ public class AssignmentManager extends ZooKeeperListener { */ static class ExistsUnassignedAsyncCallback implements AsyncCallback.StatCallback { private final Log LOG = LogFactory.getLog(ExistsUnassignedAsyncCallback.class); + private final RegionStates regionStates; private final AtomicInteger counter; private ServerName destination; - ExistsUnassignedAsyncCallback(final AtomicInteger counter, ServerName destination) { + ExistsUnassignedAsyncCallback(final RegionStates regionStates, + final AtomicInteger counter, ServerName destination) { + this.regionStates = regionStates; this.counter = counter; this.destination = destination; } @@ -1625,7 +1466,7 @@ public class AssignmentManager extends ZooKeeperListener { @Override public void processResult(int rc, String path, Object ctx, Stat stat) { if (rc != 0) { - // Thisis resultcode. If non-zero, need to resubmit. + // This is result code. If non-zero, need to resubmit. LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " + "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2"); return; @@ -1637,27 +1478,13 @@ public class AssignmentManager extends ZooKeeperListener { // yet sent out the actual open but putting this state change after the // call to open risks our writing PENDING_OPEN after state has been moved // to OPENING by the regionserver. - state.update(RegionState.State.PENDING_OPEN, System.currentTimeMillis(), destination); + regionStates.updateRegionState(state.getRegion(), + RegionState.State.PENDING_OPEN, System.currentTimeMillis(), + destination); this.counter.addAndGet(1); } } - /** - * @param region - * @return The current RegionState - */ - private RegionState addToRegionsInTransition(final HRegionInfo region) { - return addToRegionsInTransition(region, false); - } - /** - * @param region - * @param hijack - * @return The current RegionState - */ - private RegionState addToRegionsInTransition(final HRegionInfo region, - boolean hijack) { - return forceRegionStateToOffline(region, hijack); - } /** * Sets regions {@link RegionState} to {@link RegionState.State#OFFLINE}. * @param region @@ -1679,10 +1506,10 @@ public class AssignmentManager extends ZooKeeperListener { Lock lock = locker.acquireLock(encodedName); try { - RegionState state = this.regionsInTransition.get(encodedName); + RegionState state = regionStates.getRegionTransitionState(encodedName); if (state == null) { - state = new RegionState(region, RegionState.State.OFFLINE); - this.regionsInTransition.put(encodedName, state); + state = regionStates.updateRegionState( + region, RegionState.State.OFFLINE); } else { // If we are reassigning the node do not force in-memory state to OFFLINE. // Based on the znode state we will decide if to change in-memory state to @@ -1692,7 +1519,8 @@ public class AssignmentManager extends ZooKeeperListener { // assign on its tail as part of the handling of a region close. if (!hijack) { LOG.debug("Forcing OFFLINE; was=" + state); - state.update(RegionState.State.OFFLINE); + state = regionStates.updateRegionState( + region, RegionState.State.OFFLINE); } } return state; @@ -1751,8 +1579,9 @@ public class AssignmentManager extends ZooKeeperListener { LOG.info("Assigning region " + state.getRegion().getRegionNameAsString() + " to " + plan.getDestination().toString()); // Transition RegionState to PENDING_OPEN - state.update(RegionState.State.PENDING_OPEN, System.currentTimeMillis(), - plan.getDestination()); + regionStates.updateRegionState(state.getRegion(), + RegionState.State.PENDING_OPEN, System.currentTimeMillis(), + plan.getDestination()); // Send OPEN RPC. This can fail if the server on other end is is not up. // Pass the version that was obtained while setting the node to OFFLINE. RegionOpeningState regionOpenState = serverManager.sendRegionOpen(plan @@ -1781,7 +1610,8 @@ public class AssignmentManager extends ZooKeeperListener { // Clean out plan we failed execute and one that doesn't look like it'll // succeed anyways; we need a new plan! // Transition back to OFFLINE - state.update(RegionState.State.OFFLINE); + regionStates.updateRegionState( + state.getRegion(), RegionState.State.OFFLINE); // Force a new plan and reassign. Will return null if no servers. if (getRegionPlan(state, plan.getDestination(), true) == null) { this.timeoutMonitor.setAllRegionServersOffline(true); @@ -1794,7 +1624,6 @@ public class AssignmentManager extends ZooKeeperListener { } private void processAlreadyOpenedRegion(HRegionInfo region, ServerName sn) { - // Remove region from in-memory transition and unassigned node from ZK // While trying to enable the table the regions of the table were // already enabled. @@ -1813,14 +1642,8 @@ public class AssignmentManager extends ZooKeeperListener { "Error deleting OFFLINED node in ZK for transition ZK node (" + encodedRegionName + ")", e); } - // no lock concurrent ok -> sequentially consistent - this.regionsInTransition.remove(region.getEncodedName()); - - synchronized (this.regions) { - this.regions.put(region, sn); - addToServers(sn, region); - } + regionStates.regionOnline(region, sn); } private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) { @@ -1864,10 +1687,12 @@ public class AssignmentManager extends ZooKeeperListener { if (hijack && (state.getState().equals(RegionState.State.PENDING_OPEN) || state.getState().equals(RegionState.State.OPENING))) { - state.update(RegionState.State.PENDING_OPEN); + regionStates.updateRegionState(state.getRegion(), + RegionState.State.PENDING_OPEN); allowZNodeCreation = false; } else { - state.update(RegionState.State.OFFLINE); + regionStates.updateRegionState(state.getRegion(), + RegionState.State.OFFLINE); allowZNodeCreation = true; } int versionOfOfflineNode = -1; @@ -1901,7 +1726,8 @@ public class AssignmentManager extends ZooKeeperListener { new IllegalStateException()); return false; } - state.update(RegionState.State.OFFLINE); + regionStates.updateRegionState( + state.getRegion(), RegionState.State.OFFLINE); try { ZKAssign.asyncCreateNodeOffline(master.getZooKeeper(), state.getRegion(), this.master.getServerName(), cb, ctx); @@ -2009,10 +1835,10 @@ public class AssignmentManager extends ZooKeeperListener { int waitTime = this.master.getConfiguration().getInt( "hbase.bulk.waitbetween.reopen", 0); for (HRegionInfo region : regions) { - if (isRegionInTransition(region) != null) + if (regionStates.isRegionInTransition(region)) continue; unassign(region, false); - while (isRegionInTransition(region) != null) { + while (regionStates.isRegionInTransition(region)) { try { Thread.sleep(10); } catch (InterruptedException e) { @@ -2065,24 +1891,21 @@ public class AssignmentManager extends ZooKeeperListener { LOG.debug("Starting unassignment of region " + region.getRegionNameAsString() + " (offlining)"); - synchronized (this.regions) { - // Check if this region is currently assigned - if (!regions.containsKey(region)) { - LOG.debug("Attempted to unassign region " + - region.getRegionNameAsString() + " but it is not " + - "currently assigned anywhere"); - return; - } + // Check if this region is currently assigned + if (!regionStates.isRegionAssigned(region)) { + LOG.debug("Attempted to unassign region " + + region.getRegionNameAsString() + " but it is not " + + "currently assigned anywhere"); + return; } String encodedName = region.getEncodedName(); // Grab the state of this region and synchronize on it - RegionState state; int versionOfClosingNode = -1; // We need a lock here as we're going to do a put later and we don't want multiple states // creation ReentrantLock lock = locker.acquireLock(encodedName); + RegionState state = regionStates.getRegionTransitionState(encodedName); try { - state = regionsInTransition.get(encodedName); if (state == null) { // Create the znode in CLOSING state try { @@ -2124,13 +1947,12 @@ public class AssignmentManager extends ZooKeeperListener { master.abort("Unexpected ZK exception creating node CLOSING", e); return; } - state = new RegionState(region, RegionState.State.PENDING_CLOSE); - regionsInTransition.put(encodedName, state); + state = regionStates.updateRegionState(region, RegionState.State.PENDING_CLOSE); } else if (force && (state.isPendingClose() || state.isClosing())) { LOG.debug("Attempting to unassign region " + region.getRegionNameAsString() + " which is already " + state.getState() + " but forcing to send a CLOSE RPC again "); - state.update(state.getState()); + state.updateTimestampToNow(); } else { LOG.debug("Attempting to unassign region " + region.getRegionNameAsString() + " but it is " + @@ -2142,27 +1964,9 @@ public class AssignmentManager extends ZooKeeperListener { } // Send CLOSE RPC - ServerName server = null; - synchronized (this.regions) { - server = regions.get(region); - } + ServerName server = state.getServerName(); // ClosedRegionhandler can remove the server from this.regions if (server == null) { - // Possibility of disable flow removing from RIT. - lock = locker.acquireLock(encodedName); - try { - state = regionsInTransition.get(encodedName); - if (state != null) { - // remove only if the state is PENDING_CLOSE or CLOSING - RegionState.State presentState = state.getState(); - if (presentState == RegionState.State.PENDING_CLOSE - || presentState == RegionState.State.CLOSING) { - this.regionsInTransition.remove(encodedName); - } - } - } finally { - lock.unlock(); - } // delete the node. if no node exists need not bother. deleteClosingOrClosedNode(region); return; @@ -2194,22 +1998,13 @@ public class AssignmentManager extends ZooKeeperListener { + region.getTableNameAsString() + " to DISABLED state the region " + region + " was offlined but the table was in DISABLING state"); - this.regionsInTransition.remove(region.getEncodedName()); - - // Remove from the regionsMap - synchronized (this.regions) { - this.regions.remove(region); - Set serverRegions = this.servers.get(server); - if (!serverRegions.remove(region)) { - LOG.warn("No " + region + " on " + server); - } - } + regionStates.regionOffline(region); deleteClosingOrClosedNode(region); } } else if (t instanceof RegionAlreadyInTransitionException) { // RS is already processing this region, only need to update the timestamp LOG.debug("update " + state + " the timestamp."); - state.update(state.getState()); + state.updateTimestampToNow(); } LOG.info("Server " + server + " returned " + t + " for " + region.getRegionNameAsString(), t); @@ -2284,13 +2079,12 @@ public class AssignmentManager extends ZooKeeperListener { */ public void waitForAssignment(HRegionInfo regionInfo) throws InterruptedException { - synchronized(regions) { - while(!this.master.isStopped() && !regions.containsKey(regionInfo)) { - // We should receive a notification, but it's - // better to have a timeout to recheck the condition here: - // it lowers the impact of a race condition if any - regions.wait(100); - } + while(!this.master.isStopped() && + !regionStates.isRegionAssigned(regionInfo)) { + // We should receive a notification, but it's + // better to have a timeout to recheck the condition here: + // it lowers the impact of a race condition if any + regionStates.waitForUpdate(100); } } @@ -2556,12 +2350,12 @@ public class AssignmentManager extends ZooKeeperListener { // state of the Master. final long endTime = System.currentTimeMillis() + timeout; - while (!this.master.isStopped() && !regionsInTransition.isEmpty() && - endTime > System.currentTimeMillis()) { - regionsInTransition.waitForUpdate(100); + while (!this.master.isStopped() && regionStates.isRegionsInTransition() + && endTime > System.currentTimeMillis()) { + regionStates.waitForUpdate(100); } - return regionsInTransition.isEmpty(); + return !regionStates.isRegionsInTransition(); } /** @@ -2580,13 +2374,13 @@ public class AssignmentManager extends ZooKeeperListener { Iterator regionInfoIterator = regions.iterator(); while (regionInfoIterator.hasNext()) { HRegionInfo hri = regionInfoIterator.next(); - if (!regionsInTransition.containsKey(hri.getEncodedName())) { + if (!regionStates.isRegionInTransition(hri)) { regionInfoIterator.remove(); } } if (!regions.isEmpty()) { - regionsInTransition.waitForUpdate(100); + regionStates.waitForUpdate(100); } } @@ -2620,6 +2414,7 @@ public class AssignmentManager extends ZooKeeperListener { HRegionInfo regionInfo = region.getFirst(); ServerName regionLocation = region.getSecond(); if (regionInfo == null) continue; + regionStates.createRegionState(regionInfo); String tableName = regionInfo.getTableNameAsString(); if (regionLocation == null) { // regionLocation could be null if createTable didn't finish properly. @@ -2664,19 +2459,16 @@ public class AssignmentManager extends ZooKeeperListener { byte[] data = ZKUtil.getDataNoWatch(this.watcher, node, stat); // If znode does not exist dont consider this region if (data == null) { - LOG.debug("Region " + regionInfo.getRegionNameAsString() - + " split is completed. Hence need not add to regions list"); + LOG.debug("Region " + regionInfo.getRegionNameAsString() + + " split is completed. Hence need not add to regions list"); continue; } } // Region is being served and on an active server // add only if region not in disabled and enabling table - if (false == checkIfRegionBelongsToDisabled(regionInfo) - && false == checkIfRegionsBelongsToEnabling(regionInfo)) { - synchronized (this.regions) { - regions.put(regionInfo, regionLocation); - addToServers(regionLocation, regionInfo); - } + if (!checkIfRegionBelongsToDisabled(regionInfo) + && !checkIfRegionsBelongsToEnabling(regionInfo)) { + regionStates.regionOnline(regionInfo, regionLocation); } disablingOrEnabling = addTheTablesInPartialState(this.disablingTables, this.enablingTables, regionInfo, tableName); @@ -2876,32 +2668,6 @@ public class AssignmentManager extends ZooKeeperListener { } } - /* - * Presumes caller has taken care of necessary locking modifying servers Map. - * @param hsi - * @param hri - */ - private void addToServers(final ServerName sn, final HRegionInfo hri) { - Set hris = servers.get(sn); - if (hris == null) { - hris = new ConcurrentSkipListSet(); - servers.put(sn, hris); - } - if (!hris.contains(hri)) hris.add(hri); - } - - /** - * @return A copy of the Map of regions currently in transition. - */ - public NavigableMap copyRegionsInTransition() { - // no lock concurrent access ok - return regionsInTransition.copyMap(); - } - - NotifiableConcurrentSkipListMap getRegionsInTransition() { - return regionsInTransition; - } - /** * Set Regions in transitions metrics. * This takes an iterator on the RegionInTransition map (CLSM), and is not synchronized. @@ -2916,9 +2682,9 @@ public class AssignmentManager extends ZooKeeperListener { long oldestRITTime = 0; int ritThreshold = this.master.getConfiguration(). getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000); - for (Map.Entry e : this.regionsInTransition.copyEntrySet()) { + for (RegionState state: regionStates.getRegionsInTransition().values()) { totalRITs++; - long ritTime = currentTime - e.getValue().getStamp(); + long ritTime = currentTime - state.getStamp(); if (ritTime > ritThreshold) { // more than the threshold totalRITsOverThreshold++; } @@ -2933,26 +2699,6 @@ public class AssignmentManager extends ZooKeeperListener { } } - /** - * @return True if regions in transition. - */ - public boolean isRegionsInTransition() { - // no lock concurrent access ok: we could imagine that someone is currently going to remove - // it or add a region, but it's sequentially consistent. - return !(this.regionsInTransition.isEmpty()); - } - - /** - * @param hri Region to check. - * @return Returns null if passed region is not in transition else the current - * RegionState - */ - public RegionState isRegionInTransition(final HRegionInfo hri) { - // no lock concurrent access ok: we could imagine that someone is currently going to remove - // it or add it, but it's sequentially consistent. - return this.regionsInTransition.get(hri.getEncodedName()); - } - /** * @param region Region whose plan we are to clear. */ @@ -2968,14 +2714,14 @@ public class AssignmentManager extends ZooKeeperListener { * @throws IOException */ public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri) - throws IOException { - if (isRegionInTransition(hri) == null) return; + throws IOException, InterruptedException { + if (!regionStates.isRegionInTransition(hri)) return; RegionState rs = null; // There is already a timeout monitor on regions in transition so I // should not have to have one here too? - while(!this.master.isStopped() && (rs = isRegionInTransition(hri)) != null) { - Threads.sleep(1000); + while(!this.master.isStopped() && regionStates.isRegionInTransition(hri)) { LOG.info("Waiting on " + rs + " to clear regions-in-transition"); + regionStates.waitForUpdate(1000); } if (this.master.isStopped()) { LOG.info("Giving up wait on regions in " + @@ -2983,35 +2729,6 @@ public class AssignmentManager extends ZooKeeperListener { } } - - /** - * Gets the online regions of the specified table. - * This method looks at the in-memory state. It does not go to .META.. - * Only returns online regions. If a region on this table has been - * closed during a disable, etc., it will be included in the returned list. - * So, the returned list may not necessarily be ALL regions in this table, its - * all the ONLINE regions in the table. - * @param tableName - * @return Online regions from tableName - */ - public List getRegionsOfTable(byte[] tableName) { - List tableRegions = new ArrayList(); - // boundary needs to have table's name but regionID 0 so that it is sorted - // before all table's regions. - HRegionInfo boundary = - new HRegionInfo(tableName, null, null, false, 0L); - synchronized (this.regions) { - for (HRegionInfo regionInfo: this.regions.tailMap(boundary).keySet()) { - if(Bytes.equals(regionInfo.getTableName(), tableName)) { - tableRegions.add(regionInfo); - } else { - break; - } - } - } - return tableRegions; - } - /** * Update timers for all regions in transition going against the server in the * serversInUpdatingTimer. @@ -3093,7 +2810,7 @@ public class AssignmentManager extends ZooKeeperListener { long now = System.currentTimeMillis(); // no lock concurrent access ok: we will be working on a copy, and it's java-valid to do // a copy while another thread is adding/removing items - for (RegionState regionState : regionsInTransition.copyValues()) { + for (RegionState regionState : regionStates.getRegionsInTransition().values()) { if (regionState.getStamp() + timeout <= now || (this.allRegionServersOffline && !noRSAvailable)) { //decide on action upon timeout or, if some RSs just came back online, we can start the @@ -3132,8 +2849,15 @@ public class AssignmentManager extends ZooKeeperListener { case OPEN: LOG.error("Region has been OPEN for too long, " + "we don't know where region was opened so can't do anything"); - synchronized (regionState) { + // TODO: do we need synchronization here? + // could not synchronized on regionState since it can be + // an new instance + String encodedName = regionState.getRegion().getEncodedName(); + Lock lock = assignLocker.acquireLock(encodedName); + try { regionState.updateTimestampToNow(); + } finally { + lock.unlock(); } break; @@ -3242,7 +2966,7 @@ public class AssignmentManager extends ZooKeeperListener { return matchZK; } - ServerName addressFromAM = getRegionServerOfRegion(hri); + ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri); boolean matchAM = (addressFromAM != null && addressFromAM.equals(serverName)); LOG.debug("based on AM, current region=" + hri.getRegionNameAsString() + @@ -3271,33 +2995,7 @@ public class AssignmentManager extends ZooKeeperListener { } } } - // TODO: Do we want to sync on RIT here? - // Remove this server from map of servers to regions, and remove all regions - // of this server from online map of regions. - Set deadRegions = null; - List rits = new ArrayList(); - synchronized (this.regions) { - Set assignedRegions = this.servers.remove(sn); - if (assignedRegions == null || assignedRegions.isEmpty()) { - // No regions on this server, we are done, return empty list of RITs - return rits; - } - deadRegions = new TreeSet(assignedRegions); - for (HRegionInfo region : deadRegions) { - this.regions.remove(region); - } - } - // See if any of the regions that were online on this server were in RIT - // If they are, normal timeouts will deal with them appropriately so - // let's skip a manual re-assignment. - // no lock concurrent access ok: we will be working on a copy, and it's java-valid to do - // a copy while another thread is adding/removing items - for (RegionState region : this.regionsInTransition.copyValues()) { - if (deadRegions.remove(region.getRegion())) { - rits.add(region); - } - } - return rits; + return regionStates.serverOffline(sn); } /** @@ -3324,92 +3022,6 @@ public class AssignmentManager extends ZooKeeperListener { } } - /** - * This is an EXPENSIVE clone. Cloning though is the safest thing to do. - * Can't let out original since it can change and at least the loadbalancer - * wants to iterate this exported list. We need to synchronize on regions - * since all access to this.servers is under a lock on this.regions. - * - * @return A clone of current assignments by table. - */ - Map>> getAssignmentsByTable() { - Map>> result = null; - synchronized (this.regions) { - result = new HashMap>>(); - if (!this.master.getConfiguration(). - getBoolean("hbase.master.loadbalance.bytable", true)) { - result.put("ensemble", getAssignments()); - } else { - for (Map.Entry> e: this.servers.entrySet()) { - for (HRegionInfo hri : e.getValue()) { - if (hri.isMetaRegion() || hri.isRootRegion()) continue; - String tablename = hri.getTableNameAsString(); - Map> svrToRegions = result.get(tablename); - if (svrToRegions == null) { - svrToRegions = new HashMap>(this.servers.size()); - result.put(tablename, svrToRegions); - } - List regions = null; - if (!svrToRegions.containsKey(e.getKey())) { - regions = new ArrayList(); - svrToRegions.put(e.getKey(), regions); - } else { - regions = svrToRegions.get(e.getKey()); - } - regions.add(hri); - } - } - } - } - Map onlineSvrs = this.serverManager.getOnlineServers(); - // Take care of servers w/o assignments. - for (Map> map : result.values()) { - for (Map.Entry svrEntry: onlineSvrs.entrySet()) { - if (!map.containsKey(svrEntry.getKey())) { - map.put(svrEntry.getKey(), new ArrayList()); - } - } - } - return result; - } - - /** - * @return A clone of current assignments. Note, this is assignments only. - * If a new server has come in and it has no regions, it will not be included - * in the returned Map. - */ - Map> getAssignments() { - // This is an EXPENSIVE clone. Cloning though is the safest thing to do. - // Can't let out original since it can change and at least the loadbalancer - // wants to iterate this exported list. We need to synchronize on regions - // since all access to this.servers is under a lock on this.regions. - Map> result = null; - synchronized (this.regions) { - result = new HashMap>(this.servers.size()); - for (Map.Entry> e: this.servers.entrySet()) { - result.put(e.getKey(), new ArrayList(e.getValue())); - } - } - return result; - } - - /** - * @param encodedRegionName Region encoded name. - * @return Null or a {@link Pair} instance that holds the full {@link HRegionInfo} - * and the hosting servers {@link ServerName}. - */ - Pair getAssignment(final byte [] encodedRegionName) { - String name = Bytes.toString(encodedRegionName); - synchronized(this.regions) { - for (Map.Entry e: this.regions.entrySet()) { - if (e.getKey().getEncodedName().equals(name)) { - return new Pair(e.getKey(), e.getValue()); - } - } - } - return null; - } - /** * @param plan Plan to execute. */ @@ -3420,265 +3032,6 @@ public class AssignmentManager extends ZooKeeperListener { unassign(plan.getRegionInfo(), false, plan.getDestination()); } - /** - * Run through remaining regionservers and unassign all catalog regions. - */ - void unassignCatalogRegions() { - synchronized (this.regions) { - for (Map.Entry> e: this.servers.entrySet()) { - Set regions = e.getValue(); - if (regions == null || regions.isEmpty()) continue; - for (HRegionInfo hri: regions) { - if (hri.isMetaRegion()) { - unassign(hri); - } - } - } - } - } - - /** - * State of a Region while undergoing transitions. - */ - public static class RegionState implements org.apache.hadoop.io.Writable { - private HRegionInfo region; - - public enum State { - OFFLINE, // region is in an offline state - PENDING_OPEN, // sent rpc to server to open but has not begun - OPENING, // server has begun to open but not yet done - OPEN, // server opened region and updated meta - PENDING_CLOSE, // sent rpc to server to close but has not begun - CLOSING, // server has begun to close but not yet done - CLOSED, // server closed region and updated meta - SPLITTING, // server started split of a region - SPLIT // server completed split of a region - } - - private State state; - // Many threads can update the state at the stamp at the same time - private final AtomicLong stamp; - private ServerName serverName; - - public RegionState() { - this.stamp = new AtomicLong(System.currentTimeMillis()); - } - - RegionState(HRegionInfo region, State state) { - this(region, state, System.currentTimeMillis(), null); - } - - RegionState(HRegionInfo region, State state, long stamp, ServerName serverName) { - this.region = region; - this.state = state; - this.stamp = new AtomicLong(stamp); - this.serverName = serverName; - } - - public void update(State state, long stamp, ServerName serverName) { - this.state = state; - updateTimestamp(stamp); - this.serverName = serverName; - } - - public void update(State state) { - this.state = state; - updateTimestampToNow(); - this.serverName = null; - } - - public void updateTimestamp(long stamp) { - this.stamp.set(stamp); - } - - public void updateTimestampToNow() { - this.stamp.set(System.currentTimeMillis()); - } - - public State getState() { - return state; - } - - public long getStamp() { - return stamp.get(); - } - - public HRegionInfo getRegion() { - return region; - } - - public ServerName getServerName() { - return serverName; - } - - public boolean isClosing() { - return state == State.CLOSING; - } - - public boolean isClosed() { - return state == State.CLOSED; - } - - public boolean isPendingClose() { - return state == State.PENDING_CLOSE; - } - - public boolean isOpening() { - return state == State.OPENING; - } - - public boolean isOpened() { - return state == State.OPEN; - } - - public boolean isPendingOpen() { - return state == State.PENDING_OPEN; - } - - public boolean isOffline() { - return state == State.OFFLINE; - } - - public boolean isSplitting() { - return state == State.SPLITTING; - } - - public boolean isSplit() { - return state == State.SPLIT; - } - - @Override - public String toString() { - return region.getRegionNameAsString() - + " state=" + state - + ", ts=" + stamp - + ", server=" + serverName; - } - - /** - * A slower (but more easy-to-read) stringification - */ - public String toDescriptiveString() { - long lstamp = stamp.get(); - long relTime = System.currentTimeMillis() - lstamp; - - return region.getRegionNameAsString() - + " state=" + state - + ", ts=" + new Date(lstamp) + " (" + (relTime/1000) + "s ago)" - + ", server=" + serverName; - } - - /** - * Convert a RegionState to an HBaseProtos.RegionState - * - * @return the converted HBaseProtos.RegionState - */ - public ClusterStatusProtos.RegionState convert() { - ClusterStatusProtos.RegionState.Builder regionState = ClusterStatusProtos.RegionState.newBuilder(); - ClusterStatusProtos.RegionState.State rs; - switch (regionState.getState()) { - case OFFLINE: - rs = ClusterStatusProtos.RegionState.State.OFFLINE; - break; - case PENDING_OPEN: - rs = ClusterStatusProtos.RegionState.State.PENDING_OPEN; - break; - case OPENING: - rs = ClusterStatusProtos.RegionState.State.OPENING; - break; - case OPEN: - rs = ClusterStatusProtos.RegionState.State.OPEN; - break; - case PENDING_CLOSE: - rs = ClusterStatusProtos.RegionState.State.PENDING_CLOSE; - break; - case CLOSING: - rs = ClusterStatusProtos.RegionState.State.CLOSING; - break; - case CLOSED: - rs = ClusterStatusProtos.RegionState.State.CLOSED; - break; - case SPLITTING: - rs = ClusterStatusProtos.RegionState.State.SPLITTING; - break; - case SPLIT: - rs = ClusterStatusProtos.RegionState.State.SPLIT; - break; - default: - throw new IllegalStateException(""); - } - regionState.setRegionInfo(HRegionInfo.convert(region)); - regionState.setState(rs); - regionState.setStamp(getStamp()); - return regionState.build(); - } - - /** - * Convert a protobuf HBaseProtos.RegionState to a RegionState - * - * @return the RegionState - */ - public static RegionState convert(ClusterStatusProtos.RegionState proto) { - RegionState.State state; - switch (proto.getState()) { - case OFFLINE: - state = State.OFFLINE; - break; - case PENDING_OPEN: - state = State.PENDING_OPEN; - break; - case OPENING: - state = State.OPENING; - break; - case OPEN: - state = State.OPEN; - break; - case PENDING_CLOSE: - state = State.PENDING_CLOSE; - break; - case CLOSING: - state = State.CLOSING; - break; - case CLOSED: - state = State.CLOSED; - break; - case SPLITTING: - state = State.SPLITTING; - break; - case SPLIT: - state = State.SPLIT; - break; - default: - throw new IllegalStateException(""); - } - - return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),state,proto.getStamp(),null); - } - - /** - * @deprecated Writables are going away - */ - @Deprecated - @Override - public void readFields(DataInput in) throws IOException { - region = new HRegionInfo(); - region.readFields(in); - state = State.valueOf(in.readUTF()); - stamp.set(in.readLong()); - } - - /** - * @deprecated Writables are going away - */ - @Deprecated - @Override - public void write(DataOutput out) throws IOException { - region.write(out); - out.writeUTF(state.name()); - out.writeLong(stamp.get()); - } - } - public void stop() { this.timeoutMonitor.interrupt(); this.timerUpdater.interrupt(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java index b9000d8b713..1af7a2d8649 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java @@ -62,8 +62,7 @@ public class BulkReOpen extends BulkAssigner { // add plans for the regions that need to be reopened Map plans = new HashMap(); for (HRegionInfo hri : hris) { - RegionPlan reOpenPlan = new RegionPlan(hri, null, - assignmentManager.getRegionServerOfRegion(hri)); + RegionPlan reOpenPlan = assignmentManager.getRegionReopenPlan(hri); plans.put(hri.getEncodedName(), reOpenPlan); } assignmentManager.addPlans(plans); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index dc38322edc6..5e0e2e98b15 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -55,10 +54,14 @@ 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.MasterAdminProtocol; +import org.apache.hadoop.hbase.MasterMonitorProtocol; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.PleaseHoldException; +import org.apache.hadoop.hbase.RegionServerStatusProtocol; import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableNotDisabledException; @@ -77,12 +80,6 @@ import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType; import org.apache.hadoop.hbase.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.HBaseServer; -import org.apache.hadoop.hbase.MasterMonitorProtocol; -import org.apache.hadoop.hbase.MasterAdminProtocol; -import org.apache.hadoop.hbase.RegionServerStatusProtocol; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.RequestConverter; -import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.ipc.ProtocolSignature; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory; @@ -102,40 +99,17 @@ import org.apache.hadoop.hbase.master.metrics.MasterMetrics; import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; -import org.apache.hadoop.hbase.replication.regionserver.Replication; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.CompressionTest; -import org.apache.hadoop.hbase.util.FSTableDescriptors; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.HFileArchiveUtil; -import org.apache.hadoop.hbase.util.HasThread; -import org.apache.hadoop.hbase.util.InfoServer; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.Sleeper; -import org.apache.hadoop.hbase.util.Strings; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.util.VersionInfo; -import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; -import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker; -import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.hadoop.metrics.util.MBeanUtil; -import org.apache.hadoop.net.DNS; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.Watcher; -import org.apache.hadoop.hbase.ServerLoad; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; -import com.google.protobuf.RpcController; - import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest; @@ -152,10 +126,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableR import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest; @@ -164,26 +134,53 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRe import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse; +import org.apache.hadoop.hbase.replication.regionserver.Replication; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CompressionTest; +import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.apache.hadoop.hbase.util.HFileArchiveUtil; +import org.apache.hadoop.hbase.util.HasThread; +import org.apache.hadoop.hbase.util.InfoServer; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Sleeper; +import org.apache.hadoop.hbase.util.Strings; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.util.VersionInfo; +import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; +import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker; +import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; +import org.apache.hadoop.hbase.zookeeper.ZKClusterId; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.hadoop.metrics.util.MBeanUtil; +import org.apache.hadoop.net.DNS; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.Watcher; + +import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; /** @@ -205,6 +202,7 @@ import com.google.protobuf.ServiceException; * @see Watcher */ @InterfaceAudience.Private +@SuppressWarnings("deprecation") public class HMaster extends HasThread implements MasterMonitorProtocol, MasterAdminProtocol, RegionServerStatusProtocol, MasterServices, Server { @@ -505,7 +503,7 @@ Server { this.balancer = LoadBalancerFactory.getLoadBalancer(conf); this.assignmentManager = new AssignmentManager(this, serverManager, - this.catalogTracker, this.balancer, this.executorService, this.metrics); + this.catalogTracker, this.balancer, this.executorService, this.metrics); zooKeeper.registerListenerFirst(assignmentManager); this.regionServerTracker = new RegionServerTracker(zooKeeper, this, @@ -649,7 +647,7 @@ Server { } } - if (!masterRecovery) { + if (!masterRecovery) { this.assignmentManager.startTimeOutMonitor(); } // TODO: Should do this in background rather than block master startup @@ -768,6 +766,8 @@ Server { // Work on ROOT region. Is it in zk in transition? status.setStatus("Assigning ROOT region"); + assignmentManager.getRegionStates().createRegionState( + HRegionInfo.ROOT_REGIONINFO); boolean rit = this.assignmentManager. processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.ROOT_REGIONINFO); ServerName currentRootServer = null; @@ -802,6 +802,8 @@ Server { // Work on meta region status.setStatus("Assigning META region"); + assignmentManager.getRegionStates().createRegionState( + HRegionInfo.FIRST_META_REGIONINFO); rit = this.assignmentManager. processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO); boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout); @@ -1228,12 +1230,12 @@ Server { boolean balancerRan; synchronized (this.balancer) { // Only allow one balance run at at time. - if (this.assignmentManager.isRegionsInTransition()) { - LOG.debug("Not running balancer because " + - this.assignmentManager.getRegionsInTransition().size() + - " region(s) in transition: " + - org.apache.commons.lang.StringUtils. - abbreviate(this.assignmentManager.getRegionsInTransition().toString(), 256)); + if (this.assignmentManager.getRegionStates().isRegionsInTransition()) { + Map regionsInTransition = + this.assignmentManager.getRegionStates().getRegionsInTransition(); + LOG.debug("Not running balancer because " + regionsInTransition.size() + + " region(s) in transition: " + org.apache.commons.lang.StringUtils. + abbreviate(regionsInTransition.toString(), 256)); return false; } if (this.serverManager.areDeadServersInProgress()) { @@ -1255,7 +1257,7 @@ Server { } Map>> assignmentsByTable = - this.assignmentManager.getAssignmentsByTable(); + this.assignmentManager.getRegionStates().getAssignmentsByTable(); List plans = new ArrayList(); //Give the balancer the current cluster state. @@ -1373,23 +1375,24 @@ Server { LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME + " actual: " + type); } - Pair p = - this.assignmentManager.getAssignment(encodedRegionName); - if (p == null) { - throw new ServiceException( + RegionState regionState = assignmentManager.getRegionStates(). + getRegionState(Bytes.toString(encodedRegionName)); + if (regionState == null) { + throw new ServiceException( new UnknownRegionException(Bytes.toStringBinary(encodedRegionName))); } - HRegionInfo hri = p.getFirst(); + + HRegionInfo hri = regionState.getRegion(); ServerName dest; if (destServerName == null || destServerName.length == 0) { LOG.info("Passed destination servername is null/empty so " + "choosing a server at random"); final List destServers = this.serverManager.createDestinationServersList( - p.getSecond()); + regionState.getServerName()); dest = balancer.randomAssignment(hri, destServers); } else { dest = new ServerName(Bytes.toString(destServerName)); - if (dest.equals(p.getSecond())) { + if (dest.equals(regionState.getServerName())) { LOG.debug("Skipping move of region " + hri.getRegionNameAsString() + " because region already assigned to the same server " + dest + "."); return mrr; @@ -1397,7 +1400,7 @@ Server { } // Now we can do the move - RegionPlan rp = new RegionPlan(p.getFirst(), p.getSecond(), dest); + RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest); try { if (this.cpHost != null) { @@ -1729,12 +1732,6 @@ Server { } } - public void clearFromTransition(HRegionInfo hri) { - if (this.assignmentManager.isRegionInTransition(hri) != null) { - this.assignmentManager.regionOffline(hri); - } - } - @Override public GetClusterStatusResponse getClusterStatus(RpcController controller, GetClusterStatusRequest req) throws ServiceException { @@ -1787,7 +1784,7 @@ Server { this.serverManager.getDeadServers(), this.serverName, backupMasters, - this.assignmentManager.copyRegionsInTransition(), + this.assignmentManager.getRegionStates().getRegionsInTransition(), this.getCoprocessors(), this.balanceSwitch); } @@ -1952,12 +1949,11 @@ Server { public AssignmentManager getAssignmentManager() { return this.assignmentManager; } - + public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() { return rsFatals; } - @SuppressWarnings("deprecation") public void shutdown() { if (cpHost != null) { try { @@ -2082,17 +2078,16 @@ Server { LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME + " actual: " + type); } - Pair pair = - MetaReader.getRegion(this.catalogTracker, regionName); - if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName)); + HRegionInfo regionInfo = assignmentManager.getRegionStates().getRegionInfo(regionName); + if (regionInfo == null) throw new UnknownRegionException(Bytes.toString(regionName)); if (cpHost != null) { - if (cpHost.preAssign(pair.getFirst())) { + if (cpHost.preAssign(regionInfo)) { return arr; } } - assignRegion(pair.getFirst()); + assignRegion(regionInfo); if (cpHost != null) { - cpHost.postAssign(pair.getFirst()); + cpHost.postAssign(regionInfo); } return arr; @@ -2191,7 +2186,7 @@ Server { * @return the average load */ public double getAverageLoad() { - return this.assignmentManager.getAverageLoad(); + return this.assignmentManager.getRegionStates().getAverageLoad(); } /** @@ -2255,7 +2250,6 @@ Server { /** * Register bean with platform management server */ - @SuppressWarnings("deprecation") void registerMBean() { MXBeanImpl mxBeanInfo = MXBeanImpl.init(this); MBeanUtil.registerMBean("Master", "Master", mxBeanInfo); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java index 70d4581228c..922ae7f1b13 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MXBeanImpl.java @@ -25,7 +25,6 @@ import java.util.Map.Entry; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; /** * Impl for exposing HMaster Information through JMX @@ -99,8 +98,8 @@ public class MXBeanImpl implements MXBean { public RegionsInTransitionInfo[] getRegionsInTransition() { List info = new ArrayList(); - for (final Entry entry : - master.getAssignmentManager().copyRegionsInTransition().entrySet()) { + for (final Entry entry : master.getAssignmentManager() + .getRegionStates().getRegionsInTransition().entrySet()) { RegionsInTransitionInfo innerinfo = new RegionsInTransitionInfo() { @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java index 83860eb17e7..c4512cd6c7a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java @@ -24,17 +24,14 @@ import java.io.OutputStream; import java.io.PrintWriter; import java.util.Date; import java.util.Map; -import java.util.NavigableMap; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerLoad; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.monitoring.LogMonitoring; import org.apache.hadoop.hbase.monitoring.StateDumpServlet; import org.apache.hadoop.hbase.monitoring.TaskMonitor; @@ -104,8 +101,8 @@ public class MasterDumpServlet extends StateDumpServlet { private void dumpRIT(HMaster master, PrintWriter out) { - NavigableMap regionsInTransition = - master.getAssignmentManager().copyRegionsInTransition(); + Map regionsInTransition = + master.getAssignmentManager().getRegionStates().getRegionsInTransition(); for (Map.Entry e : regionsInTransition.entrySet()) { String rid = e.getKey(); RegionState rs = e.getValue(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 989e6758d79..d31fb98121d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -25,9 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.ExecutorService; -import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; /** * Services Master supplies diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/NotifiableConcurrentSkipListMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/NotifiableConcurrentSkipListMap.java deleted file mode 100644 index 55fc3108b77..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/NotifiableConcurrentSkipListMap.java +++ /dev/null @@ -1,123 +0,0 @@ -/** - * 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.master; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; -import java.util.NavigableMap; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.ConcurrentSkipListMap; - -/** - *

Encapsulate a ConcurrentSkipListMap to ensure that notifications are sent when - * the list is modified. Offers only the functions used by the AssignementManager, hence - * does not extends ConcurrentSkipListMap.

- * - *

Used only in master package (main & test), so it's package protected.

- * - * @param - class for the keys - * @param - class for the values - */ -class NotifiableConcurrentSkipListMap { - private final ConcurrentSkipListMap delegatee = new ConcurrentSkipListMap(); - - public boolean isEmpty() { - return delegatee.isEmpty(); - } - - public int size() { - return delegatee.size(); - } - - public void put(K k, V v) { - synchronized (delegatee) { - delegatee.put(k, v); - delegatee.notifyAll(); - } - } - - public V remove(K k) { - synchronized (delegatee) { - V v = delegatee.remove(k); - if (v != null) { - delegatee.notifyAll(); - } - return v; - } - } - - public void waitForUpdate(long timeout) throws InterruptedException { - synchronized (delegatee){ - delegatee.wait(timeout); - } - } - - public boolean containsKey(K k) { - return delegatee.containsKey(k); - } - - public Collection keySet() { - return delegatee.keySet(); - } - - public V get(K k) { - return delegatee.get(k); - } - - public NavigableMap copyMap() { - return delegatee.clone(); - } - - public Collection copyValues() { - Collection values = new ArrayList(size()); - synchronized (delegatee) { - values.addAll(delegatee.values()); - } - return values; - } - - public Set> copyEntrySet() { - Set> entrySet = new TreeSet>(); - synchronized (delegatee) { - Iterator> it = delegatee.entrySet().iterator(); - while (it.hasNext()) { - entrySet.add(it.next()); - } - } - return entrySet; - } - - public void waitForUpdate() throws InterruptedException { - synchronized (delegatee) { - delegatee.wait(); - } - } - - public void clear() { - if (!delegatee.isEmpty()) { - synchronized (delegatee) { - delegatee.clear(); - delegatee.notifyAll(); - } - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionState.java new file mode 100644 index 00000000000..fd1a39dff53 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionState.java @@ -0,0 +1,259 @@ +/** + * 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.master; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Date; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos; + +/** + * State of a Region while undergoing transitions. + * Region state cannot be modified except the stamp field. + * So it is almost immutable. + */ +@InterfaceAudience.Private +public class RegionState implements org.apache.hadoop.io.Writable { + public enum State { + OFFLINE, // region is in an offline state + PENDING_OPEN, // sent rpc to server to open but has not begun + OPENING, // server has begun to open but not yet done + OPEN, // server opened region and updated meta + PENDING_CLOSE, // sent rpc to server to close but has not begun + CLOSING, // server has begun to close but not yet done + CLOSED, // server closed region and updated meta + SPLITTING, // server started split of a region + SPLIT // server completed split of a region + } + + // Many threads can update the state at the stamp at the same time + private final AtomicLong stamp; + private HRegionInfo region; + + private volatile ServerName serverName; + private volatile State state; + + public RegionState() { + this.stamp = new AtomicLong(System.currentTimeMillis()); + } + + public RegionState(HRegionInfo region, State state) { + this(region, state, System.currentTimeMillis(), null); + } + + public RegionState(HRegionInfo region, + State state, long stamp, ServerName serverName) { + this.region = region; + this.state = state; + this.stamp = new AtomicLong(stamp); + this.serverName = serverName; + } + + public void updateTimestampToNow() { + this.stamp.set(System.currentTimeMillis()); + } + + public State getState() { + return state; + } + + public long getStamp() { + return stamp.get(); + } + + public HRegionInfo getRegion() { + return region; + } + + public ServerName getServerName() { + return serverName; + } + + public boolean isClosing() { + return state == State.CLOSING; + } + + public boolean isClosed() { + return state == State.CLOSED; + } + + public boolean isPendingClose() { + return state == State.PENDING_CLOSE; + } + + public boolean isOpening() { + return state == State.OPENING; + } + + public boolean isOpened() { + return state == State.OPEN; + } + + public boolean isPendingOpen() { + return state == State.PENDING_OPEN; + } + + public boolean isOffline() { + return state == State.OFFLINE; + } + + public boolean isSplitting() { + return state == State.SPLITTING; + } + + public boolean isSplit() { + return state == State.SPLIT; + } + + @Override + public String toString() { + return "{" + region.getRegionNameAsString() + + " state=" + state + + ", ts=" + stamp + + ", server=" + serverName + "}"; + } + + /** + * A slower (but more easy-to-read) stringification + */ + public String toDescriptiveString() { + long lstamp = stamp.get(); + long relTime = System.currentTimeMillis() - lstamp; + + return region.getRegionNameAsString() + + " state=" + state + + ", ts=" + new Date(lstamp) + " (" + (relTime/1000) + "s ago)" + + ", server=" + serverName; + } + + /** + * Convert a RegionState to an HBaseProtos.RegionState + * + * @return the converted HBaseProtos.RegionState + */ + public ClusterStatusProtos.RegionState convert() { + ClusterStatusProtos.RegionState.Builder regionState = ClusterStatusProtos.RegionState.newBuilder(); + ClusterStatusProtos.RegionState.State rs; + switch (regionState.getState()) { + case OFFLINE: + rs = ClusterStatusProtos.RegionState.State.OFFLINE; + break; + case PENDING_OPEN: + rs = ClusterStatusProtos.RegionState.State.PENDING_OPEN; + break; + case OPENING: + rs = ClusterStatusProtos.RegionState.State.OPENING; + break; + case OPEN: + rs = ClusterStatusProtos.RegionState.State.OPEN; + break; + case PENDING_CLOSE: + rs = ClusterStatusProtos.RegionState.State.PENDING_CLOSE; + break; + case CLOSING: + rs = ClusterStatusProtos.RegionState.State.CLOSING; + break; + case CLOSED: + rs = ClusterStatusProtos.RegionState.State.CLOSED; + break; + case SPLITTING: + rs = ClusterStatusProtos.RegionState.State.SPLITTING; + break; + case SPLIT: + rs = ClusterStatusProtos.RegionState.State.SPLIT; + break; + default: + throw new IllegalStateException(""); + } + regionState.setRegionInfo(HRegionInfo.convert(region)); + regionState.setState(rs); + regionState.setStamp(getStamp()); + return regionState.build(); + } + + /** + * Convert a protobuf HBaseProtos.RegionState to a RegionState + * + * @return the RegionState + */ + public static RegionState convert(ClusterStatusProtos.RegionState proto) { + RegionState.State state; + switch (proto.getState()) { + case OFFLINE: + state = State.OFFLINE; + break; + case PENDING_OPEN: + state = State.PENDING_OPEN; + break; + case OPENING: + state = State.OPENING; + break; + case OPEN: + state = State.OPEN; + break; + case PENDING_CLOSE: + state = State.PENDING_CLOSE; + break; + case CLOSING: + state = State.CLOSING; + break; + case CLOSED: + state = State.CLOSED; + break; + case SPLITTING: + state = State.SPLITTING; + break; + case SPLIT: + state = State.SPLIT; + break; + default: + throw new IllegalStateException(""); + } + + return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),state,proto.getStamp(),null); + } + + /** + * @deprecated Writables are going away + */ + @Deprecated + @Override + public void readFields(DataInput in) throws IOException { + region = new HRegionInfo(); + region.readFields(in); + state = State.valueOf(in.readUTF()); + stamp.set(in.readLong()); + } + + /** + * @deprecated Writables are going away + */ + @Deprecated + @Override + public void write(DataOutput out) throws IOException { + region.write(out); + out.writeUTF(state.name()); + out.writeLong(stamp.get()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java new file mode 100644 index 00000000000..7eff4129cd2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java @@ -0,0 +1,529 @@ +/** + * 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.master; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.RegionTransition; +import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerLoad; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.master.RegionState.State; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; + +/** + * Region state accountant. It holds the states of all regions in the memory. + * In normal scenario, it should match the meta table and the true region states. + * + * This map is used by AssignmentManager to track region states. + */ +@InterfaceAudience.Private +public class RegionStates { + private static final Log LOG = LogFactory.getLog(RegionStates.class); + + /** + * Regions currently in transition. + */ + final HashMap regionsInTransition; + + /** + * Region encoded name to state map. + * All the regions should be in this map. + */ + private final Map regionStates; + + /** + * Server to regions assignment map. + * Contains the set of regions currently assigned to a given server. + */ + private final Map> serverHoldings; + + /** + * Region to server assignment map. + * Contains the server a given region is currently assigned to. + */ + private final TreeMap regionAssignments; + + private final ServerManager serverManager; + private final Server server; + + RegionStates(final Server master, final ServerManager serverManager) { + regionStates = new HashMap(); + regionsInTransition = new HashMap(); + serverHoldings = new HashMap>(); + regionAssignments = new TreeMap(); + this.serverManager = serverManager; + this.server = master; + } + + /** + * @return an unmodifiable the region assignment map + */ + @SuppressWarnings("unchecked") + public synchronized Map getRegionAssignments() { + return (Map)regionAssignments.clone(); + } + + public synchronized ServerName getRegionServerOfRegion(HRegionInfo hri) { + return regionAssignments.get(hri); + } + + /** + * Get regions in transition and their states + */ + @SuppressWarnings("unchecked") + public synchronized Map getRegionsInTransition() { + return (Map)regionsInTransition.clone(); + } + + /** + * @return True if specified region in transition. + */ + public synchronized boolean isRegionInTransition(final HRegionInfo hri) { + return regionsInTransition.containsKey(hri.getEncodedName()); + } + + /** + * @return True if specified region in transition. + */ + public synchronized boolean isRegionInTransition(final String regionName) { + return regionsInTransition.containsKey(regionName); + } + + /** + * @return True if any region in transition. + */ + public synchronized boolean isRegionsInTransition() { + return !regionsInTransition.isEmpty(); + } + + /** + * @return True if specified region assigned. + */ + public synchronized boolean isRegionAssigned(final HRegionInfo hri) { + return regionAssignments.containsKey(hri); + } + + /** + * Wait for the state map to be updated by assignment manager. + */ + public synchronized void waitForUpdate( + final long timeout) throws InterruptedException { + this.wait(timeout); + } + + /** + * Get region transition state + */ + public synchronized RegionState + getRegionTransitionState(final HRegionInfo hri) { + return regionsInTransition.get(hri.getEncodedName()); + } + + /** + * Get region transition state + */ + public synchronized RegionState + getRegionTransitionState(final String regionName) { + return regionsInTransition.get(regionName); + } + + /** + * Add a list of regions to RegionStates. The initial state is OFFLINE. + * If any region is already in RegionStates, that region will be skipped. + */ + public synchronized void createRegionStates( + final List hris) { + for (HRegionInfo hri: hris) { + createRegionState(hri); + } + } + + /** + * Add a region to RegionStates. The initial state is OFFLINE. + * If it is already in RegionStates, this call has no effect, + * and the original state is returned. + */ + public synchronized RegionState createRegionState(final HRegionInfo hri) { + String regionName = hri.getEncodedName(); + RegionState regionState = regionStates.get(regionName); + if (regionState != null) { + LOG.warn("Tried to create a state of a region already in RegionStates " + + hri + ", used existing state: " + regionState + + ", ignored new state: state=OFFLINE, server=null"); + } else { + regionState = new RegionState(hri, State.OFFLINE); + regionStates.put(regionName, regionState); + } + return regionState; + } + + /** + * Update a region state. If it is not splitting, + * it will be put in transition if not already there. + */ + public synchronized RegionState updateRegionState( + final HRegionInfo hri, final State state) { + RegionState regionState = regionStates.get(hri.getEncodedName()); + ServerName serverName = (regionState == null || state == State.CLOSED + || state == State.OFFLINE) ? null : regionState.getServerName(); + return updateRegionState(hri, state, serverName); + } + + /** + * Update a region state. If it is not splitting, + * it will be put in transition if not already there. + */ + public synchronized RegionState updateRegionState( + final HRegionInfo hri, final State state, final ServerName serverName) { + return updateRegionState(hri, state, System.currentTimeMillis(), serverName); + } + + /** + * Update a region state. If it is not splitting, + * it will be put in transition if not already there. + * + * If we can't find the region info based on the region name in + * the transition, log a warning and return null. + */ + public synchronized RegionState updateRegionState( + final RegionTransition transition, final State state) { + byte[] regionName = transition.getRegionName(); + HRegionInfo regionInfo = getRegionInfo(regionName); + if (regionInfo == null) { + String prettyRegionName = HRegionInfo.prettyPrint( + HRegionInfo.encodeRegionName(regionName)); + LOG.warn("Failed to find region " + prettyRegionName + + " in updating its state to " + state + + " based on region transition " + transition); + return null; + } + return updateRegionState(regionInfo, state, + transition.getCreateTime(), transition.getServerName()); + } + + /** + * Update a region state. If it is not splitting, + * it will be put in transition if not already there. + */ + public synchronized RegionState updateRegionState(final HRegionInfo hri, + final State state, final long stamp, final ServerName serverName) { + ServerName newServerName = serverName; + if (serverName != null && + (state == State.CLOSED || state == State.OFFLINE)) { + LOG.warn("Closed region " + hri + " still on " + + serverName + "? Ignored, reset it to null"); + newServerName = null; + } + + String regionName = hri.getEncodedName(); + RegionState regionState = new RegionState(hri, state, stamp, newServerName); + RegionState oldState = regionStates.put(regionName, regionState); + LOG.info("Region " + hri + " transitioned from " + oldState + " to " + regionState); + if (state != State.SPLITTING && (newServerName != null + || (state != State.PENDING_CLOSE && state != State.CLOSING))) { + regionsInTransition.put(regionName, regionState); + } + + // notify the change + this.notifyAll(); + return regionState; + } + + /** + * A region is online, won't be in transition any more. + * We can't confirm it is really online on specified region server + * because it hasn't been put in region server's online region list yet. + */ + public synchronized void regionOnline( + final HRegionInfo hri, final ServerName serverName) { + String regionName = hri.getEncodedName(); + RegionState oldState = regionStates.get(regionName); + if (oldState == null) { + LOG.warn("Online a region not in RegionStates: " + hri); + } else { + State state = oldState.getState(); + ServerName sn = oldState.getServerName(); + if (state != State.OPEN || sn == null || !sn.equals(serverName)) { + LOG.debug("Online a region with current state=" + state + ", expected state=OPEN" + + ", assigned to server: " + sn + " expected " + serverName); + } + } + updateRegionState(hri, State.OPEN, serverName); + regionsInTransition.remove(regionName); + + ServerName oldServerName = regionAssignments.put(hri, serverName); + if (!serverName.equals(oldServerName)) { + LOG.info("Onlined region " + hri + " on " + serverName); + Set regions = serverHoldings.get(serverName); + if (regions == null) { + regions = new HashSet(); + serverHoldings.put(serverName, regions); + } + regions.add(hri); + if (oldServerName != null) { + LOG.info("Offlined region " + hri + " from " + oldServerName); + serverHoldings.get(oldServerName).remove(hri); + } + } + } + + /** + * A region is offline, won't be in transition any more. + */ + public synchronized void regionOffline(final HRegionInfo hri) { + String regionName = hri.getEncodedName(); + RegionState oldState = regionStates.get(regionName); + if (oldState == null) { + LOG.warn("Offline a region not in RegionStates: " + hri); + } else { + State state = oldState.getState(); + ServerName sn = oldState.getServerName(); + if (state != State.OFFLINE || sn != null) { + LOG.debug("Online a region with current state=" + state + ", expected state=OFFLINE" + + ", assigned to server: " + sn + ", expected null"); + } + } + updateRegionState(hri, State.OFFLINE); + regionsInTransition.remove(regionName); + + ServerName oldServerName = regionAssignments.remove(hri); + if (oldServerName != null) { + LOG.info("Offlined region " + hri + " from " + oldServerName); + serverHoldings.get(oldServerName).remove(hri); + } + } + + /** + * A server is offline, all regions on it are dead. + */ + public synchronized List serverOffline(final ServerName sn) { + // Clean up this server from map of servers to regions, and remove all regions + // of this server from online map of regions. + List rits = new ArrayList(); + Set assignedRegions = serverHoldings.get(sn); + if (assignedRegions == null || assignedRegions.isEmpty()) { + // No regions on this server, we are done, return empty list of RITs + return rits; + } + + for (HRegionInfo region : assignedRegions) { + regionAssignments.remove(region); + } + + // See if any of the regions that were online on this server were in RIT + // If they are, normal timeouts will deal with them appropriately so + // let's skip a manual re-assignment. + for (RegionState state : regionsInTransition.values()) { + if (assignedRegions.contains(state.getRegion())) { + rits.add(state); + } + } + assignedRegions.clear(); + this.notifyAll(); + return rits; + } + + /** + * Gets the online regions of the specified table. + * This method looks at the in-memory state. It does not go to .META.. + * Only returns online regions. If a region on this table has been + * closed during a disable, etc., it will be included in the returned list. + * So, the returned list may not necessarily be ALL regions in this table, its + * all the ONLINE regions in the table. + * @param tableName + * @return Online regions from tableName + */ + public synchronized List getRegionsOfTable(byte[] tableName) { + List tableRegions = new ArrayList(); + // boundary needs to have table's name but regionID 0 so that it is sorted + // before all table's regions. + HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L); + for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) { + if(Bytes.equals(hri.getTableName(), tableName)) { + tableRegions.add(hri); + } else { + break; + } + } + return tableRegions; + } + + + /** + * Wait on region to clear regions-in-transition. + *

+ * If the region isn't in transition, returns immediately. Otherwise, method + * blocks until the region is out of transition. + */ + public synchronized void waitOnRegionToClearRegionsInTransition( + final HRegionInfo hri) throws InterruptedException { + if (!isRegionInTransition(hri)) return; + + while(!server.isStopped() && isRegionInTransition(hri)) { + RegionState rs = getRegionState(hri); + LOG.info("Waiting on " + rs + " to clear regions-in-transition"); + waitForUpdate(100); + } + + if (server.isStopped()) { + LOG.info("Giving up wait on region in " + + "transition because stoppable.isStopped is set"); + } + } + + /** + * Waits until the specified region has completed assignment. + *

+ * If the region is already assigned, returns immediately. Otherwise, method + * blocks until the region is assigned. + */ + public synchronized void waitForAssignment( + final HRegionInfo hri) throws InterruptedException { + if (!isRegionAssigned(hri)) return; + + while(!server.isStopped() && !isRegionAssigned(hri)) { + RegionState rs = getRegionState(hri); + LOG.info("Waiting on " + rs + " to be assigned"); + waitForUpdate(100); + } + + if (server.isStopped()) { + LOG.info("Giving up wait on region " + + "assignment because stoppable.isStopped is set"); + } + } + + /** + * Compute the average load across all region servers. + * Currently, this uses a very naive computation - just uses the number of + * regions being served, ignoring stats about number of requests. + * @return the average load + */ + protected synchronized double getAverageLoad() { + int numServers = 0, totalLoad = 0; + for (Map.Entry> e: serverHoldings.entrySet()) { + Set regions = e.getValue(); + ServerName serverName = e.getKey(); + int regionCount = regions.size(); + if (regionCount > 0 || serverManager.isServerOnline(serverName)) { + totalLoad += regionCount; + numServers++; + } + } + return numServers == 0 ? 0.0 : + (double)totalLoad / (double)numServers; + } + + /** + * This is an EXPENSIVE clone. Cloning though is the safest thing to do. + * Can't let out original since it can change and at least the load balancer + * wants to iterate this exported list. We need to synchronize on regions + * since all access to this.servers is under a lock on this.regions. + * + * @return A clone of current assignments by table. + */ + protected Map>> getAssignmentsByTable() { + Map>> result = + new HashMap>>(); + synchronized (this) { + if (!server.getConfiguration().getBoolean("hbase.master.loadbalance.bytable", true)) { + Map> svrToRegions = + new HashMap>(serverHoldings.size()); + for (Map.Entry> e: serverHoldings.entrySet()) { + svrToRegions.put(e.getKey(), new ArrayList(e.getValue())); + } + result.put("ensemble", svrToRegions); + } else { + for (Map.Entry> e: serverHoldings.entrySet()) { + for (HRegionInfo hri: e.getValue()) { + if (hri.isMetaRegion() || hri.isRootRegion()) continue; + String tablename = hri.getTableNameAsString(); + Map> svrToRegions = result.get(tablename); + if (svrToRegions == null) { + svrToRegions = new HashMap>(serverHoldings.size()); + result.put(tablename, svrToRegions); + } + List regions = svrToRegions.get(e.getKey()); + if (regions == null) { + regions = new ArrayList(); + svrToRegions.put(e.getKey(), regions); + } + regions.add(hri); + } + } + } + } + + Map + onlineSvrs = serverManager.getOnlineServers(); + // Take care of servers w/o assignments. + for (Map> map: result.values()) { + for (ServerName svr: onlineSvrs.keySet()) { + if (!map.containsKey(svr)) { + map.put(svr, new ArrayList()); + } + } + } + return result; + } + + protected synchronized RegionState getRegionState(final HRegionInfo hri) { + return regionStates.get(hri.getEncodedName()); + } + + protected synchronized RegionState getRegionState(final String regionName) { + return regionStates.get(regionName); + } + + /** + * Get the HRegionInfo from cache, if not there, from the META table + * @param regionName + * @return HRegionInfo for the region + */ + protected HRegionInfo getRegionInfo(final byte [] regionName) { + String encodedName = HRegionInfo.encodeRegionName(regionName); + RegionState regionState = regionStates.get(encodedName); + if (regionState != null) { + return regionState.getRegion(); + } + + try { + Pair p = + MetaReader.getRegion(server.getCatalogTracker(), regionName); + return p == null ? null : p.getFirst(); + } catch (IOException e) { + server.abort("Aborting because error occoured while reading " + + Bytes.toStringBinary(regionName) + " from .META.", e); + return null; + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java index 992d3332d3b..91fc4681da1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.master.AssignmentManager; +import org.apache.hadoop.hbase.master.RegionState; /** * Handles CLOSED region event on Master. @@ -99,9 +100,8 @@ public class ClosedRegionHandler extends EventHandler implements TotesHRegionInf return; } // ZK Node is in CLOSED state, assign it. - // TODO: Should we remove the region from RIT too? We don't? Makes for - // a 'forcing' log message when we go to update state from CLOSED to OFFLINE - assignmentManager.setOffline(regionInfo); + assignmentManager.getRegionStates().updateRegionState( + regionInfo, RegionState.State.CLOSED, null); // This below has to do w/ online enable/disable of a table assignmentManager.removeClosedRegion(regionInfo); assignmentManager.assign(regionInfo, true); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java index 62cbfe6e7b9..3b77c5f69f6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java @@ -176,8 +176,9 @@ public class CreateTableHandler extends EventHandler { // 4. Trigger immediate assignment of the regions in round-robin fashion List servers = serverManager.createDestinationServersList(); try { - this.assignmentManager.assignUserRegions(Arrays.asList(newRegions), - servers); + List regions = Arrays.asList(newRegions); + assignmentManager.getRegionStates().createRegionStates(regions); + assignmentManager.assignUserRegions(regions, servers); } catch (InterruptedException ie) { LOG.error("Caught " + ie + " during round-robin assignment"); throw new IOException(ie); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java index c3db2806c8b..3df1e9464df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java @@ -62,12 +62,12 @@ public class DeleteTableHandler extends TableEventHandler { for (HRegionInfo region : regions) { long done = System.currentTimeMillis() + waitTime; while (System.currentTimeMillis() < done) { - AssignmentManager.RegionState rs = am.isRegionInTransition(region); - if (rs == null) break; + if (!am.getRegionStates().isRegionInTransition(region)) break; Threads.sleep(waitingTimeForEvents); - LOG.debug("Waiting on region to clear regions in transition; " + rs); + LOG.debug("Waiting on region to clear regions in transition; " + + am.getRegionStates().getRegionTransitionState(region)); } - if (am.isRegionInTransition(region) != null) { + if (am.getRegionStates().isRegionInTransition(region)) { throw new IOException("Waited hbase.master.wait.on.region (" + waitTime + "ms) for region to leave region " + region.getRegionNameAsString() + " in transitions"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java index 76e47df3d23..3ede0932d91 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.BulkAssigner; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; +import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.KeeperException; @@ -124,8 +125,8 @@ public class DisableTableHandler extends EventHandler { // already closed will not be included in this list; i.e. the returned // list is not ALL regions in a table, its all online regions according // to the in-memory state on this master. - final List regions = - this.assignmentManager.getRegionsOfTable(tableName); + final List regions = this.assignmentManager + .getRegionStates().getRegionsOfTable(tableName); if (regions.size() == 0) { done = true; break; @@ -162,8 +163,9 @@ public class DisableTableHandler extends EventHandler { @Override protected void populatePool(ExecutorService pool) { + RegionStates regionStates = assignmentManager.getRegionStates(); for (HRegionInfo region: regions) { - if (assignmentManager.isRegionInTransition(region) != null) continue; + if (regionStates.isRegionInTransition(region)) continue; final HRegionInfo hri = region; pool.execute(new Runnable() { public void run() { @@ -181,7 +183,7 @@ public class DisableTableHandler extends EventHandler { List regions = null; while (!server.isStopped() && remaining > 0) { Thread.sleep(waitingTimeForEvents); - regions = assignmentManager.getRegionsOfTable(tableName); + regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName); if (regions.isEmpty()) break; remaining = timeout - (System.currentTimeMillis() - startTime); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java index 51223676721..37ecd4752fd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java @@ -167,7 +167,8 @@ public class EnableTableHandler extends EventHandler { final List regionsInMeta) throws IOException { final List onlineRegions = - this.assignmentManager.getRegionsOfTable(tableName); + this.assignmentManager.getRegionStates() + .getRegionsOfTable(tableName); regionsInMeta.removeAll(onlineRegions); return regionsInMeta; } @@ -194,7 +195,8 @@ public class EnableTableHandler extends EventHandler { if (!roundRobinAssignment) { for (HRegionInfo region : regions) { - if (assignmentManager.isRegionInTransition(region) != null) { + if (assignmentManager.getRegionStates() + .isRegionInTransition(region)) { continue; } final HRegionInfo hri = region; @@ -223,7 +225,8 @@ public class EnableTableHandler extends EventHandler { int lastNumberOfRegions = 0; while (!server.isStopped() && remaining > 0) { Thread.sleep(waitingTimeForEvents); - regions = assignmentManager.getRegionsOfTable(tableName); + regions = assignmentManager.getRegionStates() + .getRegionsOfTable(tableName); if (isDone(regions)) break; // Punt on the timeout as long we make progress diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java index 2254cf46b0e..4823b5ea421 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.master.AssignmentManager; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; +import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.zookeeper.KeeperException; @@ -98,7 +98,8 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf public void process() { // Code to defend against case where we get SPLIT before region open // processing completes; temporary till we make SPLITs go via zk -- 0.92. - RegionState regionState = this.assignmentManager.isRegionInTransition(regionInfo); + RegionState regionState = this.assignmentManager.getRegionStates() + .getRegionTransitionState(regionInfo.getEncodedName()); boolean openedNodeDeleted = false; if (regionState != null && regionState.getState().equals(RegionState.State.OPEN)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java index 81de3fe3e1f..9960d754a40 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java @@ -38,9 +38,9 @@ import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.master.AssignmentManager; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; import org.apache.hadoop.hbase.master.DeadServer; import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKAssign; @@ -289,12 +289,13 @@ public class ServerShutdownHandler extends EventHandler { if (hris != null) { List toAssignRegions = new ArrayList(); for (Map.Entry e: hris.entrySet()) { - RegionState rit = this.services.getAssignmentManager().isRegionInTransition(e.getKey()); + RegionState rit = services.getAssignmentManager() + .getRegionStates().getRegionTransitionState(e.getKey()); if (processDeadRegion(e.getKey(), e.getValue(), this.services.getAssignmentManager(), this.server.getCatalogTracker())) { ServerName addressFromAM = this.services.getAssignmentManager() - .getRegionServerOfRegion(e.getKey()); + .getRegionStates().getRegionServerOfRegion(e.getKey()); if (rit != null && !rit.isClosing() && !rit.isPendingClose() && !rit.isSplitting()) { // Skip regions that were in transition unless CLOSING or // PENDING_CLOSE diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 64d5a58db5d..9a4bb677213 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -105,7 +105,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse; import org.apache.hadoop.hbase.regionserver.wal.HLog; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index e2bbb48fd25..4ed8dabb40a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -3366,8 +3366,8 @@ public class HRegionServer implements ClientProtocol, checkOpen(); requestCount.incrementAndGet(); List list = new ArrayList(onlineRegions.size()); - for (Map.Entry e: this.onlineRegions.entrySet()) { - list.add(e.getValue().getRegionInfo()); + for (HRegion region: this.onlineRegions.values()) { + list.add(region.getRegionInfo()); } Collections.sort(list); return ResponseConverter.buildGetOnlineRegionResponse(list); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java index 5f49877c35e..81a7fc3dc74 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index 9c3c9efe194..8765301cbfd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; +import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.wal.HLog; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 0259470edce..fc4fe2e4f6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; @@ -2109,6 +2110,27 @@ public class HBaseTestingUtility { return hloc.getPort(); } + /** + * Due to async racing issue, a region may not be in + * the online region list of a region server yet, after + * the assignment znode is deleted and the new assignment + * is recorded in master. + */ + public void assertRegionOnServer( + final HRegionInfo hri, final ServerName server, + final long timeout) throws IOException, InterruptedException { + long timeoutTime = System.currentTimeMillis() + timeout; + while (true) { + List regions = getHBaseAdmin().getOnlineRegions(server); + if (regions.contains(hri)) return; + long now = System.currentTimeMillis(); + if (now > timeoutTime) break; + Thread.sleep(10); + } + fail("Could not find region " + hri.getRegionNameAsString() + + " on server " + server); + } + public HRegion createTestRegion(String tableName, HColumnDescriptor hcd) throws IOException { HTableDescriptor htd = new HTableDescriptor(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java index 077fe8faf85..97abb3040ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestDrainingServer.java @@ -29,7 +29,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.master.ServerManager; @@ -223,8 +222,8 @@ public class TestDrainingServer { Assert.assertEquals("Nothing should have happened here.", regionsOnDrainingServer, drainingServer.getNumberOfOnlineRegions()); Assert.assertFalse("We should not have regions in transition here. List is: "+ - master.getAssignmentManager().copyRegionsInTransition(), - master.getAssignmentManager().isRegionsInTransition() ); + master.getAssignmentManager().getRegionStates().getRegionsInTransition(), + master.getAssignmentManager().getRegionStates().isRegionsInTransition()); // Kill a few regionservers. for (int aborted = 0; aborted <= 2; aborted++) { @@ -274,7 +273,7 @@ public class TestDrainingServer { } while (TEST_UTIL.getMiniHBaseCluster().getMaster(). - getAssignmentManager().isRegionsInTransition()) { + getAssignmentManager().getRegionStates().isRegionsInTransition()) { } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java index 63dc8dc224b..cd56c7ee387 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java @@ -39,9 +39,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 24ae44e393c..72dfb29ce97 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -1456,10 +1456,10 @@ public class TestAdmin { List tableRegions = localAdmin.getTableRegions(tableName); HRegionInfo hri = tableRegions.get(0); AssignmentManager am = master.getAssignmentManager(); - ServerName server = am.getRegionServerOfRegion(hri); + ServerName server = am.getRegionStates().getRegionServerOfRegion(hri); localAdmin.move(hri.getEncodedNameAsBytes(), Bytes.toBytes(server.getServerName())); assertEquals("Current region server and region server before move should be same.", server, - am.getRegionServerOfRegion(hri)); + am.getRegionStates().getRegionServerOfRegion(hri)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index d9c5eeacf05..56801817f53 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -159,7 +159,7 @@ public class TestHCM { // We can wait for all regions to be onlines, that makes log reading easier when debugging while (TEST_UTIL.getMiniHBaseCluster().getMaster(). - getAssignmentManager().isRegionsInTransition()) { + getAssignmentManager().getRegionStates().isRegionsInTransition()) { } // Now moving the region to the second server diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index 8d0a3b1ad61..ff8fd26b88b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; +import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -963,10 +964,10 @@ public class TestMasterObserver { // wait for assignments to finish, if any AssignmentManager mgr = master.getAssignmentManager(); - Collection transRegions = - mgr.copyRegionsInTransition().values(); - for (AssignmentManager.RegionState state : transRegions) { - mgr.waitOnRegionToClearRegionsInTransition(state.getRegion()); + Collection transRegions = + mgr.getRegionStates().getRegionsInTransition().values(); + for (RegionState state : transRegions) { + mgr.getRegionStates().waitOnRegionToClearRegionsInTransition(state.getRegion()); } // move half the open regions from RS 0 to RS 1 @@ -983,9 +984,9 @@ public class TestMasterObserver { } // wait for assignments to finish - transRegions = mgr.copyRegionsInTransition().values(); - for (AssignmentManager.RegionState state : transRegions) { - mgr.waitOnRegionToClearRegionsInTransition(state.getRegion()); + transRegions = mgr.getRegionStates().getRegionsInTransition().values(); + for (RegionState state : transRegions) { + mgr.getRegionStates().waitOnRegionToClearRegionsInTransition(state.getRegion()); } // now trigger a balance diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java index 0e265124648..0d672ca409b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java @@ -414,7 +414,7 @@ public class TestHFileOutputFormat { LOG.info("Changing regions in table"); admin.disableTable(table.getTableName()); while(util.getMiniHBaseCluster().getMaster().getAssignmentManager(). - isRegionsInTransition()) { + getRegionStates().isRegionsInTransition()) { Threads.sleep(200); LOG.info("Waiting on table to finish disabling"); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java index 5bc94b63d9e..a89db4158ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/Mocking.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -94,7 +93,8 @@ public class Mocking { boolean wait = true; while (wait) { - AssignmentManager.RegionState state = am.getRegionsInTransition().get(encodedName); + RegionState state = am.getRegionStates() + .getRegionsInTransition().get(encodedName); if (state != null && state.isPendingOpen()){ wait = false; } else { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java index 4253e8c522f..c793c07c44e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManager.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.RegionException; import org.apache.hadoop.hbase.RegionTransition; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerLoad; @@ -49,8 +50,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState.State; +import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.balancer.DefaultLoadBalancer; import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory; import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler; @@ -102,6 +102,7 @@ public class TestAssignmentManager { private ServerManager serverManager; private ZooKeeperWatcher watcher; private LoadBalancer balancer; + private HMaster master; @BeforeClass public static void beforeClass() throws Exception { @@ -155,7 +156,10 @@ public class TestAssignmentManager { Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_A, REGIONINFO, -1)). thenReturn(RegionOpeningState.OPENED); Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_B, REGIONINFO, -1)). - thenReturn(RegionOpeningState.OPENED); + thenReturn(RegionOpeningState.OPENED); + this.master = Mockito.mock(HMaster.class); + + Mockito.when(this.master.getServerManager()).thenReturn(serverManager); } @After @@ -299,8 +303,9 @@ public class TestAssignmentManager { } } - private void createRegionPlanAndBalance(final AssignmentManager am, - final ServerName from, final ServerName to, final HRegionInfo hri) { + private void createRegionPlanAndBalance( + final AssignmentManager am, final ServerName from, + final ServerName to, final HRegionInfo hri) throws RegionException { // Call the balance function but fake the region being online first at // servername from. am.regionOnline(hri, from); @@ -330,7 +335,7 @@ public class TestAssignmentManager { .getConfiguration()); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, ct, balancer, executor, null); + this.serverManager, ct, balancer, executor, null); try { // Make sure our new AM gets callbacks; once registered, can't unregister. // Thats ok because we make a new zk watcher for each test. @@ -370,7 +375,7 @@ public class TestAssignmentManager { ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO, SERVERNAME_B, versionid); assertNotSame(-1, versionid); // Wait on the handler removing the OPENED znode. - while(am.isRegionInTransition(REGIONINFO) != null) Threads.sleep(1); + while(am.getRegionStates().isRegionInTransition(REGIONINFO)) Threads.sleep(1); } finally { executor.shutdown(); am.shutdown(); @@ -397,7 +402,7 @@ public class TestAssignmentManager { .getConfiguration()); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, ct, balancer, executor, null); + this.serverManager, ct, balancer, executor, null); try { processServerShutdownHandler(ct, am, false); } finally { @@ -448,15 +453,14 @@ public class TestAssignmentManager { // We need a mocked catalog tracker. CatalogTracker ct = Mockito.mock(CatalogTracker.class); - LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration()); // Create an AM. - AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(this.server, - this.serverManager); + AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( + this.server, this.serverManager); // adding region to regions and servers maps. am.regionOnline(REGIONINFO, SERVERNAME_A); // adding region in pending close. - am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO, - State.SPLITTING, System.currentTimeMillis(), SERVERNAME_A)); + am.getRegionStates().updateRegionState( + REGIONINFO, State.SPLITTING, SERVERNAME_A); am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString()); RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING, REGIONINFO.getRegionName(), SERVERNAME_A); @@ -470,8 +474,8 @@ public class TestAssignmentManager { // In both cases the znode should be deleted. if (regionSplitDone) { - assertTrue("Region state of region in SPLITTING should be removed from rit.", - am.regionsInTransition.isEmpty()); + assertFalse("Region state of region in SPLITTING should be removed from rit.", + am.getRegionStates().isRegionsInTransition()); } else { while (!am.assignInvoked) { Thread.sleep(1); @@ -497,13 +501,12 @@ public class TestAssignmentManager { CatalogTracker ct = Mockito.mock(CatalogTracker.class); LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration()); // Create an AM. - AssignmentManager am = new AssignmentManager(this.server, this.serverManager, ct, balancer, - executor, null); + AssignmentManager am = new AssignmentManager(this.server, + this.serverManager, ct, balancer, executor, null); // adding region to regions and servers maps. am.regionOnline(REGIONINFO, SERVERNAME_A); // adding region in pending close. - am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO, - State.PENDING_CLOSE)); + am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE); if (state == Table.State.DISABLING) { am.getZKTable().setDisablingTable(REGIONINFO.getTableNameAsString()); } else { @@ -526,8 +529,8 @@ public class TestAssignmentManager { // assert will be true but the piece of code added for HBASE-5927 will not // do that. if (state == Table.State.DISABLED) { - assertTrue("Region state of region in pending close should be removed from rit.", - am.regionsInTransition.isEmpty()); + assertFalse("Region state of region in pending close should be removed from rit.", + am.getRegionStates().isRegionsInTransition()); } } finally { am.setEnabledTable(REGIONINFO.getTableNameAsString()); @@ -618,7 +621,7 @@ public class TestAssignmentManager { .getConfiguration()); // Create an AM. AssignmentManager am = new AssignmentManager(this.server, - this.serverManager, ct, balancer, null, null); + this.serverManager, ct, balancer, null, null); try { // First make sure my mock up basically works. Unassign a region. unassign(am, SERVERNAME_A, hri); @@ -636,7 +639,7 @@ public class TestAssignmentManager { // This transition should fail if the znode has been messed with. ZKAssign.transitionNode(this.watcher, hri, SERVERNAME_A, EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version); - assertTrue(am.isRegionInTransition(hri) == null); + assertFalse(am.getRegionStates().isRegionInTransition(hri)); } finally { am.shutdown(); } @@ -654,7 +657,7 @@ public class TestAssignmentManager { final RecoverableZooKeeper recoverableZk = Mockito .mock(RecoverableZooKeeper.class); AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( - this.server, this.serverManager); + this.server, this.serverManager); Watcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest", null) { public RecoverableZooKeeper getRecoverableZooKeeper() { @@ -666,7 +669,7 @@ public class TestAssignmentManager { .getChildren("/hbase/unassigned", zkw); am.setWatcher((ZooKeeperWatcher) zkw); try { - am.processDeadServersAndRegionsInTransition(); + am.processDeadServersAndRegionsInTransition(null); fail("Expected to abort"); } catch (NullPointerException e) { fail("Should not throw NPE"); @@ -678,7 +681,7 @@ public class TestAssignmentManager { * TestCase verifies that the regionPlan is updated whenever a region fails to open * and the master tries to process RS_ZK_FAILED_OPEN state.(HBASE-5546). */ - @Test + @Test(timeout = 5000) public void testRegionPlanIsUpdatedWhenRegionFailsToOpen() throws IOException, KeeperException, ServiceException, InterruptedException { this.server.getConfiguration().setClass( @@ -700,9 +703,8 @@ public class TestAssignmentManager { EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_FAILED_OPEN, v); String path = ZKAssign.getNodeName(this.watcher, REGIONINFO .getEncodedName()); - RegionState state = new RegionState(REGIONINFO, State.OPENING, System - .currentTimeMillis(), SERVERNAME_A); - am.regionsInTransition.put(REGIONINFO.getEncodedName(), state); + am.getRegionStates().updateRegionState( + REGIONINFO, State.OPENING, SERVERNAME_A); // a dummy plan inserted into the regionPlans. This plan is cleared and // new one is formed am.regionPlans.put(REGIONINFO.getEncodedName(), new RegionPlan( @@ -777,11 +779,11 @@ public class TestAssignmentManager { * region which is in Opening state on a dead RS. Master should immediately * assign the region and not wait for Timeout Monitor.(Hbase-5882). */ - @Test + @Test(timeout = 5000) public void testRegionInOpeningStateOnDeadRSWhileMasterFailover() throws IOException, KeeperException, ServiceException, InterruptedException { - AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(this.server, - this.serverManager); + AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager( + this.server, this.serverManager); ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A); int version = ZKAssign.getVersion(this.watcher, REGIONINFO); ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE, @@ -810,7 +812,7 @@ public class TestAssignmentManager { * @throws IOException * @throws Exception */ - @Test + @Test(timeout = 5000) public void testDisablingTableRegionsAssignmentDuringCleanClusterStartup() throws KeeperException, IOException, Exception { this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, @@ -888,7 +890,7 @@ public class TestAssignmentManager { } private void unassign(final AssignmentManager am, final ServerName sn, - final HRegionInfo hri) { + final HRegionInfo hri) throws RegionException { // Before I can unassign a region, I need to set it online. am.regionOnline(hri, sn); // Unassign region. @@ -905,8 +907,7 @@ public class TestAssignmentManager { * @throws KeeperException */ private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server, - final ServerManager manager) - throws IOException, KeeperException, ServiceException { + final ServerManager manager) throws IOException, KeeperException, ServiceException { // We need a mocked catalog tracker. Its used by our AM instance. CatalogTracker ct = Mockito.mock(CatalogTracker.class); // Make an RS Interface implementation. Make it so a scanner can go against @@ -938,7 +939,7 @@ public class TestAssignmentManager { ExecutorService executor = startupMasterExecutor("mockedAMExecutor"); this.balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration()); AssignmentManagerWithExtrasForTesting am = new AssignmentManagerWithExtrasForTesting( - server, manager, ct, this.balancer, executor); + server, manager, ct, this.balancer, executor); return am; } @@ -954,10 +955,10 @@ public class TestAssignmentManager { boolean assignInvoked = false; AtomicBoolean gate = new AtomicBoolean(true); - public AssignmentManagerWithExtrasForTesting(final Server master, - final ServerManager serverManager, final CatalogTracker catalogTracker, - final LoadBalancer balancer, final ExecutorService service) - throws KeeperException, IOException { + public AssignmentManagerWithExtrasForTesting( + final Server master, final ServerManager serverManager, + final CatalogTracker catalogTracker, final LoadBalancer balancer, + final ExecutorService service) throws KeeperException, IOException { super(master, serverManager, catalogTracker, balancer, service, null); this.es = service; this.ct = catalogTracker; @@ -987,12 +988,7 @@ public class TestAssignmentManager { super.assign(region, setOfflineInZK, forceNewPlan, hijack); this.gate.set(true); } - - @Override - public ServerName getRegionServerOfRegion(HRegionInfo hri) { - return SERVERNAME_A; - } - + @Override public void assign(java.util.List regions, java.util.List servers) { @@ -1037,7 +1033,7 @@ public class TestAssignmentManager { // the RIT region to our RIT Map in AM at processRegionsInTransition. // First clear any inmemory state from AM so it acts like a new master // coming on line. - am.regionsInTransition.clear(); + am.getRegionStates().regionsInTransition.clear(); am.regionPlans.clear(); try { am.joinCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java new file mode 100644 index 00000000000..a2c2d12e3fc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java @@ -0,0 +1,201 @@ +/** + * 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.master; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +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.MediumTests; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * This tests AssignmentManager with a testing cluster. + */ +@Category(MediumTests.class) +public class TestAssignmentManagerOnCluster { + private final static byte[] FAMILY = Bytes.toBytes("FAMILY"); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private final static Configuration conf = TEST_UTIL.getConfiguration(); + private static HBaseAdmin admin; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.startMiniCluster(3); + admin = TEST_UTIL.getHBaseAdmin(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * This tests region assignment + */ + @Test + public void testAssignRegion() throws Exception { + String table = "testAssignRegion"; + try { + HTableDescriptor desc = new HTableDescriptor(table); + desc.addFamily(new HColumnDescriptor(FAMILY)); + admin.createTable(desc); + + HTable meta = new HTable(conf, HConstants.META_TABLE_NAME); + HRegionInfo hri = new HRegionInfo( + desc.getName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); + Put put = new Put(hri.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(hri)); + meta.put(put); + + HMaster master = TEST_UTIL.getHBaseCluster().getMaster(); + master.assignRegion(hri); + master.getAssignmentManager().waitForAssignment(hri); + + ServerName serverName = master.getAssignmentManager(). + getRegionStates().getRegionServerOfRegion(hri); + TEST_UTIL.assertRegionOnServer(hri, serverName, 200); + } finally { + TEST_UTIL.deleteTable(Bytes.toBytes(table)); + } + } + + /** + * This tests offlining a region + */ + @Test + public void testOfflineRegion() throws Exception { + String table = "testOfflineRegion"; + try { + HRegionInfo hri = createTableAndGetOneRegion(table); + + RegionStates regionStates = TEST_UTIL.getHBaseCluster(). + getMaster().getAssignmentManager().getRegionStates(); + ServerName serverName = regionStates.getRegionServerOfRegion(hri); + TEST_UTIL.assertRegionOnServer(hri, serverName, 200); + admin.offline(hri.getRegionName()); + + long timeoutTime = System.currentTimeMillis() + 800; + while (true) { + List regions = + regionStates.getRegionsOfTable(Bytes.toBytes(table)); + if (!regions.contains(hri)) break; + long now = System.currentTimeMillis(); + if (now > timeoutTime) { + fail("Failed to offline the region in time"); + break; + } + Thread.sleep(10); + } + RegionState regionState = regionStates.getRegionState(hri); + assertTrue(regionState.isOffline()); + } finally { + TEST_UTIL.deleteTable(Bytes.toBytes(table)); + } + } + + /** + * This tests moving a region + */ + @Test + public void testMoveRegion() throws Exception { + String table = "testMoveRegion"; + try { + HRegionInfo hri = createTableAndGetOneRegion(table); + + RegionStates regionStates = TEST_UTIL.getHBaseCluster(). + getMaster().getAssignmentManager().getRegionStates(); + ServerName serverName = regionStates.getRegionServerOfRegion(hri); + ServerName destServerName = null; + for (int i = 0; i < 3; i++) { + HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(i); + if (!destServer.getServerName().equals(serverName)) { + destServerName = destServer.getServerName(); + break; + } + } + assertTrue(destServerName != null + && !destServerName.equals(serverName)); + TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(), + Bytes.toBytes(destServerName.getServerName())); + + long timeoutTime = System.currentTimeMillis() + 800; + while (true) { + ServerName sn = regionStates.getRegionServerOfRegion(hri); + if (sn != null && sn.equals(destServerName)) { + TEST_UTIL.assertRegionOnServer(hri, sn, 200); + break; + } + long now = System.currentTimeMillis(); + if (now > timeoutTime) { + fail("Failed to move the region in time"); + } + regionStates.waitForUpdate(50); + } + + } finally { + TEST_UTIL.deleteTable(Bytes.toBytes(table)); + } + } + + HRegionInfo createTableAndGetOneRegion( + final String tableName) throws IOException, InterruptedException { + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(FAMILY)); + admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5); + + // wait till the table is assigned + HMaster master = TEST_UTIL.getHBaseCluster().getMaster(); + long timeoutTime = System.currentTimeMillis() + 100; + while (true) { + List regions = master.getAssignmentManager(). + getRegionStates().getRegionsOfTable(Bytes.toBytes(tableName)); + if (regions.size() > 3) { + return regions.get(2); + } + long now = System.currentTimeMillis(); + if (now > timeoutTime) { + fail("Could not find an online region"); + } + Thread.sleep(10); + } + } + + @org.junit.Rule + public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMXBean.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMXBean.java index cdcd4fdb700..bf7361702aa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMXBean.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMXBean.java @@ -78,7 +78,7 @@ public class TestMXBean { info.getCoprocessors().length); Assert.assertEquals(master.getServerManager().getOnlineServersList().size(), info.getRegionServers().size()); - Assert.assertEquals(master.getAssignmentManager().isRegionsInTransition(), + Assert.assertEquals(master.getAssignmentManager().getRegionStates().isRegionsInTransition(), info.getRegionsInTransition().length > 0); Assert.assertTrue(info.getRegionServers().size() == 4); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index 74059612b74..acd690b61ed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.executor.EventHandler.EventType; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -770,14 +769,14 @@ public class TestMasterFailover { // PENDING_OPEN and enabled region = enabledRegions.remove(0); regionsThatShouldBeOnline.add(region); - master.assignmentManager.regionsInTransition.put(region.getEncodedName(), - new RegionState(region, RegionState.State.PENDING_OPEN, 0, null)); + master.getAssignmentManager().getRegionStates().updateRegionState( + region, RegionState.State.PENDING_OPEN, null); ZKAssign.createNodeOffline(zkw, region, master.getServerName()); // PENDING_OPEN and disabled region = disabledRegions.remove(0); regionsThatShouldBeOffline.add(region); - master.assignmentManager.regionsInTransition.put(region.getEncodedName(), - new RegionState(region, RegionState.State.PENDING_OPEN, 0, null)); + master.getAssignmentManager().getRegionStates().updateRegionState( + region, RegionState.State.PENDING_OPEN, null); ZKAssign.createNodeOffline(zkw, region, master.getServerName()); // This test is bad. It puts up a PENDING_CLOSE but doesn't say what // server we were PENDING_CLOSE against -- i.e. an entry in @@ -808,7 +807,7 @@ public class TestMasterFailover { final long maxTime = 120000; boolean done = master.assignmentManager.waitUntilNoRegionsInTransition(maxTime); if (!done) { - LOG.info("rit=" + master.assignmentManager.copyRegionsInTransition()); + LOG.info("rit=" + master.getAssignmentManager().getRegionStates().getRegionsInTransition()); } long elapsed = System.currentTimeMillis() - now; assertTrue("Elapsed=" + elapsed + ", maxTime=" + maxTime + ", done=" + done, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java index 07f306b3675..b5d116f2321 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java @@ -33,7 +33,6 @@ import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.tmpl.master.AssignmentManagerStatusTmpl; @@ -66,11 +65,11 @@ public class TestMasterStatusServlet { @Before public void setupBasicMocks() { conf = HBaseConfiguration.create(); - + master = Mockito.mock(HMaster.class); Mockito.doReturn(FAKE_HOST).when(master).getServerName(); Mockito.doReturn(conf).when(master).getConfiguration(); - + // Fake serverManager ServerManager serverManager = Mockito.mock(ServerManager.class); Mockito.doReturn(1.0).when(serverManager).getAverageLoad(); @@ -78,13 +77,15 @@ public class TestMasterStatusServlet { // Fake AssignmentManager and RIT AssignmentManager am = Mockito.mock(AssignmentManager.class); + RegionStates rs = Mockito.mock(RegionStates.class); NavigableMap regionsInTransition = Maps.newTreeMap(); regionsInTransition.put("r1", - new RegionState(FAKE_HRI, RegionState.State.CLOSING, 12345L, FAKE_HOST)); - Mockito.doReturn(regionsInTransition).when(am).copyRegionsInTransition(); + new RegionState(FAKE_HRI, RegionState.State.CLOSING, 12345L, FAKE_HOST)); + Mockito.doReturn(rs).when(am).getRegionStates(); + Mockito.doReturn(regionsInTransition).when(rs).getRegionsInTransition(); Mockito.doReturn(am).when(master).getAssignmentManager(); - + // Fake ZKW ZooKeeperWatcher zkw = Mockito.mock(ZooKeeperWatcher.class); Mockito.doReturn("fakequorum").when(zkw).getQuorum(); @@ -93,7 +94,6 @@ public class TestMasterStatusServlet { // Mock admin admin = Mockito.mock(HBaseAdmin.class); } - private void setupMockTables() throws IOException { HTableDescriptor tables[] = new HTableDescriptor[] { @@ -153,7 +153,8 @@ public class TestMasterStatusServlet { @Test public void testAssignmentManagerTruncatedList() throws IOException { AssignmentManager am = Mockito.mock(AssignmentManager.class); - + RegionStates rs = Mockito.mock(RegionStates.class); + // Add 100 regions as in-transition NavigableMap regionsInTransition = Maps.newTreeMap(); @@ -161,14 +162,15 @@ public class TestMasterStatusServlet { HRegionInfo hri = new HRegionInfo(FAKE_TABLE.getName(), new byte[]{i}, new byte[]{(byte) (i+1)}); regionsInTransition.put(hri.getEncodedName(), - new RegionState(hri, RegionState.State.CLOSING, 12345L, FAKE_HOST)); + new RegionState(hri, RegionState.State.CLOSING, 12345L, FAKE_HOST)); } // Add META in transition as well regionsInTransition.put( HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(), new RegionState(HRegionInfo.FIRST_META_REGIONINFO, RegionState.State.CLOSING, 12345L, FAKE_HOST)); - Mockito.doReturn(regionsInTransition).when(am).copyRegionsInTransition(); + Mockito.doReturn(rs).when(am).getRegionStates(); + Mockito.doReturn(regionsInTransition).when(rs).getRegionsInTransition(); // Render to a string StringWriter sw = new StringWriter(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java index aa8bdb8f10a..a7451ef49cb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java @@ -32,7 +32,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.executor.EventHandler.EventType; -import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -124,9 +123,12 @@ public class TestOpenedRegionHandler { region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); assertNotNull(region); AssignmentManager am = Mockito.mock(AssignmentManager.class); - when(am.isRegionInTransition(hri)).thenReturn( - new RegionState(region.getRegionInfo(), RegionState.State.OPEN, - System.currentTimeMillis(), server.getServerName())); + RegionStates rsm = Mockito.mock(RegionStates.class); + Mockito.doReturn(rsm).when(am).getRegionStates(); + when(rsm.isRegionInTransition(hri)).thenReturn(false); + when(rsm.getRegionState(hri)).thenReturn( + new RegionState(region.getRegionInfo(), RegionState.State.OPEN, + System.currentTimeMillis(), server.getServerName())); // create a node with OPENED state zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL, region, server.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 87cb8cae92c..765115add12 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -436,7 +436,7 @@ public class TestSplitTransactionOnCluster { hri.setOffline(true); hri.setSplit(true); ServerName regionServerOfRegion = master.getAssignmentManager() - .getRegionServerOfRegion(hri); + .getRegionStates().getRegionServerOfRegion(hri); assertTrue(regionServerOfRegion != null); } finally { @@ -515,7 +515,7 @@ public class TestSplitTransactionOnCluster { hri.setOffline(true); hri.setSplit(true); ServerName regionServerOfRegion = master.getAssignmentManager() - .getRegionServerOfRegion(hri); + .getRegionStates().getRegionServerOfRegion(hri); assertTrue(regionServerOfRegion == null); } finally { // Set this flag back. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index f0d567bdb8d..b7ab9762962 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.monitoring.MonitoredTask; -import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.regionserver.FlushRequester; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -55,7 +54,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.Strings; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -212,8 +210,12 @@ public class TestWALReplay { Bytes.toBytes(destServer.getServerName().getServerName())); while (true) { ServerName serverName = master.getAssignmentManager() - .getRegionServerOfRegion(destRegion.getRegionInfo()); - if (serverName != null && serverName.equals(destServer.getServerName())) break; + .getRegionStates().getRegionServerOfRegion(destRegion.getRegionInfo()); + if (serverName != null && serverName.equals(destServer.getServerName())) { + TEST_UTIL.assertRegionOnServer( + destRegion.getRegionInfo(), serverName, 200); + break; + } Thread.sleep(10); } } @@ -709,12 +711,10 @@ public class TestWALReplay { // Flusher used in this test. Keep count of how often we are called and // actually run the flush inside here. class TestFlusher implements FlushRequester { - private int count = 0; private HRegion r; @Override public void requestFlush(HRegion region) { - count++; try { r.flushcache(); } catch (IOException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index fdcceded8ae..8a760d3e583 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -63,12 +63,14 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.executor.EventHandler.EventType; +import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; +import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo; import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -88,7 +90,8 @@ public class TestHBaseFsck { private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final static Configuration conf = TEST_UTIL.getConfiguration(); private final static byte[] FAM = Bytes.toBytes("fam"); - private final static int REGION_ONLINE_TIMEOUT = 300; + private final static int REGION_ONLINE_TIMEOUT = 800; + private static RegionStates regionStates; // for the instance, reset every test run private HTable tbl; @@ -103,6 +106,10 @@ public class TestHBaseFsck { public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, false); TEST_UTIL.startMiniCluster(3); + + AssignmentManager assignmentManager = + TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager(); + regionStates = assignmentManager.getRegionStates(); } @AfterClass @@ -406,6 +413,8 @@ public class TestHBaseFsck { TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe); TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() .waitForAssignment(hriDupe); + ServerName server = regionStates.getRegionServerOfRegion(hriDupe); + TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT); HBaseFsck hbck = doFsck(conf, false); assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS, @@ -482,6 +491,8 @@ public class TestHBaseFsck { TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe); TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() .waitForAssignment(hriDupe); + ServerName server = regionStates.getRegionServerOfRegion(hriDupe); + TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT); // Yikes! The assignment manager can't tell between diff between two // different regions with the same start/endkeys since it doesn't @@ -531,6 +542,8 @@ public class TestHBaseFsck { TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe); TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() .waitForAssignment(hriDupe); + ServerName server = regionStates.getRegionServerOfRegion(hriDupe); + TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT); HBaseFsck hbck = doFsck(conf,false); assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DEGENERATE_REGION, @@ -568,6 +581,8 @@ public class TestHBaseFsck { TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap); TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() .waitForAssignment(hriOverlap); + ServerName server = regionStates.getRegionServerOfRegion(hriOverlap); + TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT); HBaseFsck hbck = doFsck(conf, false); assertErrors(hbck, new ERROR_CODE[] { @@ -702,6 +717,8 @@ public class TestHBaseFsck { TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap); TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() .waitForAssignment(hriOverlap); + ServerName server = regionStates.getRegionServerOfRegion(hriOverlap); + TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT); HBaseFsck hbck = doFsck(conf, false); assertErrors(hbck, new ERROR_CODE[] { @@ -739,6 +756,8 @@ public class TestHBaseFsck { TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap); TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() .waitForAssignment(hriOverlap); + ServerName server = regionStates.getRegionServerOfRegion(hriOverlap); + TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT); HBaseFsck hbck = doFsck(conf, false); assertErrors(hbck, new ERROR_CODE[] { @@ -1038,9 +1057,9 @@ public class TestHBaseFsck { int iTimes = 0; while (true) { - RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(zkw, - region.getEncodedName())); - if (rt != null && rt.getEventType() == EventType.RS_ZK_REGION_OPENED) { + byte[] data = ZKAssign.getData(zkw, region.getEncodedName()); + RegionTransition rt = data == null ? null : RegionTransition.parseFrom(data); + if (rt == null || rt.getEventType() == EventType.RS_ZK_REGION_OPENED) { break; } Thread.sleep(100);