Index: BRANCH_TODO.txt =================================================================== --- BRANCH_TODO.txt (revision 979909) +++ BRANCH_TODO.txt (working copy) @@ -1,22 +1,154 @@ List of things todo for branch, including comments from reviews not yet implemented. +harder stuff +--- -Now: +* make final decisions on root/meta timeouts. almost everyone is coordinating + access through CatalogTracker which should make it easy to standardize. + if there are operations that should just retry indefinitely, they need to + resubmit themselves to their executor service. +* move splits to RS side, integrate new patch from stack on trunk + might need a new CREATED unassigned now, or new rpc, but get rid of sending + split notification on heartbeat? + how to handle splits concurrent with disable? + +* review master startup order + we should use cluster flag in zk to signal RS to + start rather than master availability and between master up and cluster up + the master can do bulk of it's initialization. + +* figure what to do with client table admin ops (flush, split, compact) + (direct to RS rpc calls are in place, need to update client) + +* on region open (and wherever split children notify master) should check if + if the table is disabled and should close the regions... maybe. + +* in RootEditor there is a race condition between delete and watch? + +* review FileSystemManager calls + +* there are some races with master wanting to connect for rpc + to regionserver and the rs starting its rpc server, need to address + +* figure how to handle the very rare but possible race condition where two + RSs will update META and the later one can squash the valid one if there was + a long gc pause + +* review synchronization in AssignmentManager + +* migrate TestMasterTransitions or make new? + +* fix or remove last couple master tests that used RSOQ + +* write new tests!!! + + +somewhat easier stuff +--- + +* regionserver exit and expiration need to be finished in ServerManager + +* jsp pages borked + +* make sync calls for enable/disable (check and verify methods?) + this still needs some love and testing but should be much easier to control now + +* integrate load balancing + implemented but need to start a thread or chore, each time, wait for no + regions in transition, generate and iterate the plan, putting it in-memory + and then triggering the assignment. if the master crashes mid-balance, + it should finish up to the point of the last CLOSE RPC it sent out to an RS. + Regions will be sitting in CLOSED in ZK, failover master will pick it up, + re-executes the ClosedRegionHandler() on it + * synchronize all access to the boolean in ActiveMasterManager + (now this is probably just move it to extend ZKNodeTracker) -* Remove TODO in HCM about Abortable interface +* update client to use new admin functions straight to rs + possibly migrate client to use CatalogTracker? -Think about: -* renaming master file manager? MasterFS/MasterFileSystem +notes from 8/4 (with what i did tonight for them, which is most of what is different in this diff) +--- +* in CatalogTracker need to stabilize on one getRoot and one getMeta method to + use that waits and uses the default wait-for-catalogs timeout. + + We should get rid of the 'refresh' boolean that I have in there and + should always ping the server to ensure it is serving the region before we + return it. If we do eventually drop root and put the meta locations into zk + we would no longer need this, so will not always have to pay this tax. + + >> This is done. You pass default timeout in constructor. Two methods now are: + waitForRootServerConnectionDefault() + waitForMetaServerConnectionDefault() + + +* ROOT changes + + RootEditor -> RootLocationEditor, delete -> unset + + Change the way we unset the root location. Set the data to null rather than + deleting the node. Requires changes to RootLocationEditor and RootRegionTracker. + + >> Thought there was a race condition here, but there is not. In fact, we do + not even need to set the watch in the delete method. It is already properly + being handled by RootRegionTracker. + + +* In AssignmentManager.processFailure() need to insert RegionState into RIT map + + >> This is done. This needs tests but I think failover is all in place now. + + +* On RS-side, make separate OpenRootHandler and OpenMetaHandler + + >> Added four new handlers for open/close of root/meta and associated + executors + + +* Add priorities to Opened/Closed handlers on Master + + >> Added ROOT, META, USER priorities + + +* In RegionTransitionData, store the actual byte[] regionName rather than + the encoded name + + >> Done. We should also get in practice of naming variables encodedName if + it is that. + + +* Executor services need to be using a priority queue + + >> Done. I think all stuff to set pool size and add priorities is in. + + +* In EventType, completely remove differentiating between Master and RS. + This means for a given EventType, it will map to the same handler whether it + is on RS or Master. + + >> Done. + + Also in EventType, remove fromByte() and use an ordinal() method + + >> Done. Can we remove even having the (int) values for the enums now? + + Later: +* renaming master file manager? MasterFS/MasterFileSystem + * ServerStatus/MasterStatus + + We now have: Abortable as the base class (M, RS, and Client implement abort()) + + ServerController (M and RS implement getZK/ServerName/Info/etc) + + RegionServerController (RS, definitely the hacky one) + + MasterController (get/set of shutdown, close, etc) + - These need new names to be more descriptive (ServerControl?) - They should have a very clear purpose that adds value beyond passing HMaster directly @@ -26,7 +158,7 @@ * HBaseEventHandler/HBaseEventType/HBaseExecutorService - - After ZK changes, renamed to EventHandler/EventType + X (done) After ZK changes, renamed to EventHandler/EventType - Currently multiple types map to a single handler, we may want 1-to-1 - Need to do a full review of the semantics of these once bulk of master rewrite is done Index: src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (working copy) @@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper; import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -141,7 +140,7 @@ * single instance only is how the minidfscluster works. * @return The calculated cluster test build directory. */ - File setupClusterTestBuildDir() { + public File setupClusterTestBuildDir() { String randomStr = UUID.randomUUID().toString(); String dirStr = getTestDir(randomStr).toString(); File dir = new File(dirStr).getAbsoluteFile(); @@ -153,8 +152,11 @@ /** * @throws IOException If a cluster -- zk, dfs, or hbase -- already running. */ - void isRunningCluster() throws IOException { - if (this.clusterTestBuildDir == null) return; + void isRunningCluster(String passedBuildPath) throws IOException { + if (this.clusterTestBuildDir == null || + passedBuildPath != null) { + return; + } throw new IOException("Cluster already running at " + this.clusterTestBuildDir); } @@ -185,8 +187,11 @@ // base_dir = new File(System.getProperty("test.build.data", "build/test/data"), "dfs/"); // Some tests also do this: // System.getProperty("test.cache.data", "build/test/cache"); - if (dir == null) this.clusterTestBuildDir = setupClusterTestBuildDir(); - else this.clusterTestBuildDir = dir; + if (dir == null) { + this.clusterTestBuildDir = setupClusterTestBuildDir(); + } else { + this.clusterTestBuildDir = dir; + } System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestBuildDir.toString()); System.setProperty("test.cache.data", this.clusterTestBuildDir.toString()); this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true, @@ -237,7 +242,9 @@ * @see #startMiniZKCluster() */ public void shutdownMiniZKCluster() throws IOException { - if (this.zkCluster != null) this.zkCluster.shutdown(); + if (this.zkCluster != null) { + this.zkCluster.shutdown(); + } } /** @@ -267,10 +274,16 @@ throws Exception { LOG.info("Starting up minicluster"); // If we already put up a cluster, fail. - isRunningCluster(); + String testBuildPath = conf.get("hbase.test.build.dir", null); + isRunningCluster(testBuildPath); + if(testBuildPath != null) { + LOG.info("\n\nUsing passed path: " + testBuildPath + "\n\n"); + } // Make a new random dir to home everything in. Set it as system property. // minidfs reads home from system property. - this.clusterTestBuildDir = setupClusterTestBuildDir(); + this.clusterTestBuildDir = + testBuildPath == null ? + setupClusterTestBuildDir() : new File(testBuildPath); System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestBuildDir.getPath()); // Bring up mini dfs cluster. This spews a bunch of warnings about missing // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'. @@ -282,7 +295,7 @@ // Do old style too just to be safe. this.conf.set("fs.default.name", fs.getUri().toString()); this.dfsCluster.waitClusterUp(); - + // Start up a zk cluster. if (this.zkCluster == null) { startMiniZKCluster(this.clusterTestBuildDir); @@ -297,12 +310,31 @@ // Don't leave here till we've done a successful scan of the .META. HTable t = new HTable(this.conf, HConstants.META_TABLE_NAME); ResultScanner s = t.getScanner(new Scan()); - while (s.next() != null) continue; + while (s.next() != null) { + continue; + } LOG.info("Minicluster is up"); return this.hbaseCluster; } /** + * Starts the hbase cluster up again after shutting it down previously in a + * test. Use this if you want to keep dfs/zk up and just stop/start hbase. + * @param servers number of region servers + * @throws IOException + */ + public void restartHBaseCluster(int servers) throws IOException { + this.hbaseCluster = new MiniHBaseCluster(this.conf, servers); + // Don't leave here till we've done a successful scan of the .META. + HTable t = new HTable(this.conf, HConstants.META_TABLE_NAME); + ResultScanner s = t.getScanner(new Scan()); + while (s.next() != null) { + continue; + } + LOG.info("HBase has been restarted"); + } + + /** * @return Current mini hbase cluster. Only has something in it after a call * to {@link #startMiniCluster()}. * @see #startMiniCluster() @@ -513,7 +545,7 @@ results.close(); return digest.toString(); } - + /** * Creates many regions names "aaa" to "zzz". * @@ -551,7 +583,7 @@ }; return createMultiRegions(c, table, columnFamily, KEYS); } - + public int createMultiRegions(final Configuration c, final HTable table, final byte[] columnFamily, byte [][] startKeys) throws IOException { @@ -567,6 +599,7 @@ // and end key. Adding the custom regions below adds those blindly, // including the new start region from empty to "bbb". lg List rows = getMetaTableRows(htd.getName()); + List newRegions = new ArrayList(startKeys.length); // add custom ones int count = 0; for (int i = 0; i < startKeys.length; i++) { @@ -578,6 +611,7 @@ Writables.getBytes(hri)); meta.put(put); LOG.info("createMultiRegions: inserted " + hri.toString()); + newRegions.add(hri); count++; } // see comment above, remove "old" (or previous) single region @@ -589,6 +623,10 @@ // flush cache of regions HConnection conn = table.getConnection(); conn.clearRegionCache(); + // assign all the new regions + for(HRegionInfo hri : newRegions) { + hbaseCluster.getMaster().assignRegion(hri); + } return count; } @@ -609,7 +647,7 @@ s.close(); return rows; } - + /** * Returns all rows from the .META. table for a given user table * @@ -812,7 +850,7 @@ } public void cleanupTestDir() throws IOException { - getTestDir().getFileSystem(conf).delete(getTestDir(), true); + getTestDir().getFileSystem(conf).delete(getTestDir(), true); } public void waitTableAvailable(byte[] table, long timeoutMillis) @@ -872,16 +910,16 @@ * You'll get a NPE if you call before you've started a minidfscluster. * @param soft Soft limit * @param hard Hard limit - * @throws NoSuchFieldException - * @throws SecurityException - * @throws IllegalAccessException - * @throws IllegalArgumentException + * @throws NoSuchFieldException + * @throws SecurityException + * @throws IllegalAccessException + * @throws IllegalArgumentException */ public void setNameNodeNameSystemLeasePeriod(final int soft, final int hard) throws SecurityException, NoSuchFieldException, IllegalArgumentException, IllegalAccessException { // TODO: If 0.20 hadoop do one thing, if 0.21 hadoop do another. // Not available in 0.20 hdfs. Use reflection to make it happen. - + // private NameNode nameNode; Field field = this.dfsCluster.getClass().getDeclaredField("nameNode"); field.setAccessible(true); @@ -897,10 +935,10 @@ * * @param stream A DFSClient.DFSOutputStream. * @param max - * @throws NoSuchFieldException - * @throws SecurityException - * @throws IllegalAccessException - * @throws IllegalArgumentException + * @throws NoSuchFieldException + * @throws SecurityException + * @throws IllegalAccessException + * @throws IllegalArgumentException */ public static void setMaxRecoveryErrorCount(final OutputStream stream, final int max) { @@ -943,14 +981,18 @@ for (Result r = null; (r = s.next()) != null;) { byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); - if (b == null || b.length <= 0) break; + if (b == null || b.length <= 0) { + break; + } rows++; } s.close(); // If I get to here and all rows have a Server, then all have been assigned. - if (rows == countOfRegions) break; + if (rows == countOfRegions) { + break; + } LOG.info("Found=" + rows); - Threads.sleep(1000); + Threads.sleep(1000); } } } \ No newline at end of file Index: src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressManager.java (working copy) @@ -63,7 +63,7 @@ // Should not have a master yet MasterAddressManager addressManager = new MasterAddressManager(zk, null); - addressManager.monitorMaster(); + addressManager.start(); assertFalse(addressManager.hasMaster()); zk.registerListener(addressManager); Index: src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (working copy) @@ -19,6 +19,13 @@ */ package org.apache.hadoop.hbase; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +import java.io.IOException; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -32,22 +39,13 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; -import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import java.io.IOException; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; - public class TestZooKeeper { private final Log LOG = LogFactory.getLog(this.getClass()); Index: src/test/java/org/apache/hadoop/hbase/master/TestZKBasedReopenRegion.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/master/TestZKBasedReopenRegion.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/master/TestZKBasedReopenRegion.java (working copy) @@ -27,24 +27,22 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; 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.master.HMaster; -import org.apache.hadoop.hbase.master.ProcessRegionClose; -import org.apache.hadoop.hbase.master.ProcessRegionOpen; -import org.apache.hadoop.hbase.master.RegionServerOperation; -import org.apache.hadoop.hbase.master.RegionServerOperationListener; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener; +import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; +import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Writables; @@ -83,7 +81,7 @@ // Need at least two servers. LOG.info("Started new server=" + TEST_UTIL.getHBaseCluster().startRegionServer()); - + } } @@ -95,23 +93,29 @@ int rsIdx = 0; HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx); Collection regions = regionServer.getOnlineRegions(); - HRegion region = regions.iterator().next(); + HRegion region; + while((region = regions.iterator().next()) != null) { + if(!region.getRegionInfo().isMetaRegion() && !region.getRegionInfo().isRootRegion()) { + break; + } + } LOG.debug("Asking RS to close region " + region.getRegionNameAsString()); AtomicBoolean closeEventProcessed = new AtomicBoolean(false); AtomicBoolean reopenEventProcessed = new AtomicBoolean(false); - RegionServerOperationListener listener = - new ReopenRegionEventListener(region.getRegionNameAsString(), - closeEventProcessed, - reopenEventProcessed); - HMaster master = TEST_UTIL.getHBaseCluster().getMaster(); - master.getRegionServerOperationQueue().registerRegionServerOperationListener(listener); - HMsg closeRegionMsg = new HMsg(HMsg.Type.MSG_REGION_CLOSE, - region.getRegionInfo(), - Bytes.toBytes("Forcing close in test") - ); - TEST_UTIL.getHBaseCluster().addMessageToSendRegionServer(rsIdx, closeRegionMsg); - + + EventHandlerListener closeListener = + new RegionEventListener(region.getRegionNameAsString(), + closeEventProcessed, EventType.M2RS_CLOSE_REGION); + EventHandler.registerListener(closeListener); + + EventHandlerListener openListener = + new RegionEventListener(region.getRegionNameAsString(), + reopenEventProcessed, EventType.M2RS_OPEN_REGION); + EventHandler.registerListener(openListener); + + regionServer.closeRegion(region.getRegionInfo()); + synchronized(closeEventProcessed) { closeEventProcessed.wait(3*60*1000); } @@ -124,69 +128,54 @@ } if(!reopenEventProcessed.get()) { throw new Exception("Timed out, open event not called on master after region close."); - } - - LOG.info("Done with test, RS informed master successfully."); + } + + LOG.info("\n\n\nDone with test, RS informed master successfully.\n\n\n"); } - - public static class ReopenRegionEventListener implements RegionServerOperationListener { - - private static final Log LOG = LogFactory.getLog(ReopenRegionEventListener.class); - String regionToClose; - AtomicBoolean closeEventProcessed; - AtomicBoolean reopenEventProcessed; - public ReopenRegionEventListener(String regionToClose, - AtomicBoolean closeEventProcessed, - AtomicBoolean reopenEventProcessed) { - this.regionToClose = regionToClose; - this.closeEventProcessed = closeEventProcessed; - this.reopenEventProcessed = reopenEventProcessed; - } - @Override - public boolean process(HServerInfo serverInfo, HMsg incomingMsg) { - return true; - } + public static class RegionEventListener implements EventHandlerListener { + private static final Log LOG = LogFactory.getLog(RegionEventListener.class); + String regionName; + AtomicBoolean eventProcessed; + EventType eventType; - @Override - public boolean process(RegionServerOperation op) throws IOException { - return true; + public RegionEventListener(String regionName, + AtomicBoolean eventProcessed, EventType eventType) { + this.regionName = regionName; + this.eventProcessed = eventProcessed; + this.eventType = eventType; } @Override - public void processed(RegionServerOperation op) { - LOG.debug("Master processing object: " + op.getClass().getCanonicalName()); - if(op instanceof ProcessRegionClose) { - ProcessRegionClose regionCloseOp = (ProcessRegionClose)op; - String region = regionCloseOp.getRegionInfo().getRegionNameAsString(); - LOG.debug("Finished closing region " + region + ", expected to close region " + regionToClose); - if(regionToClose.equals(region)) { - closeEventProcessed.set(true); + public void afterProcess(EventHandler event) { + LOG.info("afterProcess(" + event + ")"); + if(event.getEventType() == eventType) { + LOG.info("Finished processing " + eventType); + String regionName = ""; + if(eventType == EventType.M2RS_OPEN_REGION) { + OpenRegionHandler openHandler = (OpenRegionHandler)event; + regionName = openHandler.getRegionInfo().getRegionNameAsString(); + } else if(eventType == EventType.M2RS_CLOSE_REGION) { + CloseRegionHandler closeHandler = (CloseRegionHandler)event; + regionName = closeHandler.getRegionInfo().getRegionNameAsString(); } - synchronized(closeEventProcessed) { - closeEventProcessed.notifyAll(); + if(this.regionName.equals(regionName)) { + eventProcessed.set(true); } + synchronized(eventProcessed) { + eventProcessed.notifyAll(); + } } - // Wait for open event AFTER we have closed the region - if(closeEventProcessed.get()) { - if(op instanceof ProcessRegionOpen) { - ProcessRegionOpen regionOpenOp = (ProcessRegionOpen)op; - String region = regionOpenOp.getRegionInfo().getRegionNameAsString(); - LOG.debug("Finished closing region " + region + ", expected to close region " + regionToClose); - if(regionToClose.equals(region)) { - reopenEventProcessed.set(true); - } - synchronized(reopenEventProcessed) { - reopenEventProcessed.notifyAll(); - } - } + } + + @Override + public void beforeProcess(EventHandler event) { + if(event.getEventType() == eventType) { + LOG.info("Received " + eventType + " and beginning to process it"); } - } - } - private static void waitUntilAllRegionsAssigned(final int countOfRegions) throws IOException { @@ -200,14 +189,18 @@ for (Result r = null; (r = s.next()) != null;) { byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); - if (b == null || b.length <= 0) break; + if (b == null || b.length <= 0) { + break; + } rows++; } s.close(); // If I get to here and all rows have a Server, then all have been assigned. - if (rows == countOfRegions) break; + if (rows == countOfRegions) { + break; + } LOG.info("Found=" + rows); - Threads.sleep(1000); + Threads.sleep(1000); } } @@ -229,7 +222,9 @@ for (Result r = null; (r = s.next()) != null;) { byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - if (b == null || b.length <= 0) break; + if (b == null || b.length <= 0) { + break; + } HRegionInfo hri = Writables.getHRegionInfo(b); // If start key, add 'aaa'. byte [] row = getStartKey(hri); @@ -255,14 +250,14 @@ private static byte [] getTestQualifier() { return getTestFamily(); } - + public static void main(String args[]) throws Exception { TestZKBasedReopenRegion.beforeAllTests(); - + TestZKBasedReopenRegion test = new TestZKBasedReopenRegion(); test.setup(); test.testOpenRegion(); - + TestZKBasedReopenRegion.afterAllTests(); } } Index: src/test/java/org/apache/hadoop/hbase/master/TestZKBasedCloseRegion.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/master/TestZKBasedCloseRegion.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/master/TestZKBasedCloseRegion.java (working copy) @@ -27,23 +27,21 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.HMsg; +import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.MiniHBaseCluster; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; 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.master.HMaster; -import org.apache.hadoop.hbase.master.ProcessRegionClose; -import org.apache.hadoop.hbase.master.RegionServerOperation; -import org.apache.hadoop.hbase.master.RegionServerOperationListener; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener; +import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Writables; @@ -82,7 +80,7 @@ // Need at least two servers. LOG.info("Started new server=" + TEST_UTIL.getHBaseCluster().startRegionServer()); - + } } @@ -95,20 +93,22 @@ int rsIdx = 0; HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx); Collection regions = regionServer.getOnlineRegions(); - HRegion region = regions.iterator().next(); + HRegion region; + while((region = regions.iterator().next()) != null) { + if(!region.getRegionInfo().isMetaRegion() && !region.getRegionInfo().isRootRegion()) { + break; + } + } LOG.debug("Asking RS to close region " + region.getRegionNameAsString()); AtomicBoolean closeEventProcessed = new AtomicBoolean(false); - RegionServerOperationListener listener = - new CloseRegionEventListener(region.getRegionNameAsString(), closeEventProcessed); - HMaster master = TEST_UTIL.getHBaseCluster().getMaster(); - master.getRegionServerOperationQueue().registerRegionServerOperationListener(listener); - HMsg closeRegionMsg = new HMsg(HMsg.Type.MSG_REGION_CLOSE, - region.getRegionInfo(), - Bytes.toBytes("Forcing close in test") - ); - TEST_UTIL.getHBaseCluster().addMessageToSendRegionServer(rsIdx, closeRegionMsg); - + EventHandlerListener listener = + new CloseRegionEventListener(region.getRegionNameAsString(), + closeEventProcessed); + EventHandler.registerListener(listener); + + regionServer.closeRegion(region.getRegionInfo()); + synchronized(closeEventProcessed) { // wait for 3 minutes closeEventProcessed.wait(3*60*1000); @@ -120,36 +120,25 @@ LOG.info("Done with test, RS informed master successfully."); } } - - public static class CloseRegionEventListener implements RegionServerOperationListener { - + + public static class CloseRegionEventListener implements EventHandlerListener { private static final Log LOG = LogFactory.getLog(CloseRegionEventListener.class); String regionToClose; AtomicBoolean closeEventProcessed; - public CloseRegionEventListener(String regionToClose, AtomicBoolean closeEventProcessed) { + public CloseRegionEventListener(String regionToClose, + AtomicBoolean closeEventProcessed) { this.regionToClose = regionToClose; this.closeEventProcessed = closeEventProcessed; } @Override - public boolean process(HServerInfo serverInfo, HMsg incomingMsg) { - return true; - } - - @Override - public boolean process(RegionServerOperation op) throws IOException { - return true; - } - - @Override - public void processed(RegionServerOperation op) { - LOG.debug("Master processing object: " + op.getClass().getCanonicalName()); - if(op instanceof ProcessRegionClose) { - ProcessRegionClose regionCloseOp = (ProcessRegionClose)op; - String region = regionCloseOp.getRegionInfo().getRegionNameAsString(); - LOG.debug("Finished closing region " + region + ", expected to close region " + regionToClose); - if(regionToClose.equals(region)) { + public void afterProcess(EventHandler event) { + LOG.info("afterProcess(" + event + ")"); + if(event.getEventType() == EventType.M2RS_CLOSE_REGION) { + LOG.info("Finished processing CLOSE REGION"); + CloseRegionHandler closeHandler = (CloseRegionHandler)event; + if(regionToClose.equals(closeHandler.getRegionInfo().getRegionNameAsString())) { closeEventProcessed.set(true); } synchronized(closeEventProcessed) { @@ -157,9 +146,14 @@ } } } - + + @Override + public void beforeProcess(EventHandler event) { + if(event.getEventType() == EventType.M2RS_CLOSE_REGION) { + LOG.info("Received CLOSE RPC and beginning to process it"); + } + } } - private static void waitUntilAllRegionsAssigned(final int countOfRegions) throws IOException { @@ -173,14 +167,18 @@ for (Result r = null; (r = s.next()) != null;) { byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); - if (b == null || b.length <= 0) break; + if (b == null || b.length <= 0) { + break; + } rows++; } s.close(); // If I get to here and all rows have a Server, then all have been assigned. - if (rows == countOfRegions) break; + if (rows == countOfRegions) { + break; + } LOG.info("Found=" + rows); - Threads.sleep(1000); + Threads.sleep(1000); } } @@ -202,7 +200,9 @@ for (Result r = null; (r = s.next()) != null;) { byte [] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - if (b == null || b.length <= 0) break; + if (b == null || b.length <= 0) { + break; + } HRegionInfo hri = Writables.getHRegionInfo(b); // If start key, add 'aaa'. byte [] row = getStartKey(hri); @@ -228,14 +228,14 @@ private static byte [] getTestQualifier() { return getTestFamily(); } - + public static void main(String args[]) throws Exception { TestZKBasedCloseRegion.beforeAllTests(); - + TestZKBasedCloseRegion test = new TestZKBasedCloseRegion(); test.setup(); test.testCloseRegion(); - + TestZKBasedCloseRegion.afterAllTests(); } } Index: src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java (working copy) @@ -189,7 +189,7 @@ } } - public static class DummyMasterStatus implements MasterStatus { + public static class DummyMasterStatus implements MasterController { private AtomicBoolean closed = new AtomicBoolean(false); @@ -204,11 +204,6 @@ } @Override - public RegionManager getRegionManager() { - return null; - } - - @Override public ServerConnection getServerConnection() { return null; } @@ -265,5 +260,20 @@ return null; } + @Override + public String getServerName() { + return null; + } + + @Override + public boolean isRegionServer() { + return false; + } + + @Override + public long getTimeout() { + return 0; + } + } } Index: src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (working copy) @@ -19,6 +19,8 @@ */ package org.apache.hadoop.hbase.master; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import org.apache.commons.logging.Log; @@ -27,7 +29,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; +import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKUtil; @@ -45,18 +48,68 @@ private static final byte[] TABLENAME = Bytes.toBytes("master_transitions"); private static final byte [][] FAMILIES = new byte [][] {Bytes.toBytes("a")}; + + private static final byte [][] TABLES = new byte[][] { + Bytes.toBytes("restartTableOne"), + Bytes.toBytes("restartTableTwo"), + Bytes.toBytes("restartTableThree") + }; + private static final byte [] FAMILY = Bytes.toBytes("family"); + @BeforeClass public static void beforeAllTests() throws Exception { conf = HBaseConfiguration.create(); utility = new HBaseTestingUtility(conf); } @AfterClass public static void afterAllTests() throws IOException { - utility.shutdownMiniCluster(); +// utility.shutdownMiniCluster(); } @Before public void setup() throws IOException { } + @Test (timeout=300000) + public void testClusterRestart() throws Exception { + + utility.getConfiguration().set("hbase.test.build.dir", + utility.setupClusterTestBuildDir().getAbsolutePath()); + + LOG.info("\n\nStarting cluster the first time"); + utility.startMiniCluster(3); + + LOG.info("\n\nCreating tables"); + for(byte [] TABLE : TABLES) { + utility.createTable(TABLE, FAMILY); + utility.waitTableAvailable(TABLE, 30000); + } + + LOG.info("\n\nShutting down cluster"); + utility.getHBaseCluster().shutdown(); + utility.getHBaseCluster().join(); + + LOG.info("\n\nSleeping a bit"); + Thread.sleep(2000); + + LOG.info("\n\nStarting cluster the second time"); + utility.restartHBaseCluster(3); + + LOG.info("\n\nWaiting for tables to be available"); + for(byte [] TABLE : TABLES) { + try { + utility.createTable(TABLE, FAMILY); + assertTrue("Able to create table that should already exist", false); + } catch(TableExistsException tee) { + LOG.info("Table already exists as expected"); + } + utility.waitTableAvailable(TABLE, 30000); + } + + LOG.info("\n\nShutting stuff down now"); + utility.shutdownMiniCluster(); + + LOG.info("\n\nDone!"); + } + @Test (timeout=300000) public void testRestartClusterAfterKill()throws Exception { utility.startMiniZKCluster(); zooKeeper = new ZooKeeperWatcher(conf, "cluster1", null); @@ -72,7 +125,7 @@ HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(), HMaster.MASTER); LOG.debug("Created UNASSIGNED zNode for ROOT and META regions in state " + - HBaseEventType.M2ZK_REGION_OFFLINE); + EventType.M2ZK_REGION_OFFLINE); // start the HB cluster LOG.info("Starting HBase cluster..."); Index: src/test/java/org/apache/hadoop/hbase/master/TestMaster.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/master/TestMaster.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/master/TestMaster.java (working copy) @@ -23,16 +23,15 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MiniHBaseCluster; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.executor.HBaseEventHandler; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventHandlerListener; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener; +import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -76,7 +75,8 @@ TEST_UTIL.loadTable(new HTable(TABLENAME), FAMILYNAME); List> tableRegions = - m.getTableRegions(TABLENAME); + MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(), + Bytes.toString(TABLENAME)); LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions)); assertEquals(1, tableRegions.size()); assertArrayEquals(HConstants.EMPTY_START_ROW, @@ -85,11 +85,11 @@ tableRegions.get(0).getFirst().getEndKey()); // Now trigger a split and stop when the split is in progress - + CountDownLatch aboutToOpen = new CountDownLatch(1); CountDownLatch proceed = new CountDownLatch(1); RegionOpenListener list = new RegionOpenListener(aboutToOpen, proceed); - HBaseEventHandler.registerListener(list); + EventHandler.registerListener(list); LOG.info("Splitting table"); admin.split(TABLENAME); @@ -97,7 +97,9 @@ aboutToOpen.await(60, TimeUnit.SECONDS); try { LOG.info("Making sure we can call getTableRegions while opening"); - tableRegions = m.getTableRegions(TABLENAME); + tableRegions = MetaReader.getTableRegionsAndLocations( + m.getCatalogTracker(), Bytes.toString(TABLENAME)); + LOG.info("Regions: " + Joiner.on(',').join(tableRegions)); // We have three regions because one is split-in-progress assertEquals(3, tableRegions.size()); @@ -105,14 +107,16 @@ Pair pair = m.getTableRegionForRow(TABLENAME, Bytes.toBytes("cde")); LOG.info("Result is: " + pair); - Pair tableRegionFromName = m.getTableRegionFromName(pair.getFirst().getRegionName()); + Pair tableRegionFromName = + MetaReader.getRegion(m.getCatalogTracker(), + pair.getFirst().getRegionName()); assertEquals(tableRegionFromName.getFirst(), pair.getFirst()); } finally { proceed.countDown(); } } - static class RegionOpenListener implements HBaseEventHandlerListener { + static class RegionOpenListener implements EventHandlerListener { CountDownLatch aboutToOpen, proceed; public RegionOpenListener(CountDownLatch aboutToOpen, CountDownLatch proceed) @@ -122,8 +126,8 @@ } @Override - public void afterProcess(HBaseEventHandler event) { - if (event.getHBEvent() != HBaseEventType.RS2ZK_REGION_OPENED) { + public void afterProcess(EventHandler event) { + if (event.getEventType() != EventType.RS2ZK_REGION_OPENED) { return; } try { @@ -136,7 +140,7 @@ } @Override - public void beforeProcess(HBaseEventHandler event) { + public void beforeProcess(EventHandler event) { } } Index: src/test/java/org/apache/hadoop/hbase/util/TestBytes.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/util/TestBytes.java (revision 979909) +++ src/test/java/org/apache/hadoop/hbase/util/TestBytes.java (working copy) @@ -93,6 +93,14 @@ // If split more than once, this should fail parts = Bytes.split(low, high, 2); assertTrue("Returned split but should have failed", parts == null); + + // Split 0 times should throw IAE + try { + parts = Bytes.split(low, high, 0); + assertTrue("Should not be able to split 0 times", false); + } catch(IllegalArgumentException iae) { + // Correct + } } public void testToLong() throws Exception { Index: src/main/java/org/apache/hadoop/hbase/ServerController.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/ServerController.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/ServerController.java (working copy) @@ -20,25 +20,44 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.ServerConnection; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; /** - * Set of functions that are exposed by any HBase process (implemented by the - * master, region server, and client). + * Defines the set of shared functions implemented by all HBase servers (Masters + * and RegionServers). */ public interface ServerController extends Abortable { /** - * Return the address of the current server. + * Returns the address of the current server. */ public HServerAddress getHServerAddress(); /** - * Get the configuration object for this server. + * Gets the configuration object for this server. */ public Configuration getConfiguration(); /** - * Get the ZooKeeper instance for this server. + * Gets the ZooKeeper instance for this server. */ public ZooKeeperWatcher getZooKeeper(); + + /** + * Gets the unique server name for this server. + * @return unique server name + */ + public String getServerName(); + + /** + * Return the server RPC connection object + */ + public ServerConnection getServerConnection(); + + /** + * Returns the timeout to use for operations such as waiting on root and meta + * availability. + * @return default timeout to wait for root and meta + */ + public long getTimeout(); } Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableDisable.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableDisable.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableDisable.java (revision 0) @@ -0,0 +1,70 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.zookeeper; + +import java.util.List; + +import org.apache.zookeeper.KeeperException; + +/** + * Helper class for table disable tracking in zookeeper. + *

+ * The node /disabled will contain a child node for every table which should be + * disabled, for example, /disabled/table. + */ +public class ZKTableDisable { + + /** + * Sets the specified table as disabled in zookeeper. Fails silently if the + * table is already disabled in zookeeper. Sets no watches. + * @param zkw + * @param tableName + * @throws KeeperException unexpected zookeeper exception + */ + public static void disableTable(ZooKeeperWatcher zkw, String tableName) + throws KeeperException { + ZKUtil.createAndFailSilent(zkw, ZKUtil.joinZNode(zkw.tableZNode, + tableName)); + } + + /** + * Unsets the specified table as disabled in zookeeper. Fails silently if the + * table is not currently disabled in zookeeper. Sets no watches. + * @param zkw + * @param tableName + * @throws KeeperException unexpected zookeeper exception + */ + public static void undisableTable(ZooKeeperWatcher zkw, String tableName) + throws KeeperException { + ZKUtil.deleteNodeFailSilent(zkw, ZKUtil.joinZNode(zkw.tableZNode, + tableName)); + } + + /** + * Gets a list of all the tables set as disabled in zookeeper. + * @param zkw + * @return list of disabled tables, empty list if none + * @throws KeeperException + */ + public static List getDisabledTables(ZooKeeperWatcher zkw) + throws KeeperException { + return ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); + } +} Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java (working copy) @@ -69,7 +69,12 @@ */ public void setClusterDown() throws KeeperException { - ZKUtil.deleteNode(watcher, watcher.clusterStateZNode); + try { + ZKUtil.deleteNode(watcher, watcher.clusterStateZNode); + } catch(KeeperException.NoNodeException nne) { + LOG.warn("Attempted to set cluster as down but already down, cluster " + + "state node (" + watcher.clusterStateZNode + ") not found"); + } } @Override Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (working copy) @@ -81,6 +81,8 @@ public String clusterStateZNode; // znode used for region transitioning and assignment public String assignmentZNode; + // znode used for table disabling/enabling + public String tableZNode; /** * Instantiate a ZooKeeper connection and watcher. @@ -103,6 +105,7 @@ ZKUtil.createAndFailSilent(this, baseZNode); ZKUtil.createAndFailSilent(this, assignmentZNode); ZKUtil.createAndFailSilent(this, rsZNode); + ZKUtil.createAndFailSilent(this, tableZNode); } catch (KeeperException e) { error("Unexpected KeeperException creating base node", e); error("Message: " + e.getMessage()); @@ -125,7 +128,9 @@ clusterStateZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.state", "shutdown")); assignmentZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.regionInTransition", "unassigned")); + conf.get("zookeeper.znode.unassigned", "unassigned")); + tableZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.tableEnableDisable", "table")); } /** Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (working copy) @@ -207,22 +207,22 @@ * * @param zkw zk reference * @param znode path of node to watch - * @return true if znode exists, false if does not exist or error + * @return version of the node if it exists, -1 if does not exist * @throws KeeperException if unexpected zookeeper exception */ - public static boolean checkExists(ZooKeeperWatcher zkw, String znode) + public static int checkExists(ZooKeeperWatcher zkw, String znode) throws KeeperException { try { Stat s = zkw.getZooKeeper().exists(znode, null); - return s != null ? true : false; + return s != null ? s.getVersion() : -1; } catch (KeeperException e) { zkw.warn("Unable to set watcher on znode (" + znode + ")", e); zkw.keeperException(e); - return false; + return -1; } catch (InterruptedException e) { zkw.warn("Unable to set watcher on znode (" + znode + ")", e); zkw.interruptedException(e); - return false; + return -1; } } @@ -370,6 +370,7 @@ public byte [] getData() { return data; } + @Override public String toString() { return node + " (" + RegionTransitionData.fromBytes(data) + ")"; } @@ -732,21 +733,25 @@ * *

The node created is persistent and open access. * + *

Returns the version number of the created node if successful. + * * @param zkw zk reference * @param znode path of node to create * @param data data of node to create + * @return version of node created * @throws KeeperException if unexpected zookeeper exception * @throws KeeperException.NodeExistsException if node already exists */ - public static void createAndWatch(ZooKeeperWatcher zkw, + public static int createAndWatch(ZooKeeperWatcher zkw, String znode, byte [] data) throws KeeperException, KeeperException.NodeExistsException { try { zkw.getZooKeeper().create(znode, data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zkw.getZooKeeper().exists(znode, zkw); + return zkw.getZooKeeper().exists(znode, zkw).getVersion(); } catch (InterruptedException e) { zkw.interruptedException(e); + return -1; } } @@ -833,6 +838,22 @@ } /** + * Deletes the specified node. Fails silent if the node does not exist. + * @param zkw + * @param joinZNode + * @throws KeeperException + */ + public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node) + throws KeeperException { + try { + zkw.getZooKeeper().delete(node, -1); + } catch(KeeperException.NoNodeException nne) { + } catch(InterruptedException ie) { + zkw.interruptedException(ie); + } + } + + /** * Delete the specified node and all of it's children. * * Sets no watches. Throws all exceptions besides dealing with deletion of Index: src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/RootRegionTracker.java (working copy) @@ -24,8 +24,10 @@ import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.zookeeper.KeeperException; +/** + * Tracks the root region server location node in zookeeper. + */ public class RootRegionTracker extends ZooKeeperNodeTracker { private static final Log LOG = LogFactory.getLog(RootRegionTracker.class); @@ -60,19 +62,16 @@ } /** - * Sets the root region location. - * @param address - * @throws KeeperException unexpected zk exception + * Gets the root region location, if available, and waits for up to the + * specified timeout if not immediately available. + * @param timeout maximum time to wait, in millis, use 0 for forever + * @return server address for server hosting root region, null if timed out + * @throws InterruptedException if interrupted while waiting */ - public void setRootRegionLocation(HServerAddress address) - throws KeeperException { - try { - ZKUtil.createAndWatch(watcher, watcher.rootServerZNode, - Bytes.toBytes(address.toString())); - } catch(KeeperException.NodeExistsException nee) { - ZKUtil.setData(watcher, watcher.rootServerZNode, - Bytes.toBytes(address.toString())); - } + public HServerAddress waitRootRegionLocation(long timeout) + throws InterruptedException { + byte [] data = super.blockUntilAvailable(timeout); + return data == null ? null : new HServerAddress(Bytes.toString(data)); } @Override Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java (working copy) @@ -1,1149 +0,0 @@ -/** - * Copyright 2009 The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.zookeeper; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.PrintWriter; -import java.net.Socket; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.ZooDefs.Ids; -import org.apache.zookeeper.ZooKeeper.States; -import org.apache.zookeeper.data.Stat; - -/** - * Wraps a ZooKeeper instance and adds HBase specific functionality. - * - * This class provides methods to: - * - read/write/delete the root region location in ZooKeeper. - * - set/check out of safe mode flag. - * - * ------------------------------------------ - * The following STATIC ZNodes are created: - * ------------------------------------------ - * - parentZNode : All the HBase directories are hosted under this parent - * node, default = "/hbase" - * - rsZNode : This is the directory where the RS's create ephemeral - * nodes. The master watches these nodes, and their expiry - * indicates RS death. The default location is "/hbase/rs" - * - * ------------------------------------------ - * The following DYNAMIC ZNodes are created: - * ------------------------------------------ - * - rootRegionZNode : Specifies the RS hosting root. - * - masterElectionZNode : ZNode used for election of the primary master when - * there are secondaries. All the masters race to write - * their addresses into this location, the one that - * succeeds is the primary. Others block. - * - clusterStateZNode : Determines if the cluster is running. Its default - * location is "/hbase/shutdown". It always has a value - * of "up". If present with the valus, cluster is up - * and running. If deleted, the cluster is shutting - * down. - * - rgnsInTransitZNode : All the nodes under this node are names of regions - * in transition. The first byte of the data for each - * of these nodes is the event type. This is used to - * deserialize the rest of the data. - * - * TODO: Eventually this class will be removed and everything put into ZKUtil - * and ZooKeeperWatcher. Can rename stuff once it's done. - */ -public class ZooKeeperWrapper implements Watcher { - protected static final Log LOG = LogFactory.getLog(ZooKeeperWrapper.class); - - // name of this instance - private String instanceName; - - // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved. - private static final char ZNODE_PATH_SEPARATOR = '/'; - - private String quorumServers = null; - private final int sessionTimeout; - private ZooKeeper zooKeeper; - - /* - * All the HBase directories are hosted under this parent - */ - public final String parentZNode; - /* - * Specifies the RS hosting root - */ - private final String rootRegionZNode; - /* - * This is the directory where the RS's create ephemeral nodes. The master - * watches these nodes, and their expiry indicates RS death. - */ - private final String rsZNode; - /* - * ZNode used for election of the primary master when there are secondaries. - */ - private final String masterElectionZNode; - /* - * State of the cluster - if up and running or shutting down - */ - public final String clusterStateZNode; - /* - * Regions that are in transition - */ - private final String rgnsInTransitZNode; - /* - * List of ZNodes in the unassgined region that are already being watched - */ - private Set unassignedZNodesWatched = new HashSet(); - - private List listeners = Collections.synchronizedList(new ArrayList()); - - /** - * Create a ZooKeeperWrapper. The Zookeeper wrapper listens to all messages - * from Zookeeper, and notifies all the listeners about all the messages. Any - * component can subscribe to these messages by adding itself as a listener, - * and remove itself from being a listener. - * - * @param conf HBaseConfiguration to read settings from. - * @param instanceName name of this zk instance, used for debugging - * @throws IOException If a connection error occurs. - */ - protected ZooKeeperWrapper(Configuration conf, String instanceName) - throws IOException { - this.instanceName = instanceName; - Properties properties = ZKConfig.makeZKProps(conf); - quorumServers = ZKConfig.getZKQuorumServersString(properties); - if (quorumServers == null) { - throw new IOException("Could not read quorum servers from " + - HConstants.ZOOKEEPER_CONFIG_NAME); - } - sessionTimeout = conf.getInt("zookeeper.session.timeout", 60 * 1000); - reconnectToZk(); - - parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - - String rootServerZNodeName = conf.get("zookeeper.znode.rootserver", "root-region-server"); - String rsZNodeName = conf.get("zookeeper.znode.rs", "rs"); - String masterAddressZNodeName = conf.get("zookeeper.znode.master", "master"); - String stateZNodeName = conf.get("zookeeper.znode.state", "shutdown"); - String regionsInTransitZNodeName = conf.get("zookeeper.znode.regionInTransition", "unassigned"); - - rootRegionZNode = getZNode(parentZNode, rootServerZNodeName); - rsZNode = getZNode(parentZNode, rsZNodeName); - rgnsInTransitZNode = getZNode(parentZNode, regionsInTransitZNodeName); - masterElectionZNode = getZNode(parentZNode, masterAddressZNodeName); - clusterStateZNode = getZNode(parentZNode, stateZNodeName); - } - - public void reconnectToZk() throws IOException { - try { - LOG.info("Reconnecting to zookeeper"); - if(zooKeeper != null) { - zooKeeper.close(); - LOG.debug("<" + instanceName + ">" + "Closed existing zookeeper client"); - } - zooKeeper = new ZooKeeper(quorumServers, sessionTimeout, this); - LOG.debug("<" + instanceName + ">" + "Connected to zookeeper again"); - } catch (IOException e) { - LOG.error("<" + instanceName + ">" + "Failed to create ZooKeeper object: " + e); - throw new IOException(e); - } catch (InterruptedException e) { - LOG.error("<" + instanceName + ">" + "Error closing ZK connection: " + e); - throw new IOException(e); - } - } - - public synchronized void registerListener(Watcher watcher) { - listeners.add(watcher); - } - - public synchronized void unregisterListener(Watcher watcher) { - listeners.remove(watcher); - } - - /** - * This is the primary ZK watcher - * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatchedEvent) - */ - @Override - public synchronized void process(WatchedEvent event) { - for(Watcher w : listeners) { - try { - w.process(event); - } catch (Throwable t) { - LOG.error("<"+instanceName+">" + "ZK updates listener threw an exception in process()", t); - } - } - } - - /** @return String dump of everything in ZooKeeper. */ - public String dump() { - StringBuilder sb = new StringBuilder(); - sb.append("\nHBase tree in ZooKeeper is rooted at ").append(parentZNode); - sb.append("\n Cluster up? ").append(exists(clusterStateZNode, true)); - sb.append("\n Master address: ").append(readMasterAddress(null)); - sb.append("\n Region server holding ROOT: ").append(readRootRegionLocation()); - sb.append("\n Region servers:"); - for (HServerAddress address : scanRSDirectory()) { - sb.append("\n - ").append(address); - } - sb.append("\n Quorum Server Statistics:"); - String[] servers = quorumServers.split(","); - for (String server : servers) { - sb.append("\n - ").append(server); - try { - String[] stat = getServerStats(server); - for (String s : stat) { - sb.append("\n ").append(s); - } - } catch (Exception e) { - sb.append("\n ERROR: ").append(e.getMessage()); - } - } - return sb.toString(); - } - - /** - * Gets the statistics from the given server. Uses a 1 minute timeout. - * - * @param server The server to get the statistics from. - * @return The array of response strings. - * @throws IOException When the socket communication fails. - */ - public String[] getServerStats(String server) - throws IOException { - return getServerStats(server, 60 * 1000); - } - - /** - * Gets the statistics from the given server. - * - * @param server The server to get the statistics from. - * @param timeout The socket timeout to use. - * @return The array of response strings. - * @throws IOException When the socket communication fails. - */ - public String[] getServerStats(String server, int timeout) - throws IOException { - String[] sp = server.split(":"); - Socket socket = new Socket(sp[0], - sp.length > 1 ? Integer.parseInt(sp[1]) : 2181); - socket.setSoTimeout(timeout); - PrintWriter out = new PrintWriter(socket.getOutputStream(), true); - BufferedReader in = new BufferedReader(new InputStreamReader( - socket.getInputStream())); - out.println("stat"); - out.flush(); - ArrayList res = new ArrayList(); - while (true) { - String line = in.readLine(); - if (line != null) res.add(line); - else break; - } - socket.close(); - return res.toArray(new String[res.size()]); - } - - public boolean exists(String znode, boolean watch) { - try { - return zooKeeper.exists(getZNode(parentZNode, znode), watch?this:null) != null; - } catch (KeeperException.SessionExpiredException e) { - // if the session has expired try to reconnect to ZK, then perform query - try { - // TODO: ZK-REFACTOR: We should not reconnect - we should just quit and restart. - reconnectToZk(); - return zooKeeper.exists(getZNode(parentZNode, znode), watch?this:null) != null; - } catch (IOException e1) { - LOG.error("Error reconnecting to zookeeper", e1); - throw new RuntimeException("Error reconnecting to zookeeper", e1); - } catch (KeeperException e1) { - LOG.error("Error reading after reconnecting to zookeeper", e1); - throw new RuntimeException("Error reading after reconnecting to zookeeper", e1); - } catch (InterruptedException e1) { - LOG.error("Error reading after reconnecting to zookeeper", e1); - throw new RuntimeException("Error reading after reconnecting to zookeeper", e1); - } - } catch (KeeperException e) { - return false; - } catch (InterruptedException e) { - return false; - } - } - - /** @return ZooKeeper used by this wrapper. */ - public ZooKeeper getZooKeeper() { - return zooKeeper; - } - - /** - * This is for testing KeeperException.SessionExpiredException. - * See HBASE-1232. - * @return long session ID of this ZooKeeper session. - */ - public long getSessionID() { - return zooKeeper.getSessionId(); - } - - /** - * This is for testing KeeperException.SessionExpiredException. - * See HBASE-1232. - * @return byte[] password of this ZooKeeper session. - */ - public byte[] getSessionPassword() { - return zooKeeper.getSessionPasswd(); - } - - /** @return host:port list of quorum servers. */ - public String getQuorumServers() { - return quorumServers; - } - - /** @return true if currently connected to ZooKeeper, false otherwise. */ - public boolean isConnected() { - return zooKeeper.getState() == States.CONNECTED; - } - - /** - * Read location of server storing root region. - * @return HServerAddress pointing to server serving root region or null if - * there was a problem reading the ZNode. - */ - public HServerAddress readRootRegionLocation() { - return readAddress(rootRegionZNode, null); - } - - /** - * Read address of master server. - * @return HServerAddress of master server. - * @throws IOException if there's a problem reading the ZNode. - */ - public HServerAddress readMasterAddressOrThrow() throws IOException { - return readAddressOrThrow(masterElectionZNode, null); - } - - /** - * Read master address and set a watch on it. - * @param watcher Watcher to set on master address ZNode if not null. - * @return HServerAddress of master or null if there was a problem reading the - * ZNode. The watcher is set only if the result is not null. - */ - public HServerAddress readMasterAddress(Watcher watcher) { - return readAddress(masterElectionZNode, watcher); - } - - /** - * Watch the state of the cluster, up or down - * @param watcher Watcher to set on cluster state node - */ - public void setClusterStateWatch() { - try { - zooKeeper.exists(clusterStateZNode, this); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to check on ZNode " + clusterStateZNode, e); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to check on ZNode " + clusterStateZNode, e); - } - } - - /** - * Set the cluster state, up or down - * @param up True to write the node, false to delete it - * @return true if it worked, else it's false - */ - public boolean setClusterState(boolean up) { - if (!ensureParentExists(clusterStateZNode)) { - return false; - } - try { - if(up) { - byte[] data = Bytes.toBytes("up"); - zooKeeper.create(clusterStateZNode, data, - Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - LOG.debug("<" + instanceName + ">" + "State node wrote in ZooKeeper"); - } else { - zooKeeper.delete(clusterStateZNode, -1); - LOG.debug("<" + instanceName + ">" + "State node deleted in ZooKeeper"); - } - return true; - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to set state node in ZooKeeper", e); - } catch (KeeperException e) { - if(e.code() == KeeperException.Code.NODEEXISTS) { - LOG.debug("<" + instanceName + ">" + "State node exists."); - } else { - LOG.warn("<" + instanceName + ">" + "Failed to set state node in ZooKeeper", e); - } - } - - return false; - } - - /** - * Set a watcher on the master address ZNode. The watcher will be set unless - * an exception occurs with ZooKeeper. - * @param watcher Watcher to set on master address ZNode. - * @return true if watcher was set, false otherwise. - */ - public boolean watchMasterAddress(Watcher watcher) { - try { - zooKeeper.exists(masterElectionZNode, watcher); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to set watcher on ZNode " + masterElectionZNode, e); - return false; - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to set watcher on ZNode " + masterElectionZNode, e); - return false; - } - LOG.debug("<" + instanceName + ">" + "Set watcher on master address ZNode " + masterElectionZNode); - return true; - } - - private HServerAddress readAddress(String znode, Watcher watcher) { - try { - return readAddressOrThrow(znode, watcher); - } catch (IOException e) { - LOG.debug("<" + instanceName + ">" + "Failed to read " + e.getMessage()); - return null; - } - } - - private HServerAddress readAddressOrThrow(String znode, Watcher watcher) throws IOException { - byte[] data; - try { - data = zooKeeper.getData(znode, watcher, null); - } catch (InterruptedException e) { - throw new IOException(e); - } catch (KeeperException e) { - throw new IOException(e); - } - - String addressString = Bytes.toString(data); - LOG.debug("<" + instanceName + ">" + "Read ZNode " + znode + " got " + addressString); - return new HServerAddress(addressString); - } - - /** - * Make sure this znode exists by creating it if it's missing - * @param znode full path to znode - * @return true if it works - */ - public boolean ensureExists(final String znode) { - try { - Stat stat = zooKeeper.exists(znode, false); - if (stat != null) { - return true; - } - zooKeeper.create(znode, new byte[0], - Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - LOG.debug("<" + instanceName + ">" + "Created ZNode " + znode); - return true; - } catch (KeeperException.NodeExistsException e) { - return true; // ok, move on. - } catch (KeeperException.NoNodeException e) { - return ensureParentExists(znode) && ensureExists(znode); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create " + znode + - " -- check quorum servers, currently=" + this.quorumServers, e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create " + znode + - " -- check quorum servers, currently=" + this.quorumServers, e); - } - return false; - } - - private boolean ensureParentExists(final String znode) { - int index = znode.lastIndexOf(ZNODE_PATH_SEPARATOR); - if (index <= 0) { // Parent is root, which always exists. - return true; - } - return ensureExists(znode.substring(0, index)); - } - - /** - * Delete ZNode containing root region location. - * @return true if operation succeeded, false otherwise. - */ - public boolean deleteRootRegionLocation() { - if (!ensureParentExists(rootRegionZNode)) { - return false; - } - - try { - deleteZNode(rootRegionZNode); - return true; - } catch (KeeperException.NoNodeException e) { - return true; // ok, move on. - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to delete " + rootRegionZNode + ": " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to delete " + rootRegionZNode + ": " + e); - } - - return false; - } - - /** - * Unrecursive deletion of specified znode - * @param znode - * @throws KeeperException - * @throws InterruptedException - */ - public void deleteZNode(String znode) - throws KeeperException, InterruptedException { - deleteZNode(znode, false); - } - - /** - * Optionnally recursive deletion of specified znode - * @param znode - * @param recursive - * @throws KeeperException - * @throws InterruptedException - */ - public void deleteZNode(String znode, boolean recursive) - throws KeeperException, InterruptedException { - if (recursive) { - LOG.info("<" + instanceName + ">" + "deleteZNode get children for " + znode); - List znodes = this.zooKeeper.getChildren(znode, false); - if (znodes.size() > 0) { - for (String child : znodes) { - String childFullPath = getZNode(znode, child); - LOG.info("<" + instanceName + ">" + "deleteZNode recursive call " + childFullPath); - this.deleteZNode(childFullPath, true); - } - } - } - this.zooKeeper.delete(znode, -1); - LOG.debug("<" + instanceName + ">" + "Deleted ZNode " + znode); - } - - private boolean createRootRegionLocation(String address) { - byte[] data = Bytes.toBytes(address); - try { - zooKeeper.create(rootRegionZNode, data, Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - LOG.debug("<" + instanceName + ">" + "Created ZNode " + rootRegionZNode + " with data " + address); - return true; - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create root region in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create root region in ZooKeeper: " + e); - } - - return false; - } - - private boolean updateRootRegionLocation(String address) { - byte[] data = Bytes.toBytes(address); - try { - zooKeeper.setData(rootRegionZNode, data, -1); - LOG.debug("<" + instanceName + ">" + "SetData of ZNode " + rootRegionZNode + " with " + address); - return true; - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to set root region location in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to set root region location in ZooKeeper: " + e); - } - - return false; - } - - /** - * Write root region location to ZooKeeper. If address is null, delete ZNode. - * containing root region location. - * @param address HServerAddress to write to ZK. - * @return true if operation succeeded, false otherwise. - */ - public boolean writeRootRegionLocation(HServerAddress address) { - if (address == null) { - return deleteRootRegionLocation(); - } - - if (!ensureParentExists(rootRegionZNode)) { - return false; - } - - String addressString = address.toString(); - - if (checkExistenceOf(rootRegionZNode)) { - return updateRootRegionLocation(addressString); - } - - return createRootRegionLocation(addressString); - } - - /** - * Write address of master to ZooKeeper. - * @param address HServerAddress of master. - * @return true if operation succeeded, false otherwise. - */ - public boolean writeMasterAddress(final HServerAddress address) { - LOG.debug("<" + instanceName + ">" + "Writing master address " + address.toString() + " to znode " + masterElectionZNode); - if (!ensureParentExists(masterElectionZNode)) { - return false; - } - LOG.debug("<" + instanceName + ">" + "Znode exists : " + masterElectionZNode); - - String addressStr = address.toString(); - byte[] data = Bytes.toBytes(addressStr); - try { - zooKeeper.create(masterElectionZNode, data, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); - LOG.debug("<" + instanceName + ">" + "Wrote master address " + address + " to ZooKeeper"); - return true; - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to write master address " + address + " to ZooKeeper", e); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to write master address " + address + " to ZooKeeper", e); - } - - return false; - } - - /** - * Write in ZK this RS startCode and address. - * Ensures that the full path exists. - * @param info The RS info - * @return true if the location was written, false if it failed - */ - public boolean writeRSLocation(HServerInfo info) { - ensureExists(rsZNode); - byte[] data = Bytes.toBytes(info.getServerAddress().toString()); - String znode = joinPath(rsZNode, info.getServerName()); - try { - zooKeeper.create(znode, data, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); - LOG.debug("<" + instanceName + ">" + "Created ZNode " + znode - + " with data " + info.getServerAddress().toString()); - return true; - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create " + znode + " znode in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create " + znode + " znode in ZooKeeper: " + e); - } - return false; - } - - /** - * Update the RS address and set a watcher on the znode - * @param info The RS info - * @param watcher The watcher to put on the znode - * @return true if the update is done, false if it failed - */ - public boolean updateRSLocationGetWatch(HServerInfo info, Watcher watcher) { - byte[] data = Bytes.toBytes(info.getServerAddress().toString()); - String znode = rsZNode + ZNODE_PATH_SEPARATOR + info.getServerName(); - try { - zooKeeper.setData(znode, data, -1); - LOG.debug("<" + instanceName + ">" + "Updated ZNode " + znode - + " with data " + info.getServerAddress().toString()); - zooKeeper.getData(znode, watcher, null); - return true; - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to update " + znode + " znode in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to update " + znode + " znode in ZooKeeper: " + e); - } - - return false; - } - - /** - * Scans the regions servers directory - * @return A list of server addresses - */ - public List scanRSDirectory() { - return scanAddressDirectory(rsZNode, null); - } - - /** - * Scans the regions servers directory and sets a watch on each znode - * @param watcher a watch to use for each znode - * @return A list of server addresses - */ - public List scanRSDirectory(Watcher watcher) { - return scanAddressDirectory(rsZNode, watcher); - } - - /** - * Method used to make sure the region server directory is empty. - * - */ - public void clearRSDirectory() { - try { - List nodes = zooKeeper.getChildren(rsZNode, false); - for (String node : nodes) { - LOG.debug("<" + instanceName + ">" + "Deleting node: " + node); - zooKeeper.delete(joinPath(this.rsZNode, node), -1); - } - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to delete " + rsZNode + " znodes in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to delete " + rsZNode + " znodes in ZooKeeper: " + e); - } - } - - private boolean checkExistenceOf(String path) { - Stat stat = null; - try { - stat = zooKeeper.exists(path, false); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "checking existence of " + path, e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "checking existence of " + path, e); - } - - return stat != null; - } - - /** - * Close this ZooKeeper session. - */ - public void close() { - try { - zooKeeper.close(); - LOG.debug("<" + instanceName + ">" + "Closed connection with ZooKeeper; " + this.rootRegionZNode); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to close connection with ZooKeeper"); - } - } - - public String getZNode(String parentZNode, String znodeName) { - return znodeName.charAt(0) == ZNODE_PATH_SEPARATOR ? - znodeName : joinPath(parentZNode, znodeName); - } - - public String getZNodePathForHBase(String znodeName) { - return getZNode(parentZNode, znodeName); - } - - private String joinPath(String parent, String child) { - return parent + ZNODE_PATH_SEPARATOR + child; - } - - /** - * Get the path of the masterElectionZNode - * @return the path to masterElectionZNode - */ - public String getMasterElectionZNode() { - return masterElectionZNode; - } - - /** - * Get the path of the parent ZNode - * @return path of that znode - */ - public String getParentZNode() { - return parentZNode; - } - - /** - * Scan a directory of address data. - * @param znode The parent node - * @param watcher The watcher to put on the found znodes, if not null - * @return The directory contents - */ - public List scanAddressDirectory(String znode, - Watcher watcher) { - List list = new ArrayList(); - List nodes = this.listZnodes(znode); - if(nodes == null) { - return list; - } - for (String node : nodes) { - String path = joinPath(znode, node); - list.add(readAddress(path, watcher)); - } - return list; - } - - /** - * List all znodes in the specified path - * @param znode path to list - * @return a list of all the znodes - */ - public List listZnodes(String znode) { - return listZnodes(znode, this); - } - - /** - * List all znodes in the specified path and set a watcher on each - * @param znode path to list - * @param watcher watch to set, can be null - * @return a list of all the znodes - */ - public List listZnodes(String znode, Watcher watcher) { - List nodes = null; - if (watcher == null) { - watcher = this; - } - try { - if (checkExistenceOf(znode)) { - nodes = zooKeeper.getChildren(znode, this); - for (String node : nodes) { - getDataAndWatch(znode, node, this); - } - } - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e); - } - return nodes; - } - - public byte[] getData(String parentZNode, String znode) { - return getDataAndWatch(parentZNode, znode, null); - } - - public byte[] getDataAndWatch(String parentZNode, - String znode, Watcher watcher) { - byte[] data = null; - try { - String path = joinPath(parentZNode, znode); - // TODO: ZK-REFACTOR: remove existance check? - if (checkExistenceOf(path)) { - data = zooKeeper.getData(path, watcher, null); - } - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e); - } - return data; - } - - /** - * Write a znode and fail if it already exists - * @param parentPath parent path to the new znode - * @param child name of the znode - * @param strData data to insert - * @throws InterruptedException - * @throws KeeperException - */ - public void writeZNode(String parentPath, String child, String strData) - throws InterruptedException, KeeperException { - writeZNode(parentPath, child, strData, false); - } - - - /** - * Write (and optionally over-write) a znode - * @param parentPath parent path to the new znode - * @param child name of the znode - * @param strData data to insert - * @param failOnWrite true if an exception should be returned if the znode - * already exists, false if it should be overwritten - * @throws InterruptedException - * @throws KeeperException - */ - public void writeZNode(String parentPath, String child, String strData, - boolean failOnWrite) throws InterruptedException, KeeperException { - String path = joinPath(parentPath, child); - if (!ensureExists(parentPath)) { - LOG.error("<" + instanceName + ">" + "unable to ensure parent exists: " + parentPath); - } - byte[] data = Bytes.toBytes(strData); - Stat stat = this.zooKeeper.exists(path, false); - if (failOnWrite || stat == null) { - this.zooKeeper.create(path, data, - Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - LOG.debug("<" + instanceName + ">" + "Created " + path); - } else { - this.zooKeeper.setData(path, data, -1); - LOG.debug("<" + instanceName + ">" + "Updated " + path); - } - } - - /** - * Get the key to the ZK ensemble for this configuration without - * adding a name at the end - * @param conf Configuration to use to build the key - * @return ensemble key without a name - */ - public static String getZookeeperClusterKey(Configuration conf) { - return getZookeeperClusterKey(conf, null); - } - - /** - * Get the key to the ZK ensemble for this configuration and append - * a name at the end - * @param conf Configuration to use to build the key - * @param name Name that should be appended at the end if not empty or null - * @return ensemble key with a name (if any) - */ - public static String getZookeeperClusterKey(Configuration conf, String name) { - String quorum = conf.get(HConstants.ZOOKEEPER_QUORUM.replaceAll( - "[\\t\\n\\x0B\\f\\r]", "")); - StringBuilder builder = new StringBuilder(quorum); - builder.append(":"); - builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); - if (name != null && !name.isEmpty()) { - builder.append(","); - builder.append(name); - } - return builder.toString(); - } - - /** - * Get the znode that has all the regions in transition. - * @return path to znode - */ - public String getRegionInTransitionZNode() { - return this.rgnsInTransitZNode; - } - - /** - * Get the path of this region server's znode - * @return path to znode - */ - public String getRsZNode() { - return this.rsZNode; - } - - public void deleteZNode(String zNodeName, int version) { - String fullyQualifiedZNodeName = getZNode(parentZNode, zNodeName); - try - { - zooKeeper.delete(fullyQualifiedZNodeName, version); - } - catch (InterruptedException e) - { - LOG.warn("<" + instanceName + ">" + "Failed to delete ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e); - } - catch (KeeperException e) - { - LOG.warn("<" + instanceName + ">" + "Failed to delete ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e); - } - } - - public String createZNodeIfNotExists(String zNodeName) { - return createZNodeIfNotExists(zNodeName, null, CreateMode.PERSISTENT, true); - } - - public void watchZNode(String zNodeName) { - String fullyQualifiedZNodeName = getZNode(parentZNode, zNodeName); - - try { - zooKeeper.exists(fullyQualifiedZNodeName, this); - zooKeeper.getData(fullyQualifiedZNodeName, this, null); - zooKeeper.getChildren(fullyQualifiedZNodeName, this); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e); - } - } - - public String createZNodeIfNotExists(String zNodeName, byte[] data, CreateMode createMode, boolean watch) { - String fullyQualifiedZNodeName = getZNode(parentZNode, zNodeName); - - if (!ensureParentExists(fullyQualifiedZNodeName)) { - return null; - } - - try { - // create the znode - zooKeeper.create(fullyQualifiedZNodeName, data, Ids.OPEN_ACL_UNSAFE, createMode); - LOG.debug("<" + instanceName + ">" + "Created ZNode " + fullyQualifiedZNodeName + " in ZooKeeper"); - // watch the znode for deletion, data change, creation of children - if(watch) { - watchZNode(zNodeName); - } - return fullyQualifiedZNodeName; - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to create ZNode " + fullyQualifiedZNodeName + " in ZooKeeper", e); - } - - return null; - } - - public byte[] readZNode(String znodeName, Stat stat) throws IOException { - byte[] data; - try { - String fullyQualifiedZNodeName = getZNode(parentZNode, znodeName); - data = zooKeeper.getData(fullyQualifiedZNodeName, this, stat); - } catch (InterruptedException e) { - throw new IOException(e); - } catch (KeeperException e) { - throw new IOException(e); - } - return data; - } - - // TODO: perhaps return the version number from this write? - public boolean writeZNode(String znodeName, byte[] data, int version, boolean watch) throws IOException { - try { - String fullyQualifiedZNodeName = getZNode(parentZNode, znodeName); - zooKeeper.setData(fullyQualifiedZNodeName, data, version); - if(watch) { - zooKeeper.getData(fullyQualifiedZNodeName, this, null); - } - return true; - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to write data to ZooKeeper", e); - throw new IOException(e); - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to write data to ZooKeeper", e); - throw new IOException(e); - } - } - - public void createUnassignedRegion(String regionName, byte[] data) { - String znode = getZNode(getRegionInTransitionZNode(), regionName); - if(LOG.isDebugEnabled()) { - // check if this node already exists - - // - it should not exist - // - if it does, it should be in the CLOSED state - if(exists(znode, true)) { - Stat stat = new Stat(); - byte[] oldData = null; - try { - oldData = readZNode(znode, stat); - } catch (IOException e) { - LOG.error("Error reading data for " + znode); - } - if(oldData == null) { - LOG.debug("While creating UNASSIGNED region " + regionName + " exists with no data" ); - } - else { - LOG.debug("While creating UNASSIGNED region " + regionName + " exists, state = " + (HBaseEventType.fromByte(oldData[0]))); - } - } - else { - if(data == null) { - LOG.debug("Creating UNASSIGNED region " + regionName + " with no data" ); - } - else { - LOG.debug("Creating UNASSIGNED region " + regionName + " in state = " + (HBaseEventType.fromByte(data[0]))); - } - } - } - synchronized(unassignedZNodesWatched) { - unassignedZNodesWatched.add(znode); - createZNodeIfNotExists(znode, data, CreateMode.PERSISTENT, true); - } - } - - public void deleteUnassignedRegion(String regionName) { - String znode = getZNode(getRegionInTransitionZNode(), regionName); - try { - LOG.debug("Deleting ZNode " + znode + " in ZooKeeper as region is open..."); - synchronized(unassignedZNodesWatched) { - unassignedZNodesWatched.remove(znode); - deleteZNode(znode); - } - } catch (KeeperException.SessionExpiredException e) { - LOG.error("Zookeeper session has expired", e); - // if the session has expired try to reconnect to ZK, then perform query - try { - // TODO: ZK-REFACTOR: should just quit on reconnect?? - reconnectToZk(); - synchronized(unassignedZNodesWatched) { - unassignedZNodesWatched.remove(znode); - deleteZNode(znode); - } - } catch (IOException e1) { - LOG.error("Error reconnecting to zookeeper", e1); - throw new RuntimeException("Error reconnecting to zookeeper", e1); - } catch (KeeperException.SessionExpiredException e1) { - LOG.error("Error reading after reconnecting to zookeeper", e1); - throw new RuntimeException("Error reading after reconnecting to zookeeper", e1); - } catch (KeeperException e1) { - LOG.error("Error reading after reconnecting to zookeeper", e1); - } catch (InterruptedException e1) { - LOG.error("Error reading after reconnecting to zookeeper", e1); - } - } catch (KeeperException e) { - LOG.error("Error deleting region " + regionName, e); - } catch (InterruptedException e) { - LOG.error("Error deleting region " + regionName, e); - } - } - - /** - * Atomically adds a watch and reads data from the unwatched znodes in the - * UNASSGINED region. This works because the master is the only person - * deleting nodes. - * @param znode - * @return - */ - public List watchAndGetNewChildren(String znode) { - List nodes = null; - List newNodes = new ArrayList(); - try { - if (checkExistenceOf(znode)) { - synchronized(unassignedZNodesWatched) { - nodes = zooKeeper.getChildren(znode, this); - for (String node : nodes) { - String znodePath = joinPath(znode, node); - if(!unassignedZNodesWatched.contains(znodePath)) { - byte[] data = getDataAndWatch(znode, node, this); - newNodes.add(new ZNodePathAndData(znodePath, data)); - unassignedZNodesWatched.add(znodePath); - } - } - } - } - } catch (KeeperException e) { - LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e); - } catch (InterruptedException e) { - LOG.warn("<" + instanceName + ">" + "Failed to read " + znode + " znode in ZooKeeper: " + e); - } - return newNodes; - } - - public static class ZNodePathAndData { - private String zNodePath; - private byte[] data; - - public ZNodePathAndData(String zNodePath, byte[] data) { - this.zNodePath = zNodePath; - this.data = data; - } - - public String getzNodePath() { - return zNodePath; - } - public byte[] getData() { - return data; - } - - } -} Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java (working copy) @@ -35,13 +35,13 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { /** Path of node being tracked */ - private String node; + protected String node; /** Data of the node being tracked */ private byte [] data; /** Used to abort if a fatal error occurs */ - private Abortable abortable; + protected Abortable abortable; /** * Constructs a new ZK node tracker. @@ -98,6 +98,23 @@ } /** + * Gets the data of the node, blocking until the node is available or the + * specified timeout has elapsed. + * + * @param timeout maximum time to wait for the node data to be available, + * in milliseconds + * @return data of the node + * @throws InterruptedException if the waiting thread is interrupted + */ + public synchronized byte [] blockUntilAvailable(long timeout) + throws InterruptedException { + while(data == null) { + wait(timeout); + } + return data; + } + + /** * Gets the data of the node. * *

If the node is currently available, the most up-to-date known version of Index: src/main/java/org/apache/hadoop/hbase/zookeeper/MetaNodeTracker.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/MetaNodeTracker.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/MetaNodeTracker.java (revision 0) @@ -0,0 +1,73 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.zookeeper; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.catalog.CatalogTracker; + +/** + * Tracks the unassigned zookeeper node used by the META table. + *

+ * A callback is made into a {@link CatalogTracker} when META completes a new + * assignment. + *

+ * If META is already assigned when instantiating this class, you will not + * receive any notification for that assignment. You will receive a + * notification after META has been successfully assigned to a new location. + */ +public class MetaNodeTracker extends ZooKeeperListener { + private static final Log LOG = LogFactory.getLog(MetaNodeTracker.class); + + private final String node; + + /** Catalog tracker to notify when META has a new assignment completed. */ + private final CatalogTracker catalogTracker; + + /** + * Creates a meta node tracker. + * @param watcher + * @param abortable + */ + public MetaNodeTracker(ZooKeeperWatcher watcher, + CatalogTracker catalogTracker) { + super(watcher); + this.catalogTracker = catalogTracker; + node = ZKUtil.joinZNode(watcher.assignmentZNode, + HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + } + + @Override + public void nodeDeleted(String path) { + if(path.equals(node)) { + LOG.info("Detected completed assignment of META, notifying catalog " + + "tracker"); + try { + catalogTracker.waitForMetaServerConnectionDefault(); + } catch (IOException e) { + LOG.warn("Tried to reset META server location after seeing the " + + "completion of a new META assignment but got an IOE", e); + } + } + } +} Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java (working copy) @@ -19,8 +19,9 @@ */ package org.apache.hadoop.hbase.zookeeper; +import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.executor.RegionTransitionData; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; +import org.apache.hadoop.hbase.executor.EventHandler.EventType; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.KeeperException.NoNodeException; @@ -119,20 +120,20 @@ * of a table. * * @param zkw zk reference - * @param regionName region to be created as offline + * @param region region to be created as offline * @param serverName server event originates from * @throws KeeperException if unexpected zookeeper exception * @throws KeeperException.NodeExistsException if node already exists */ - public static void createNodeOffline(ZooKeeperWatcher zkw, String regionName, + public static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region, String serverName) throws KeeperException, KeeperException.NodeExistsException { - zkw.debug("Creating an unassigned node for " + regionName + + zkw.debug("Creating an unassigned node for " + region.getEncodedName() + " in an OFFLINE state"); RegionTransitionData data = new RegionTransitionData( - HBaseEventType.M2ZK_REGION_OFFLINE, regionName, serverName); + EventType.M2ZK_REGION_OFFLINE, region.getRegionName(), serverName); synchronized(zkw.getNodes()) { - String node = getNodeName(zkw, regionName); + String node = getNodeName(zkw, region.getEncodedName()); zkw.getNodes().add(node); ZKUtil.createAndWatch(zkw, node, data.getBytes()); } @@ -151,26 +152,65 @@ *

This method should only be used during recovery of regionserver failure. * * @param zkw zk reference - * @param regionName region to be forced as offline + * @param region region to be forced as offline * @param serverName server event originates from * @throws KeeperException if unexpected zookeeper exception * @throws KeeperException.NoNodeException if node does not exist */ - public static void forceNodeOffline(ZooKeeperWatcher zkw, String regionName, + public static void forceNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region, String serverName) throws KeeperException, KeeperException.NoNodeException { - zkw.debug("Forcing an existing unassigned node for " + regionName + - " to an OFFLINE state"); + zkw.debug("Forcing an existing unassigned node for " + + region.getEncodedName() + " to an OFFLINE state"); RegionTransitionData data = new RegionTransitionData( - HBaseEventType.M2ZK_REGION_OFFLINE, regionName, serverName); + EventType.M2ZK_REGION_OFFLINE, region.getRegionName(), serverName); synchronized(zkw.getNodes()) { - String node = getNodeName(zkw, regionName); + String node = getNodeName(zkw, region.getEncodedName()); zkw.getNodes().add(node); ZKUtil.setData(zkw, node, data.getBytes()); } } + /** + * Creates or force updates an unassigned node to the OFFLINE state for the + * specified region. + *

+ * Attempts to create the node but if it exists will force it to transition to + * and OFFLINE state. + * + *

Sets a watcher on the unassigned region node if the method is + * successful. + * + *

This method should be used when assigning a region. + * + * @param zkw zk reference + * @param region region to be created as offline + * @param serverName server event originates from + * @throws KeeperException if unexpected zookeeper exception + * @throws KeeperException.NodeExistsException if node already exists + */ + public static boolean createOrForceNodeOffline(ZooKeeperWatcher zkw, + HRegionInfo region, String serverName) + throws KeeperException { + zkw.debug("Creating or updating an unassigned node for " + + region.getEncodedName() + " with an OFFLINE state"); + RegionTransitionData data = new RegionTransitionData( + EventType.M2ZK_REGION_OFFLINE, region.getRegionName(), serverName); + synchronized(zkw.getNodes()) { + String node = getNodeName(zkw, region.getEncodedName()); + zkw.getNodes().add(node); + int version = ZKUtil.checkExists(zkw, node); + if(version == -1) { + ZKUtil.createAndWatch(zkw, node, data.getBytes()); + return true; + } else { + return ZKUtil.setData(zkw, node, data.getBytes(), version); + } + } + } + + /** * Deletes an existing unassigned node that is in the OPENED state for the * specified region. * @@ -186,15 +226,94 @@ * of the specified regions transition. * * @param zkw zk reference - * @param regionName opened region to be deleted from zk + * @param region opened region to be deleted from zk * @throws KeeperException if unexpected zookeeper exception * @throws KeeperException.NoNodeException if node does not exist */ public static boolean deleteOpenedNode(ZooKeeperWatcher zkw, String regionName) throws KeeperException, KeeperException.NoNodeException { + return deleteNode(zkw, regionName, EventType.RS2ZK_REGION_OPENED); + } + + /** + * Deletes an existing unassigned node that is in the CLOSED state for the + * specified region. + * + *

If a node does not already exist for this region, a + * {@link NoNodeException} will be thrown. + * + *

No watcher is set whether this succeeds or not. + * + *

Returns false if the node was not in the proper state but did exist. + * + *

This method is used during table disables when a region finishes + * successfully closing. This is the Master acknowledging completion + * of the specified regions transition to being closed. + * + * @param zkw zk reference + * @param region closed region to be deleted from zk + * @throws KeeperException if unexpected zookeeper exception + * @throws KeeperException.NoNodeException if node does not exist + */ + public static boolean deleteClosedNode(ZooKeeperWatcher zkw, + String regionName) + throws KeeperException, KeeperException.NoNodeException { + return deleteNode(zkw, regionName, EventType.RS2ZK_REGION_CLOSED); + } + + /** + * Deletes an existing unassigned node that is in the CLOSING state for the + * specified region. + * + *

If a node does not already exist for this region, a + * {@link NoNodeException} will be thrown. + * + *

No watcher is set whether this succeeds or not. + * + *

Returns false if the node was not in the proper state but did exist. + * + *

This method is used during table disables when a region finishes + * successfully closing. This is the Master acknowledging completion + * of the specified regions transition to being closed. + * + * @param zkw zk reference + * @param region closing region to be deleted from zk + * @throws KeeperException if unexpected zookeeper exception + * @throws KeeperException.NoNodeException if node does not exist + */ + public static boolean deleteClosingNode(ZooKeeperWatcher zkw, + String regionName) + throws KeeperException, KeeperException.NoNodeException { + return deleteNode(zkw, regionName, EventType.RS2ZK_REGION_CLOSING); + } + + /** + * Deletes an existing unassigned node that is in the specified state for the + * specified region. + * + *

If a node does not already exist for this region, a + * {@link NoNodeException} will be thrown. + * + *

No watcher is set whether this succeeds or not. + * + *

Returns false if the node was not in the proper state but did exist. + * + *

This method is used during table disables when a region finishes + * successfully closing. This is the Master acknowledging completion + * of the specified regions transition to being closed. + * + * @param zkw zk reference + * @param region region to be deleted from zk + * @param expectedState state region must be in for delete to complete + * @throws KeeperException if unexpected zookeeper exception + * @throws KeeperException.NoNodeException if node does not exist + */ + private static boolean deleteNode(ZooKeeperWatcher zkw, String regionName, + EventType expectedState) + throws KeeperException, KeeperException.NoNodeException { zkw.debug("Deleting an existing unassigned node for " + regionName + - " that is in a OPENED state"); + " that is in a " + expectedState + " state"); String node = getNodeName(zkw, regionName); Stat stat = new Stat(); byte [] bytes = ZKUtil.getDataNoWatch(zkw, node, stat); @@ -202,16 +321,17 @@ throw KeeperException.create(Code.NONODE); } RegionTransitionData data = RegionTransitionData.fromBytes(bytes); - if(!data.getEventType().equals(HBaseEventType.RS2ZK_REGION_OPENED)) { - zkw.warn("Attempting to delete an unassigned node in OPENED state but " + - "node is in " + data.getEventType() + " state"); + if(!data.getEventType().equals(expectedState)) { + zkw.warn("Attempting to delete an unassigned node in " + expectedState + + " state but node is in " + data.getEventType() + " state"); return false; } synchronized(zkw.getNodes()) { // TODO: Does this go here or only if we successfully delete node? zkw.getNodes().remove(node); if(!ZKUtil.deleteNode(zkw, node, stat.getVersion())) { - zkw.warn("Attempting to delete an unassigned node in OPENED state but " + + zkw.warn("Attempting to delete an unassigned node in " + expectedState + + " state but " + "after verifying it was in OPENED state, we got a version mismatch"); return false; } @@ -245,28 +365,32 @@ *

Does not transition nodes from any states. If a node already exists * for this region, a {@link NodeExistsException} will be thrown. * + *

If creation is successful, returns the version number of the CLOSING + * node created. + * *

Does not set any watches. * *

This method should only be used by a RegionServer when initiating a * close of a region after receiving a CLOSE RPC from the Master. * * @param zkw zk reference - * @param regionName region to be created as closing + * @param region region to be created as closing * @param serverName server event originates from + * @return version of node after transition, -1 if unsuccessful transition * @throws KeeperException if unexpected zookeeper exception * @throws KeeperException.NodeExistsException if node already exists */ - public static void createNodeClosing(ZooKeeperWatcher zkw, String regionName, + public static int createNodeClosing(ZooKeeperWatcher zkw, HRegionInfo region, String serverName) throws KeeperException, KeeperException.NodeExistsException { - zkw.debug("Creating an unassigned node for " + regionName + + zkw.debug("Creating an unassigned node for " + region.getEncodedName() + " in a CLOSING state"); RegionTransitionData data = new RegionTransitionData( - HBaseEventType.RS2ZK_REGION_CLOSING, regionName, serverName); + EventType.RS2ZK_REGION_CLOSING, region.getRegionName(), serverName); synchronized(zkw.getNodes()) { - String node = getNodeName(zkw, regionName); + String node = getNodeName(zkw, region.getEncodedName()); zkw.getNodes().add(node); - ZKUtil.createAndWatch(zkw, node, data.getBytes()); + return ZKUtil.createAndWatch(zkw, node, data.getBytes()); } } @@ -275,7 +399,8 @@ * currently in the CLOSING state to be in the CLOSED state. * *

Does not transition nodes from other states. If for some reason the - * node could not be transitioned, the method returns false. + * node could not be transitioned, the method returns -1. If the transition + * is successful, the version of the node after transition is returned. * *

This method can fail and return false for three different reasons: *

  • Unassigned node for this region does not exist
  • @@ -290,17 +415,17 @@ * close of a region after receiving a CLOSE RPC from the Master. * * @param zkw zk reference - * @param regionName region to be transitioned to closed + * @param region region to be transitioned to closed * @param serverName server event originates from - * @return true if transition was successful, false if not + * @return version of node after transition, -1 if unsuccessful transition * @throws KeeperException if unexpected zookeeper exception */ - public static boolean transitionNodeClosed(ZooKeeperWatcher zkw, - String regionName, String serverName) + public static int transitionNodeClosed(ZooKeeperWatcher zkw, + HRegionInfo region, String serverName, int expectedVersion) throws KeeperException { - return transitionNode(zkw, regionName, serverName, - HBaseEventType.RS2ZK_REGION_CLOSING, - HBaseEventType.RS2ZK_REGION_CLOSED); + return transitionNode(zkw, region, serverName, + EventType.RS2ZK_REGION_CLOSING, + EventType.RS2ZK_REGION_CLOSED, expectedVersion); } /** @@ -308,9 +433,10 @@ * currently in the OFFLINE state to be in the OPENING state. * *

    Does not transition nodes from other states. If for some reason the - * node could not be transitioned, the method returns false. + * node could not be transitioned, the method returns -1. If the transition + * is successful, the version of the node written as OPENING is returned. * - *

    This method can fail and return false for three different reasons: + *

    This method can fail and return -1 for three different reasons: *

    • Unassigned node for this region does not exist
    • *
    • Unassigned node for this region is not in OFFLINE state
    • *
    • After verifying OFFLINE state, update fails because of wrong version @@ -323,25 +449,59 @@ * open of a region after receiving an OPEN RPC from the Master. * * @param zkw zk reference - * @param regionName region to be transitioned to opening + * @param region region to be transitioned to opening * @param serverName server event originates from - * @return true if transition was successful, false if not + * @return version of node after transition, -1 if unsuccessful transition * @throws KeeperException if unexpected zookeeper exception */ - public static boolean transitionNodeOpening(ZooKeeperWatcher zkw, - String regionName, String serverName) + public static int transitionNodeOpening(ZooKeeperWatcher zkw, + HRegionInfo region, String serverName) throws KeeperException { - return transitionNode(zkw, regionName, serverName, - HBaseEventType.M2ZK_REGION_OFFLINE, - HBaseEventType.RS2ZK_REGION_OPENING); + return transitionNode(zkw, region, serverName, + EventType.M2ZK_REGION_OFFLINE, + EventType.RS2ZK_REGION_OPENING, -1); } + /** + * Retransitions an existing unassigned node for the specified region which is + * currently in the OPENING state to be in the OPENING state. + * + *

      Does not transition nodes from other states. If for some reason the + * node could not be transitioned, the method returns -1. If the transition + * is successful, the version of the node rewritten as OPENING is returned. + * + *

      This method can fail and return -1 for three different reasons: + *

      • Unassigned node for this region does not exist
      • + *
      • Unassigned node for this region is not in OPENING state
      • + *
      • After verifying OPENING state, update fails because of wrong version + * (someone else already transitioned the node)
      • + *
      + * + *

      Does not set any watches. + * + *

      This method should only be used by a RegionServer when initiating an + * open of a region after receiving an OPEN RPC from the Master. + * + * @param zkw zk reference + * @param region region to be transitioned to opening + * @param serverName server event originates from + * @return version of node after transition, -1 if unsuccessful transition + * @throws KeeperException if unexpected zookeeper exception + */ + public static int retransitionNodeOpening(ZooKeeperWatcher zkw, + HRegionInfo region, String serverName, int expectedVersion) + throws KeeperException { + return transitionNode(zkw, region, serverName, + EventType.RS2ZK_REGION_OPENING, + EventType.RS2ZK_REGION_OPENING, expectedVersion); + } /** * Transitions an existing unassigned node for the specified region which is * currently in the OPENING state to be in the OPENED state. * *

      Does not transition nodes from other states. If for some reason the - * node could not be transitioned, the method returns false. + * node could not be transitioned, the method returns -1. If the transition + * is successful, the version of the node after transition is returned. * *

      This method can fail and return false for three different reasons: *

      • Unassigned node for this region does not exist
      • @@ -358,17 +518,17 @@ * open of a region. * * @param zkw zk reference - * @param regionName region to be transitioned to opened + * @param region region to be transitioned to opened * @param serverName server event originates from - * @return true if transition was successful, false if not + * @return version of node after transition, -1 if unsuccessful transition * @throws KeeperException if unexpected zookeeper exception */ - public static boolean transitionNodeOpened(ZooKeeperWatcher zkw, - String regionName, String serverName) + public static int transitionNodeOpened(ZooKeeperWatcher zkw, + HRegionInfo region, String serverName, int expectedVersion) throws KeeperException { - return transitionNode(zkw, regionName, serverName, - HBaseEventType.RS2ZK_REGION_OPENING, - HBaseEventType.RS2ZK_REGION_OPENED); + return transitionNode(zkw, region, serverName, + EventType.RS2ZK_REGION_OPENING, + EventType.RS2ZK_REGION_OPENED, expectedVersion); } /** @@ -379,33 +539,36 @@ * *

        Method first reads existing data and verifies it is in the expected * state. If the node does not exist or the node is not in the expected - * state, the method returns false. + * state, the method returns -1. If the transition is successful, the + * version number of the node following the transition is returned. * *

        If the read state is what is expected, it attempts to write the new * state and data into the node. When doing this, it includes the expected * version (determined when the existing state was verified) to ensure that * only one transition is successful. If there is a version mismatch, the - * method returns false. + * method returns -1. * *

        If the write is successful, no watch is set and the method returns true. * * @param zkw zk reference - * @param regionName region to be transitioned to opened + * @param region region to be transitioned to opened * @param serverName server event originates from * @param beginState state the node must currently be in to do transition * @param endState state to transition node to if all checks pass - * @return true if transition was successful, false if not + * @return version of node after transition, -1 if unsuccessful transition * @throws KeeperException if unexpected zookeeper exception */ - private static boolean transitionNode(ZooKeeperWatcher zkw, String regionName, - String serverName, HBaseEventType beginState, HBaseEventType endState) + private static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region, + String serverName, EventType beginState, EventType endState, + int expectedVersion) throws KeeperException { + String encoded = region.getEncodedName(); if(zkw.isDebugEnabled()) { - zkw.debug("Attempting to transition node for " + regionName + + zkw.debug("Attempting to transition node for " + encoded + " from " + beginState.toString() + " to " + endState.toString()); } - String node = getNodeName(zkw, regionName); + String node = getNodeName(zkw, encoded); // Read existing data of the node Stat stat = new Stat(); @@ -414,37 +577,45 @@ RegionTransitionData existingData = RegionTransitionData.fromBytes(existingBytes); + // Verify it is the expected version + if(expectedVersion != -1 && stat.getVersion() != expectedVersion) { + zkw.warn("Attempt to transition the unassigned node for " + encoded + + " from " + beginState + " to " + endState + " failed, " + + "the node existed but was version " + stat.getVersion() + + " not the expected version " + expectedVersion); + return -1; + } + // Verify it is in expected state if(!existingData.getEventType().equals(beginState)) { - zkw.warn("Attempt to transition the unassigned node for " + regionName + + zkw.warn("Attempt to transition the unassigned node for " + encoded + " from " + beginState + " to " + endState + " failed, " + "the node existed but was in the state " + existingData.getEventType()); - return false; + return -1; } // Write new data, ensuring data has not changed since we last read it try { RegionTransitionData data = new RegionTransitionData(endState, - regionName, serverName); - if(!ZKUtil.setData(zkw, node, data.getBytes(), - stat.getVersion())) { - zkw.warn("Attempt to transition the unassigned node for " + regionName + + region.getRegionName(), serverName); + if(!ZKUtil.setData(zkw, node, data.getBytes(), stat.getVersion())) { + zkw.warn("Attempt to transition the unassigned node for " + encoded + " from " + beginState + " to " + endState + " failed, " + "the node existed and was in the expected state but then when " + "setting data we got a version mismatch"); - return false; + return -1; } if(zkw.isDebugEnabled()) { - zkw.debug("Successfully transitioned node for " + regionName + + zkw.debug("Successfully transitioned node for " + encoded + " from " + beginState + " to " + endState); } - return true; + return stat.getVersion() + 1; } catch (KeeperException.NoNodeException nne) { - zkw.warn("Attempt to transition the unassigned node for " + regionName + + zkw.warn("Attempt to transition the unassigned node for " + encoded + " from " + beginState + " to " + endState + " failed, " + "the node existed and was in the expected state but then when " + "setting data it no longer existed"); - return false; + return -1; } } Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java (working copy) @@ -22,13 +22,13 @@ /** * Base class for internal listeners of ZooKeeper events. - * + * * The {@link ZooKeeperWatcher} for a process will execute the appropriate * methods of implementations of this class. In order to receive events from * the watcher, every listener must register itself via {@link ZooKeeperWatcher#registerListener}. - * + * * Subclasses need only override those methods in which they are interested. - * + * * Note that the watcher will be blocked when invoking methods in listeners so * they must not be long-running. */ @@ -36,10 +36,9 @@ // Reference to the zk watcher which also contains configuration and constants protected ZooKeeperWatcher watcher; - + /** * Construct a ZooKeeper event listener. - * TODO: This should take ServerStatus which will contain ZKWatcher ref? */ public ZooKeeperListener(ZooKeeperWatcher watcher) { this.watcher = watcher; Index: src/main/java/org/apache/hadoop/hbase/ClusterStatus.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/ClusterStatus.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/ClusterStatus.java (working copy) @@ -27,9 +27,9 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; -import java.util.NavigableMap; import java.util.TreeMap; +import org.apache.hadoop.hbase.master.AssignmentManager.RegionState; import org.apache.hadoop.io.VersionedWritable; /** @@ -53,7 +53,7 @@ private String hbaseVersion; private Collection liveServerInfo; private Collection deadServers; - private NavigableMap intransition; + private Map intransition; /** * Constructor, for Writable @@ -186,11 +186,11 @@ this.deadServers = deadServers; } - public Map getRegionsInTransition() { + public Map getRegionsInTransition() { return this.intransition; } - public void setRegionsInTransition(final NavigableMap m) { + public void setRegionsInTransition(final Map m) { this.intransition = m; } @@ -210,9 +210,9 @@ out.writeUTF(server); } out.writeInt(this.intransition.size()); - for (Map.Entry e: this.intransition.entrySet()) { + for (Map.Entry e: this.intransition.entrySet()) { out.writeUTF(e.getKey()); - out.writeUTF(e.getValue()); + e.getValue().write(out); } } @@ -232,11 +232,12 @@ deadServers.add(in.readUTF()); } count = in.readInt(); - this.intransition = new TreeMap(); + this.intransition = new TreeMap(); for (int i = 0; i < count; i++) { String key = in.readUTF(); - String value = in.readUTF(); - this.intransition.put(key, value); + RegionState regionState = new RegionState(); + regionState.readFields(in); + this.intransition.put(key, regionState); } } } Index: src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java (revision 0) @@ -0,0 +1,50 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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; + +import java.io.IOException; + +/** + * Thrown if a request is table schema modification is requested but + * made for an invalid family name. + */ +public class InvalidFamilyOperationException extends IOException { + private static final long serialVersionUID = 1L << 22 - 1L; + /** default constructor */ + public InvalidFamilyOperationException() { + super(); + } + + /** + * Constructor + * @param s message + */ + public InvalidFamilyOperationException(String s) { + super(s); + } + + /** + * Constructor taking another exception. + * @param e Exception to grab data from. + */ + public InvalidFamilyOperationException(Exception e) { + super(e); + } +} Index: src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (working copy) @@ -71,13 +71,13 @@ import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.ServerController; import org.apache.hadoop.hbase.UnknownRowLockException; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.HMsg.Type; import org.apache.hadoop.hbase.Leases.LeaseStillHeldException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.MultiPut; @@ -87,6 +87,8 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ServerConnection; import org.apache.hadoop.hbase.client.ServerConnectionManager; +import org.apache.hadoop.hbase.executor.HBaseExecutorService; +import org.apache.hadoop.hbase.executor.HBaseExecutorService.HBaseExecutorServiceType; import org.apache.hadoop.hbase.io.hfile.LruBlockCache; import org.apache.hadoop.hbase.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler; @@ -94,6 +96,12 @@ import org.apache.hadoop.hbase.ipc.HBaseServer; import org.apache.hadoop.hbase.ipc.HMasterRegionInterface; import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler; +import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; +import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler; +import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; +import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; +import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler; import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics; import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.util.Bytes; @@ -112,26 +120,26 @@ import org.apache.zookeeper.KeeperException; /** - * HRegionServer makes a set of HRegions available to clients. It checks in with + * HRegionServer makes a set of HRegions available to clients. It checks in with * the HMaster. There are many HRegionServers in a single HBase deployment. */ -public class HRegionServer implements HRegionInterface, - HBaseRPCErrorHandler, Runnable, Stoppable, ServerController { +public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler, + Runnable, Stoppable, RegionServerController { public static final Log LOG = LogFactory.getLog(HRegionServer.class); private static final HMsg REPORT_EXITING = new HMsg(Type.MSG_REPORT_EXITING); private static final HMsg REPORT_QUIESCED = new HMsg(Type.MSG_REPORT_QUIESCED); - private static final HMsg [] EMPTY_HMSG_ARRAY = new HMsg [] {}; + private static final HMsg[] EMPTY_HMSG_ARRAY = new HMsg[] {}; // Set when a report to the master comes back with a message asking us to - // shutdown. Also set by call to stop when debugging or running unit tests + // shutdown. Also set by call to stop when debugging or running unit tests // of HRegionServer in isolation. We use AtomicBoolean rather than - // plain boolean so we can pass a reference to Chore threads. Otherwise, + // plain boolean so we can pass a reference to Chore threads. Otherwise, // Chore threads need to know about the hosting class. protected final AtomicBoolean stopRequested = new AtomicBoolean(false); protected final AtomicBoolean quiesced = new AtomicBoolean(false); - // Go down hard. Used if file system becomes unavailable and also in + // Go down hard. Used if file system becomes unavailable and also in // debugging and unit tests. protected volatile boolean abortRequested; @@ -149,15 +157,14 @@ private Path rootDir; private final Random rand = new Random(); - // Key is Bytes.hashCode of region name byte array and the value is HRegion - // in both of the maps below. Use Bytes.mapKey(byte []) generating key for - // below maps. - protected final Map onlineRegions = - new ConcurrentHashMap(); + /** + * Map of regions currently being served by this region server. Key is the + * encoded region name. + */ + protected final Map onlineRegions = new ConcurrentHashMap(); protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - private final LinkedBlockingQueue outboundMsgs = - new LinkedBlockingQueue(); + private final LinkedBlockingQueue outboundMsgs = new LinkedBlockingQueue(); final int numRetries; protected final int threadWakeFrequency; @@ -170,7 +177,7 @@ // Remote HMaster private HMasterRegionInterface hbaseMaster; - // Server to handle client requests. Default access so can be accessed by + // Server to handle client requests. Default access so can be accessed by // unit tests. HBaseServer server; @@ -180,7 +187,7 @@ // Request counter private volatile AtomicInteger requestCount = new AtomicInteger(); - // Info server. Default access so can be used by unit tests. REGIONSERVER + // Info server. Default access so can be used by unit tests. REGIONSERVER // is name of the webapp and the attribute name used stuffing this instance // into web context. InfoServer infoServer; @@ -189,11 +196,11 @@ public static final String REGIONSERVER = "regionserver"; /* - * Space is reserved in HRS constructor and then released when aborting - * to recover from an OOME. See HBASE-706. TODO: Make this percentage of the - * heap or a minimum. + * Space is reserved in HRS constructor and then released when aborting to + * recover from an OOME. See HBASE-706. TODO: Make this percentage of the heap + * or a minimum. */ - private final LinkedList reservedSpace = new LinkedList(); + private final LinkedList reservedSpace = new LinkedList(); private RegionServerMetrics metrics; @@ -203,11 +210,12 @@ // Cache flushing MemStoreFlusher cacheFlusher; - /* Check for major compactions. + /* + * Check for major compactions. */ Chore majorCompactionChecker; - // HLog and HLog roller. log is protected rather than private to avoid + // HLog and HLog roller. log is protected rather than private to avoid // eclipse warning when accessed by inner classes protected volatile HLog hlog; LogRoller hlogRoller; @@ -215,44 +223,49 @@ // flag set after we're done setting up server threads (used for testing) protected volatile boolean isOnline; - final Map scanners = - new ConcurrentHashMap(); + final Map scanners = new ConcurrentHashMap(); // zookeeper connection and watcher private ZooKeeperWatcher zooKeeper; - - // master address manager and watecher + + // master address manager and watcher private MasterAddressManager masterAddressManager; + // catalog tracker + private CatalogTracker catalogTracker; + // A sleeper that sleeps for msgInterval. private final Sleeper sleeper; private final long rpcTimeout; - // Address passed in to constructor. This is not always the address we run - // with. For example, if passed port is 0, then we are to pick a port. The + // Address passed in to constructor. This is not always the address we run + // with. For example, if passed port is 0, then we are to pick a port. The // actual address we run with is in the #serverInfo data member. private final HServerAddress address; // The main region server thread. + @SuppressWarnings("unused") private Thread regionServerThread; private final String machineName; /** * Starts a HRegionServer at the default location + * * @param conf * @throws IOException */ public HRegionServer(Configuration conf) throws IOException { - machineName = DNS.getDefaultHost( - conf.get("hbase.regionserver.dns.interface","default"), - conf.get("hbase.regionserver.dns.nameserver","default")); - String addressStr = machineName + ":" + - conf.get(HConstants.REGIONSERVER_PORT, - Integer.toString(HConstants.DEFAULT_REGIONSERVER_PORT)); - // This is not necessarily the address we will run with. The address we - // use will be in #serverInfo data member. For example, we may have been + machineName = DNS.getDefaultHost(conf.get( + "hbase.regionserver.dns.interface", "default"), conf.get( + "hbase.regionserver.dns.nameserver", "default")); + String addressStr = machineName + + ":" + + conf.get(HConstants.REGIONSERVER_PORT, Integer + .toString(HConstants.DEFAULT_REGIONSERVER_PORT)); + // This is not necessarily the address we will run with. The address we + // use will be in #serverInfo data member. For example, we may have been // passed a port of 0 which means we should pick some ephemeral port to bind // to. address = new HServerAddress(addressStr); @@ -266,7 +279,7 @@ this.isOnline = false; // Config'ed params - this.numRetries = conf.getInt("hbase.client.retries.number", 2); + this.numRetries = conf.getInt("hbase.client.retries.number", 2); this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000); this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 1 * 1000); @@ -274,26 +287,27 @@ sleeper = new Sleeper(this.msgInterval, this.stopRequested); this.maxScannerResultSize = conf.getLong( - HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, - HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE); + HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, + HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE); // Task thread to process requests from Master this.worker = new Worker(); - this.numRegionsToReport = - conf.getInt("hbase.regionserver.numregionstoreport", 10); + this.numRegionsToReport = conf.getInt( + "hbase.regionserver.numregionstoreport", 10); - this.rpcTimeout = - conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, - HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD); + this.rpcTimeout = conf.getLong( + HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, + HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD); initialize(); } /** * Creates all of the state that needs to be reconstructed in case we are - * doing a restart. This is shared between the constructor and restart(). - * Both call it. + * doing a restart. This is shared between the constructor and restart(). Both + * call it. + * * @throws IOException */ private void initialize() throws IOException { @@ -301,41 +315,68 @@ this.stopRequested.set(false); // Server to handle client requests - this.server = HBaseRPC.getServer(this, address.getBindAddress(), - address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10), - false, conf); + this.server = HBaseRPC.getServer(this, address.getBindAddress(), address + .getPort(), conf.getInt("hbase.regionserver.handler.count", 10), false, + conf); this.server.setErrorHandler(this); // Address is giving a default IP for the moment. Will be changed after // calling the master. - this.serverInfo = new HServerInfo(new HServerAddress( - new InetSocketAddress(address.getBindAddress(), - this.server.getListenerAddress().getPort())), System.currentTimeMillis(), - this.conf.getInt("hbase.regionserver.info.port", 60030), machineName); + this.serverInfo = new HServerInfo(new HServerAddress(new InetSocketAddress( + address.getBindAddress(), this.server.getListenerAddress().getPort())), + System.currentTimeMillis(), this.conf.getInt( + "hbase.regionserver.info.port", 60030), machineName); if (this.serverInfo.getServerAddress() == null) { - throw new NullPointerException("Server address cannot be null; " + - "hbase-958 debugging"); + throw new NullPointerException("Server address cannot be null; " + + "hbase-958 debugging"); } + initializeZooKeeper(); initializeThreads(); - initializeZooKeeper(); int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4); - for(int i = 0; i < nbBlocks; i++) { + for (int i = 0; i < nbBlocks; i++) { reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]); } } private void initializeZooKeeper() throws IOException { // open connection to zookeeper and set primary watcher - zooKeeper = new ZooKeeperWatcher(conf, serverInfo.getServerName(), this); - + zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER + "-" + + serverInfo.getServerName(), this); + // create the master address manager, register with zk, and start it masterAddressManager = new MasterAddressManager(zooKeeper, this); zooKeeper.registerListener(masterAddressManager); - masterAddressManager.monitorMaster(); + masterAddressManager.start(); + + // create the catalog tracker and start it + this.catalogTracker = new CatalogTracker(zooKeeper, connection, this, + conf.getInt("hbase.regionserver.catalog.timeout", 30000)); + catalogTracker.start(); } - private void initializeThreads() { + private void initializeThreads() throws IOException { this.workerThread = new Thread(worker); + // Start executor services + + HBaseExecutorServiceType.RS_OPEN_REGION.startExecutorService( + getServerName(), + conf.getInt("hbase.regionserver.executor.openregion.threads", 5)); + HBaseExecutorServiceType.RS_OPEN_ROOT.startExecutorService( + getServerName(), + conf.getInt("hbase.regionserver.executor.openroot.threads", 1)); + HBaseExecutorServiceType.RS_OPEN_META.startExecutorService( + getServerName(), + conf.getInt("hbase.regionserver.executor.openmeta.threads", 1)); + HBaseExecutorServiceType.RS_CLOSE_REGION.startExecutorService( + getServerName(), + conf.getInt("hbase.regionserver.executor.closeregion.threads", 5)); + HBaseExecutorServiceType.RS_CLOSE_ROOT.startExecutorService( + getServerName(), + conf.getInt("hbase.regionserver.executor.closeroot.threads", 1)); + HBaseExecutorServiceType.RS_CLOSE_META.startExecutorService( + getServerName(), + conf.getInt("hbase.regionserver.executor.closemeta.threads", 1)); + // Cache flushing thread. this.cacheFlusher = new MemStoreFlusher(conf, this); @@ -346,21 +387,21 @@ this.hlogRoller = new LogRoller(this); // Background thread to check for major compactions; needed if region - // has not gotten updates in a while. Make it run at a lesser frequency. - int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY + - ".multiplier", 1000); + // has not gotten updates in a while. Make it run at a lesser frequency. + int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY + + ".multiplier", 1000); this.majorCompactionChecker = new MajorCompactionChecker(this, - this.threadWakeFrequency * multiplier, this.stopRequested); + this.threadWakeFrequency * multiplier, this.stopRequested); - this.leases = new Leases( - (int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, - HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD), + this.leases = new Leases((int) conf.getLong( + HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, + HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD), this.threadWakeFrequency); } /** - * The HRegionServer sticks in this loop until closed. It repeatedly checks - * in with the HMaster, sending heartbeats & reports, and receiving HRegion + * The HRegionServer sticks in this loop until closed. It repeatedly checks in + * with the HMaster, sending heartbeats & reports, and receiving HRegion * load/unload instructions. */ public void run() { @@ -375,54 +416,54 @@ break; } sleeper.sleep(); - LOG.warn("No response from master on reportForDuty. Sleeping and " + - "then trying again."); + LOG.warn("No response from master on reportForDuty. Sleeping and " + + "then trying again."); } List outboundMessages = new ArrayList(); long lastMsg = 0; // Now ask master what it wants us to do and tell it what we have done for (int tries = 0; !stopRequested.get() && isHealthy();) { - // Try to get the root region location from the master. + // Try to get the root region location from zookeeper. if (!haveRootRegion.get()) { - HServerAddress rootServer = - ZKUtil.getDataAsAddress(zooKeeper, zooKeeper.rootServerZNode); + HServerAddress rootServer = catalogTracker.getRootLocation(); if (rootServer != null) { - // By setting the root region location, we bypass the wait imposed on + // By setting the root region location, we bypass the wait imposed + // on // HTable for all regions being assigned. - this.connection.setRootRegionLocation( - new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, rootServer)); + this.connection.setRootRegionLocation(new HRegionLocation( + HRegionInfo.ROOT_REGIONINFO, rootServer)); haveRootRegion.set(true); } } long now = System.currentTimeMillis(); // Drop into the send loop if msgInterval has elapsed or if something - // to send. If we fail talking to the master, then we'll sleep below + // to send. If we fail talking to the master, then we'll sleep below // on poll of the outboundMsgs blockingqueue. if ((now - lastMsg) >= msgInterval || !outboundMessages.isEmpty()) { try { doMetrics(); - MemoryUsage memory = - ManagementFactory.getMemoryMXBean().getHeapMemoryUsage(); - HServerLoad hsl = new HServerLoad(requestCount.get(), - (int)(memory.getUsed()/1024/1024), - (int)(memory.getMax()/1024/1024)); - for (HRegion r: onlineRegions.values()) { + MemoryUsage memory = ManagementFactory.getMemoryMXBean() + .getHeapMemoryUsage(); + HServerLoad hsl = new HServerLoad(requestCount.get(), (int) (memory + .getUsed() / 1024 / 1024), + (int) (memory.getMax() / 1024 / 1024)); + for (HRegion r : onlineRegions.values()) { hsl.addRegionInfo(createRegionLoad(r)); } this.serverInfo.setLoad(hsl); this.requestCount.set(0); addOutboundMsgs(outboundMessages); - HMsg msgs[] = this.hbaseMaster.regionServerReport( - serverInfo, outboundMessages.toArray(EMPTY_HMSG_ARRAY), - getMostLoadedRegions()); + HMsg msgs[] = this.hbaseMaster.regionServerReport(serverInfo, + outboundMessages.toArray(EMPTY_HMSG_ARRAY), + getMostLoadedRegions()); lastMsg = System.currentTimeMillis(); updateOutboundMsgs(outboundMessages); outboundMessages.clear(); if (this.quiesced.get() && onlineRegions.size() == 0) { // We've just told the master we're exiting because we aren't // serving any regions. So set the stop bit and exit. - LOG.info("Server quiesced and not serving any regions. " + - "Starting shutdown"); + LOG.info("Server quiesced and not serving any regions. " + + "Starting shutdown"); stopRequested.set(true); this.outboundMsgs.clear(); continue; @@ -430,38 +471,36 @@ // Queue up the HMaster's instruction stream for processing boolean restart = false; - for(int i = 0; - !restart && !stopRequested.get() && i < msgs.length; - i++) { + for (int i = 0; !restart && !stopRequested.get() && i < msgs.length; i++) { LOG.info(msgs[i].toString()); this.connection.unsetRootRegionLocation(); - switch(msgs[i].getType()) { + switch (msgs[i].getType()) { - case MSG_REGIONSERVER_STOP: - stopRequested.set(true); - break; + case MSG_REGIONSERVER_STOP: + stopRequested.set(true); + break; - case MSG_REGIONSERVER_QUIESCE: - if (!quiesceRequested) { - try { - toDo.put(new ToDoEntry(msgs[i])); - } catch (InterruptedException e) { - throw new RuntimeException("Putting into msgQueue was " + - "interrupted.", e); + case MSG_REGIONSERVER_QUIESCE: + if (!quiesceRequested) { + try { + toDo.put(new ToDoEntry(msgs[i])); + } catch (InterruptedException e) { + throw new RuntimeException("Putting into msgQueue was " + + "interrupted.", e); + } + quiesceRequested = true; } - quiesceRequested = true; - } - break; + break; - default: - if (fsOk) { - try { - toDo.put(new ToDoEntry(msgs[i])); - } catch (InterruptedException e) { - throw new RuntimeException("Putting into msgQueue was " + - "interrupted.", e); + default: + if (fsOk) { + try { + toDo.put(new ToDoEntry(msgs[i])); + } catch (InterruptedException e) { + throw new RuntimeException("Putting into msgQueue was " + + "interrupted.", e); + } } - } } } // Reset tries count if we had a successful transaction. @@ -493,17 +532,17 @@ } LOG.warn("Attempt=" + tries, e); // No point retrying immediately; this is probably connection to - // master issue. Doing below will cause us to sleep. + // master issue. Doing below will cause us to sleep. lastMsg = System.currentTimeMillis(); } } now = System.currentTimeMillis(); HMsg msg = this.outboundMsgs.poll((msgInterval - (now - lastMsg)), - TimeUnit.MILLISECONDS); + TimeUnit.MILLISECONDS); // If we got something, add it to list of things to send. - if (msg != null) outboundMessages.add(msg); - // Do some housekeeping before going back around - housekeeping(); + if (msg != null) { + outboundMessages.add(msg); + } } // for } catch (Throwable t) { if (!checkOOME(t)) { @@ -522,11 +561,13 @@ } } // Send cache a shutdown. - LruBlockCache c = (LruBlockCache)StoreFile.getBlockCache(this.conf); - if (c != null) c.shutdown(); + LruBlockCache c = (LruBlockCache) StoreFile.getBlockCache(this.conf); + if (c != null) { + c.shutdown(); + } // Send interrupts to wake up threads if sleeping so they notice shutdown. - // TODO: Should we check they are alive? If OOME could have exited already + // TODO: Should we check they are alive? If OOME could have exited already cacheFlusher.interruptIfNecessary(); compactSplitThread.interruptIfNecessary(); hlogRoller.interruptIfNecessary(); @@ -543,8 +584,8 @@ LOG.info("On abort, closed hlog"); } } catch (Throwable e) { - LOG.error("Unable to close log in abort", - RemoteExceptionHandler.checkThrowable(e)); + LOG.error("Unable to close log in abort", RemoteExceptionHandler + .checkThrowable(e)); } closeAllRegions(); // Don't leave any open file handles } @@ -556,25 +597,26 @@ hlog.closeAndDelete(); } } catch (Throwable e) { - LOG.error("Close and delete failed", - RemoteExceptionHandler.checkThrowable(e)); + LOG.error("Close and delete failed", RemoteExceptionHandler + .checkThrowable(e)); } try { HMsg[] exitMsg = new HMsg[closedRegions.size() + 1]; exitMsg[0] = REPORT_EXITING; // Tell the master what regions we are/were serving int i = 1; - for (HRegion region: closedRegions) { - exitMsg[i++] = new HMsg(HMsg.Type.MSG_REPORT_CLOSE, - region.getRegionInfo()); + for (HRegion region : closedRegions) { + exitMsg[i++] = new HMsg(HMsg.Type.MSG_REPORT_CLOSE, region + .getRegionInfo()); } - LOG.info("telling master that region server is shutting down at: " + - serverInfo.getServerName()); - hbaseMaster.regionServerReport(serverInfo, exitMsg, (HRegionInfo[])null); + LOG.info("telling master that region server is shutting down at: " + + serverInfo.getServerName()); + hbaseMaster.regionServerReport(serverInfo, exitMsg, + (HRegionInfo[]) null); } catch (Throwable e) { LOG.warn("Failed to send exiting message to master: ", - RemoteExceptionHandler.checkThrowable(e)); + RemoteExceptionHandler.checkThrowable(e)); } LOG.info("stopping server at: " + this.serverInfo.getServerName()); } @@ -586,7 +628,7 @@ } this.zooKeeper.close(); - + if (!killed) { join(); } @@ -595,6 +637,7 @@ /* * Add to the passed msgs messages to pass to the master. + * * @param msgs Current outboundMsgs array; we'll add messages to this List. */ private void addOutboundMsgs(final List msgs) { @@ -602,8 +645,8 @@ this.outboundMsgs.drainTo(msgs); return; } - OUTER: for (HMsg m: this.outboundMsgs) { - for (HMsg mm: msgs) { + OUTER: for (HMsg m : this.outboundMsgs) { + for (HMsg mm : msgs) { // Be careful don't add duplicates. if (mm.equals(m)) { continue OUTER; @@ -615,12 +658,15 @@ /* * Remove from this.outboundMsgs those messsages we sent the master. + * * @param msgs Messages we sent the master. */ private void updateOutboundMsgs(final List msgs) { - if (msgs.isEmpty()) return; - for (HMsg m: this.outboundMsgs) { - for (HMsg mm: msgs) { + if (msgs.isEmpty()) { + return; + } + for (HMsg m : this.outboundMsgs) { + for (HMsg mm : msgs) { if (mm.equals(m)) { this.outboundMsgs.remove(m); break; @@ -631,11 +677,12 @@ /* * Run init. Sets up hlog and starts up all server threads. + * * @param c Extra configuration. */ protected void init(final MapWritable c) throws IOException { try { - for (Map.Entry e: c.entrySet()) { + for (Map.Entry e : c.entrySet()) { String key = e.getKey().toString(); String value = e.getValue().toString(); if (LOG.isDebugEnabled()) { @@ -646,21 +693,23 @@ // Master may have sent us a new address with the other configs. // Update our address in this case. See HBASE-719 String hra = conf.get("hbase.regionserver.address"); - // TODO: The below used to be this.address != null. Was broken by what + // TODO: The below used to be this.address != null. Was broken by what // looks like a mistake in: // - // HBASE-1215 migration; metautils scan of meta region was broken; wouldn't see first row + // HBASE-1215 migration; metautils scan of meta region was broken; + // wouldn't see first row // ------------------------------------------------------------------------ - // r796326 | stack | 2009-07-21 07:40:34 -0700 (Tue, 21 Jul 2009) | 38 lines + // r796326 | stack | 2009-07-21 07:40:34 -0700 (Tue, 21 Jul 2009) | 38 + // lines if (hra != null) { - HServerAddress hsa = new HServerAddress (hra, - this.serverInfo.getServerAddress().getPort()); - LOG.info("Master passed us address to use. Was=" + - this.serverInfo.getServerAddress() + ", Now=" + hra); + HServerAddress hsa = new HServerAddress(hra, this.serverInfo + .getServerAddress().getPort()); + LOG.info("Master passed us address to use. Was=" + + this.serverInfo.getServerAddress() + ", Now=" + hra); this.serverInfo.setServerAddress(hsa); } // Master sent us hbase.rootdir to use. Should be fully qualified - // path with file system specification included. Set 'fs.defaultFS' + // path with file system specification included. Set 'fs.defaultFS' // to match the filesystem on hbase.rootdir else underlying hadoop hdfs // accessors will be going against wrong filesystem (unless all is set // to defaults). @@ -677,13 +726,15 @@ this.isOnline = false; this.stopRequested.set(true); throw convertThrowableToIOE(cleanup(e, "Failed init"), - "Region server startup failed"); + "Region server startup failed"); } } /* * @param r Region to get RegionLoad for. + * * @return RegionLoad instance. + * * @throws IOException */ private HServerLoad.RegionLoad createRegionLoad(final HRegion r) { @@ -691,20 +742,18 @@ int stores = 0; int storefiles = 0; int storefileSizeMB = 0; - int memstoreSizeMB = (int)(r.memstoreSize.get()/1024/1024); + int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024); int storefileIndexSizeMB = 0; synchronized (r.stores) { stores += r.stores.size(); - for (Store store: r.stores.values()) { + for (Store store : r.stores.values()) { storefiles += store.getStorefilesCount(); - storefileSizeMB += - (int)(store.getStorefilesSize()/1024/1024); - storefileIndexSizeMB += - (int)(store.getStorefilesIndexSize()/1024/1024); + storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024); + storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024); } } return new HServerLoad.RegionLoad(name, stores, storefiles, - storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB); + storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB); } /** @@ -712,14 +761,16 @@ * @return An instance of RegionLoad. * @throws IOException */ - public HServerLoad.RegionLoad createRegionLoad(final byte [] regionName) { + public HServerLoad.RegionLoad createRegionLoad(final byte[] regionName) { return createRegionLoad(this.onlineRegions.get(Bytes.mapKey(regionName))); } /* - * Cleanup after Throwable caught invoking method. Converts t - * to IOE if it isn't already. + * Cleanup after Throwable caught invoking method. Converts t to + * IOE if it isn't already. + * * @param t Throwable + * * @return Throwable converted to an IOE; methods can only let out IOEs. */ private Throwable cleanup(final Throwable t) { @@ -727,10 +778,13 @@ } /* - * Cleanup after Throwable caught invoking method. Converts t - * to IOE if it isn't already. + * Cleanup after Throwable caught invoking method. Converts t to + * IOE if it isn't already. + * * @param t Throwable - * @param msg Message to log in error. Can be null. + * + * @param msg Message to log in error. Can be null. + * * @return Throwable converted to an IOE; methods can only let out IOEs. */ private Throwable cleanup(final Throwable t, final String msg) { @@ -747,6 +801,7 @@ /* * @param t + * * @return Make t an IOE if it isn't already. */ private IOException convertThrowableToIOE(final Throwable t) { @@ -755,36 +810,38 @@ /* * @param t + * * @param msg Message to put in new IOE if passed t is not an IOE + * * @return Make t an IOE if it isn't already. */ - private IOException convertThrowableToIOE(final Throwable t, - final String msg) { - return (t instanceof IOException? (IOException)t: - msg == null || msg.length() == 0? - new IOException(t): new IOException(msg, t)); + private IOException convertThrowableToIOE(final Throwable t, final String msg) { + return (t instanceof IOException ? (IOException) t : msg == null + || msg.length() == 0 ? new IOException(t) : new IOException(msg, t)); } + /* * Check if an OOME and if so, call abort. + * * @param e + * * @return True if we OOME'd and are aborting. */ public boolean checkOOME(final Throwable e) { boolean stop = false; - if (e instanceof OutOfMemoryError || - (e.getCause() != null && e.getCause() instanceof OutOfMemoryError) || - (e.getMessage() != null && - e.getMessage().contains("java.lang.OutOfMemoryError"))) { + if (e instanceof OutOfMemoryError + || (e.getCause() != null && e.getCause() instanceof OutOfMemoryError) + || (e.getMessage() != null && e.getMessage().contains( + "java.lang.OutOfMemoryError"))) { abort("OutOfMemoryError, aborting", e); stop = true; } return stop; } - /** - * Checks to see if the file system is still accessible. - * If not, sets abortRequested and stopRequested + * Checks to see if the file system is still accessible. If not, sets + * abortRequested and stopRequested * * @return false if file system is not available */ @@ -807,8 +864,8 @@ private static class MajorCompactionChecker extends Chore { private final HRegionServer instance; - MajorCompactionChecker(final HRegionServer h, - final int sleepTime, final AtomicBoolean stopper) { + MajorCompactionChecker(final HRegionServer h, final int sleepTime, + final AtomicBoolean stopper) { super("MajorCompactionChecker", sleepTime, stopper); this.instance = h; LOG.info("Runs every " + sleepTime + "ms"); @@ -816,14 +873,12 @@ @Override protected void chore() { - Set keys = this.instance.onlineRegions.keySet(); - for (Integer i: keys) { - HRegion r = this.instance.onlineRegions.get(i); + for (HRegion r : this.instance.onlineRegions.values()) { try { if (r != null && r.isMajorCompaction()) { - // Queue a compaction. Will recognize if major is needed. - this.instance.compactSplitThread. - compactionRequested(r, getName() + " requests major compaction"); + // Queue a compaction. Will recognize if major is needed. + this.instance.compactSplitThread.compactionRequested(r, getName() + + " requests major compaction"); } } catch (IOException e) { LOG.warn("Failed major compaction check on " + r, e); @@ -833,9 +888,10 @@ } /** - * Report the status of the server. A server is online once all the startup - * is completed (setting up filesystem, starting service threads, etc.). This + * Report the status of the server. A server is online once all the startup is + * completed (setting up filesystem, starting service threads, etc.). This * method is designed mostly to be useful in tests. + * * @return true if online, false if not. */ public boolean isOnline() { @@ -849,22 +905,22 @@ LOG.debug("Log dir " + logdir); } if (fs.exists(logdir)) { - throw new RegionServerRunningException("region server already " + - "running at " + this.serverInfo.getServerName() + - " because logdir " + logdir.toString() + " exists"); + throw new RegionServerRunningException("region server already " + + "running at " + this.serverInfo.getServerName() + + " because logdir " + logdir.toString() + " exists"); } HLog newlog = instantiateHLog(logdir, oldLogDir); return newlog; } // instantiate - protected HLog instantiateHLog(Path logdir, Path oldLogDir) throws IOException { + protected HLog instantiateHLog(Path logdir, Path oldLogDir) + throws IOException { HLog newlog = new HLog(fs, logdir, oldLogDir, conf, hlogRoller, null, serverInfo.getServerAddress().toString()); return newlog; } - protected LogRoller getLogRoller() { return hlogRoller; } @@ -884,19 +940,19 @@ this.metrics.regions.set(this.onlineRegions.size()); this.metrics.incrementRequests(this.requestCount.get()); // Is this too expensive every three seconds getting a lock on onlineRegions - // and then per store carried? Can I make metrics be sloppier and avoid + // and then per store carried? Can I make metrics be sloppier and avoid // the synchronizations? int stores = 0; int storefiles = 0; long memstoreSize = 0; long storefileIndexSize = 0; synchronized (this.onlineRegions) { - for (Map.Entry e: this.onlineRegions.entrySet()) { + for (Map.Entry e : this.onlineRegions.entrySet()) { HRegion r = e.getValue(); memstoreSize += r.memstoreSize.get(); synchronized (r.stores) { stores += r.stores.size(); - for(Map.Entry ee: r.stores.entrySet()) { + for (Map.Entry ee : r.stores.entrySet()) { Store store = ee.getValue(); storefiles += store.getStorefilesCount(); storefileIndexSize += store.getStorefilesIndexSize(); @@ -906,12 +962,13 @@ } this.metrics.stores.set(stores); this.metrics.storefiles.set(storefiles); - this.metrics.memstoreSizeMB.set((int)(memstoreSize/(1024*1024))); - this.metrics.storefileIndexSizeMB.set((int)(storefileIndexSize/(1024*1024))); - this.metrics.compactionQueueSize.set(compactSplitThread. - getCompactionQueueSize()); + this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024))); + this.metrics.storefileIndexSizeMB + .set((int) (storefileIndexSize / (1024 * 1024))); + this.metrics.compactionQueueSize.set(compactSplitThread + .getCompactionQueueSize()); - LruBlockCache lruBlockCache = (LruBlockCache)StoreFile.getBlockCache(conf); + LruBlockCache lruBlockCache = (LruBlockCache) StoreFile.getBlockCache(conf); if (lruBlockCache != null) { this.metrics.blockCacheCount.set(lruBlockCache.size()); this.metrics.blockCacheFree.set(lruBlockCache.getFreeSize()); @@ -932,14 +989,14 @@ /* * Start maintanence Threads, Server, Worker and lease checker threads. * Install an UncaughtExceptionHandler that calls abort of RegionServer if we - * get an unhandled exception. We cannot set the handler on all threads. - * Server's internal Listener thread is off limits. For Server, if an OOME, - * it waits a while then retries. Meantime, a flush or a compaction that - * tries to run should trigger same critical condition and the shutdown will - * run. On its way out, this server will shut down Server. Leases are sort - * of inbetween. It has an internal thread that while it inherits from - * Chore, it keeps its own internal stop mechanism so needs to be stopped - * by this hosting server. Worker logs the exception and exits. + * get an unhandled exception. We cannot set the handler on all threads. + * Server's internal Listener thread is off limits. For Server, if an OOME, it + * waits a while then retries. Meantime, a flush or a compaction that tries to + * run should trigger same critical condition and the shutdown will run. On + * its way out, this server will shut down Server. Leases are sort of + * inbetween. It has an internal thread that while it inherits from Chore, it + * keeps its own internal stop mechanism so needs to be stopped by this + * hosting server. Worker logs the exception and exits. */ private void startServiceThreads() throws IOException { String n = Thread.currentThread().getName(); @@ -948,17 +1005,16 @@ abort("Uncaught exception in service thread " + t.getName(), e); } }; - Threads.setDaemonThreadRunning(this.hlogRoller, n + ".logRoller", + Threads.setDaemonThreadRunning(this.hlogRoller, n + ".logRoller", handler); + Threads.setDaemonThreadRunning(this.cacheFlusher, n + ".cacheFlusher", handler); - Threads.setDaemonThreadRunning(this.cacheFlusher, n + ".cacheFlusher", - handler); Threads.setDaemonThreadRunning(this.compactSplitThread, n + ".compactor", handler); Threads.setDaemonThreadRunning(this.workerThread, n + ".worker", handler); - Threads.setDaemonThreadRunning(this.majorCompactionChecker, - n + ".majorCompactionChecker", handler); + Threads.setDaemonThreadRunning(this.majorCompactionChecker, n + + ".majorCompactionChecker", handler); - // Leases is not a Thread. Internally it runs a daemon thread. If it gets + // Leases is not a Thread. Internally it runs a daemon thread. If it gets // an unhandled exception, it will just exit. this.leases.setName(n + ".leaseChecker"); this.leases.start(); @@ -966,7 +1022,8 @@ int port = this.conf.getInt("hbase.regionserver.info.port", 60030); // -1 is for disabling info server if (port >= 0) { - String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0"); + String addr = this.conf.get("hbase.regionserver.info.bindAddress", + "0.0.0.0"); // check if auto port bind enabled boolean auto = this.conf.getBoolean("hbase.regionserver.info.port.auto", false); @@ -977,7 +1034,7 @@ this.infoServer.start(); break; } catch (BindException e) { - if (!auto){ + if (!auto) { // auto bind disabled throw BindException throw e; } @@ -986,17 +1043,17 @@ port++; // update HRS server info port. this.serverInfo = new HServerInfo(this.serverInfo.getServerAddress(), - this.serverInfo.getStartCode(), port, - this.serverInfo.getHostname()); + this.serverInfo.getStartCode(), port, this.serverInfo + .getHostname()); } } } - // Start Server. This service is like leases in that it internally runs + // Start Server. This service is like leases in that it internally runs // a thread. this.server.start(); - LOG.info("HRegionServer started at: " + - this.serverInfo.getServerAddress().toString()); + LOG.info("HRegionServer started at: " + + this.serverInfo.getServerAddress().toString()); } /* @@ -1008,9 +1065,9 @@ return false; } // Verify that all threads are alive - if (!(leases.isAlive() && compactSplitThread.isAlive() && - cacheFlusher.isAlive() && hlogRoller.isAlive() && - workerThread.isAlive() && this.majorCompactionChecker.isAlive())) { + if (!(leases.isAlive() && compactSplitThread.isAlive() + && cacheFlusher.isAlive() && hlogRoller.isAlive() + && workerThread.isAlive() && this.majorCompactionChecker.isAlive())) { // One or more threads are no longer alive - shut down stop(); return false; @@ -1018,58 +1075,32 @@ return true; } - /* - * Run some housekeeping tasks. - */ - private void housekeeping() { - // If the todo list has > 0 messages, iterate looking for open region - // messages. Send the master a message that we're working on its - // processing so it doesn't assign the region elsewhere. - if (this.toDo.isEmpty()) { - return; - } - // This iterator isn't safe if elements are gone and HRS.Worker could - // remove them (it already checks for null there). Goes from oldest. - for (ToDoEntry e: this.toDo) { - if(e == null) { - LOG.warn("toDo gave a null entry during iteration"); - break; - } - HMsg msg = e.msg; - if (msg != null) { - if (msg.isType(HMsg.Type.MSG_REGION_OPEN)) { - addProcessingMessage(msg.getRegionInfo()); - } - } else { - LOG.warn("Message is empty: " + e); - } - } - } - /** @return the HLog */ public HLog getLog() { return this.hlog; } /** - * Sets a flag that will cause all the HRegionServer threads to shut down - * in an orderly fashion. Used by unit tests. + * Sets a flag that will cause all the HRegionServer threads to shut down in + * an orderly fashion. Used by unit tests. */ public void stop() { this.stopRequested.set(true); - synchronized(this) { + synchronized (this) { // Wakes run() if it is sleeping notifyAll(); // FindBugs NN_NAKED_NOTIFY } } /** - * Cause the server to exit without closing the regions it is serving, the - * log it is using and without notifying the master. - * Used unit testing and on catastrophic events such as HDFS is yanked out - * from under hbase or we OOME. - * @param reason the reason we are aborting - * @param cause the exception that caused the abort, or null + * Cause the server to exit without closing the regions it is serving, the log + * it is using and without notifying the master. Used unit testing and on + * catastrophic events such as HDFS is yanked out from under hbase or we OOME. + * + * @param reason + * the reason we are aborting + * @param cause + * the exception that caused the abort, or null */ public void abort(String reason, Throwable cause) { if (cause != null) { @@ -1084,7 +1115,7 @@ } stop(); } - + /** * @see HRegionServer#abort(String, Throwable) */ @@ -1093,9 +1124,9 @@ } /* - * Simulate a kill -9 of this server. - * Exits w/o closing regions or cleaninup logs but it does close socket in - * case want to bring up server on old hostname+port immediately. + * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup + * logs but it does close socket in case want to bring up server on old + * hostname+port immediately. */ protected void kill() { this.killed = true; @@ -1103,8 +1134,8 @@ } /** - * Wait on all threads to finish. - * Presumption is that all closes and stops have already been called. + * Wait on all threads to finish. Presumption is that all closes and stops + * have already been called. */ protected void join() { Threads.shutdown(this.majorCompactionChecker); @@ -1112,20 +1143,21 @@ Threads.shutdown(this.cacheFlusher); Threads.shutdown(this.compactSplitThread); Threads.shutdown(this.hlogRoller); + HBaseExecutorService.shutdown(); } /** * Get the current master from ZooKeeper and open the RPC connection to it. - * - * Method will block until a master is available. You can break from this + * + * Method will block until a master is available. You can break from this * block by requesting the server stop. - * + * * @return */ private boolean getMaster() { HServerAddress masterAddress = null; - while((masterAddress = masterAddressManager.getMasterAddress()) == null) { - if(stopRequested.get()) { + while ((masterAddress = masterAddressManager.getMasterAddress()) == null) { + if (stopRequested.get()) { return false; } LOG.debug("No master found, will retry"); @@ -1135,11 +1167,12 @@ HMasterRegionInterface master = null; while (!stopRequested.get() && master == null) { try { - // Do initial RPC setup. The final argument indicates that the RPC + // Do initial RPC setup. The final argument indicates that the RPC // should retry indefinitely. - master = (HMasterRegionInterface)HBaseRPC.waitForProxy( - HMasterRegionInterface.class, HBaseRPCProtocolVersion.versionID, - masterAddress.getInetSocketAddress(), this.conf, -1, this.rpcTimeout); + master = (HMasterRegionInterface) HBaseRPC.waitForProxy( + HMasterRegionInterface.class, HBaseRPCProtocolVersion.versionID, + masterAddress.getInetSocketAddress(), this.conf, -1, + this.rpcTimeout); } catch (IOException e) { LOG.warn("Unable to connect to master. Retrying. Error was:", e); sleeper.sleep(); @@ -1150,8 +1183,8 @@ } /* - * Let the master know we're here - * Run initialization using parameters passed us by the master. + * Let the master know we're here Run initialization using parameters passed + * us by the master. */ private MapWritable reportForDuty() { while (!stopRequested.get() && !getMaster()) { @@ -1161,20 +1194,21 @@ MapWritable result = null; long lastMsg = 0; - while(!stopRequested.get()) { + while (!stopRequested.get()) { try { this.requestCount.set(0); - MemoryUsage memory = - ManagementFactory.getMemoryMXBean().getHeapMemoryUsage(); - HServerLoad hsl = new HServerLoad(0, (int)memory.getUsed()/1024/1024, - (int)memory.getMax()/1024/1024); + MemoryUsage memory = ManagementFactory.getMemoryMXBean() + .getHeapMemoryUsage(); + HServerLoad hsl = new HServerLoad(0, + (int) memory.getUsed() / 1024 / 1024, + (int) memory.getMax() / 1024 / 1024); this.serverInfo.setLoad(hsl); - if (LOG.isDebugEnabled()) + if (LOG.isDebugEnabled()) { LOG.debug("sending initial server load: " + hsl); + } lastMsg = System.currentTimeMillis(); - ZKUtil.setAddressAndWatch(zooKeeper, - ZKUtil.joinZNode(zooKeeper.rsZNode, ZKUtil.getNodeName(serverInfo)), - address); + ZKUtil.setAddressAndWatch(zooKeeper, ZKUtil.joinZNode( + zooKeeper.rsZNode, ZKUtil.getNodeName(serverInfo)), address); result = this.hbaseMaster.regionServerStartup(this.serverInfo); break; } catch (IOException e) { @@ -1199,16 +1233,16 @@ */ void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA, HRegionInfo newRegionB) { - this.outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT_INCLUDES_DAUGHTERS, - oldRegion, newRegionA, newRegionB, - Bytes.toBytes("Daughters; " + - newRegionA.getRegionNameAsString() + ", " + - newRegionB.getRegionNameAsString()))); + this.outboundMsgs.add(new HMsg( + HMsg.Type.MSG_REPORT_SPLIT_INCLUDES_DAUGHTERS, oldRegion, newRegionA, + newRegionB, Bytes.toBytes("Daughters; " + + newRegionA.getRegionNameAsString() + ", " + + newRegionB.getRegionNameAsString()))); } - ////////////////////////////////////////////////////////////////////////////// + // //////////////////////////////////////////////////////////////////////////// // HMaster-given operations - ////////////////////////////////////////////////////////////////////////////// + // //////////////////////////////////////////////////////////////////////////// /* * Data structure to hold a HMsg and retries count. @@ -1229,90 +1263,65 @@ /** Thread that performs long running requests from the master */ class Worker implements Runnable { void stop() { - synchronized(toDo) { + synchronized (toDo) { toDo.notifyAll(); } } public void run() { try { - while(!stopRequested.get()) { + while (!stopRequested.get()) { ToDoEntry e = null; try { e = toDo.poll(threadWakeFrequency, TimeUnit.MILLISECONDS); - if(e == null || stopRequested.get()) { + if (e == null || stopRequested.get()) { continue; } LOG.info("Worker: " + e.msg); HRegion region = null; HRegionInfo info = e.msg.getRegionInfo(); - switch(e.msg.getType()) { + switch (e.msg.getType()) { - case MSG_REGIONSERVER_QUIESCE: - closeUserRegions(); - break; + case MSG_REGIONSERVER_QUIESCE: + closeUserRegions(); + break; - case MSG_REGION_OPEN: - // Open a region - if (!haveRootRegion.get() && !info.isRootRegion()) { - // root region is not online yet. requeue this task - LOG.info("putting region open request back into queue because" + - " root region is not yet available"); - try { - toDo.put(e); - } catch (InterruptedException ex) { - LOG.warn("insertion into toDo queue was interrupted", ex); - break; - } - } - openRegion(info); - break; + case MSG_REGION_SPLIT: + region = getRegion(info.getRegionName()); + region.flushcache(); + region.shouldSplit(true); + // force a compaction; split will be side-effect. + compactSplitThread.compactionRequested(region, e.msg.getType() + .name()); + break; - case MSG_REGION_CLOSE: - // Close a region - closeRegion(e.msg.getRegionInfo(), true); - break; + case MSG_REGION_MAJOR_COMPACT: + case MSG_REGION_COMPACT: + // Compact a region + region = getRegion(info.getRegionName()); + compactSplitThread.compactionRequested(region, e.msg + .isType(Type.MSG_REGION_MAJOR_COMPACT), e.msg.getType() + .name()); + break; - case MSG_REGION_CLOSE_WITHOUT_REPORT: - // Close a region, don't reply - closeRegion(e.msg.getRegionInfo(), false); - break; + case MSG_REGION_FLUSH: + region = getRegion(info.getRegionName()); + region.flushcache(); + break; - case MSG_REGION_SPLIT: - region = getRegion(info.getRegionName()); - region.flushcache(); - region.shouldSplit(true); - // force a compaction; split will be side-effect. - compactSplitThread.compactionRequested(region, - e.msg.getType().name()); - break; + case TESTING_MSG_BLOCK_RS: + while (!stopRequested.get()) { + Threads.sleep(1000); + LOG.info("Regionserver blocked by " + + HMsg.Type.TESTING_MSG_BLOCK_RS + "; " + + stopRequested.get()); + } + break; - case MSG_REGION_MAJOR_COMPACT: - case MSG_REGION_COMPACT: - // Compact a region - region = getRegion(info.getRegionName()); - compactSplitThread.compactionRequested(region, - e.msg.isType(Type.MSG_REGION_MAJOR_COMPACT), - e.msg.getType().name()); - break; - - case MSG_REGION_FLUSH: - region = getRegion(info.getRegionName()); - region.flushcache(); - break; - - case TESTING_MSG_BLOCK_RS: - while (!stopRequested.get()) { - Threads.sleep(1000); - LOG.info("Regionserver blocked by " + - HMsg.Type.TESTING_MSG_BLOCK_RS + "; " + stopRequested.get()); - } - break; - - default: - throw new AssertionError( - "Impossible state during msg processing. Instruction: " - + e.msg.toString()); + default: + throw new AssertionError( + "Impossible state during msg processing. Instruction: " + + e.msg.toString()); } } catch (InterruptedException ex) { LOG.warn("Processing Worker queue", ex); @@ -1320,25 +1329,25 @@ if (ex instanceof IOException) { ex = RemoteExceptionHandler.checkIOException((IOException) ex); } - if(e != null && e.tries.get() < numRetries) { + if (e != null && e.tries.get() < numRetries) { LOG.warn(ex); e.tries.incrementAndGet(); try { toDo.put(e); } catch (InterruptedException ie) { - throw new RuntimeException("Putting into msgQueue was " + - "interrupted.", ex); + throw new RuntimeException("Putting into msgQueue was " + + "interrupted.", ex); } } else { - LOG.error("unable to process message" + - (e != null ? (": " + e.msg.toString()) : ""), ex); + LOG.error("unable to process message" + + (e != null ? (": " + e.msg.toString()) : ""), ex); if (!checkFileSystem()) { break; } } } } - } catch(Throwable t) { + } catch (Throwable t) { if (!checkOOME(t)) { LOG.fatal("Unhandled exception", t); } @@ -1348,111 +1357,34 @@ } } - void openRegion(final HRegionInfo regionInfo) { - Integer mapKey = Bytes.mapKey(regionInfo.getRegionName()); - HRegion region = this.onlineRegions.get(mapKey); - RSZookeeperUpdater zkUpdater = - new RSZookeeperUpdater(zooKeeper, serverInfo.getServerName(), - regionInfo.getEncodedName()); - if (region == null) { - try { - zkUpdater.startRegionOpenEvent(null, true); - region = instantiateRegion(regionInfo, this.hlog); - // Startup a compaction early if one is needed, if region has references - // or if a store has too many store files - if (region.hasReferences() || region.hasTooManyStoreFiles()) { - this.compactSplitThread.compactionRequested(region, - region.hasReferences() ? "Region has references on open" : - "Region has too many store files"); - } - } catch (Throwable e) { - Throwable t = cleanup(e, - "Error opening " + regionInfo.getRegionNameAsString()); - // TODO: add an extra field in HRegionInfo to indicate that there is - // an error. We can't do that now because that would be an incompatible - // change that would require a migration - try { - HMsg hmsg = new HMsg(HMsg.Type.MSG_REPORT_CLOSE, - regionInfo, - StringUtils.stringifyException(t).getBytes()); - zkUpdater.abortOpenRegion(hmsg); - } catch (IOException e1) { - // TODO: Can we recover? Should be throw RTE? - LOG.error("Failed to abort open region " + regionInfo.getRegionNameAsString(), e1); - } - return; - } - this.lock.writeLock().lock(); - try { - this.onlineRegions.put(mapKey, region); - } finally { - this.lock.writeLock().unlock(); - } - } - try { - HMsg hmsg = new HMsg(HMsg.Type.MSG_REPORT_OPEN, regionInfo); - zkUpdater.finishRegionOpenEvent(hmsg); - } catch (IOException e) { - LOG.error("Failed to mark region " + regionInfo.getRegionNameAsString() + " as opened", e); - } - } - /* * @param regionInfo RegionInfo for the Region we're to instantiate and * initialize. + * * @param wal Set into here the regions' seqid. + * + * @param reporter periodic callback + * * @return + * * @throws IOException */ - protected HRegion instantiateRegion(final HRegionInfo regionInfo, final HLog wal) - throws IOException { - Path dir = - HTableDescriptor.getTableDir(rootDir, regionInfo.getTableDesc().getName()); + public HRegion instantiateRegion(final HRegionInfo regionInfo, + final HLog wal, Progressable reporter) throws IOException { + Path dir = HTableDescriptor.getTableDir(rootDir, regionInfo.getTableDesc() + .getName()); HRegion r = HRegion.newHRegion(dir, this.hlog, this.fs, conf, regionInfo, - this.cacheFlusher); - long seqid = r.initialize(new Progressable() { - public void progress() { - addProcessingMessage(regionInfo); - } - }); + this.cacheFlusher); + long seqid = r.initialize(reporter); // If a wal and its seqid is < that of new region, use new regions seqid. if (wal != null) { - if (seqid > wal.getSequenceNumber()) wal.setSequenceNumber(seqid); + if (seqid > wal.getSequenceNumber()) { + wal.setSequenceNumber(seqid); + } } return r; } - /** - * Add a MSG_REPORT_PROCESS_OPEN to the outbound queue. - * This method is called while region is in the queue of regions to process - * and then while the region is being opened, it is called from the Worker - * thread that is running the region open. - * @param hri Region to add the message for - */ - public void addProcessingMessage(final HRegionInfo hri) { - getOutboundMsgs().add(new HMsg(HMsg.Type.MSG_REPORT_PROCESS_OPEN, hri)); - } - - protected void closeRegion(final HRegionInfo hri, final boolean reportWhenCompleted) - throws IOException { - RSZookeeperUpdater zkUpdater = null; - if(reportWhenCompleted) { - zkUpdater = new RSZookeeperUpdater(zooKeeper, - serverInfo.getServerName(), hri.getEncodedName()); - zkUpdater.startRegionCloseEvent(null, false); - } - HRegion region = this.removeFromOnlineRegions(hri); - if (region != null) { - region.close(); - if(reportWhenCompleted) { - if(zkUpdater != null) { - HMsg hmsg = new HMsg(HMsg.Type.MSG_REPORT_CLOSE, hri, null); - zkUpdater.finishRegionCloseEvent(hmsg); - } - } - } - } - /** Called either when the master tells us to restart or from stop() */ ArrayList closeAllRegions() { ArrayList regionsToClose = new ArrayList(); @@ -1463,21 +1395,22 @@ } finally { this.lock.writeLock().unlock(); } - // Close any outstanding scanners. Means they'll get an UnknownScanner + // Close any outstanding scanners. Means they'll get an UnknownScanner // exception next time they come in. - for (Map.Entry e: this.scanners.entrySet()) { + for (Map.Entry e : this.scanners.entrySet()) { try { e.getValue().close(); } catch (IOException ioe) { LOG.warn("Closing scanner " + e.getKey(), ioe); } } - for (HRegion region: regionsToClose) { + for (HRegion region : regionsToClose) { if (LOG.isDebugEnabled()) { LOG.debug("closing region " + Bytes.toString(region.getRegionName())); } try { - region.close(abortRequested); + new CloseRegionHandler(this, region.getRegionInfo(), abortRequested) + .execute(); } catch (Throwable e) { cleanup(e, "Error closing " + Bytes.toString(region.getRegionName())); } @@ -1505,7 +1438,7 @@ r.close(); } catch (Throwable e) { LOG.error("Error closing region " + r.toString(), - RemoteExceptionHandler.checkThrowable(e)); + RemoteExceptionHandler.checkThrowable(e)); } } } @@ -1516,9 +1449,9 @@ this.lock.writeLock().lock(); try { synchronized (onlineRegions) { - for (Iterator> i = - onlineRegions.entrySet().iterator(); i.hasNext();) { - Map.Entry e = i.next(); + for (Iterator> i = onlineRegions.entrySet() + .iterator(); i.hasNext();) { + Map.Entry e = i.next(); HRegion r = e.getValue(); if (!r.getRegionInfo().isMetaRegion()) { regionsToClose.add(r); @@ -1560,16 +1493,14 @@ // HRegionInterface // - public HRegionInfo getRegionInfo(final byte [] regionName) - throws NotServingRegionException { + public HRegionInfo getRegionInfo(final byte[] regionName) + throws NotServingRegionException { requestCount.incrementAndGet(); return getRegion(regionName).getRegionInfo(); } - - public Result getClosestRowBefore(final byte [] regionName, - final byte [] row, final byte [] family) - throws IOException { + public Result getClosestRowBefore(final byte[] regionName, final byte[] row, + final byte[] family) throws IOException { checkOpen(); requestCount.incrementAndGet(); try { @@ -1585,33 +1516,33 @@ } /** {@inheritDoc} */ - public Result get(byte [] regionName, Get get) throws IOException { + public Result get(byte[] regionName, Get get) throws IOException { checkOpen(); requestCount.incrementAndGet(); try { HRegion region = getRegion(regionName); return region.get(get, getLockFromId(get.getLockId())); - } catch(Throwable t) { + } catch (Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } - public boolean exists(byte [] regionName, Get get) throws IOException { + public boolean exists(byte[] regionName, Get get) throws IOException { checkOpen(); requestCount.incrementAndGet(); try { HRegion region = getRegion(regionName); Result r = region.get(get, getLockFromId(get.getLockId())); return r != null && !r.isEmpty(); - } catch(Throwable t) { + } catch (Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } - public void put(final byte [] regionName, final Put put) - throws IOException { - if (put.getRow() == null) + public void put(final byte[] regionName, final Put put) throws IOException { + if (put.getRow() == null) { throw new IllegalArgumentException("update has null row"); + } checkOpen(); this.requestCount.incrementAndGet(); @@ -1628,7 +1559,7 @@ } public int put(final byte[] regionName, final List puts) - throws IOException { + throws IOException { checkOpen(); HRegion region = null; try { @@ -1636,21 +1567,22 @@ if (!region.getRegionInfo().isMetaTable()) { this.cacheFlusher.reclaimMemStoreMemory(); } - + @SuppressWarnings("unchecked") Pair[] putsWithLocks = new Pair[puts.size()]; - + int i = 0; for (Put p : puts) { Integer lock = getLockFromId(p.getLockId()); putsWithLocks[i++] = new Pair(p, lock); } - + this.requestCount.addAndGet(puts.size()); OperationStatusCode[] codes = region.put(putsWithLocks); for (i = 0; i < codes.length; i++) { - if (codes[i] != OperationStatusCode.SUCCESS) + if (codes[i] != OperationStatusCode.SUCCESS) { return i; + } } return -1; } catch (Throwable t) { @@ -1658,8 +1590,8 @@ } } - private boolean checkAndMutate(final byte[] regionName, final byte [] row, - final byte [] family, final byte [] qualifier, final byte [] value, + private boolean checkAndMutate(final byte[] regionName, final byte[] row, + final byte[] family, final byte[] qualifier, final byte[] value, final Writable w, Integer lock) throws IOException { checkOpen(); this.requestCount.incrementAndGet(); @@ -1668,28 +1600,28 @@ if (!region.getRegionInfo().isMetaTable()) { this.cacheFlusher.reclaimMemStoreMemory(); } - return region.checkAndMutate(row, family, qualifier, value, w, lock, - true); + return region + .checkAndMutate(row, family, qualifier, value, w, lock, true); } catch (Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } - /** * * @param regionName * @param row * @param family * @param qualifier - * @param value the expected value + * @param value + * the expected value * @param put * @throws IOException * @return true if the new put was execute, false otherwise */ - public boolean checkAndPut(final byte[] regionName, final byte [] row, - final byte [] family, final byte [] qualifier, final byte [] value, - final Put put) throws IOException{ + public boolean checkAndPut(final byte[] regionName, final byte[] row, + final byte[] family, final byte[] qualifier, final byte[] value, + final Put put) throws IOException { return checkAndMutate(regionName, row, family, qualifier, value, put, getLockFromId(put.getLockId())); } @@ -1700,14 +1632,15 @@ * @param row * @param family * @param qualifier - * @param value the expected value + * @param value + * the expected value * @param delete * @throws IOException * @return true if the new put was execute, false otherwise */ - public boolean checkAndDelete(final byte[] regionName, final byte [] row, - final byte [] family, final byte [] qualifier, final byte [] value, - final Delete delete) throws IOException{ + public boolean checkAndDelete(final byte[] regionName, final byte[] row, + final byte[] family, final byte[] qualifier, final byte[] value, + final Delete delete) throws IOException { return checkAndMutate(regionName, row, family, qualifier, value, delete, getLockFromId(delete.getLockId())); } @@ -1716,8 +1649,7 @@ // remote scanner interface // - public long openScanner(byte [] regionName, Scan scan) - throws IOException { + public long openScanner(byte[] regionName, Scan scan) throws IOException { checkOpen(); NullPointerException npe = null; if (regionName == null) { @@ -1742,20 +1674,19 @@ scannerId = rand.nextLong(); String scannerName = String.valueOf(scannerId); scanners.put(scannerName, s); - this.leases. - createLease(scannerName, new ScannerListener(scannerName)); + this.leases.createLease(scannerName, new ScannerListener(scannerName)); return scannerId; } public Result next(final long scannerId) throws IOException { - Result [] res = next(scannerId, 1); - if(res == null || res.length == 0) { + Result[] res = next(scannerId, 1); + if (res == null || res.length == 0) { return null; } return res[0]; } - public Result [] next(final long scannerId, int nbRows) throws IOException { + public Result[] next(final long scannerId, int nbRows) throws IOException { try { String scannerName = String.valueOf(scannerId); InternalScanner s = this.scanners.get(scannerName); @@ -1774,7 +1705,8 @@ List results = new ArrayList(nbRows); long currentScanResultSize = 0; List values = new ArrayList(); - for (int i = 0; i < nbRows && currentScanResultSize < maxScannerResultSize; i++) { + for (int i = 0; i < nbRows + && currentScanResultSize < maxScannerResultSize; i++) { requestCount.incrementAndGet(); // Collect values to be returned here boolean moreRows = s.next(values); @@ -1790,13 +1722,13 @@ values.clear(); } // Below is an ugly hack where we cast the InternalScanner to be a - // HRegion.RegionScanner. The alternative is to change InternalScanner + // HRegion.RegionScanner. The alternative is to change InternalScanner // interface but its used everywhere whereas we just need a bit of info // from HRegion.RegionScanner, IF its filter if any is done with the scan - // and wants to tell the client to stop the scan. This is done by passing + // and wants to tell the client to stop the scan. This is done by passing // a null result. - return ((HRegion.RegionScanner)s).isFilterDone() && results.isEmpty()? - null: results.toArray(new Result[0]); + return ((HRegion.RegionScanner) s).isFilterDone() && results.isEmpty() ? null + : results.toArray(new Result[0]); } catch (Throwable t) { if (t instanceof NotServingRegionException) { String scannerName = String.valueOf(scannerId); @@ -1822,8 +1754,8 @@ } /** - * Instantiated as a scanner lease. - * If the lease times out, the scanner is closed + * Instantiated as a scanner lease. If the lease times out, the scanner is + * closed */ private class ScannerListener implements LeaseListener { private final String scannerName; @@ -1848,8 +1780,8 @@ // // Methods that do the actual work for the remote API // - public void delete(final byte [] regionName, final Delete delete) - throws IOException { + public void delete(final byte[] regionName, final Delete delete) + throws IOException { checkOpen(); try { boolean writeToWAL = true; @@ -1866,7 +1798,7 @@ } public int delete(final byte[] regionName, final List deletes) - throws IOException { + throws IOException { // Count of Deletes processed. int i = 0; checkOpen(); @@ -1879,7 +1811,7 @@ } int size = deletes.size(); Integer[] locks = new Integer[size]; - for (Delete delete: deletes) { + for (Delete delete : deletes) { this.requestCount.incrementAndGet(); locks[i] = getLockFromId(delete.getLockId()); region.delete(delete, locks[i], writeToWAL); @@ -1896,16 +1828,15 @@ return -1; } - public long lockRow(byte [] regionName, byte [] row) - throws IOException { + public long lockRow(byte[] regionName, byte[] row) throws IOException { checkOpen(); NullPointerException npe = null; - if(regionName == null) { + if (regionName == null) { npe = new NullPointerException("regionName is null"); - } else if(row == null) { + } else if (row == null) { npe = new NullPointerException("row to lock is null"); } - if(npe != null) { + if (npe != null) { IOException io = new IOException("Invalid arguments to lockRow"); io.initCause(npe); throw io; @@ -1914,34 +1845,36 @@ try { HRegion region = getRegion(regionName); Integer r = region.obtainRowLock(row); - long lockId = addRowLock(r,region); + long lockId = addRowLock(r, region); LOG.debug("Row lock " + lockId + " explicitly acquired by client"); return lockId; } catch (Throwable t) { - throw convertThrowableToIOE(cleanup(t, - "Error obtaining row lock (fsOk: " + this.fsOk + ")")); + throw convertThrowableToIOE(cleanup(t, "Error obtaining row lock (fsOk: " + + this.fsOk + ")")); } } - protected long addRowLock(Integer r, HRegion region) throws LeaseStillHeldException { + protected long addRowLock(Integer r, HRegion region) + throws LeaseStillHeldException { long lockId = -1L; lockId = rand.nextLong(); String lockName = String.valueOf(lockId); rowlocks.put(lockName, r); - this.leases. - createLease(lockName, new RowLockListener(lockName, region)); + this.leases.createLease(lockName, new RowLockListener(lockName, region)); return lockId; } /** - * Method to get the Integer lock identifier used internally - * from the long lock identifier used by the client. - * @param lockId long row lock identifier from client + * Method to get the Integer lock identifier used internally from the long + * lock identifier used by the client. + * + * @param lockId + * long row lock identifier from client * @return intId Integer row lock used internally in HRegion - * @throws IOException Thrown if this is not a valid client lock id. + * @throws IOException + * Thrown if this is not a valid client lock id. */ - Integer getLockFromId(long lockId) - throws IOException { + Integer getLockFromId(long lockId) throws IOException { if (lockId == -1L) { return null; } @@ -1954,16 +1887,15 @@ return rl; } - public void unlockRow(byte [] regionName, long lockId) - throws IOException { + public void unlockRow(byte[] regionName, long lockId) throws IOException { checkOpen(); NullPointerException npe = null; - if(regionName == null) { + if (regionName == null) { npe = new NullPointerException("regionName is null"); - } else if(lockId == -1L) { + } else if (lockId == -1L) { npe = new NullPointerException("lockId is null"); } - if(npe != null) { + if (npe != null) { IOException io = new IOException("Invalid arguments to unlockRow"); io.initCause(npe); throw io; @@ -1973,31 +1905,30 @@ HRegion region = getRegion(regionName); String lockName = String.valueOf(lockId); Integer r = rowlocks.remove(lockName); - if(r == null) { + if (r == null) { throw new UnknownRowLockException(lockName); } region.releaseRowLock(r); this.leases.cancelLease(lockName); - LOG.debug("Row lock " + lockId + " has been explicitly released by client"); + LOG.debug("Row lock " + lockId + + " has been explicitly released by client"); } catch (Throwable t) { throw convertThrowableToIOE(cleanup(t)); } } @Override - public void bulkLoadHFile( - String hfilePath, byte[] regionName, byte[] familyName) - throws IOException { + public void bulkLoadHFile(String hfilePath, byte[] regionName, + byte[] familyName) throws IOException { HRegion region = getRegion(regionName); region.bulkLoadHFile(hfilePath, familyName); } - Map rowlocks = - new ConcurrentHashMap(); + Map rowlocks = new ConcurrentHashMap(); /** - * Instantiated as a row lock lease. - * If the lease times out, the row lock is released + * Instantiated as a row lock lease. If the lease times out, the row lock is + * released */ private class RowLockListener implements LeaseListener { private final String lockName; @@ -2011,12 +1942,80 @@ public void leaseExpired() { LOG.info("Row Lock " + this.lockName + " lease expired"); Integer r = rowlocks.remove(this.lockName); - if(r != null) { + if (r != null) { region.releaseRowLock(r); } } } + // Region open/close direct RPCs + + @Override + public void openRegion(HRegionInfo region) { + LOG.info("Received request to open region: " + + region.getRegionNameAsString()); + if(region.isRootRegion()) { + new OpenRootHandler(this, catalogTracker, region).submit(); + } else if(region.isMetaRegion()) { + new OpenMetaHandler(this, catalogTracker, region).submit(); + } else { + new OpenRegionHandler(this, catalogTracker, region).submit(); + } + } + + @Override + public boolean closeRegion(HRegionInfo region) + throws NotServingRegionException { + LOG.info("Received request to close region: " + + region.getRegionNameAsString()); + // TODO: Need to check if this is being served here but currently undergoing + // a split (so master needs to retry close after split is complete) + if (!onlineRegions.containsKey(region.getEncodedName())) { + LOG.warn("Received close for region we are not serving"); + throw new NotServingRegionException("Received close for " + + region.getRegionNameAsString() + " but we are not serving it"); + } + if(region.isRootRegion()) { + new CloseRootHandler(this, region).submit(); + } else if(region.isMetaRegion()) { + new CloseMetaHandler(this, region).submit(); + } else { + new CloseRegionHandler(this, region).submit(); + } + return true; + } + + // Manual remote region administration RPCs + + @Override + public void flushRegion(HRegionInfo regionInfo) + throws NotServingRegionException, IOException { + HRegion region = getRegion(regionInfo.getRegionName()); + region.flushcache(); + } + + @Override + public void splitRegion(HRegionInfo regionInfo) + throws NotServingRegionException, IOException { + HRegion region = getRegion(regionInfo.getRegionName()); + region.flushcache(); + region.shouldSplit(true); + // force a compaction, split will be side-effect + // TODO: flush/compact/split refactor will make it trivial to do this + // sync/async (and won't require us to do a compaction to split!) + compactSplitThread.compactionRequested(region, "User-triggered split"); + } + + @Override + public void compactRegion(HRegionInfo regionInfo, boolean major) + throws NotServingRegionException, IOException { + HRegion region = getRegion(regionInfo.getRegionName()); + region.flushcache(); + region.shouldSplit(true); + compactSplitThread.compactionRequested(region, major, "User-triggered " + + (major ? "major " : "") + "compaction"); + } + /** @return the info server */ public InfoServer getInfoServer() { return infoServer; @@ -2049,7 +2048,7 @@ return Collections.unmodifiableCollection(onlineRegions.values()); } - public HRegion [] getOnlineRegionsAsArray() { + public HRegion[] getOnlineRegionsAsArray() { return getOnlineRegions().toArray(new HRegion[0]); } @@ -2058,21 +2057,34 @@ */ public SortedSet getSortedOnlineRegionInfos() { SortedSet result = new TreeSet(); - synchronized(this.onlineRegions) { - for (HRegion r: this.onlineRegions.values()) { + synchronized (this.onlineRegions) { + for (HRegion r : this.onlineRegions.values()) { result.add(r.getRegionInfo()); } } return result; } + @Override + public void addToOnlineRegions(HRegion region) { + lock.writeLock().lock(); + try { + onlineRegions.put(region.getRegionInfo().getEncodedName(), region); + } finally { + lock.writeLock().unlock(); + } + } + /** - * This method removes HRegion corresponding to hri from the Map of onlineRegions. + * This method removes HRegion corresponding to hri from the Map of + * onlineRegions. * - * @param hri the HRegionInfo corresponding to the HRegion to-be-removed. - * @return the removed HRegion, or null if the HRegion was not in onlineRegions. + * @param hri + * the HRegionInfo corresponding to the HRegion to-be-removed. + * @return the removed HRegion, or null if the HRegion was not in + * onlineRegions. */ - HRegion removeFromOnlineRegions(HRegionInfo hri) { + public HRegion removeFromOnlineRegions(HRegionInfo hri) { this.lock.writeLock().lock(); HRegion toReturn = null; try { @@ -2085,7 +2097,7 @@ /** * @return A new Map of online regions sorted by region size with the first - * entry being the biggest. + * entry being the biggest. */ public SortedMap getCopyOfOnlineRegionsSortedBySize() { // we'll sort the regions in reverse @@ -2106,13 +2118,22 @@ /** * @param regionName - * @return HRegion for the passed regionName or null if named - * region is not member of the online regions. + * @return HRegion for the passed encoded regionName or null if + * named region is not member of the online regions. */ - public HRegion getOnlineRegion(final byte [] regionName) { - return onlineRegions.get(Bytes.mapKey(regionName)); + public HRegion getOnlineRegion(final String encodedRegionName) { + return onlineRegions.get(encodedRegionName); } + /** + * @param regionName + * @return HRegion for the passed binary regionName or null if + * named region is not member of the online regions. + */ + public HRegion getOnlineRegion(final byte[] regionName) { + return getOnlineRegion(HRegionInfo.encodeRegionName(regionName)); + } + /** @return the request count */ public AtomicInteger getRequestCount() { return this.requestCount; @@ -2125,16 +2146,18 @@ /** * Protected utility method for safely obtaining an HRegion handle. - * @param regionName Name of online {@link HRegion} to return + * + * @param regionName + * Name of online {@link HRegion} to return * @return {@link HRegion} for regionName * @throws NotServingRegionException */ - protected HRegion getRegion(final byte [] regionName) - throws NotServingRegionException { + protected HRegion getRegion(final byte[] regionName) + throws NotServingRegionException { HRegion region = null; this.lock.readLock().lock(); try { - region = onlineRegions.get(Integer.valueOf(Bytes.hashCode(regionName))); + region = getOnlineRegion(regionName); if (region == null) { throw new NotServingRegionException(regionName); } @@ -2145,10 +2168,10 @@ } /** - * Get the top N most loaded regions this server is serving so we can - * tell the master which regions it can reallocate if we're overloaded. - * TODO: actually calculate which regions are most loaded. (Right now, we're - * just grabbing the first N regions being served regardless of load.) + * Get the top N most loaded regions this server is serving so we can tell the + * master which regions it can reallocate if we're overloaded. TODO: actually + * calculate which regions are most loaded. (Right now, we're just grabbing + * the first N regions being served regardless of load.) */ protected HRegionInfo[] getMostLoadedRegions() { ArrayList regions = new ArrayList(); @@ -2174,8 +2197,8 @@ */ protected void checkOpen() throws IOException { if (this.stopRequested.get() || this.abortRequested) { - throw new IOException("Server not running" + - (this.abortRequested? ", aborting": "")); + throw new IOException("Server not running" + + (this.abortRequested ? ", aborting" : "")); } if (!fsOk) { throw new IOException("File system not available"); @@ -2183,12 +2206,12 @@ } /** - * @return Returns list of non-closed regions hosted on this server. If no - * regions to check, returns an empty list. + * @return Returns list of non-closed regions hosted on this server. If no + * regions to check, returns an empty list. */ protected Set getRegionsToCheck() { HashSet regionsToCheck = new HashSet(); - //TODO: is this locking necessary? + // TODO: is this locking necessary? lock.readLock().lock(); try { regionsToCheck.addAll(this.onlineRegions.values()); @@ -2205,9 +2228,8 @@ return regionsToCheck; } - public long getProtocolVersion(final String protocol, - final long clientVersion) - throws IOException { + public long getProtocolVersion(final String protocol, final long clientVersion) + throws IOException { if (protocol.equals(HRegionInterface.class.getName())) { return HBaseRPCProtocolVersion.versionID; } @@ -2223,6 +2245,7 @@ /** * Return the total size of all memstores in every region. + * * @return memstore size in bytes */ public long getGlobalMemStoreSize() { @@ -2259,17 +2282,19 @@ /** * @return Info on port this server has bound to, etc. */ - public HServerInfo getServerInfo() { return this.serverInfo; } + public HServerInfo getServerInfo() { + return this.serverInfo; + } /** {@inheritDoc} */ - public long incrementColumnValue(byte [] regionName, byte [] row, - byte [] family, byte [] qualifier, long amount, boolean writeToWAL) - throws IOException { + public long incrementColumnValue(byte[] regionName, byte[] row, + byte[] family, byte[] qualifier, long amount, boolean writeToWAL) + throws IOException { checkOpen(); if (regionName == null) { - throw new IOException("Invalid arguments to incrementColumnValue " + - "regionName is null"); + throw new IOException("Invalid arguments to incrementColumnValue " + + "regionName is null"); } requestCount.incrementAndGet(); try { @@ -2288,7 +2313,7 @@ public HRegionInfo[] getRegionsAssignment() throws IOException { HRegionInfo[] regions = new HRegionInfo[onlineRegions.size()]; Iterator ite = onlineRegions.values().iterator(); - for(int i = 0; ite.hasNext(); i++) { + for (int i = 0; ite.hasNext(); i++) { regions[i] = ite.next().getRegionInfo(); } return regions; @@ -2304,7 +2329,7 @@ MultiPutResponse resp = new MultiPutResponse(); // do each region as it's own. - for( Map.Entry> e: puts.puts.entrySet()) { + for (Map.Entry> e : puts.puts.entrySet()) { int result = put(e.getKey(), e.getValue()); resp.addResult(e.getKey(), result); @@ -2320,6 +2345,7 @@ /** * Interval at which threads should run + * * @return the interval */ public int getThreadWakeFrequency() { @@ -2327,7 +2353,7 @@ } // ServerStatus - + @Override public void abort() { this.abort("Received abort call"); @@ -2342,7 +2368,7 @@ public ZooKeeperWatcher getZooKeeper() { return zooKeeper; } - + // // Main program and support routines // @@ -2353,9 +2379,9 @@ * @throws IOException */ public static Thread startRegionServer(final HRegionServer hrs) - throws IOException { - return startRegionServer(hrs, - "regionserver" + hrs.getServerInfo().getServerAddress().getPort()); + throws IOException { + return startRegionServer(hrs, "regionserver" + + hrs.getServerInfo().getServerAddress().getPort()); } /** @@ -2365,15 +2391,14 @@ * @throws IOException */ public static Thread startRegionServer(final HRegionServer hrs, - final String name) - throws IOException { + final String name) throws IOException { Thread t = new Thread(hrs); t.setName(name); t.start(); // Install shutdown hook that will catch signals and run an orderly shutdown // of the hrs. - ShutdownHook.install(hrs.getConfiguration(), - FileSystem.get(hrs.getConfiguration()), hrs, t); + ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs + .getConfiguration()), hrs, t); return t; } @@ -2391,28 +2416,32 @@ /** * Utility for constructing an instance of the passed HRegionServer class. + * * @param regionServerClass * @param conf2 * @return HRegionServer instance. */ - public static HRegionServer constructRegionServer(Class regionServerClass, - final Configuration conf2) { + public static HRegionServer constructRegionServer( + Class regionServerClass, + final Configuration conf2) { try { - Constructor c = - regionServerClass.getConstructor(Configuration.class); + Constructor c = regionServerClass + .getConstructor(Configuration.class); return c.newInstance(conf2); } catch (Exception e) { - throw new RuntimeException("Failed construction of " + - "Master: " + regionServerClass.toString(), e); + throw new RuntimeException("Failed construction of " + "Master: " + + regionServerClass.toString(), e); } } /** * Do class main. + * * @param args - * @param regionServerClass HRegionServer to instantiate. + * @param regionServerClass + * HRegionServer to instantiate. */ - protected static void doMain(final String [] args, + protected static void doMain(final String[] args, final Class regionServerClass) { if (args.length < 1) { printUsageAndExit(); @@ -2421,14 +2450,14 @@ // Process command-line args. TODO: Better cmd-line processing // (but hopefully something not as painful as cli options). - for (String cmd: args) { + for (String cmd : args) { if (cmd.equals("start")) { try { - // If 'local', don't start a region server here. Defer to - // LocalHBaseCluster. It manages 'local' clusters. + // If 'local', don't start a region server here. Defer to + // LocalHBaseCluster. It manages 'local' clusters. if (LocalHBaseCluster.isLocal(conf)) { - LOG.warn("Not starting a distinct region server because " + - HConstants.CLUSTER_DISTRIBUTED + " is false"); + LOG.warn("Not starting a distinct region server because " + + HConstants.CLUSTER_DISTRIBUTED + " is false"); } else { RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean(); if (runtime != null) { @@ -2438,16 +2467,16 @@ startRegionServer(hrs); } } catch (Throwable t) { - LOG.error( "Can not start region server because "+ - StringUtils.stringifyException(t) ); + LOG.error("Can not start region server because " + + StringUtils.stringifyException(t)); } break; } if (cmd.equals("stop")) { - printUsageAndExit("To shutdown the regionserver run " + - "bin/hbase-daemon.sh stop regionserver or send a kill signal to" + - "the regionserver pid"); + printUsageAndExit("To shutdown the regionserver run " + + "bin/hbase-daemon.sh stop regionserver or send a kill signal to" + + "the regionserver pid"); } // Print out usage if we get to here. @@ -2458,12 +2487,32 @@ /** * @param args */ - public static void main(String [] args) { + public static void main(String[] args) { Configuration conf = HBaseConfiguration.create(); @SuppressWarnings("unchecked") - Class regionServerClass = - (Class) conf.getClass(HConstants.REGION_SERVER_IMPL, - HRegionServer.class); + Class regionServerClass = (Class) conf + .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class); doMain(args, regionServerClass); } + + @Override + public String getServerName() { + return serverInfo.getServerName(); + } + + @Override + public CompactSplitThread getCompactSplitThread() { + return compactSplitThread; + } + + @Override + public ServerConnection getServerConnection() { + return connection; + } + + @Override + public long getTimeout() { + // TODO: use configuration + return 5000; + } } \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java (revision 0) @@ -0,0 +1,220 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.handler; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.catalog.RootLocationEditor; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.RegionServerController; +import org.apache.hadoop.hbase.zookeeper.ZKAssign; +import org.apache.hadoop.util.Progressable; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.Code; + +/** + * Handles opening of a region on a region server. + *

        + * This is executed after receiving an OPEN RPC from the master. + */ +public class OpenRegionHandler extends EventHandler { + private static final Log LOG = LogFactory.getLog(OpenRegionHandler.class); + + private final RegionServerController server; + + private final CatalogTracker catalogTracker; + + private final HRegionInfo regionInfo; + + public OpenRegionHandler(RegionServerController server, + CatalogTracker catalogTracker, HRegionInfo regionInfo) { + this(server, catalogTracker, regionInfo, EventType.M2RS_OPEN_REGION); + } + + protected OpenRegionHandler(RegionServerController server, + CatalogTracker catalogTracker, HRegionInfo regionInfo, + EventType eventType) { + super(server, eventType); + this.server = server; + this.catalogTracker = catalogTracker; + this.regionInfo = regionInfo; + } + + public HRegionInfo getRegionInfo() { + return regionInfo; + } + + @Override + public void process() { + LOG.debug("Processing open region of " + + regionInfo.getRegionNameAsString()); + + final String regionName = regionInfo.getEncodedName(); + + // Previously we would check for root region availability (but only that it + // was initially available, does not check if it later went away) + // Do we need to wait on both root and meta to be available to open a region + // now since we edit meta? + + // Check that this region is not already online + HRegion region = server.getOnlineRegion(regionName); + if(region != null) { + LOG.warn("Attemping to open region " + regionInfo.getRegionNameAsString() + + " but it's already online on this server"); + return; + } + + // Transition ZK node from OFFLINE to OPENING + // TODO: should also handle transition from CLOSED? + int openingVersion; + try { + if((openingVersion = ZKAssign.transitionNodeOpening(server.getZooKeeper(), + regionInfo, server.getServerName())) == -1) { + LOG.warn("Error transitioning node from OFFLINE to OPENING, " + + "aborting open"); + return; + } + } catch (KeeperException e) { + LOG.error("Error transitioning node from OFFLINE to OPENING for region " + + regionName, e); + return; + } + + // Open the region + final AtomicInteger openingInteger = new AtomicInteger(openingVersion); + try { + // Instantiate the region. This also periodically updates OPENING. + region = server.instantiateRegion(regionInfo, server.getLog(), + new Progressable() { + public void progress() { + try { + int vsn = ZKAssign.retransitionNodeOpening( + server.getZooKeeper(), regionInfo, server.getServerName(), + openingInteger.get()); + if (vsn == -1) { + throw KeeperException.create(Code.BADVERSION); + } + openingInteger.set(vsn); + } catch (KeeperException e) { + LOG.error("ZK exception refreshing OPENING node", e); + server.abort(); + } + } + }); + } catch (IOException e) { + LOG.error("IOException instantiating region for " + regionInfo); + LOG.debug("Resetting state of transition node from OPENING to OFFLINE"); + try { + // TODO: We should rely on the master timing out OPENING instead of this + ZKAssign.forceNodeOffline(server.getZooKeeper(), regionInfo, + server.getServerName()); + } catch (KeeperException e1) { + LOG.error("Error forcing node back to OFFLINE from OPENING"); + return; + } + return; + } + + // Re-transition node to OPENING again to verify someone else has not + // stomped on us + openingVersion = openingInteger.get(); + try { + if((openingVersion = ZKAssign.retransitionNodeOpening( + server.getZooKeeper(), regionInfo, server.getServerName(), + openingVersion)) == -1) { + LOG.warn("Completed the OPEN of a region but when transitioning from " + + " OPENING to OPENED got a version mismatch, someone else clashed " + + "so now unassigning"); + region.close(); + return; + } + } catch (KeeperException e) { + LOG.error("Failed transitioning node from OPENING to OPENED", e); + return; + } catch (IOException e) { + LOG.error("Failed to close region after failing to transition", e); + return; + } + + LOG.debug("Re-transitioned node to OPENING, completing OPEN by adding to " + + "online regions, doing on-open checks, and updating ROOT or META " + + "for region " + region.getRegionNameAsString()); + + // Do checks to see if we need to compact (references or too many files) + if(region.hasReferences() || region.hasTooManyStoreFiles()) { + server.getCompactSplitThread().compactionRequested(region, + region.hasReferences() ? "Region has references on open" : + "Region has too many store files"); + } + + // Add to online regions + server.addToOnlineRegions(region); + + // Update ZK, ROOT or META + try { + if(regionInfo.isRootRegion()) { + RootLocationEditor.setRootLocation(server.getZooKeeper(), + server.getServerInfo().getServerAddress()); + } else if(regionInfo.isMetaRegion()) { + // TODO: doh, this has weird naming between RootEditor/MetaEditor + MetaEditor.updateMetaLocation(catalogTracker, regionInfo, + server.getServerInfo()); + } else { + MetaEditor.updateRegionLocation(catalogTracker, region.getRegionInfo(), + server.getServerInfo()); + } + } catch (IOException e) { + // TODO: rollback the open? + LOG.error("Error updating region location in catalog table", e); + } catch (KeeperException e) { + // TODO: rollback the open? + LOG.error("ZK Error updating region location in catalog table", e); + } + + // Finally, Transition ZK node to OPENED + try { + if(ZKAssign.transitionNodeOpened(server.getZooKeeper(), regionInfo, + server.getServerName(), openingVersion) == -1) { + LOG.warn("Completed the OPEN of a region but when transitioning from " + + " OPENING to OPENED got a version mismatch, someone else clashed " + + "so now unassigning"); + region.close(); + return; + } + } catch (KeeperException e) { + LOG.error("Failed transitioning node from OPENING to OPENED", e); + return; + } catch (IOException e) { + LOG.error("Failed to close region after failing to transition", e); + return; + } + + // Done! Successful region open + LOG.debug("Completed region open and successfully transitioned node to " + + "OPENED for region " + region.getRegionNameAsString()); + } +} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java (revision 0) @@ -0,0 +1,36 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.handler; + +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.regionserver.RegionServerController; + +/** + * Handles opening of the root region on a region server. + *

        + * This is executed after receiving an OPEN RPC from the master for root. + */ +public class OpenRootHandler extends OpenRegionHandler { + public OpenRootHandler(RegionServerController server, + CatalogTracker catalogTracker, HRegionInfo regionInfo) { + super(server, catalogTracker, regionInfo, EventType.M2RS_OPEN_ROOT); + } +} Index: src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java (revision 0) @@ -0,0 +1,36 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.handler; + +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.regionserver.RegionServerController; + +/** + * Handles opening of a meta region on a region server. + *

        + * This is executed after receiving an OPEN RPC from the master for meta. + */ +public class OpenMetaHandler extends OpenRegionHandler { + public OpenMetaHandler(RegionServerController server, + CatalogTracker catalogTracker, HRegionInfo regionInfo) { + super(server, catalogTracker, regionInfo, EventType.M2RS_OPEN_META); + } +} Index: src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java (revision 0) @@ -0,0 +1,141 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.handler; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.RegionServerController; +import org.apache.hadoop.hbase.zookeeper.ZKAssign; +import org.apache.zookeeper.KeeperException; + +/** + * Handles closing of a region on a region server. + *

        + * This is executed after receiving an CLOSE RPC from the master. + */ +public class CloseRegionHandler extends EventHandler { + private static final Log LOG = LogFactory.getLog(CloseRegionHandler.class); + + private final RegionServerController server; + + private final HRegionInfo regionInfo; + + private final boolean abort; + + public CloseRegionHandler(RegionServerController server, + HRegionInfo regionInfo) { + this(server, regionInfo, false); + } + + public CloseRegionHandler(RegionServerController server, + HRegionInfo regionInfo, boolean abort) { + this(server, regionInfo, abort, EventType.M2RS_CLOSE_REGION); + } + + protected CloseRegionHandler(RegionServerController server, + HRegionInfo regionInfo, boolean abort, EventType eventType) { + super(server, eventType); + this.server = server; + this.regionInfo = regionInfo; + this.abort = abort; + } + + public HRegionInfo getRegionInfo() { + return regionInfo; + } + + @Override + public void process() { + LOG.debug("Processing close region of " + + regionInfo.getRegionNameAsString()); + + String regionName = regionInfo.getEncodedName(); + + // Check that this region is being served here + HRegion region = server.getOnlineRegion(regionName); + if(region == null) { + LOG.warn("Received a CLOSE for the region " + + regionInfo.getRegionNameAsString() + " but not currently serving " + + "this region"); + return; + } + + // Create ZK node in CLOSING state + int expectedVersion; + try { + if((expectedVersion = ZKAssign.createNodeClosing(server.getZooKeeper(), + regionInfo, server.getServerName())) == -1) { + LOG.warn("Error creating node in CLOSING state, aborting close of " + + regionInfo.getRegionNameAsString()); + return; + } + } catch (KeeperException e) { + LOG.warn("Error creating node in CLOSING state, aborting close of " + + regionInfo.getRegionNameAsString()); + return; + } + + // Close the region + try { + // TODO: If we need to keep updating CLOSING stamp to prevent against + // a timeout if this is long-running, need to spin up a thread? + server.removeFromOnlineRegions(regionInfo); + region.close(abort); + } catch (IOException e) { + LOG.error("IOException closing region for " + regionInfo); + LOG.debug("Deleting transition node that was in CLOSING"); + try { + ZKAssign.deleteClosingNode(server.getZooKeeper(), regionName); + } catch (KeeperException e1) { + LOG.error("Error deleting CLOSING node"); + return; + } + return; + } + + // Transition ZK node to CLOSED + try { + if(ZKAssign.transitionNodeClosed(server.getZooKeeper(), regionInfo, + server.getServerName(), expectedVersion) == -1) { + LOG.warn("Completed the OPEN of a region but when transitioning from " + + " OPENING to OPENED got a version mismatch, someone else clashed " + + "so now unassigning"); + region.close(); + return; + } + } catch (KeeperException e) { + LOG.error("Failed transitioning node from OPENING to OPENED", e); + return; + } catch (IOException e) { + LOG.error("Failed to close region after failing to transition", e); + return; + } + + // Done! Successful region open + LOG.debug("Completed region close and successfully transitioned node to " + + "CLOSED for region " + region.getRegionNameAsString() + " (" + + regionName + ")"); + } +} Index: src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRootHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRootHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRootHandler.java (revision 0) @@ -0,0 +1,35 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.handler; + +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.regionserver.RegionServerController; + +/** + * Handles closing of the root region on a region server. + *

        + * This is executed after receiving an CLOSE RPC from the master for root. + */ +public class CloseRootHandler extends CloseRegionHandler { + public CloseRootHandler(RegionServerController server, + HRegionInfo regionInfo) { + super(server, regionInfo, false, EventType.M2RS_CLOSE_ROOT); + } +} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseMetaHandler.java (revision 0) @@ -0,0 +1,35 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.handler; + +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.regionserver.RegionServerController; + +/** + * Handles closing of the root region on a region server. + *

        + * This is executed after receiving an CLOSE RPC from the master for root. + */ +public class CloseMetaHandler extends CloseRegionHandler { + public CloseMetaHandler(RegionServerController server, + HRegionInfo regionInfo) { + super(server, regionInfo, false, EventType.M2RS_CLOSE_META); + } +} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerController.java (revision 0) @@ -0,0 +1,51 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; + +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerInfo; +import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.util.Progressable; + +public interface RegionServerController extends ServerController { + + // this is unfortunate but otherwise all the implementation of region + // open/close must happen in HRS itself and not in handlers + // handlers could make calls to HRS methods and logic still in HRS, just + // trying to reduce the already massive class + + public HRegion getOnlineRegion(String regionName); + + public void addToOnlineRegions(HRegion region); + + public HRegion instantiateRegion(final HRegionInfo regionInfo, + final HLog wal, Progressable progressable) throws IOException; + + public HLog getLog(); + + public CompactSplitThread getCompactSplitThread(); + + public HServerInfo getServerInfo(); + + public HRegion removeFromOnlineRegions(HRegionInfo regionInfo); +} Index: src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (working copy) @@ -40,7 +40,7 @@ /** * Compact region on request and then run split if appropriate */ -class CompactSplitThread extends Thread { +public class CompactSplitThread extends Thread { static final Log LOG = LogFactory.getLog(CompactSplitThread.class); private HTable root = null; Index: src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -385,7 +385,7 @@ /** * @return True if this region has references. */ - boolean hasReferences() { + public boolean hasReferences() { for (Store store : this.stores.values()) { for (StoreFile sf : store.getStorefiles()) { // Found a reference, return. Index: src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/regionserver/MasterAddressManager.java (working copy) @@ -21,49 +21,41 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; /** * Manages the location of the current active Master for this RegionServer. - * + *

        * Listens for ZooKeeper events related to the master address. The node /master * will contain the address of the current master. This listener is interested * in NodeDeleted and NodeCreated events on /master. - * - * This class is thread-safe and takes care of re-setting all watchers to - * ensure it always knows the up-to-date master. To kick it off, instantiate - * the class and run the {@link #monitorMaster()} method. - * + *

        + * Utilizes {@link ZooKeeperNodeTracker} for zk interactions. + *

        * You can get the current master via {@link #getMasterAddress()} or the * blocking method {@link #waitMasterAddress()}. */ -public class MasterAddressManager extends ZooKeeperListener { +public class MasterAddressManager extends ZooKeeperNodeTracker { private static final Log LOG = LogFactory.getLog(MasterAddressManager.class); - // Address of the current primary master, null if no primary master - private HServerAddress masterAddress; - - // Status and controller for the regionserver - private ServerController status; - /** * Construct a master address listener with the specified zookeeper reference. - * + *

        * This constructor does not trigger any actions, you must call methods - * explicitly. Normally you will just want to execute {@link #monitorMaster()} - * and you will ensure to + * explicitly. Normally you will just want to execute {@link #start()} to + * begin tracking of the master address. * * @param watcher zk reference and watcher + * @param abortable abortable in case of fatal error */ - public MasterAddressManager(ZooKeeperWatcher watcher, ServerController status) { - super(watcher); - this.status = status; - this.masterAddress = null; + public MasterAddressManager(ZooKeeperWatcher watcher, Abortable abortable) { + super(watcher, watcher.masterAddressZNode, abortable); } /** @@ -74,16 +66,17 @@ * available. * @return server address of current active master, or null if none available */ - public synchronized HServerAddress getMasterAddress() { - return masterAddress; + public HServerAddress getMasterAddress() { + byte [] data = super.getData(); + return data == null ? null : new HServerAddress(Bytes.toString(data)); } /** * Check if there is a master available. * @return true if there is a master set, false if not. */ - public synchronized boolean hasMaster() { - return masterAddress != null; + public boolean hasMaster() { + return super.getData() != null; } /** @@ -91,113 +84,18 @@ * will block until one is available, the thread is interrupted, or timeout * has passed. * - * TODO: Make this work, currently unused, kept with existing retry semantics. - * + * @param timeout maximum time to wait for master in millis, 0 for forever * @return server address of current active master, null if timed out * @throws InterruptedException if the thread is interrupted while waiting */ - public synchronized HServerAddress waitForMaster() + public synchronized HServerAddress waitForMaster(long timeout) throws InterruptedException { - return masterAddress; + byte [] data = super.blockUntilAvailable(timeout); + return data == null ? null : new HServerAddress(Bytes.toString(data)); } - /** - * Setup to watch for the primary master of the cluster. - * - * If the master is already available in ZooKeeper, this method will ensure - * it gets set and that any further changes are also watched for. - * - * If no master is available, this method ensures we become aware of it and - * will take care of setting it. - */ - public void monitorMaster() { - try { - if(ZKUtil.watchAndCheckExists(watcher, watcher.masterAddressZNode)) { - handleNewMaster(); - } - } catch(KeeperException ke) { - // If we have a ZK exception trying to find the master we must abort - LOG.fatal("Unexpected ZooKeeper exception", ke); - status.abort(); - } - } - @Override - public void nodeCreated(String path) { - LOG.info("nodeCreated(" + path + ")"); - if(path.equals(watcher.masterAddressZNode)) { - handleNewMaster(); - } - monitorMaster(); + protected Log getLog() { + return LOG; } - - @Override - public void nodeDeleted(String path) { - if(path.equals(watcher.masterAddressZNode)) { - handleDeadMaster(); - } - monitorMaster(); - } - - /** - * Set the master address to the specified address. This operation is - * idempotent, a master will only be set if there is currently no master set. - */ - private synchronized void setMasterAddress(HServerAddress address) { - if(masterAddress == null) { - LOG.info("Found and set master address: " + address); - masterAddress = address; - } - } - - /** - * Unsets the master address. Used when the master goes offline so none is - * available. - */ - private synchronized void unsetMasterAddress() { - if(masterAddress != null) { - LOG.info("Master has been unset. There is no current master available"); - masterAddress = null; - } - } - - /** - * Handle a new master being set. - * - * This method should be called to check if there is a new master. If there - * is already a master set, this method returns immediately. If none is set, - * this will attempt to grab the master location from ZooKeeper and will set - * it. - * - * This method uses an atomic operation to ensure a new master is only set - * once. - */ - private void handleNewMaster() { - if(hasMaster()) { - return; - } - HServerAddress address = null; - try { - address = ZKUtil.getDataAsAddress(watcher, watcher.masterAddressZNode); - } catch (KeeperException ke) { - // If we have a ZK exception trying to find the master we must abort - LOG.fatal("Unexpected ZooKeeper exception", ke); - status.abort(); - } - if(address != null) { - setMasterAddress(address); - } - } - - /** - * Handle a master failure. - * - * Triggered when a master node is deleted. - * - * TODO: Other ways we figure master is "dead"? What do we do if set in ZK - * but we can't communicate with TCP? - */ - private void handleDeadMaster() { - unsetMasterAddress(); - } } Index: src/main/java/org/apache/hadoop/hbase/regionserver/RSZookeeperUpdater.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/RSZookeeperUpdater.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/regionserver/RSZookeeperUpdater.java (working copy) @@ -1,194 +0,0 @@ -package org.apache.hadoop.hbase.regionserver; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.executor.RegionTransitionData; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.data.Stat; - -/** - * This is a helper class for region servers to update various states in - * Zookeeper. The various updates are abstracted out here. - * - * The "startRegionXXX" methods are to be called first, followed by the - * "finishRegionXXX" methods. Supports updating zookeeper periodically as a - * part of the "startRegionXXX". Currently handles the following state updates: - * - Close region - * - Open region - */ -// TODO: make this thread local, in which case it is re-usable per thread -// TODO: After open/close is direct RPC, move this logic into Handlers -public class RSZookeeperUpdater { - private static final Log LOG = LogFactory.getLog(RSZookeeperUpdater.class); - private final String regionServerName; - private String regionName = null; - private String regionZNode = null; - private ZooKeeperWatcher zooKeeper = null; - private int zkVersion = 0; - HBaseEventType lastUpdatedState; - - public RSZookeeperUpdater(ZooKeeperWatcher zooKeeper, String regionServerName, - String regionName) { - this(zooKeeper, regionServerName, regionName, 0); - } - - public RSZookeeperUpdater(ZooKeeperWatcher zooKeeper, String regionServerName, - String regionName, int zkVersion) { - this.zooKeeper = zooKeeper; - this.regionServerName = regionServerName; - this.regionName = regionName; - // get the region ZNode we have to create - this.regionZNode = ZKUtil.joinZNode(zooKeeper.assignmentZNode, regionName); - this.zkVersion = zkVersion; - } - - /** - * This method updates the various states in ZK to inform the master that the - * region server has started closing the region. - * @param updatePeriodically - if true, periodically updates the state in ZK - */ - public void startRegionCloseEvent(HMsg hmsg, boolean updatePeriodically) throws IOException { - // Try to create the node with a CLOSING state, if already exists, - // something is wrong - try { - if(ZKUtil.createNodeIfNotExistsAndWatch(zooKeeper, regionZNode, - makeZKEventData(HBaseEventType.RS2ZK_REGION_CLOSING, - regionName, hmsg))) { - String msg = "ZNode " + regionZNode + " already exists in ZooKeeper, will NOT close region."; - LOG.error(msg); - throw new IOException(msg); - } - } catch (KeeperException e) { - zooKeeper.error("Unexpected exception trying to create unassigned node", e); - throw new IOException(e); - } - - // TODO: implement the updatePeriodically logic here - } - - /** - * This method updates the states in ZK to signal that the region has been - * closed. This will stop the periodic updater thread if one was started. - * @throws IOException - */ - public void finishRegionCloseEvent(HMsg hmsg) throws IOException { - // TODO: stop the updatePeriodically here - - // update the data for "regionName" ZNode in unassigned to CLOSED - updateZKWithEventData(HBaseEventType.RS2ZK_REGION_CLOSED, hmsg); - } - - /** - * This method updates the various states in ZK to inform the master that the - * region server has started opening the region. - * @param updatePeriodically - if true, periodically updates the state in ZK - */ - public void startRegionOpenEvent(HMsg hmsg, boolean updatePeriodically) - throws IOException { - Stat stat = new Stat(); - byte[] data = null; - try { - data = ZKUtil.getDataNoWatch(zooKeeper, regionZNode, stat); - } catch (KeeperException e) { - zooKeeper.error("ZooKeeper error", e); - throw new IOException(e); - } - // if there is no ZNode for this region, something is wrong - if(data == null) { - String msg = "ZNode " + regionZNode + " does not exist in ZooKeeper, will NOT open region."; - LOG.error(msg); - throw new IOException(msg); - } - // if the ZNode is not in the closed state, something is wrong - HBaseEventType rsEvent = HBaseEventType.fromByte(data[0]); - if(rsEvent != HBaseEventType.RS2ZK_REGION_CLOSED && rsEvent != HBaseEventType.M2ZK_REGION_OFFLINE) { - String msg = "ZNode " + regionZNode + " is not in CLOSED/OFFLINE state (state = " + rsEvent + "), will NOT open region."; - LOG.error(msg); - throw new IOException(msg); - } - - // get the version to update from ZK - zkVersion = stat.getVersion(); - - // update the data for "regionName" ZNode in unassigned to CLOSING - updateZKWithEventData(HBaseEventType.RS2ZK_REGION_OPENING, hmsg); - - // TODO: implement the updatePeriodically logic here - } - - /** - * This method updates the states in ZK to signal that the region has been - * opened. This will stop the periodic updater thread if one was started. - * @throws IOException - */ - public void finishRegionOpenEvent(HMsg hmsg) throws IOException { - // TODO: stop the updatePeriodically here - - // update the data for "regionName" ZNode in unassigned to CLOSED - updateZKWithEventData(HBaseEventType.RS2ZK_REGION_OPENED, hmsg); - } - - public boolean isClosingRegion() { - return (lastUpdatedState == HBaseEventType.RS2ZK_REGION_CLOSING); - } - - public boolean isOpeningRegion() { - return (lastUpdatedState == HBaseEventType.RS2ZK_REGION_OPENING); - } - - public void abortOpenRegion(HMsg hmsg) throws IOException { - LOG.error("Aborting open of region " + regionName); - - // TODO: stop the updatePeriodically for start open region here - - // update the data for "regionName" ZNode in unassigned to CLOSED - updateZKWithEventData(HBaseEventType.RS2ZK_REGION_CLOSED, hmsg); - } - - /** - * Make the serialized data to put into unassigned znodes for the specified - * event type and message. - * @param eventType - * @param hmsg - * @return serialized data - */ - private byte [] makeZKEventData(HBaseEventType eventType, String regionName, - HMsg hmsg) - throws IOException { - return Writables.getBytes(new RegionTransitionData(eventType, regionName, - regionServerName, hmsg)); - } - - /** - * Update the data for this region to the serialized form of the specified - * event type and message. - * @param hbEventType - * @param hmsg - * @throws IOException - */ - private void updateZKWithEventData(HBaseEventType eventType, HMsg hmsg) - throws IOException { - byte[] data = makeZKEventData(eventType, regionName, hmsg); - LOG.debug("Updating ZNode " + regionZNode + - " with [" + eventType + "]" + - " expected version = " + zkVersion); - try { - ZKUtil.updateExistingNodeData(zooKeeper, regionZNode, data, zkVersion); - } catch(KeeperException.BadVersionException e) { - zooKeeper.error("Version mismatch on unassigned znode when updating", e); - throw new IOException(e); - } catch(KeeperException e) { - zooKeeper.error("Unexpected exception trying to update unassigned node", e); - throw new IOException(e); - } - lastUpdatedState = eventType; - zkVersion++; - } -} Index: src/main/java/org/apache/hadoop/hbase/HServerInfo.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/HServerInfo.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/HServerInfo.java (working copy) @@ -148,6 +148,8 @@ } /** + * Gets the unique server instance name. Includes the hostname, port, and + * start code. * @return Server name made of the concatenation of hostname, port and * startcode formatted as <hostname> ',' <port> ',' <startcode> */ Index: src/main/java/org/apache/hadoop/hbase/HRegionInfo.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/HRegionInfo.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/HRegionInfo.java (working copy) @@ -316,6 +316,24 @@ } /** + * Gets the table name from the specified region name. + * @param regionName + * @return + */ + public static byte [] getTableName(byte [] regionName) { + int offset = -1; + for (int i = 0; i < regionName.length; i++) { + if (regionName[i] == DELIMITER) { + offset = i; + break; + } + } + byte [] tableName = new byte[offset]; + System.arraycopy(regionName, 0, tableName, 0, offset); + return tableName; + } + + /** * Separate elements of a regionName. * @param regionName * @return Array of byte[] containing tableName, startKey and id Index: src/main/java/org/apache/hadoop/hbase/master/MasterStatus.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/MasterStatus.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/MasterStatus.java (working copy) @@ -1,83 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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.concurrent.atomic.AtomicBoolean; - -import org.apache.hadoop.hbase.ServerController; -import org.apache.hadoop.hbase.client.ServerConnection; - -/** - * These are the set of functions implemented by the HMaster and accessed by - * the other packages in the master. - * - * TODO: this list has to be cleaned up, this is a re-factor only change that - * preserves the functions in the interface. - */ -public interface MasterStatus extends ServerController { - - /** - * Return the server manager for region server related info - */ - public ServerManager getServerManager(); - - /** - * Return the region manager for region related info - */ - public RegionManager getRegionManager(); - - /** - * Return the file system manager for dealing with FS related stuff - */ - public FileSystemManager getFileSystemManager(); - - /** - * Is this the master that is starting the cluster up? If true, yes. - * Otherwise this is a failed over master. - */ - public boolean isClusterStartup(); - - /** - * Set whether this is a cluster starting up. - * @param isClusterStartup whether this is a cluster startup or failover - */ - public void setClusterStartup(boolean isClusterStartup); - - /** - * Return the server RPC connection - */ - public ServerConnection getServerConnection(); - - // TODO: the semantics of the following methods should be defined. Once that - // is clear, most of these should move to server status - - // start shutting down the server - public void startShutdown(); - // is a shutdown requested - public AtomicBoolean getShutdownRequested(); - // sets the closed variable in the master to true - public void setClosed(); - // returns the closed atomic boolean - public AtomicBoolean getClosed(); - // returns the boolean value of the closed atomic boolean - public boolean isClosed(); - // is the server shutdown - public void shutdown(); -} Index: src/main/java/org/apache/hadoop/hbase/master/MasterController.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/MasterController.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/MasterController.java (revision 0) @@ -0,0 +1,86 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.concurrent.atomic.AtomicBoolean; + +import org.apache.hadoop.hbase.ServerController; + +/** + * Defines the set of functions implemented by the HMaster related to control + * of the master process and cluster shutdown. + */ +public interface MasterController extends ServerController { + + // TODO: getServerManager and getFileManager exist because each references the + // other in a single call. should figure how to clean this up. + + /** + * Returns the server manager which manages for region server related info + */ + public ServerManager getServerManager(); + + /** + * Return the file system manager for dealing with FS related stuff + */ + public FileSystemManager getFileSystemManager(); + + /** + * Is this the master that is starting the cluster up? If true, yes. + * Otherwise this is a failed over master. + */ + public boolean isClusterStartup(); + + /** + * Set whether this is a cluster starting up. + * @param isClusterStartup whether this is a cluster startup or failover + */ + public void setClusterStartup(boolean isClusterStartup); + + /** + * Requests a shutdown of the cluster. + *

        + * Requesting a shutdown + */ + public void requestShutdown(); + + /** + * Gets a boolean representing whether a shutdown has been requested or not. + * @return if a shutdown has been requested or not + */ + public AtomicBoolean getShutdownRequested(); + + /** + * Sets the cluster as closed. + */ + public void setClosed(); + + /** + * Gets an atomic boolean that represents whether the master is closed. + * @return boolean used to get/set master closed status + */ + public AtomicBoolean getClosed(); + + /** + * Returns true if the master is closed, false if not. + * @return if master is closed + */ + public boolean isClosed(); +} Index: src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationQueue.java (working copy) @@ -1,257 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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.Set; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.DelayQueue; -import java.util.concurrent.PriorityBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.util.Sleeper; -import org.apache.hadoop.ipc.RemoteException; - -/** - * Keeps up the queue of {@link RegionServerOperation}s. - * Has both live queue and a temporary put-aside queue; if processing of the - * live todo queue fails for some reason, we'll add the item back on the delay - * queue for retry later. Call {@link #shutdown()} to effect a cleanup of - * queues when done. Listen to this queue by registering - * {@link RegionServerOperationListener}s. - * @see #registerRegionServerOperationListener(RegionServerOperationListener) - * @see #unregisterRegionServerOperationListener(RegionServerOperationListener) - */ -public class RegionServerOperationQueue { - // TODO: Build up the junit test of this class. - private final Log LOG = LogFactory.getLog(this.getClass()); - - /** - * Enums returned by {@link RegionServerOperationQueue#process()}; - */ - public static enum ProcessingResultCode { - /** - * Operation was processed successfully. - */ - PROCESSED, - /** - * Nothing to do. - */ - NOOP, - /** - * Operation was put-aside for now. Will be retried later. - */ - REQUEUED, - /** - * Failed processing of the operation. - */ - FAILED, - /** - * Operation was requeued but we failed its processing for some reason - * (Bad filesystem?). - */ - REQUEUED_BUT_PROBLEM - }; - - /* - * Do not put items directly on this queue. Use {@link #putOnDelayQueue(RegionServerOperation)}. - * It makes sure the expiration on the RegionServerOperation added is updated. - */ - private final DelayQueue delayedToDoQueue = - new DelayQueue(); - private final BlockingQueue toDoQueue = - new PriorityBlockingQueue(); - private final Set listeners = - new CopyOnWriteArraySet(); - private final int threadWakeFrequency; - private final AtomicBoolean closed; - private final Sleeper sleeper; - - RegionServerOperationQueue(final Configuration c, final AtomicBoolean closed) { - this.threadWakeFrequency = c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000); - this.closed = closed; - this.sleeper = new Sleeper(this.threadWakeFrequency, this.closed); - } - - public void put(final RegionServerOperation op) { - try { - this.toDoQueue.put(op); - } catch (InterruptedException e) { - LOG.warn("Insertion into todo queue interrupted; putting on delay queue", e); - putOnDelayQueue(op); - } - } - - /** - * Try to get an operation off of the queue and process it. - * @return {@link ProcessingResultCode#PROCESSED}, - * {@link ProcessingResultCode#REQUEUED}, - * {@link ProcessingResultCode#REQUEUED_BUT_PROBLEM} - */ - public synchronized ProcessingResultCode process() { - RegionServerOperation op = null; - // Only process the delayed queue if root region is online. If offline, - // the operation to put it online is probably in the toDoQueue. Process - // it first. - if (toDoQueue.isEmpty()) { - op = delayedToDoQueue.poll(); - } - if (op == null) { - try { - op = toDoQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - LOG.debug("Interrupted", e); - } - } - - // At this point, if there's still no todo operation, or we're supposed to - // be closed, return. - if (op == null || closed.get()) { - return ProcessingResultCode.NOOP; - } - - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Processing todo: " + op.toString()); - } - if (!process(op)) { - // Add it back on the queue. - putOnDelayQueue(op); - } else if (op.process()) { - processed(op); - } else { - // Operation would have blocked because not all meta regions are - // online. This could cause a deadlock, because this thread is waiting - // for the missing meta region(s) to come back online, but since it - // is waiting, it cannot process the meta region online operation it - // is waiting for. So put this operation back on the queue for now. - if (toDoQueue.size() == 0) { - // The queue is currently empty so wait for a while to see if what - // we need comes in first - this.sleeper.sleep(); - } - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Put " + op.toString() + " back on queue"); - } - toDoQueue.put(op); - } catch (InterruptedException e) { - throw new RuntimeException( - "Putting into toDoQueue was interrupted.", e); - } - } - } catch (Exception ex) { - // There was an exception performing the operation. - if (ex instanceof RemoteException) { - try { - ex = RemoteExceptionHandler.decodeRemoteException( - (RemoteException)ex); - } catch (IOException e) { - ex = e; - LOG.warn("main processing loop: " + op.toString(), e); - } - } - LOG.warn("Failed processing: " + op.toString() + - "; putting onto delayed todo queue", ex); - putOnDelayQueue(op); - return ProcessingResultCode.REQUEUED_BUT_PROBLEM; - } - return ProcessingResultCode.REQUEUED; - } - - void putOnDelayQueue(final RegionServerOperation op) { - op.resetExpiration(); - this.delayedToDoQueue.put(op); - } - - /** - * Clean up the queues. - */ - public synchronized void shutdown() { - this.toDoQueue.clear(); - this.delayedToDoQueue.clear(); - } - - /** - * @param l Register this listener of RegionServerOperation events. - */ - public void registerRegionServerOperationListener(final RegionServerOperationListener l) { - this.listeners.add(l); - } - - /** - * @param l Unregister this listener for RegionServerOperation events. - * @return True if this listener was registered. - */ - public boolean unregisterRegionServerOperationListener(final RegionServerOperationListener l) { - return this.listeners.remove(l); - } - - /* - * Tell listeners that we processed a RegionServerOperation. - * @param op Operation to tell the world about. - */ - private void processed(final RegionServerOperation op) { - if (this.listeners.isEmpty()) return; - for (RegionServerOperationListener listener: this.listeners) { - listener.processed(op); - } - } - - /** - * Called for each message passed the master. Most of the messages that come - * in here will go on to become {@link #process(RegionServerOperation)}s but - * others like {@linke HMsg.Type#MSG_REPORT_PROCESS_OPEN} go no further; - * only in here can you see them come in. - * @param serverInfo Server we got the message from. - * @param incomingMsg The message received. - * @return True to continue processing, false to skip. - */ - boolean process(final HServerInfo serverInfo, - final HMsg incomingMsg) { - if (this.listeners.isEmpty()) return true; - for (RegionServerOperationListener listener: this.listeners) { - if (!listener.process(serverInfo, incomingMsg)) return false; - } - return true; - } - - /* - * Tell listeners that we processed a RegionServerOperation. - * @param op Operation to tell the world about. - */ - private boolean process(final RegionServerOperation op) throws IOException { - if (this.listeners.isEmpty()) return true; - for (RegionServerOperationListener listener: this.listeners) { - if (!listener.process(op)) return false; - } - return true; - } -} Index: src/main/java/org/apache/hadoop/hbase/master/MetaScanner.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/MetaScanner.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/MetaScanner.java (working copy) @@ -1,185 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.RemoteExceptionHandler; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; - -/** - * MetaScanner META table. - * - * When a META server comes on line, a MetaRegion object is - * queued up by regionServerReport() and this thread wakes up. - * - * It's important to do this work in a separate thread, or else the blocking - * action would prevent other work from getting done. - */ -class MetaScanner extends BaseScanner { - /** Initial work for the meta scanner is queued up here */ - private volatile BlockingQueue metaRegionsToScan = - new LinkedBlockingQueue(); - - private final List metaRegionsToRescan = - new ArrayList(); - - protected static int threadWakeFrequency; - - /** - * Constructor - * - * @param master - */ - public MetaScanner(MasterStatus masterStatus) { - super(masterStatus, false, masterStatus.getShutdownRequested()); - threadWakeFrequency = masterStatus.getConfiguration().getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000); - } - - // Don't retry if we get an error while scanning. Errors are most often - // caused by the server going away. Wait until next rescan interval when - // things should be back to normal. - private boolean scanOneMetaRegion(MetaRegion region) { - while (!this.masterStatus.isClosed() && - !this.masterStatus.getRegionManager().isInitialRootScanComplete() && - this.masterStatus.getRegionManager().getRootRegionLocation() == null) { - sleep(); - } - if (this.masterStatus.isClosed()) { - return false; - } - - try { - // Don't interrupt us while we're working - synchronized (scannerLock) { - scanRegion(region); - this.masterStatus.getRegionManager().putMetaRegionOnline(region); - } - } catch (IOException e) { - e = RemoteExceptionHandler.checkIOException(e); - LOG.warn("Scan one META region: " + region.toString(), e); - // The region may have moved (TestRegionServerAbort, etc.). If - // so, either it won't be in the onlineMetaRegions list or its host - // address has changed and the containsValue will fail. If not - // found, best thing to do here is probably return. - if (!this.masterStatus.getRegionManager().isMetaRegionOnline(region.getStartKey())) { - LOG.debug("Scanned region is no longer in map of online " + - "regions or its value has changed"); - return false; - } - // Make sure the file system is still available - this.masterStatus.getFileSystemManager().checkFileSystem(); - } catch (Exception e) { - // If for some reason we get some other kind of exception, - // at least log it rather than go out silently. - LOG.error("Unexpected exception", e); - } - return true; - } - - @Override - protected boolean initialScan() { - MetaRegion region = null; - while (!this.masterStatus.isClosed() && - (region == null && metaRegionsToScan.size() > 0) && - !metaRegionsScanned()) { - try { - region = metaRegionsToScan.poll(this.threadWakeFrequency, - TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - // continue - } - if (region == null && metaRegionsToRescan.size() != 0) { - region = metaRegionsToRescan.remove(0); - } - if (region != null) { - if (!scanOneMetaRegion(region)) { - metaRegionsToRescan.add(region); - } - } - } - initialScanComplete = true; - return true; - } - - @Override - protected void maintenanceScan() { - List regions = - this.masterStatus.getRegionManager().getListOfOnlineMetaRegions(); - int regionCount = 0; - for (MetaRegion r: regions) { - scanOneMetaRegion(r); - regionCount++; - } - LOG.info("All " + regionCount + " .META. region(s) scanned"); - metaRegionsScanned(); - } - - /* - * Called by the meta scanner when it has completed scanning all meta - * regions. This wakes up any threads that were waiting for this to happen. - * @param totalRows Total rows scanned. - * @param regionCount Count of regions in .META. table. - * @return False if number of meta regions matches count of online regions. - */ - private synchronized boolean metaRegionsScanned() { - if (!this.masterStatus.getRegionManager().isInitialRootScanComplete() || - this.masterStatus.getRegionManager().numMetaRegions() != - this.masterStatus.getRegionManager().numOnlineMetaRegions()) { - return false; - } - notifyAll(); - return true; - } - - /** - * Other threads call this method to wait until all the meta regions have - * been scanned. - */ - synchronized boolean waitForMetaRegionsOrClose() { - while (!this.masterStatus.isClosed()) { - synchronized (masterStatus.getRegionManager()) { - if (this.masterStatus.getRegionManager().isInitialRootScanComplete() && - this.masterStatus.getRegionManager().numMetaRegions() == - this.masterStatus.getRegionManager().numOnlineMetaRegions()) { - break; - } - } - try { - wait(this.threadWakeFrequency); - } catch (InterruptedException e) { - // continue - } - } - return this.masterStatus.isClosed(); - } - - /** - * Add another meta region to scan to the queue. - */ - void addMetaRegionToScan(MetaRegion m) { - metaRegionsToScan.add(m); - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/master/ServerManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (working copy) @@ -19,58 +19,68 @@ */ package org.apache.hadoop.hbase.master; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Chore; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HMsg; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HServerLoad; +import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.YouAreDeadException; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.Leases.LeaseStillHeldException; import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.master.RegionManager.RegionState; +import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler; import org.apache.hadoop.hbase.master.metrics.MasterMetrics; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.Watcher.Event.EventType; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - /** * The ServerManager class manages info about region servers - HServerInfo, * load numbers, dying servers, etc. + *

        + * Maintains lists of online and dead servers. Processes the startups, + * shutdowns, and deaths of region servers. + *

        + * Servers are distinguished in two different ways. A given server has a + * location, specified by hostname and port, and of which there can only be one + * online at any given time. A server instance is specified by the location + * (hostname and port) as well as the startcode (timestamp from when the server + * was started). This is used to differentiate a restarted instance of a given + * server from the original instance. */ public class ServerManager { - private static final Log LOG = - LogFactory.getLog(ServerManager.class.getName()); + private static final Log LOG = LogFactory.getLog(ServerManager.class); private final AtomicInteger quiescedServers = new AtomicInteger(0); + private final AtomicInteger availableServers = new AtomicInteger(0); - // The map of known server names to server info - private final Map serversToServerInfo = + /** The map of known server names to server info */ + private final Map onlineServers = new ConcurrentHashMap(); - /* + // TODO: This is strange to have two maps but HSI above is used on both sides + /** + * Map from full server-instance name to the RPC connection for this server. + */ + private final Map serverConnections = + new HashMap(); + + /** * Set of known dead servers. On znode expiration, servers are added here. * This is needed in case of a network partitioning where the server's lease * expires, but the server is still running. After the network is healed, @@ -80,29 +90,17 @@ private final Set deadServers = Collections.synchronizedSet(new HashSet()); - // SortedMap server load -> Set of server names - private final SortedMap> loadToServers = - Collections.synchronizedSortedMap(new TreeMap>()); - // Map of server names -> server load - private final Map serversToLoad = - new ConcurrentHashMap(); + private MasterController masterStatus; - private MasterStatus masterStatus; - private RegionServerOperationQueue regionServerOperationQueue; private MasterMetrics masterMetrics; - /* The regionserver will not be assigned or asked close regions if it - * is currently opening >= this many regions. - */ - private final int nobalancingCount; - private final ServerMonitor serverMonitorThread; private int minimumServerCount; private final OldLogsCleaner oldLogCleaner; - /* + /** * Dumps into log current stats on dead servers and number of servers * TODO: Make this a metric; dump metrics into log. */ @@ -113,7 +111,7 @@ @Override protected void chore() { - int numServers = serversToServerInfo.size(); + int numServers = availableServers.get(); int numDeadServers = deadServers.size(); double averageLoad = getAverageLoad(); String deadServersList = null; @@ -142,16 +140,15 @@ * Constructor. * @param masterStatus */ - public ServerManager(MasterStatus masterStatus, MasterMetrics masterMetrics, - RegionServerOperationQueue regionServerOperationQueue) { + public ServerManager(MasterController masterStatus, + MasterMetrics masterMetrics, + FileSystemManager fileSystemManager) { this.masterStatus = masterStatus; this.masterMetrics = masterMetrics; - this.regionServerOperationQueue = regionServerOperationQueue; Configuration c = masterStatus.getConfiguration(); - this.nobalancingCount = c.getInt("hbase.regions.nobalancing.count", 4); int metaRescanInterval = c.getInt("hbase.master.meta.thread.rescanfrequency", 60 * 1000); - this.minimumServerCount = c.getInt("hbase.regions.server.count.min", 0); + this.minimumServerCount = c.getInt("hbase.regions.server.count.min", 1); this.serverMonitorThread = new ServerMonitor(metaRescanInterval, this.masterStatus.getShutdownRequested()); String n = Thread.currentThread().getName(); @@ -160,8 +157,8 @@ this.oldLogCleaner = new OldLogsCleaner( c.getInt("hbase.master.meta.thread.rescanfrequency",60 * 1000), this.masterStatus.getShutdownRequested(), c, - masterStatus.getFileSystemManager().getFileSystem(), - masterStatus.getFileSystemManager().getOldLogDir()); + fileSystemManager.getFileSystem(), + fileSystemManager.getOldLogDir()); Threads.setDaemonThreadRunning(oldLogCleaner, n + ".oldLogCleaner"); @@ -200,8 +197,8 @@ } private HServerInfo haveServerWithSameHostAndPortAlready(final String hostnamePort) { - synchronized (this.serversToServerInfo) { - for (Map.Entry e: this.serversToServerInfo.entrySet()) { + synchronized (this.onlineServers) { + for (Map.Entry e: this.onlineServers.entrySet()) { if (e.getValue().getHostnamePort().equals(hostnamePort)) { return e.getValue(); } @@ -210,7 +207,7 @@ return null; } - /* + /** * If this server is on the dead list, reject it with a LeaseStillHeldException * @param serverName Server name formatted as host_port_startcode. * @param what START or REPORT @@ -218,7 +215,9 @@ */ private void checkIsDead(final String serverName, final String what) throws YouAreDeadException { - if (!isDead(serverName)) return; + if (!isDead(serverName)) { + return; + } String message = "Server " + what + " rejected; currently processing " + serverName + " as dead server"; LOG.debug(message); @@ -230,7 +229,7 @@ * @param info The region server informations */ public void recordNewServer(HServerInfo info) { - recordNewServer(info, false); + recordNewServer(info, false, null); } /** @@ -239,21 +238,19 @@ * @param useInfoLoad True if the load from the info should be used * like under a master failover */ - void recordNewServer(HServerInfo info, boolean useInfoLoad) { + void recordNewServer(HServerInfo info, boolean useInfoLoad, + HRegionInterface hri) { HServerLoad load = useInfoLoad ? info.getLoad() : new HServerLoad(); String serverName = info.getServerName(); info.setLoad(load); // TODO: Why did we update the RS location ourself? Shouldn't RS do this? // masterStatus.getZooKeeper().updateRSLocationGetWatch(info, watcher); - this.serversToServerInfo.put(serverName, info); - this.serversToLoad.put(serverName, load); - synchronized (this.loadToServers) { - Set servers = this.loadToServers.get(load); - if (servers == null) { - servers = new HashSet(); - } - servers.add(serverName); - this.loadToServers.put(load, servers); + onlineServers.put(serverName, info); + availableServers.incrementAndGet(); + if(hri == null) { + serverConnections.remove(serverName); + } else { + serverConnections.put(serverName, hri); } } @@ -285,11 +282,11 @@ } } if (this.masterStatus.getShutdownRequested().get()) { - if (quiescedServers.get() >= serversToServerInfo.size()) { + if (quiescedServers.get() >= availableServers.get()) { // If the only servers we know about are meta servers, then we can // proceed with shutdown LOG.info("All user tables quiesced. Proceeding with shutdown"); - this.masterStatus.startShutdown(); + this.masterStatus.requestShutdown(); } if (!this.masterStatus.isClosed()) { if (msgs.length > 0 && @@ -309,7 +306,7 @@ return new HMsg [] {HMsg.REGIONSERVER_STOP}; } - HServerInfo storedInfo = this.serversToServerInfo.get(info.getServerName()); + HServerInfo storedInfo = this.onlineServers.get(info.getServerName()); if (storedInfo == null) { LOG.warn("Received report from unknown server -- telling it " + "to " + HMsg.REGIONSERVER_STOP + ": " + info.getServerName()); @@ -331,7 +328,7 @@ info.getServerName()); } - synchronized (this.serversToServerInfo) { + synchronized (this.onlineServers) { removeServerInfo(info.getServerName()); notifyServers(); } @@ -342,7 +339,7 @@ } } - /* + /** * Region server is exiting with a clean shutdown. * * In this case, the server sends MSG_REPORT_EXITING in msgs[0] followed by @@ -351,7 +348,12 @@ * @param msgs */ private void processRegionServerExit(HServerInfo serverInfo, HMsg[] msgs) { - synchronized (this.serversToServerInfo) { + // TODO: won't send MSG_REPORT_CLOSE for each region + // TODO: Change structure/naming of removeServerInfo (does much more) + // TODO: should we keep this? seems like master could issue closes + // and then rs would stagger rather than closing all before + // reporting back to master they are closed? + synchronized (this.onlineServers) { // This method removes ROOT/META from the list and marks them to be // reassigned in addition to other housework. if (removeServerInfo(serverInfo.getServerName())) { @@ -369,32 +371,32 @@ HRegionInfo info = msgs[i].getRegionInfo(); // Meta/root region offlining is handed in removeServerInfo above. if (!info.isMetaRegion()) { - synchronized (masterStatus.getRegionManager()) { - if (!masterStatus.getRegionManager().isOfflined(info.getRegionNameAsString())) { - masterStatus.getRegionManager().setUnassigned(info, true); - } else { - masterStatus.getRegionManager().removeRegion(info); - } - } +// synchronized (masterStatus.getRegionManager()) { +// if (!masterStatus.getRegionManager().isOfflined(info.getRegionNameAsString())) { +// masterStatus.getRegionManager().setUnassigned(info, true); +// } else { +// masterStatus.getRegionManager().removeRegion(info); +// } +// } } } } // There should not be any regions in transition for this server - the // server should finish transitions itself before closing - Map inTransition = masterStatus.getRegionManager() - .getRegionsInTransitionOnServer(serverInfo.getServerName()); - for (Map.Entry entry : inTransition.entrySet()) { - LOG.warn("Region server " + serverInfo.getServerName() - + " shut down with region " + entry.getKey() + " in transition " - + "state " + entry.getValue()); - masterStatus.getRegionManager().setUnassigned(entry.getValue().getRegionInfo(), - true); - } +// Map inTransition = masterStatus.getRegionManager() +// .getRegionsInTransitionOnServer(serverInfo.getServerName()); +// for (Map.Entry entry : inTransition.entrySet()) { +// LOG.warn("Region server " + serverInfo.getServerName() +// + " shut down with region " + entry.getKey() + " in transition " +// + "state " + entry.getValue()); +// masterStatus.getRegionManager().setUnassigned(entry.getValue().getRegionInfo(), +// true); +// } } } } - /* + /** * RegionServer is checking in, no exceptional circumstances * @param serverInfo * @param mostLoadedRegions @@ -406,113 +408,16 @@ final HRegionInfo[] mostLoadedRegions, HMsg[] msgs) throws IOException { // Refresh the info object and the load information - this.serversToServerInfo.put(serverInfo.getServerName(), serverInfo); - HServerLoad load = this.serversToLoad.get(serverInfo.getServerName()); - if (load != null) { + this.onlineServers.put(serverInfo.getServerName(), serverInfo); + HServerLoad load = serverInfo.getLoad(); + if(load != null) { masterMetrics.incrementRequests(load.getNumberOfRequests()); - if (!load.equals(serverInfo.getLoad())) { - updateLoadToServers(serverInfo.getServerName(), load); - } } - - // Set the current load information - load = serverInfo.getLoad(); - this.serversToLoad.put(serverInfo.getServerName(), load); - synchronized (loadToServers) { - Set servers = this.loadToServers.get(load); - if (servers == null) { - servers = new HashSet(); - } - servers.add(serverInfo.getServerName()); - this.loadToServers.put(load, servers); - } - - // Next, process messages for this server - return processMsgs(serverInfo, mostLoadedRegions, msgs); + // No more piggyback messages on heartbeats for other stuff + return msgs; } - /* - * Process all the incoming messages from a server that's contacted us. - * Note that we never need to update the server's load information because - * that has already been done in regionServerReport. - * @param serverInfo - * @param mostLoadedRegions - * @param incomingMsgs - * @return - */ - private HMsg[] processMsgs(HServerInfo serverInfo, - HRegionInfo[] mostLoadedRegions, HMsg incomingMsgs[]) { - ArrayList returnMsgs = new ArrayList(); - if (serverInfo.getServerAddress() == null) { - throw new NullPointerException("Server address cannot be null; " + - "hbase-958 debugging"); - } - // Get reports on what the RegionServer did. - // Be careful that in message processors we don't throw exceptions that - // break the switch below because then we might drop messages on the floor. - int openingCount = 0; - for (int i = 0; i < incomingMsgs.length; i++) { - HRegionInfo region = incomingMsgs[i].getRegionInfo(); - LOG.info("Processing " + incomingMsgs[i] + " from " + - serverInfo.getServerName() + "; " + (i + 1) + " of " + - incomingMsgs.length); - if (!regionServerOperationQueue.process(serverInfo, incomingMsgs[i])) { - continue; - } - switch (incomingMsgs[i].getType()) { - case MSG_REPORT_PROCESS_OPEN: - openingCount++; - break; - - case MSG_REPORT_OPEN: - processRegionOpen(serverInfo, region, returnMsgs); - break; - - case MSG_REPORT_CLOSE: - processRegionClose(region); - break; - - case MSG_REPORT_SPLIT: - processSplitRegion(region, incomingMsgs[++i].getRegionInfo(), - incomingMsgs[++i].getRegionInfo()); - break; - - case MSG_REPORT_SPLIT_INCLUDES_DAUGHTERS: - processSplitRegion(region, incomingMsgs[i].getDaughterA(), - incomingMsgs[i].getDaughterB()); - break; - - default: - LOG.warn("Impossible state during message processing. Instruction: " + - incomingMsgs[i].getType()); - } - } - - synchronized (this.masterStatus.getRegionManager()) { - // Tell the region server to close regions that we have marked for closing. - for (HRegionInfo i: - this.masterStatus.getRegionManager().getMarkedToClose(serverInfo.getServerName())) { - returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE, i)); - // Transition the region from toClose to closing state - this.masterStatus.getRegionManager().setPendingClose(i.getRegionNameAsString()); - } - - // Figure out what the RegionServer ought to do, and write back. - - // Should we tell it close regions because its overloaded? If its - // currently opening regions, leave it alone till all are open. - if (openingCount < this.nobalancingCount) { - this.masterStatus.getRegionManager().assignRegions(serverInfo, mostLoadedRegions, - returnMsgs); - } - - // Send any pending table actions. - this.masterStatus.getRegionManager().applyActions(serverInfo, returnMsgs); - } - return returnMsgs.toArray(new HMsg[returnMsgs.size()]); - } - - /* + /** * A region has split. * * @param region @@ -521,22 +426,24 @@ * @param returnMsgs */ private void processSplitRegion(HRegionInfo region, HRegionInfo a, HRegionInfo b) { - synchronized (masterStatus.getRegionManager()) { - // Cancel any actions pending for the affected region. - // This prevents the master from sending a SPLIT message if the table - // has already split by the region server. - this.masterStatus.getRegionManager().endActions(region.getRegionName()); - assignSplitDaughter(a); - assignSplitDaughter(b); - if (region.isMetaTable()) { - // A meta region has split. - this. masterStatus.getRegionManager().offlineMetaRegionWithStartKey(region.getStartKey()); - this.masterStatus.getRegionManager().incrementNumMetaRegions(); - } - } +// synchronized (masterStatus.getRegionManager()) { +// // TODO: This will just delete the parent off fs? or will this completely +// // go away and we will rely on RS doing stuff and new ZK nodes +// // Cancel any actions pending for the affected region. +// // This prevents the master from sending a SPLIT message if the table +// // has already split by the region server. +// this.masterStatus.getRegionManager().endActions(region.getRegionName()); +// assignSplitDaughter(a); +// assignSplitDaughter(b); +// if (region.isMetaTable()) { +// // A meta region has split. +// this. masterStatus.getRegionManager().offlineMetaRegionWithStartKey(region.getStartKey()); +// this.masterStatus.getRegionManager().incrementNumMetaRegions(); +// } +// } } - /* + /** * Assign new daughter-of-a-split UNLESS its already been assigned. * It could have been assigned already in rare case where there was a large * gap between insertion of the daughter region into .META. by the @@ -544,177 +451,55 @@ * HBASE-1784). * @param hri Region to assign. */ - private void assignSplitDaughter(final HRegionInfo hri) { - MetaRegion mr = - this.masterStatus.getRegionManager().getFirstMetaRegionForRegion(hri); - Get g = new Get(hri.getRegionName()); - g.addFamily(HConstants.CATALOG_FAMILY); - try { - HRegionInterface server = - this.masterStatus.getServerConnection().getHRegionConnection(mr.getServer()); - Result r = server.get(mr.getRegionName(), g); - // If size > 3 -- presume regioninfo, startcode and server -- then presume - // that this daughter already assigned and return. - if (r.size() >= 3) return; - } catch (IOException e) { - LOG.warn("Failed get on " + HConstants.CATALOG_FAMILY_STR + - "; possible double-assignment?", e); - } - this.masterStatus.getRegionManager().setUnassigned(hri, false); - } +// private void assignSplitDaughter(final HRegionInfo hri) { +// MetaRegion mr = +// this.masterStatus.getRegionManager().getFirstMetaRegionForRegion(hri); +// Get g = new Get(hri.getRegionName()); +// g.addFamily(HConstants.CATALOG_FAMILY); +// try { +// HRegionInterface server = +// this.masterStatus.getServerConnection().getHRegionConnection(mr.getServer()); +// Result r = server.get(mr.getRegionName(), g); +// // If size > 3 -- presume regioninfo, startcode and server -- then presume +// // that this daughter already assigned and return. +// if (r.size() >= 3) { +// return; +// } +// } catch (IOException e) { +// LOG.warn("Failed get on " + HConstants.CATALOG_FAMILY_STR + +// "; possible double-assignment?", e); +// } +// this.masterStatus.getRegionManager().setUnassigned(hri, false); +// } - /* - * Region server is reporting that a region is now opened - * @param serverInfo - * @param region - * @param returnMsgs - */ - public void processRegionOpen(HServerInfo serverInfo, - HRegionInfo region, ArrayList returnMsgs) { - boolean duplicateAssignment = false; - synchronized (masterStatus.getRegionManager()) { - if (!this.masterStatus.getRegionManager().isUnassigned(region) && - !this.masterStatus.getRegionManager().isPendingOpen(region.getRegionNameAsString())) { - if (region.isRootRegion()) { - // Root region - HServerAddress rootServer = - this.masterStatus.getRegionManager().getRootRegionLocation(); - if (rootServer != null) { - if (rootServer.compareTo(serverInfo.getServerAddress()) == 0) { - // A duplicate open report from the correct server - return; - } - // We received an open report on the root region, but it is - // assigned to a different server - duplicateAssignment = true; - } - } else { - // Not root region. If it is not a pending region, then we are - // going to treat it as a duplicate assignment, although we can't - // tell for certain that's the case. - if (this.masterStatus.getRegionManager().isPendingOpen( - region.getRegionNameAsString())) { - // A duplicate report from the correct server - return; - } - duplicateAssignment = true; - } - } - - if (duplicateAssignment) { - LOG.warn("region server " + serverInfo.getServerAddress().toString() + - " should not have opened region " + Bytes.toString(region.getRegionName())); - - // This Region should not have been opened. - // Ask the server to shut it down, but don't report it as closed. - // Otherwise the HMaster will think the Region was closed on purpose, - // and then try to reopen it elsewhere; that's not what we want. - returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE_WITHOUT_REPORT, - region, "Duplicate assignment".getBytes())); - } else { - if (region.isRootRegion()) { - // it was assigned, and it's not a duplicate assignment, so take it out - // of the unassigned list. - this.masterStatus.getRegionManager().removeRegion(region); - - // Store the Root Region location (in memory) - HServerAddress rootServer = serverInfo.getServerAddress(); - this.masterStatus.getServerConnection().setRootRegionLocation( - new HRegionLocation(region, rootServer)); - this.masterStatus.getRegionManager().setRootRegionLocation(rootServer); - } else { - // Note that the table has been assigned and is waiting for the - // meta table to be updated. - this.masterStatus.getRegionManager().setOpen(region.getRegionNameAsString()); - RegionServerOperation op = - new ProcessRegionOpen(masterStatus, serverInfo, region); - regionServerOperationQueue.put(op); - } - } - } - } - - /* - * @param region - * @throws Exception - */ - public void processRegionClose(HRegionInfo region) { - synchronized (this.masterStatus.getRegionManager()) { - if (region.isRootRegion()) { - // Root region - this.masterStatus.getRegionManager().unsetRootRegion(); - if (region.isOffline()) { - // Can't proceed without root region. Shutdown. - LOG.fatal("root region is marked offline"); - this.masterStatus.shutdown(); - return; - } - - } else if (region.isMetaTable()) { - // Region is part of the meta table. Remove it from onlineMetaRegions - this.masterStatus.getRegionManager().offlineMetaRegionWithStartKey(region.getStartKey()); - } - - boolean offlineRegion = - this.masterStatus.getRegionManager().isOfflined(region.getRegionNameAsString()); - boolean reassignRegion = !region.isOffline() && !offlineRegion; - - // NOTE: If the region was just being closed and not offlined, we cannot - // mark the region unassignedRegions as that changes the ordering of - // the messages we've received. In this case, a close could be - // processed before an open resulting in the master not agreeing on - // the region's state. - this.masterStatus.getRegionManager().setClosed(region.getRegionNameAsString()); - RegionServerOperation op = - new ProcessRegionClose(masterStatus, region, offlineRegion, reassignRegion); - regionServerOperationQueue.put(op); - } - } - /** Update a server load information because it's shutting down*/ private boolean removeServerInfo(final String serverName) { boolean infoUpdated = false; - HServerInfo info = this.serversToServerInfo.remove(serverName); - // Only update load information once. - // This method can be called a couple of times during shutdown. - if (info != null) { - LOG.info("Removing server's info " + serverName); - this.masterStatus.getRegionManager().offlineMetaServer(info.getServerAddress()); - - //HBASE-1928: Check whether this server has been transitioning the ROOT table - if (this.masterStatus.getRegionManager().isRootInTransitionOnThisServer(serverName)) { - this.masterStatus.getRegionManager().unsetRootRegion(); - this.masterStatus.getRegionManager().reassignRootRegion(); - } - - //HBASE-1928: Check whether this server has been transitioning the META table - HRegionInfo metaServerRegionInfo = this.masterStatus.getRegionManager().getMetaServerRegionInfo (serverName); - if (metaServerRegionInfo != null) { - this.masterStatus.getRegionManager().setUnassigned(metaServerRegionInfo, true); - } - - infoUpdated = true; - // update load information - updateLoadToServers(serverName, this.serversToLoad.remove(serverName)); - } + HServerInfo info = this.onlineServers.remove(serverName); + this.availableServers.decrementAndGet(); +// // Only update load information once. +// // This method can be called a couple of times during shutdown. +// if (info != null) { +// LOG.info("Removing server's info " + serverName); +// this.masterStatus.getRegionManager().offlineMetaServer(info.getServerAddress()); +// +// //HBASE-1928: Check whether this server has been transitioning the ROOT table +// if (this.masterStatus.getRegionManager().isRootInTransitionOnThisServer(serverName)) { +// this.masterStatus.getRegionManager().unsetRootRegion(); +// this.masterStatus.getRegionManager().reassignRootRegion(); +// } +// +// //HBASE-1928: Check whether this server has been transitioning the META table +// HRegionInfo metaServerRegionInfo = this.masterStatus.getRegionManager().getMetaServerRegionInfo (serverName); +// if (metaServerRegionInfo != null) { +// this.masterStatus.getRegionManager().setUnassigned(metaServerRegionInfo, true); +// } +// +// infoUpdated = true; +// } return infoUpdated; } - private void updateLoadToServers(final String serverName, - final HServerLoad load) { - if (load == null) return; - synchronized (this.loadToServers) { - Set servers = this.loadToServers.get(load); - if (servers != null) { - servers.remove(serverName); - if (servers.size() > 0) - this.loadToServers.put(load, servers); - else - this.loadToServers.remove(load); - } - } - } - /** * Compute the average load across all region servers. * Currently, this uses a very naive computation - just uses the number of @@ -725,19 +510,17 @@ int totalLoad = 0; int numServers = 0; double averageLoad = 0.0; - synchronized (serversToLoad) { - numServers = serversToLoad.size(); - for (HServerLoad load : serversToLoad.values()) { - totalLoad += load.getNumberOfRegions(); - } - averageLoad = (double)totalLoad / (double)numServers; + for (HServerInfo hsi : onlineServers.values()) { + numServers++; + totalLoad += hsi.getLoad().getNumberOfRegions(); } + averageLoad = (double)totalLoad / (double)numServers; return averageLoad; } /** @return the number of active servers */ public int numServers() { - return this.serversToServerInfo.size(); + return availableServers.get(); } /** @@ -745,15 +528,15 @@ * @return HServerInfo for the given server address */ public HServerInfo getServerInfo(String name) { - return this.serversToServerInfo.get(name); + return this.onlineServers.get(name); } /** * @return Read-only map of servers to serverinfo. */ - public Map getServersToServerInfo() { - synchronized (this.serversToServerInfo) { - return Collections.unmodifiableMap(this.serversToServerInfo); + public Map getOnlineServers() { + synchronized (this.onlineServers) { + return Collections.unmodifiableMap(this.onlineServers); } } @@ -763,39 +546,23 @@ * if nothing found. */ public HServerInfo getHServerInfo(final HServerAddress hsa) { - synchronized(this.serversToServerInfo) { + synchronized(this.onlineServers) { // TODO: This is primitive. Do a better search. - for (Map.Entry e: this.serversToServerInfo.entrySet()) { - if (e.getValue().getServerAddress().equals(hsa)) return e.getValue(); + for (Map.Entry e: this.onlineServers.entrySet()) { + if (e.getValue().getServerAddress().equals(hsa)) { + return e.getValue(); + } } } return null; } /** - * @return Read-only map of servers to load. - */ - public Map getServersToLoad() { - synchronized (this.serversToLoad) { - return Collections.unmodifiableMap(serversToLoad); - } - } - - /** - * @return Read-only map of load to servers. - */ - public SortedMap> getLoadToServers() { - synchronized (this.loadToServers) { - return Collections.unmodifiableSortedMap(this.loadToServers); - } - } - - /** * Wakes up threads waiting on serversToServerInfo */ public void notifyServers() { - synchronized (this.serversToServerInfo) { - this.serversToServerInfo.notifyAll(); + synchronized (this.onlineServers) { + this.onlineServers.notifyAll(); } } @@ -811,12 +578,12 @@ // away. Just exit as quickly as possible. return; } - synchronized (serversToServerInfo) { - while (serversToServerInfo.size() > 0) { + synchronized (onlineServers) { + while (onlineServers.size() > 0) { LOG.info("Waiting on following regionserver(s) to go down " + - this.serversToServerInfo.values()); + this.onlineServers.values()); try { - this.serversToServerInfo.wait(500); + this.onlineServers.wait(500); } catch (InterruptedException e) { // continue } @@ -824,24 +591,6 @@ } } - /** Watcher triggered when a RS znode is deleted */ - private class ServerExpirer implements Watcher { - private HServerInfo server; - - ServerExpirer(final HServerInfo hsi) { - this.server = hsi; - } - - public void process(WatchedEvent event) { - if (!event.getType().equals(EventType.NodeDeleted)) { - LOG.warn("Unexpected event=" + event); - return; - } - LOG.info(this.server.getServerName() + " znode expired"); - expireServer(this.server); - } - } - /* * Expire the passed server. Add it to list of deadservers and queue a * shutdown processing. @@ -850,32 +599,27 @@ // First check a server to expire. ServerName is of the form: // , , String serverName = hsi.getServerName(); - HServerInfo info = this.serversToServerInfo.get(serverName); + HServerInfo info = this.onlineServers.get(serverName); if (info == null) { - LOG.warn("No HServerInfo for " + serverName); + LOG.warn("Received expiration of " + hsi.getServerName() + + " but server is not currently online"); return; } if (this.deadServers.contains(serverName)) { - LOG.warn("Already processing shutdown of " + serverName); + // TODO: Can this happen? It shouldn't be online in this case? + LOG.warn("Received expiration of " + hsi.getServerName() + + " but server shutdown is already in progress"); return; } // Remove the server from the known servers lists and update load info - this.serversToServerInfo.remove(serverName); - HServerLoad load = this.serversToLoad.remove(serverName); - if (load != null) { - synchronized (this.loadToServers) { - Set servers = this.loadToServers.get(load); - if (servers != null) { - servers.remove(serverName); - if (servers.isEmpty()) this.loadToServers.remove(load); - } - } - } + this.onlineServers.remove(serverName); + this.availableServers.decrementAndGet(); + this.serverConnections.remove(serverName); // Add to dead servers and queue a shutdown processing. - LOG.debug("Added=" + serverName + - " to dead servers, added shutdown processing operation"); this.deadServers.add(serverName); - regionServerOperationQueue.put(new ProcessServerShutdown(masterStatus, info)); + new ServerShutdownHandler(masterStatus).submit(); + LOG.debug("Added=" + serverName + + " to dead servers, submitted shutdown handler to be executed"); } /** @@ -914,19 +658,6 @@ return this.deadServers; } - /** - * Add to the passed m servers that are loaded less than - * l. - * @param l - * @param m - */ - void getLightServers(final HServerLoad l, - SortedMap> m) { - synchronized (this.loadToServers) { - m.putAll(this.loadToServers.headMap(l)); - } - } - public boolean canAssignUserRegions() { if (minimumServerCount == 0) { return true; @@ -937,8 +668,86 @@ public void setMinimumServerCount(int minimumServerCount) { this.minimumServerCount = minimumServerCount; } - - public RegionServerOperationQueue getRegionServerOperationQueue() { - return this.regionServerOperationQueue; + + // RPC methods to region servers + + /** + * Sends an OPEN RPC to the specified server to open the specified region. + *

        + * There is no reason a region server should reject this open. + *

        + * @param server server to open a region + * @param regionName region to open + */ + public void sendRegionOpen(HServerInfo server, HRegionInfo region) { + HRegionInterface hri = getServerConnection(server); + if(hri == null) { + LOG.warn("Attempting to send OPEN RPC to server " + server.getServerName() + + " failed because no RPC connection found to this server"); + return; + } + hri.openRegion(region); } + + /** + * Sends an CLOSE RPC to the specified server to close the specified region. + *

        + * A region server could reject the close request because it either does not + * have the specified region or the region is being split. + * @param server server to open a region + * @param regionName region to open + * @return true if server acknowledged close, false if not + * @throws NotServingRegionException + */ + public void sendRegionClose(HServerInfo server, HRegionInfo region) + throws NotServingRegionException { + HRegionInterface hri = getServerConnection(server); + if(hri == null) { + LOG.warn("Attempting to send CLOSE RPC to server " + server.getServerName() + + " failed because no RPC connection found to this server"); + return; + } + hri.closeRegion(region); + } + + private HRegionInterface getServerConnection(HServerInfo info) { + try { + HRegionInterface hri = serverConnections.get(info.getServerName()); + if(hri == null) { + LOG.info("new connection"); + hri = masterStatus.getServerConnection().getHRegionConnection( + info.getServerAddress(), false); + serverConnections.put(info.getServerName(), hri); + } + return hri; + } catch (IOException e) { + LOG.error("Error connecting to region server", e); + throw new RuntimeException("Fatal error connection to RS", e); + } + } + + /** + * Waits for the minimum number of servers to be running. + */ + public void waitForMinServers() { + while(numServers() < minimumServerCount) { +// !masterStatus.getShutdownRequested().get()) { + LOG.info("Waiting for enough servers to check in. Currently have " + + numServers() + " but need at least " + minimumServerCount); + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + LOG.warn("Got interrupted waiting for servers to check in, looping"); + } + } + } + + public List getOnlineServersList() { + // TODO: optimize the load balancer call so we don't need to make a new list + return new ArrayList(onlineServers.values()); + } + + public boolean isServerOnline(String serverName) { + return onlineServers.containsKey(serverName); + } } Index: src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationListener.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationListener.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/RegionServerOperationListener.java (working copy) @@ -1,58 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.HServerInfo; - -/** - * Listener for regionserver events in master. - * @see HMaster#registerRegionServerOperationListener(RegionServerOperationListener) - * @see HMaster#unregisterRegionServerOperationListener(RegionServerOperationListener) - */ -public interface RegionServerOperationListener { - /** - * Called for each message passed the master. Most of the messages that come - * in here will go on to become {@link #process(RegionServerOperation)}s but - * others like {@linke HMsg.Type#MSG_REPORT_PROCESS_OPEN} go no further; - * only in here can you see them come in. - * @param serverInfo Server we got the message from. - * @param incomingMsg The message received. - * @return True to continue processing, false to skip. - */ - public boolean process(final HServerInfo serverInfo, - final HMsg incomingMsg); - - /** - * Called before processing op - * @param op - * @return True if we are to proceed w/ processing. - * @exception IOException - */ - public boolean process(final RegionServerOperation op) throws IOException; - - /** - * Called after op has been processed. - * @param op The operation that just completed. - */ - public void processed(final RegionServerOperation op); -} Index: src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java (working copy) @@ -1,134 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKAssign; -import org.apache.zookeeper.KeeperException; - -/** - * ProcessRegionOpen is instantiated when a region server reports that it is - * serving a region. This applies to all meta and user regions except the - * root region which is handled specially. - */ -public class ProcessRegionOpen extends ProcessRegionStatusChange { - protected final HServerInfo serverInfo; - - /** - * @param masterStatus - * @param info - * @param regionInfo - */ - public ProcessRegionOpen(MasterStatus masterStatus, HServerInfo info, - HRegionInfo regionInfo) { - super(masterStatus, regionInfo); - if (info == null) { - throw new NullPointerException("HServerInfo cannot be null; " + - "hbase-958 debugging"); - } - this.serverInfo = info; - } - - @Override - public String toString() { - return "PendingOpenOperation from " + serverInfo.getServerName(); - } - - @Override - protected boolean process() throws IOException { - // TODO: The below check is way too convoluted!!! - if (!metaRegionAvailable()) { - // We can't proceed unless the meta region we are going to update - // is online. metaRegionAvailable() has put this operation on the - // delayedToDoQueue, so return true so the operation is not put - // back on the toDoQueue - return true; - } - HRegionInterface server = - masterStatus.getServerConnection().getHRegionConnection(getMetaRegion().getServer()); - LOG.info(regionInfo.getRegionNameAsString() + " open on " + - serverInfo.getServerName()); - - // Register the newly-available Region's location. - Put p = new Put(regionInfo.getRegionName()); - p.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, - Bytes.toBytes(serverInfo.getHostnamePort())); - p.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, - Bytes.toBytes(serverInfo.getStartCode())); - server.put(metaRegionName, p); - LOG.info("Updated row " + regionInfo.getRegionNameAsString() + - " in region " + Bytes.toString(metaRegionName) + " with startcode=" + - serverInfo.getStartCode() + ", server=" + serverInfo.getHostnamePort()); - synchronized (masterStatus.getRegionManager()) { - if (isMetaTable) { - // It's a meta region. - MetaRegion m = - new MetaRegion(new HServerAddress(serverInfo.getServerAddress()), - regionInfo); - if (!masterStatus.getRegionManager().isInitialMetaScanComplete()) { - // Put it on the queue to be scanned for the first time. - if (LOG.isDebugEnabled()) { - LOG.debug("Adding " + m.toString() + " to regions to scan"); - } - masterStatus.getRegionManager().addMetaRegionToScan(m); - } else { - // Add it to the online meta regions - if (LOG.isDebugEnabled()) { - LOG.debug("Adding to onlineMetaRegions: " + m.toString()); - } - masterStatus.getRegionManager().putMetaRegionOnline(m); - // Interrupting the Meta Scanner sleep so that it can - // process regions right away - masterStatus.getRegionManager().metaScannerThread.triggerNow(); - } - } - // If updated successfully, remove from pending list if the state - // is consistent. For example, a disable could be called before the - // synchronization. - if(masterStatus.getRegionManager(). - isOfflined(regionInfo.getRegionNameAsString())) { - LOG.warn("We opened a region while it was asked to be closed."); - } else { - masterStatus.getRegionManager().removeRegion(regionInfo); - } - try { - ZKAssign.deleteOpenedNode(masterStatus.getZooKeeper(), - regionInfo.getEncodedName()); - } catch (KeeperException e) { - LOG.error("ZK error deleting opened node", e); - throw new IOException(e); - } - return true; - } - } - - @Override - protected int getPriority() { - return 0; // highest priority - } -} Index: src/main/java/org/apache/hadoop/hbase/master/ModifyColumn.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ModifyColumn.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ModifyColumn.java (working copy) @@ -1,56 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Bytes; - -import java.io.IOException; - -/** Instantiated to modify an existing column family on a table */ -class ModifyColumn extends ColumnOperation { - private final HColumnDescriptor descriptor; - private final byte [] columnName; - - ModifyColumn(final HMaster master, final byte [] tableName, - final byte [] columnName, HColumnDescriptor descriptor) - throws IOException { - super(master, tableName); - this.descriptor = descriptor; - this.columnName = columnName; - } - - @Override - protected void postProcessMeta(MetaRegion m, HRegionInterface server) - throws IOException { - for (HRegionInfo i: unservedRegions) { - if (i.getTableDesc().hasFamily(columnName)) { - i.getTableDesc().addFamily(descriptor); - updateRegionInfo(server, m.getRegionName(), i); - } else { // otherwise, we have an error. - throw new InvalidColumnNameException("Column family '" + - Bytes.toString(columnName) + - "' doesn't exist, so cannot be modified."); - } - } - } -} Index: src/main/java/org/apache/hadoop/hbase/master/TableOperation.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/TableOperation.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/TableOperation.java (working copy) @@ -1,180 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Bytes; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - -/** - * Abstract base class for operations that need to examine all HRegionInfo - * objects in a table. (For a table, operate on each of its rows - * in .META.). - */ -abstract class TableOperation { - private final Set metaRegions; - protected final byte [] tableName; - // Do regions in order. - protected final Set unservedRegions = new TreeSet(); - protected HMaster master; - - protected TableOperation(final HMaster master, final byte [] tableName) - throws IOException { - this.master = master; - if (!this.master.isMasterRunning()) { - throw new MasterNotRunningException(); - } - // add the delimiters. - // TODO maybe check if this is necessary? - this.tableName = tableName; - - // Don't wait for META table to come on line if we're enabling it - if (!Bytes.equals(HConstants.META_TABLE_NAME, this.tableName)) { - // We can not access any meta region if they have not already been - // assigned and scanned. - if (master.getRegionManager().metaScannerThread.waitForMetaRegionsOrClose()) { - // We're shutting down. Forget it. - throw new MasterNotRunningException(); - } - } - this.metaRegions = master.getRegionManager().getMetaRegionsForTable(tableName); - } - - private class ProcessTableOperation extends RetryableMetaOperation { - ProcessTableOperation(MetaRegion m, HMaster master) { - super(m, master); - } - - public Boolean call() throws IOException { - boolean tableExists = false; - - // Open a scanner on the meta region - byte [] tableNameMetaStart = - Bytes.toBytes(Bytes.toString(tableName) + ",,"); - final Scan scan = new Scan(tableNameMetaStart) - .addFamily(HConstants.CATALOG_FAMILY); - long scannerId = this.server.openScanner(m.getRegionName(), scan); - int rows = this.masterStatus.getConfiguration(). - getInt("hbase.meta.scanner.caching", 100); - scan.setCaching(rows); - List emptyRows = new ArrayList(); - try { - while (true) { - Result values = this.server.next(scannerId); - if (values == null || values.isEmpty()) { - break; - } - HRegionInfo info = RegionManager.getHRegionInfo(values.getRow(), values); - if (info == null) { - emptyRows.add(values.getRow()); - LOG.error(Bytes.toString(HConstants.CATALOG_FAMILY) + ":" - + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) - + " not found on " - + Bytes.toStringBinary(values.getRow())); - continue; - } - final String serverAddress = BaseScanner.getServerAddress(values); - String serverName = null; - if (serverAddress != null && serverAddress.length() > 0) { - long startCode = BaseScanner.getStartCode(values); - serverName = HServerInfo.getServerName(serverAddress, startCode); - } - if (Bytes.compareTo(info.getTableDesc().getName(), tableName) > 0) { - break; // Beyond any more entries for this table - } - - tableExists = true; - if (!isBeingServed(serverName) || !isEnabled(info)) { - unservedRegions.add(info); - } - processScanItem(serverName, info); - } - } finally { - if (scannerId != -1L) { - try { - this.server.close(scannerId); - } catch (IOException e) { - e = RemoteExceptionHandler.checkIOException(e); - LOG.error("closing scanner", e); - } - } - scannerId = -1L; - } - - // Get rid of any rows that have a null HRegionInfo - - if (emptyRows.size() > 0) { - LOG.warn("Found " + emptyRows.size() + - " rows with empty HRegionInfo while scanning meta region " + - Bytes.toString(m.getRegionName())); - RegionManager.deleteEmptyMetaRows(server, m.getRegionName(), emptyRows); - } - - if (!tableExists) { - throw new TableNotFoundException(Bytes.toString(tableName)); - } - - postProcessMeta(m, server); - unservedRegions.clear(); - return Boolean.TRUE; - } - } - - void process() throws IOException { - // Prevent meta scanner from running - synchronized(master.getRegionManager().metaScannerThread.scannerLock) { - for (MetaRegion m: metaRegions) { - new ProcessTableOperation(m, master).doWithRetries(); - } - } - } - - protected boolean isBeingServed(String serverName) { - boolean result = false; - if (serverName != null && serverName.length() > 0) { - HServerInfo s = master.getServerManager().getServerInfo(serverName); - result = s != null; - } - return result; - } - - protected boolean isEnabled(HRegionInfo info) { - return !info.isOffline(); - } - - protected abstract void processScanItem(String serverName, HRegionInfo info) - throws IOException; - - protected abstract void postProcessMeta(MetaRegion m, - HRegionInterface server) throws IOException; -} Index: src/main/java/org/apache/hadoop/hbase/master/TableDelete.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/TableDelete.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/TableDelete.java (working copy) @@ -1,78 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.TableNotDisabledException; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.util.Bytes; - -import java.io.IOException; - -/** - * Instantiated to delete a table. Table must be offline. - */ -class TableDelete extends TableOperation { - private final Log LOG = LogFactory.getLog(this.getClass()); - - TableDelete(final HMaster master, final byte [] tableName) throws IOException { - super(master, tableName); - } - - @Override - protected void processScanItem(String serverName, - final HRegionInfo info) throws IOException { - if (isEnabled(info)) { - LOG.debug("Region still enabled: " + info.toString()); - throw new TableNotDisabledException(tableName); - } - } - - @Override - protected void postProcessMeta(MetaRegion m, HRegionInterface server) - throws IOException { - for (HRegionInfo i: unservedRegions) { - if (!Bytes.equals(this.tableName, i.getTableDesc().getName())) { - // Don't delete regions that are not from our table. - continue; - } - // Delete the region - try { - HRegion.removeRegionFromMETA(server, m.getRegionName(), i.getRegionName()); - HRegion.deleteRegion(this.master.getFileSystemManager().getFileSystem(), - this.master.getFileSystemManager().getRootDir(), i); - - } catch (IOException e) { - LOG.error("failed to delete region " + Bytes.toString(i.getRegionName()), - RemoteExceptionHandler.checkIOException(e)); - } - } - - // delete the table's folder from fs. - this.master.getFileSystemManager().getFileSystem().delete( - new Path(this.master.getFileSystemManager().getRootDir(), - Bytes.toString(this.tableName)), true); - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/RetryableMetaOperation.java (working copy) @@ -1,113 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.TableNotDisabledException; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Sleeper; -import org.apache.hadoop.ipc.RemoteException; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; - -/** - * Uses Callable pattern so that operations against meta regions do not need - * to duplicate retry logic. - */ -abstract class RetryableMetaOperation implements Callable { - protected final Log LOG = LogFactory.getLog(this.getClass()); - protected final Sleeper sleeper; - protected final MetaRegion m; - protected final MasterStatus masterStatus; - protected static int numRetries; - protected static int threadWakeFrequency; - - protected HRegionInterface server; - - protected RetryableMetaOperation(MetaRegion m, MasterStatus masterStatus) { - this.m = m; - this.masterStatus = masterStatus; - threadWakeFrequency = - masterStatus.getConfiguration().getInt( - HConstants.THREAD_WAKE_FREQUENCY, - HConstants.DEFAULT_THREAD_WAKE_FREQUENCY); - this.sleeper = new Sleeper(threadWakeFrequency, this.masterStatus.getClosed()); - numRetries = - masterStatus.getConfiguration().getInt( - HConstants.NUM_CLIENT_RETRIES, - HConstants.DEFAULT_NUM_CLIENT_RETRIES); - } - - protected T doWithRetries() - throws IOException, RuntimeException { - List exceptions = new ArrayList(); - for (int tries = 0; tries < numRetries; tries++) { - if (this.masterStatus.isClosed()) { - return null; - } - try { - this.server = - this.masterStatus.getServerConnection().getHRegionConnection(m.getServer()); - return this.call(); - } catch (IOException e) { - if (e instanceof TableNotFoundException || - e instanceof TableNotDisabledException || - e instanceof InvalidColumnNameException) { - throw e; - } - if (e instanceof RemoteException) { - e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e); - } - if (tries == numRetries - 1) { - if (LOG.isDebugEnabled()) { - StringBuilder message = new StringBuilder( - "Trying to contact region server for regionName '" + - Bytes.toString(m.getRegionName()) + "', but failed after " + - (tries + 1) + " attempts.\n"); - int i = 1; - for (IOException e2 : exceptions) { - message.append("Exception " + i + ":\n" + e2); - } - LOG.debug(message); - } - this.masterStatus.getFileSystemManager().checkFileSystem(); - throw e; - } - if (LOG.isDebugEnabled()) { - exceptions.add(e); - } - } catch (Exception e) { - LOG.debug("Exception in RetryableMetaOperation: ", e); - throw new RuntimeException(e); - } - this.sleeper.sleep(); - } - return null; - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java (working copy) @@ -48,10 +48,10 @@ final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false); private final HServerAddress address; - private final MasterStatus status; + private final MasterController status; ActiveMasterManager(ZooKeeperWatcher watcher, HServerAddress address, - MasterStatus status) { + MasterController status) { super(watcher); this.address = address; this.status = status; Index: src/main/java/org/apache/hadoop/hbase/master/RegionServerOperation.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/RegionServerOperation.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/RegionServerOperation.java (working copy) @@ -1,123 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import java.io.IOException; -import java.util.concurrent.Delayed; -import java.util.concurrent.TimeUnit; - -abstract class RegionServerOperation implements Delayed { - protected static final Log LOG = - LogFactory.getLog(RegionServerOperation.class.getName()); - - private long expire; - protected final MasterStatus masterStatus; - /* How long we stay on queue. - */ - private int delay; - - protected RegionServerOperation(MasterStatus masterStatus) { - this.masterStatus = masterStatus; - this.delay = this.masterStatus.getConfiguration(). - getInt("hbase.server.thread.wakefrequency", 10 * 1000); - // Set the future time at which we expect to be released from the - // DelayQueue we're inserted in on lease expiration. - resetExpiration(); - } - - /** - * Call before putting this back on the delay queue. - * @return When we will expire next. - */ - long resetExpiration() { - // Set the future time at which we expect to be released from the - // DelayQueue we're inserted in on lease expiration. - this.expire = System.currentTimeMillis() + this.delay; - return this.expire; - } - - public long getDelay(TimeUnit unit) { - return unit.convert(this.expire - System.currentTimeMillis(), - TimeUnit.MILLISECONDS); - } - - void setDelay(final int d) { - this.delay = d; - } - - public int compareTo(Delayed o) { - return Long.valueOf(getDelay(TimeUnit.MILLISECONDS) - - o.getDelay(TimeUnit.MILLISECONDS)).intValue(); - } - - protected void requeue() { - masterStatus.getServerManager().getRegionServerOperationQueue().putOnDelayQueue(this); - } - - private long whenToExpire() { - return System.currentTimeMillis() + this.delay; - } - - protected boolean rootAvailable() { - boolean available = true; - if (this.masterStatus.getRegionManager().getRootRegionLocation() == null) { - available = false; - requeue(); - } - return available; - } - - protected boolean metaTableAvailable() { - boolean available = true; - if ((masterStatus.getRegionManager().numMetaRegions() != - masterStatus.getRegionManager().numOnlineMetaRegions()) || - masterStatus.getRegionManager().metaRegionsInTransition()) { - // We can't proceed because not all of the meta regions are online. - // We can't block either because that would prevent the meta region - // online message from being processed. In order to prevent spinning - // in the run queue, put this request on the delay queue to give - // other threads the opportunity to get the meta regions on-line. - if (LOG.isDebugEnabled()) { - LOG.debug("numberOfMetaRegions: " + - masterStatus.getRegionManager().numMetaRegions() + - ", onlineMetaRegions.size(): " + - masterStatus.getRegionManager().numOnlineMetaRegions()); - LOG.debug("Requeuing because not all meta regions are online"); - } - available = false; - requeue(); - } - return available; - } - - public int compareTo(RegionServerOperation other) { - return getPriority() - other.getPriority(); - } - - // the Priority of this operation, 0 is lowest priority - protected int getPriority() { - return Integer.MAX_VALUE; - } - - protected abstract boolean process() throws IOException; -} Index: src/main/java/org/apache/hadoop/hbase/master/RootScanner.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/RootScanner.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/RootScanner.java (working copy) @@ -1,81 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.RemoteExceptionHandler; - -import java.io.IOException; - -/** Scanner for the ROOT HRegion. */ -class RootScanner extends BaseScanner { - /** - * Constructor - * @param masterStatus - */ - public RootScanner(MasterStatus masterStatus) { - super(masterStatus, true, masterStatus.getShutdownRequested()); - } - - /** - * Don't retry if we get an error while scanning. Errors are most often - * - * caused by the server going away. Wait until next rescan interval when - * things should be back to normal. - * @return True if successfully scanned. - */ - private boolean scanRoot() { - masterStatus.getRegionManager().waitForRootRegionLocation(); - if (masterStatus.isClosed()) { - return false; - } - - try { - // Don't interrupt us while we're working - synchronized(scannerLock) { - if (masterStatus.getRegionManager().getRootRegionLocation() != null) { - scanRegion(new MetaRegion(masterStatus.getRegionManager().getRootRegionLocation(), - HRegionInfo.ROOT_REGIONINFO)); - } - } - } catch (IOException e) { - e = RemoteExceptionHandler.checkIOException(e); - LOG.warn("Scan ROOT region", e); - // Make sure the file system is still available - masterStatus.getFileSystemManager().checkFileSystem(); - } catch (Exception e) { - // If for some reason we get some other kind of exception, - // at least log it rather than go out silently. - LOG.error("Unexpected exception", e); - } - return true; - } - - @Override - protected boolean initialScan() { - this.initialScanComplete = scanRoot(); - return initialScanComplete; - } - - @Override - protected void maintenanceScan() { - scanRoot(); - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/master/handler/MasterOpenRegionHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/MasterOpenRegionHandler.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/handler/MasterOpenRegionHandler.java (working copy) @@ -1,111 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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.handler; - -import java.io.IOException; -import java.util.ArrayList; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.executor.RegionTransitionData; -import org.apache.hadoop.hbase.executor.HBaseEventHandler; -import org.apache.hadoop.hbase.master.ServerManager; -import org.apache.hadoop.hbase.util.Writables; - -/** - * This is the event handler for all events relating to opening regions on the - * HMaster. This could be one of the following: - * - notification that a region server is "OPENING" a region - * - notification that a region server has "OPENED" a region - * The following event types map to this handler: - * - RS_REGION_OPENING - * - RS_REGION_OPENED - */ -public class MasterOpenRegionHandler extends HBaseEventHandler { - private static final Log LOG = LogFactory.getLog(MasterOpenRegionHandler.class); - // other args passed in a byte array form - protected byte[] serializedData; - private String regionName; - private RegionTransitionData hbEventData; - ServerManager serverManager; - - public MasterOpenRegionHandler(HBaseEventType eventType, - ServerManager serverManager, - String serverName, - String regionName, - byte[] serData) { - super(false, serverName, eventType); - this.regionName = regionName; - this.serializedData = serData; - this.serverManager = serverManager; - } - - /** - * Handle the various events relating to opening regions. We can get the - * following events here: - * - RS_REGION_OPENING : Keep track to see how long the region open takes. - * If the RS is taking too long, then revert the - * region back to closed state so that it can be - * re-assigned. - * - RS_REGION_OPENED : The region is opened. Add an entry into META for - * the RS having opened this region. Then delete this - * entry in ZK. - */ - @Override - public void process() - { - LOG.debug("Event = " + getHBEvent() + ", region = " + regionName); - if(this.getHBEvent() == HBaseEventType.RS2ZK_REGION_OPENING) { - handleRegionOpeningEvent(); - } - else if(this.getHBEvent() == HBaseEventType.RS2ZK_REGION_OPENED) { - handleRegionOpenedEvent(); - } - } - - private void handleRegionOpeningEvent() { - // TODO: not implemented. - LOG.debug("NO-OP call to handling region opening event"); - // Keep track to see how long the region open takes. If the RS is taking too - // long, then revert the region back to closed state so that it can be - // re-assigned. - } - - private void handleRegionOpenedEvent() { - try { - if(hbEventData == null) { - hbEventData = new RegionTransitionData(); - Writables.getWritable(serializedData, hbEventData); - } - } catch (IOException e) { - LOG.error("Could not deserialize additional args for Open region", e); - } - LOG.debug("RS " + hbEventData.getServerName() + " has opened region " + regionName); - HServerInfo serverInfo = serverManager.getServerInfo(hbEventData.getServerName()); - ArrayList returnMsgs = new ArrayList(); - serverManager.processRegionOpen(serverInfo, hbEventData.getHmsg().getRegionInfo(), returnMsgs); - if(returnMsgs.size() > 0) { - LOG.error("Open region tried to send message: " + returnMsgs.get(0).getType() + - " about " + returnMsgs.get(0).getRegionInfo().getRegionNameAsString()); - } - } -} Index: src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java (revision 0) @@ -0,0 +1,68 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.InvalidFamilyOperationException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.master.FileSystemManager; +import org.apache.hadoop.hbase.master.MasterController; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Handles adding a new family to an existing table. + */ +public class TableModifyFamilyHandler extends TableEventHandler { + + private final HColumnDescriptor familyDesc; + + public TableModifyFamilyHandler(byte[] tableName, + HColumnDescriptor familyDesc, MasterController server, + CatalogTracker catalogTracker, FileSystemManager fileManager) { + super(EventType.C2M_ADD_FAMILY, tableName, server, + catalogTracker, fileManager); + this.familyDesc = familyDesc; + } + + @Override + protected void handleTableOperation(List regions) throws IOException { + HTableDescriptor htd = regions.get(0).getTableDesc(); + byte [] familyName = familyDesc.getName(); + if(htd.hasFamily(familyName)) { + throw new InvalidFamilyOperationException( + "Family '" + Bytes.toString(familyName) + "' already exists so " + + "cannot be modified"); + } + for(HRegionInfo region : regions) { + // Update the HTD + region.getTableDesc().addFamily(familyDesc); + // Update region in META + MetaEditor.updateRegionInfo(catalogTracker, region); + // Update region info in FS + fileManager.updateRegionInfo(region); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (revision 0) @@ -0,0 +1,37 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.executor.EventHandler; + + +public class ServerShutdownHandler extends EventHandler { + + public ServerShutdownHandler(ServerController server) { + super(server, EventType.M_SERVER_SHUTDOWN); + } + + @Override + public void process() { + // TODO: implement this + } + +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java (revision 0) @@ -0,0 +1,71 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.InvalidFamilyOperationException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.master.FileSystemManager; +import org.apache.hadoop.hbase.master.MasterController; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Handles adding a new family to an existing table. + */ +public class TableAddFamilyHandler extends TableEventHandler { + + private final HColumnDescriptor familyDesc; + + public TableAddFamilyHandler(byte[] tableName, HColumnDescriptor familyDesc, + MasterController server, CatalogTracker catalogTracker, + FileSystemManager fileManager) { + super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker, + fileManager); + this.familyDesc = familyDesc; + } + + @Override + protected void handleTableOperation(List regions) + throws IOException { + HTableDescriptor htd = regions.get(0).getTableDesc(); + byte [] familyName = familyDesc.getName(); + if(htd.hasFamily(familyName)) { + throw new InvalidFamilyOperationException( + "Family '" + Bytes.toString(familyName) + "' already exists so " + + "cannot be added"); + } + for(HRegionInfo region : regions) { + // Update the HTD + region.getTableDesc().addFamily(familyDesc); + // Update region in META + MetaEditor.updateRegionInfo(catalogTracker, region); + // Update region info in FS + fileManager.updateRegionInfo(region); + // Add directory to FS + fileManager.addFamily(region, familyName); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/MasterCloseRegionHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/MasterCloseRegionHandler.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/handler/MasterCloseRegionHandler.java (working copy) @@ -1,93 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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.handler; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.executor.RegionTransitionData; -import org.apache.hadoop.hbase.executor.HBaseEventHandler; -import org.apache.hadoop.hbase.master.ServerManager; -import org.apache.hadoop.hbase.util.Writables; - -/** - * This is the event handler for all events relating to closing regions on the - * HMaster. The following event types map to this handler: - * - RS_REGION_CLOSING - * - RS_REGION_CLOSED - */ -public class MasterCloseRegionHandler extends HBaseEventHandler -{ - private static final Log LOG = LogFactory.getLog(MasterCloseRegionHandler.class); - - private String regionName; - protected byte[] serializedData; - RegionTransitionData hbEventData; - ServerManager serverManager; - - public MasterCloseRegionHandler(HBaseEventType eventType, - ServerManager serverManager, - String serverName, - String regionName, - byte[] serializedData) { - super(false, serverName, eventType); - this.regionName = regionName; - this.serializedData = serializedData; - this.serverManager = serverManager; - } - - /** - * Handle the various events relating to closing regions. We can get the - * following events here: - * - RS_REGION_CLOSING : No-op - * - RS_REGION_CLOSED : The region is closed. If we are not in a shutdown - * state, find the RS to open this region. This could - * be a part of a region move, or just that the RS has - * died. Should result in a M_REQUEST_OPENREGION event - * getting created. - */ - @Override - public void process() - { - LOG.debug("Event = " + getHBEvent() + ", region = " + regionName); - // handle RS_REGION_CLOSED events - handleRegionClosedEvent(); - } - - private void handleRegionClosedEvent() { - try { - if(hbEventData == null) { - hbEventData = new RegionTransitionData(); - Writables.getWritable(serializedData, hbEventData); - } - } catch (IOException e) { - LOG.error("Could not deserialize additional args for Close region", e); - } - // process the region close - this will cause the reopening of the - // region as a part of the heartbeat of some RS - serverManager.processRegionClose(hbEventData.getHmsg().getRegionInfo()); - LOG.info("Processed close of region " + hbEventData.getHmsg().getRegionInfo().getRegionNameAsString()); - } - - public String getRegionName() { - return regionName; - } -} Index: src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java (revision 0) @@ -0,0 +1,92 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.TableNotDisabledException; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.master.FileSystemManager; +import org.apache.hadoop.hbase.master.MasterController; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Base class for performing operations against tables. + *

        + * Ensures all regions of the table are offline and then executes + * {@link #handleTableOperation(List)} with a list of regions of the + * table. + */ +public abstract class TableEventHandler extends EventHandler { + private static final Log LOG = LogFactory.getLog(TableEventHandler.class); + + protected final byte [] tableName; + protected final CatalogTracker catalogTracker; + protected final FileSystemManager fileManager; + + public TableEventHandler(EventType eventType, byte [] tableName, + MasterController server, CatalogTracker catalogTracker, + FileSystemManager fileManager) { + super(server, eventType); + this.tableName = tableName; + this.catalogTracker = catalogTracker; + this.fileManager = fileManager; + } + + @Override + public void process() { + try { + LOG.info("Handling table operation " + eventType + " on table " + + Bytes.toString(tableName)); + handleTableOperation(tableChecks()); + } catch (IOException e) { + LOG.error("Error trying to delete the table " + Bytes.toString(tableName), + e); + } + } + + private List tableChecks() throws IOException { + // Check if table exists + if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) { + throw new TableNotFoundException(Bytes.toString(tableName)); + } + // Get the regions of this table + // TODO: Use in-memory state of master? + List regions = MetaReader.getTableRegions(catalogTracker, + tableName); + // Verify all regions of table are disabled + for(HRegionInfo region : regions) { + if(!region.isOffline()) { + throw new TableNotDisabledException(tableName); + } + } + return regions; + } + + protected abstract void handleTableOperation(List regions) + throws IOException; +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java (revision 0) @@ -0,0 +1,55 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.master.FileSystemManager; +import org.apache.hadoop.hbase.master.MasterController; + +public class DeleteTableHandler extends TableEventHandler { + private static final Log LOG = LogFactory.getLog(DeleteTableHandler.class); + + public DeleteTableHandler(byte [] tableName, MasterController server, + CatalogTracker catalogTracker, FileSystemManager fileManager) { + super(EventType.C2M_DELETE_TABLE, tableName, server, catalogTracker, + fileManager); + } + + @Override + protected void handleTableOperation(List regions) + throws IOException { + for(HRegionInfo region : regions) { + LOG.debug("Deleting region " + region + " from META and FS"); + // Remove region from META + MetaEditor.deleteRegion(catalogTracker, region); + // Delete region from FS + fileManager.deleteRegion(region); + } + // Delete table from FS + fileManager.deleteTable(tableName); + } +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java (revision 0) @@ -0,0 +1,104 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.master.AssignmentManager; +import org.apache.hadoop.hbase.util.Bytes; + + +public class DisableTableHandler extends EventHandler { + private static final Log LOG = LogFactory.getLog(DisableTableHandler.class); + + private final byte [] tableName; + private final CatalogTracker catalogTracker; + private final AssignmentManager assignmentManager; + + public DisableTableHandler(ServerController server, byte [] tableName, + CatalogTracker catalogTracker, AssignmentManager assignmentManager) { + super(server, EventType.C2M_DISABLE_TABLE); + this.tableName = tableName; + this.catalogTracker = catalogTracker; + this.assignmentManager = assignmentManager; + } + + @Override + public void process() { + try { + LOG.info("Attemping to disable the table " + Bytes.toString(tableName)); + handleDisableTable(); + } catch (IOException e) { + LOG.error("Error trying to disable the table " + Bytes.toString(tableName), + e); + } + } + + // Meta scan + @SuppressWarnings("unused") + private void oldDisableTable() throws IOException { + // Check if table exists + if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) { + throw new TableNotFoundException(Bytes.toString(tableName)); + } + // Set the table as disabled so it doesn't get re-onlined + assignmentManager.disableTable(Bytes.toString(tableName)); + // Get the regions of this table + // TODO: should we use in-memory state? need to deal with concurrent splits + List regions = MetaReader.getTableRegions(catalogTracker, + tableName); + // Verify all regions of table are offline + for(HRegionInfo region : regions) { + if(region.isOffline()) { + continue; + } + // Unassign any regions still online + assignmentManager.unassign(region); + } + } + + // In-memory scan + private void handleDisableTable() throws IOException { + // Check if table exists + // TODO: do we want to keep this in-memory as well? i guess this is + // part of old master rewrite, schema to zk to check for table + // existence and such + if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) { + throw new TableNotFoundException(Bytes.toString(tableName)); + } + // Set the table as disabled so it doesn't get re-onlined + assignmentManager.disableTable(Bytes.toString(tableName)); + // Get the online regions of this table + List regions = assignmentManager.getRegionsOfTable(tableName); + // Unassign the online regions + for(HRegionInfo region : regions) { + assignmentManager.unassign(region); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java (revision 0) @@ -0,0 +1,68 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.InvalidFamilyOperationException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.master.FileSystemManager; +import org.apache.hadoop.hbase.master.MasterController; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Handles adding a new family to an existing table. + */ +public class TableDeleteFamilyHandler extends TableEventHandler { + + private final byte [] familyName; + + public TableDeleteFamilyHandler(byte[] tableName, byte [] familyName, + MasterController server, CatalogTracker catalogTracker, + FileSystemManager fileManager) { + super(EventType.C2M_ADD_FAMILY, tableName, server, catalogTracker, + fileManager); + this.familyName = familyName; + } + + @Override + protected void handleTableOperation(List regions) throws IOException { + HTableDescriptor htd = regions.get(0).getTableDesc(); + if(!htd.hasFamily(familyName)) { + throw new InvalidFamilyOperationException( + "Family '" + Bytes.toString(familyName) + "' does not exist so " + + "cannot be deleted"); + } + for(HRegionInfo region : regions) { + // Update the HTD + region.getTableDesc().removeFamily(familyName); + // Update region in META + MetaEditor.updateRegionInfo(catalogTracker, region); + // Update region info in FS + fileManager.updateRegionInfo(region); + // Delete directory in FS + fileManager.deleteFamily(region, familyName); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java (revision 0) @@ -0,0 +1,110 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.executor.RegionTransitionData; +import org.apache.hadoop.hbase.master.AssignmentManager; +import org.apache.hadoop.hbase.zookeeper.ZKAssign; +import org.apache.zookeeper.KeeperException; + +/** + * Handles CLOSED region event on Master. + *

        + * If table is being disabled, deletes ZK unassigned node and removes from + * regions in transition. + *

        + * Otherwise, assigns the region to another server. + */ +public class ClosedRegionHandler extends EventHandler { + private static final Log LOG = LogFactory.getLog(ClosedRegionHandler.class); + + private final AssignmentManager assignmentManager; + private final RegionTransitionData data; + private final HRegionInfo regionInfo; + private final ClosedPriority priority; + + private enum ClosedPriority { + ROOT (1), + META (2), + USER (3); + + private final int value; + ClosedPriority(int value) { + this.value = value; + } + public int getValue() { + return value; + } + }; + + public ClosedRegionHandler(ServerController server, + AssignmentManager assignmentManager, RegionTransitionData data, + HRegionInfo regionInfo) { + super(server, EventType.RS2ZK_REGION_CLOSED); + this.assignmentManager = assignmentManager; + this.data = data; + this.regionInfo = regionInfo; + if(regionInfo.isRootRegion()) { + priority = ClosedPriority.ROOT; + } else if(regionInfo.isMetaRegion()) { + priority = ClosedPriority.META; + } else { + priority = ClosedPriority.USER; + } + } + + @Override + public int getPriority() { + return priority.getValue(); + } + + @Override + public void process() { + LOG.debug("Handling CLOSED event with data: " + data); + // Check if this table is being disabled or not + if(assignmentManager.isTableOfRegionDisabled(regionInfo.getRegionName())) { + // Disabling so should not be reassigned, just delete the CLOSED node + LOG.debug("Table being disabled so deleting ZK node and removing from " + + "regions in transition, skipping assignment"); + try { + ZKAssign.deleteClosedNode(server.getZooKeeper(), + regionInfo.getEncodedName()); + } catch (KeeperException.NoNodeException nne) { + LOG.warn("Tried to delete closed node for " + data + " but it does " + + "not exist"); + return; + } catch (KeeperException e) { + LOG.fatal("Error deleting CLOSED node in ZK", e); + server.abort(); + } + assignmentManager.regionOffline(regionInfo); + return; + } + // ZK Node is in CLOSED state, assign it (transition to OFFLINE done here) + assignmentManager.setOffline(regionInfo); + assignmentManager.assign(regionInfo); + } + +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java (revision 0) @@ -0,0 +1,97 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerInfo; +import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.executor.RegionTransitionData; +import org.apache.hadoop.hbase.master.AssignmentManager; +import org.apache.hadoop.hbase.zookeeper.ZKAssign; +import org.apache.zookeeper.KeeperException; + +/** + * Handles OPENED region event on Master. + */ +public class OpenedRegionHandler extends EventHandler { + private static final Log LOG = LogFactory.getLog(OpenedRegionHandler.class); + + private final AssignmentManager assignmentManager; + private final RegionTransitionData data; + private final HRegionInfo regionInfo; + private final HServerInfo serverInfo; + private final OpenedPriority priority; + + private enum OpenedPriority { + ROOT (1), + META (2), + USER (3); + + private final int value; + OpenedPriority(int value) { + this.value = value; + } + public int getValue() { + return value; + } + }; + + public OpenedRegionHandler(ServerController server, + AssignmentManager assignmentManager, RegionTransitionData data, + HRegionInfo regionInfo, HServerInfo serverInfo) { + super(server, EventType.RS2ZK_REGION_OPENED); + this.assignmentManager = assignmentManager; + this.data = data; + this.regionInfo = regionInfo; + this.serverInfo = serverInfo; + if(regionInfo.isRootRegion()) { + priority = OpenedPriority.ROOT; + } else if(regionInfo.isMetaRegion()) { + priority = OpenedPriority.META; + } else { + priority = OpenedPriority.USER; + } + } + + @Override + public int getPriority() { + return priority.getValue(); + } + + @Override + public void process() { + LOG.debug("Handling OPENED event with data: " + data); + // TODO: should we check if this table was disabled and get it closed? + // Remove region from in-memory transition and unassigned node from ZK + try { + ZKAssign.deleteOpenedNode(server.getZooKeeper(), + regionInfo.getEncodedName()); + } catch (KeeperException e) { + LOG.fatal("Error deleting OPENED node in ZK", e); + server.abort(); + } + assignmentManager.regionOnline(regionInfo, serverInfo); + LOG.debug("Opened region " + regionInfo); + } + +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (revision 0) @@ -0,0 +1,79 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.master.AssignmentManager; +import org.apache.hadoop.hbase.util.Bytes; + + +public class EnableTableHandler extends EventHandler { + private static final Log LOG = LogFactory.getLog(EnableTableHandler.class); + + private final byte [] tableName; + private final CatalogTracker catalogTracker; + private final AssignmentManager assignmentManager; + + public EnableTableHandler(ServerController server, byte [] tableName, + CatalogTracker catalogTracker, AssignmentManager assignmentManager) { + super(server, EventType.C2M_ENABLE_TABLE); + this.tableName = tableName; + this.catalogTracker = catalogTracker; + this.assignmentManager = assignmentManager; + } + + @Override + public void process() { + try { + LOG.info("Attemping to enable the table " + Bytes.toString(tableName)); + handleEnableTable(); + } catch (IOException e) { + LOG.error("Error trying to enable the table " + Bytes.toString(tableName), + e); + } + } + + private void handleEnableTable() throws IOException { + // Check if table exists + if(!MetaReader.tableExists(catalogTracker, Bytes.toString(tableName))) { + throw new TableNotFoundException(Bytes.toString(tableName)); + } + // Get the regions of this table + List regions = MetaReader.getTableRegions(catalogTracker, + tableName); + // Verify all regions of table are disabled + for(HRegionInfo region : regions) { + if(!region.isOffline()) { + continue; + } + assignmentManager.assign(region); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java (revision 0) @@ -0,0 +1,52 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.handler; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.master.FileSystemManager; +import org.apache.hadoop.hbase.master.MasterController; + +public class ModifyTableHandler extends TableEventHandler { + private static final Log LOG = LogFactory.getLog(ModifyTableHandler.class); + + public ModifyTableHandler(byte [] tableName, MasterController server, + CatalogTracker catalogTracker, FileSystemManager fileManager) { + super(EventType.C2M_MODIFY_TABLE, tableName, server, catalogTracker, + fileManager); + } + + @Override + protected void handleTableOperation(List regions) + throws IOException { + for(HRegionInfo region : regions) { + // Update region info in META + MetaEditor.updateRegionInfo(catalogTracker, region); + // Update region info in FS + fileManager.updateRegionInfo(region); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/master/ChangeTableState.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ChangeTableState.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ChangeTableState.java (working copy) @@ -1,162 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Writables; - -import java.io.IOException; -import java.util.HashSet; -import java.util.Map; -import java.util.TreeMap; - -/** - * Instantiated to enable or disable a table - */ -class ChangeTableState extends TableOperation { - private final Log LOG = LogFactory.getLog(this.getClass()); - private boolean online; - // Do in order. - protected final Map> servedRegions = - new TreeMap>(); - protected long lockid; - - ChangeTableState(final HMaster master, final byte [] tableName, - final boolean onLine) - throws IOException { - super(master, tableName); - this.online = onLine; - } - - @Override - protected void processScanItem(String serverName, HRegionInfo info) { - if (isBeingServed(serverName)) { - HashSet regions = this.servedRegions.get(serverName); - if (regions == null) { - regions = new HashSet(); - } - regions.add(info); - this.servedRegions.put(serverName, regions); - } - } - - @Override - protected void postProcessMeta(MetaRegion m, HRegionInterface server) - throws IOException { - // Process regions not being served - if (LOG.isDebugEnabled()) { - LOG.debug("Processing unserved regions"); - } - for (HRegionInfo i: this.unservedRegions) { - if (i.isOffline() && i.isSplit()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skipping region " + i.toString() + - " because it is offline and split"); - } - continue; - } - - if(!this.online && this.master.getRegionManager(). - isPendingOpen(i.getRegionNameAsString())) { - LOG.debug("Skipping region " + i.toString() + - " because it is pending open, will tell it to close later"); - continue; - } - - // If it's already offline then don't set it a second/third time, skip - // Same for online, don't set again if already online - if (!(i.isOffline() && !online) && !(!i.isOffline() && online)) { - // Update meta table - Put put = updateRegionInfo(i); - server.put(m.getRegionName(), put); - Delete delete = new Delete(i.getRegionName()); - delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); - delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER); - server.delete(m.getRegionName(), delete); - } - if (LOG.isDebugEnabled()) { - LOG.debug("Removed server and startcode from row and set online=" + - this.online + ": " + i.getRegionNameAsString()); - } - synchronized (master.getRegionManager()) { - if (this.online) { - // Bring offline regions on-line - if (!this.master.getRegionManager().regionIsOpening(i.getRegionNameAsString())) { - this.master.getRegionManager().setUnassigned(i, false); - } - } else { - // Prevent region from getting assigned. - this.master.getRegionManager().removeRegion(i); - } - } - } - - // Process regions currently being served - if (LOG.isDebugEnabled()) { - LOG.debug("Processing regions currently being served"); - } - synchronized (this.master.getRegionManager()) { - for (Map.Entry> e: - this.servedRegions.entrySet()) { - String serverName = e.getKey(); - if (this.online) { - LOG.debug("Already online"); - continue; // Already being served - } - - // Cause regions being served to be taken off-line and disabled - for (HRegionInfo i: e.getValue()) { - // The scan we did could be totally staled, get the freshest data - Get get = new Get(i.getRegionName()); - get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); - Result values = server.get(m.getRegionName(), get); - String serverAddress = BaseScanner.getServerAddress(values); - // If this region is unassigned, skip! - if(serverAddress.length() == 0) { - continue; - } - if (LOG.isDebugEnabled()) { - LOG.debug("Adding region " + i.getRegionNameAsString() + - " to setClosing list"); - } - // this marks the regions to be closed - this.master.getRegionManager().setClosing(serverName, i, true); - } - } - } - this.servedRegions.clear(); - } - - protected Put updateRegionInfo(final HRegionInfo i) - throws IOException { - i.setOffline(!online); - Put put = new Put(i.getRegionName()); - put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, Writables.getBytes(i)); - return put; - } -} Index: src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java (working copy) @@ -1,380 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.wal.HLog; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.master.RegionManager.RegionState; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * Instantiated when a server's lease has expired, meaning it has crashed. - * The region server's log file needs to be split up for each region it was - * serving, and the regions need to get reassigned. - */ -class ProcessServerShutdown extends RegionServerOperation { - // Server name made of the concatenation of hostname, port and startcode - // formatted as <hostname> ',' <port> ',' <startcode> - private final String deadServer; - private boolean isRootServer; - private List metaRegions; - - private Path rsLogDir; - private boolean logSplit; - private boolean rootRescanned; - private HServerAddress deadServerAddress; - - private static class ToDoEntry { - boolean regionOffline; - final HRegionInfo info; - - ToDoEntry(final HRegionInfo info) { - this.regionOffline = false; - this.info = info; - } - } - - /** - * @param masterStatus - * @param serverInfo - */ - public ProcessServerShutdown(MasterStatus masterStatus, HServerInfo serverInfo) { - super(masterStatus); - this.deadServer = serverInfo.getServerName(); - this.deadServerAddress = serverInfo.getServerAddress(); - this.logSplit = false; - this.rootRescanned = false; - this.rsLogDir = - new Path(masterStatus.getFileSystemManager().getRootDir(), HLog.getHLogDirectoryName(serverInfo)); - - // check to see if I am responsible for either ROOT or any of the META tables. - - // TODO Why do we do this now instead of at processing time? - closeMetaRegions(); - } - - private void closeMetaRegions() { - this.isRootServer = - this.masterStatus.getRegionManager().isRootServer(this.deadServerAddress) || - this.masterStatus.getRegionManager().isRootInTransitionOnThisServer(deadServer); - if (this.isRootServer) { - this.masterStatus.getRegionManager().unsetRootRegion(); - } - List metaStarts = - this.masterStatus.getRegionManager().listMetaRegionsForServer(deadServerAddress); - - this.metaRegions = new ArrayList(); - for (byte [] startKey: metaStarts) { - MetaRegion r = masterStatus.getRegionManager().offlineMetaRegionWithStartKey(startKey); - this.metaRegions.add(r); - } - - //HBASE-1928: Check whether this server has been transitioning the META table - HRegionInfo metaServerRegionInfo = masterStatus.getRegionManager().getMetaServerRegionInfo (deadServer); - if (metaServerRegionInfo != null) { - metaRegions.add (new MetaRegion (deadServerAddress, metaServerRegionInfo)); - } - } - - /** - * @return Name of server we are processing. - */ - public HServerAddress getDeadServerAddress() { - return this.deadServerAddress; - } - - private void closeRegionsInTransition() { - Map inTransition = - masterStatus.getRegionManager().getRegionsInTransitionOnServer(deadServer); - for (Map.Entry entry : inTransition.entrySet()) { - String regionName = entry.getKey(); - RegionState state = entry.getValue(); - - LOG.info("Region " + regionName + " was in transition " + - state + " on dead server " + deadServer + " - marking unassigned"); - masterStatus.getRegionManager().setUnassigned(state.getRegionInfo(), true); - } - } - - @Override - public String toString() { - return "ProcessServerShutdown of " + this.deadServer; - } - - /** Finds regions that the dead region server was serving - */ - protected void scanMetaRegion(HRegionInterface server, long scannerId, - byte [] regionName) - throws IOException { - List toDoList = new ArrayList(); - Set regions = new HashSet(); - List emptyRows = new ArrayList(); - try { - while (true) { - Result values = null; - try { - values = server.next(scannerId); - } catch (IOException e) { - LOG.error("Shutdown scanning of meta region", - RemoteExceptionHandler.checkIOException(e)); - break; - } - if (values == null || values.size() == 0) { - break; - } - byte [] row = values.getRow(); - // Check server name. If null, skip (We used to consider it was on - // shutdown server but that would mean that we'd reassign regions that - // were already out being assigned, ones that were product of a split - // that happened while the shutdown was being processed). - String serverAddress = BaseScanner.getServerAddress(values); - long startCode = BaseScanner.getStartCode(values); - - String serverName = null; - if (serverAddress != null && serverAddress.length() > 0) { - serverName = HServerInfo.getServerName(serverAddress, startCode); - } - if (serverName == null || !deadServer.equals(serverName)) { - // This isn't the server you're looking for - move along - continue; - } - - if (LOG.isDebugEnabled() && row != null) { - LOG.debug("Shutdown scanner for " + serverName + " processing " + - Bytes.toString(row)); - } - - HRegionInfo info = RegionManager.getHRegionInfo(row, values); - if (info == null) { - emptyRows.add(row); - continue; - } - - synchronized (masterStatus.getRegionManager()) { - if (info.isMetaTable()) { - if (LOG.isDebugEnabled()) { - LOG.debug("removing meta region " + - Bytes.toString(info.getRegionName()) + - " from online meta regions"); - } - masterStatus.getRegionManager().offlineMetaRegionWithStartKey(info.getStartKey()); - } - - ToDoEntry todo = new ToDoEntry(info); - toDoList.add(todo); - - if (masterStatus.getRegionManager().isOfflined(info.getRegionNameAsString()) || - info.isOffline()) { - masterStatus.getRegionManager().removeRegion(info); - // Mark region offline - if (!info.isOffline()) { - todo.regionOffline = true; - } - } else { - if (!info.isOffline() && !info.isSplit()) { - // Get region reassigned - regions.add(info); - } - } - } - } - } finally { - if (scannerId != -1L) { - try { - server.close(scannerId); - } catch (IOException e) { - LOG.error("Closing scanner", - RemoteExceptionHandler.checkIOException(e)); - } - } - } - - // Scan complete. Remove any rows which had empty HRegionInfos - - if (emptyRows.size() > 0) { - LOG.warn("Found " + emptyRows.size() + - " rows with empty HRegionInfo while scanning meta region " + - Bytes.toString(regionName)); - RegionManager.deleteEmptyMetaRows(server, regionName, emptyRows); - } - // Update server in root/meta entries - for (ToDoEntry e: toDoList) { - if (e.regionOffline) { - HRegion.offlineRegionInMETA(server, regionName, e.info); - } - } - - // Get regions reassigned - for (HRegionInfo info: regions) { - masterStatus.getRegionManager().setUnassigned(info, true); - } - } - - private class ScanRootRegion extends RetryableMetaOperation { - ScanRootRegion(MetaRegion m, MasterStatus masterStatus) { - super(m, masterStatus); - } - - public Boolean call() throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("Process server shutdown scanning root region on " + - masterStatus.getRegionManager().getRootRegionLocation().getBindAddress()); - } - Scan scan = new Scan(); - scan.addFamily(HConstants.CATALOG_FAMILY); - long scannerId = server.openScanner( - HRegionInfo.ROOT_REGIONINFO.getRegionName(), scan); - scanMetaRegion(server, scannerId, - HRegionInfo.ROOT_REGIONINFO.getRegionName()); - return true; - } - } - - private class ScanMetaRegions extends RetryableMetaOperation { - ScanMetaRegions(MetaRegion m, MasterStatus masterStatus) { - super(m, masterStatus); - } - - public Boolean call() throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("process server shutdown scanning " + - Bytes.toString(m.getRegionName()) + " on " + m.getServer()); - } - Scan scan = new Scan(); - scan.addFamily(HConstants.CATALOG_FAMILY); - long scannerId = server.openScanner( - m.getRegionName(), scan); - scanMetaRegion(server, scannerId, m.getRegionName()); - return true; - } - } - - @Override - protected boolean process() throws IOException { - LOG.info("Process shutdown of server " + this.deadServer + - ": logSplit: " + logSplit + ", rootRescanned: " + rootRescanned + - ", numberOfMetaRegions: " + masterStatus.getRegionManager().numMetaRegions() + - ", onlineMetaRegions.size(): " + - masterStatus.getRegionManager().numOnlineMetaRegions()); - if (!logSplit) { - // Process the old log file - if (this.masterStatus.getFileSystemManager().getFileSystem().exists(rsLogDir)) { - if (!masterStatus.getFileSystemManager().getSplitLogLock().tryLock()) { - return false; - } - try { - HLog.splitLog(masterStatus.getFileSystemManager().getRootDir(), rsLogDir, - this.masterStatus.getFileSystemManager().getOldLogDir(), - this.masterStatus.getFileSystemManager().getFileSystem(), - this.masterStatus.getConfiguration()); - } finally { - masterStatus.getFileSystemManager().getSplitLogLock().unlock(); - } - } - logSplit = true; - } - LOG.info("Log split complete, meta reassignment and scanning:"); - if (this.isRootServer) { - LOG.info("ProcessServerShutdown reassigning ROOT region"); - masterStatus.getRegionManager().reassignRootRegion(); - isRootServer = false; // prevent double reassignment... heh. - } - - for (MetaRegion metaRegion : metaRegions) { - LOG.info("ProcessServerShutdown setting to unassigned: " + metaRegion.toString()); - masterStatus.getRegionManager().setUnassigned(metaRegion.getRegionInfo(), true); - } - // one the meta regions are online, "forget" about them. Since there are explicit - // checks below to make sure meta/root are online, this is likely to occur. - metaRegions.clear(); - - if (!rootAvailable()) { - // Return true so that worker does not put this request back on the - // toDoQueue. - // rootAvailable() has already put it on the delayedToDoQueue - return true; - } - - if (!rootRescanned) { - // Scan the ROOT region - Boolean result = new ScanRootRegion( - new MetaRegion(masterStatus.getRegionManager().getRootRegionLocation(), - HRegionInfo.ROOT_REGIONINFO), this.masterStatus).doWithRetries(); - if (result == null) { - // Master is closing - give up - return true; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Process server shutdown scanning root region on " + - masterStatus.getRegionManager().getRootRegionLocation().getBindAddress() + - " finished " + Thread.currentThread().getName()); - } - rootRescanned = true; - } - - if (!metaTableAvailable()) { - // We can't proceed because not all meta regions are online. - // metaAvailable() has put this request on the delayedToDoQueue - // Return true so that worker does not put this on the toDoQueue - return true; - } - - List regions = masterStatus.getRegionManager().getListOfOnlineMetaRegions(); - for (MetaRegion r: regions) { - Boolean result = new ScanMetaRegions(r, this.masterStatus).doWithRetries(); - if (result == null) { - break; - } - if (LOG.isDebugEnabled()) { - LOG.debug("process server shutdown finished scanning " + - Bytes.toString(r.getRegionName()) + " on " + r.getServer()); - } - } - - closeRegionsInTransition(); - this.masterStatus.getServerManager().removeDeadServer(deadServer); - if (LOG.isDebugEnabled()) { - LOG.debug("Removed " + deadServer + " from deadservers Map"); - } - return true; - } - - @Override - protected int getPriority() { - return 2; // high but not highest priority - } -} Index: src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (working copy) @@ -19,43 +19,93 @@ */ package org.apache.hadoop.hbase.master; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Chore; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.HServerInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.executor.RegionTransitionData; -import org.apache.hadoop.hbase.master.handler.MasterCloseRegionHandler; -import org.apache.hadoop.hbase.master.handler.MasterOpenRegionHandler; +import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan; +import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler; +import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.ZKAssign; +import org.apache.hadoop.hbase.zookeeper.ZKTableDisable; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData; +import org.apache.hadoop.io.Writable; import org.apache.zookeeper.KeeperException; /** - * Manages region assignment. - * - *

        Monitors ZooKeeper for events related to regions in transition. - * - *

        Handles existing regions in transition during master failover. + * Manages and performs region assignment. + *

        + * Monitors ZooKeeper for events related to regions in transition. + *

        + * Handles existing regions in transition during master failover. */ public class AssignmentManager extends ZooKeeperListener { private static final Log LOG = LogFactory.getLog(AssignmentManager.class); - private MasterStatus status; + protected MasterController master; private ServerManager serverManager; - private RegionManager regionManager; + private CatalogTracker catalogTracker; - private String serverName; + private TimeoutMonitor timeoutMonitor; -// TODO: Eventually RIT will move here? -// private final Map regionsInTransition = -// new TreeMap(); + /** Regions currently in transition. */ + private final Map regionsInTransition = + new TreeMap(); + /** Plans for region movement. */ + private final Map regionPlans = + new TreeMap(); + + /** Set of tables that have been disabled. */ + private final Set disabledTables = + Collections.synchronizedSet(new HashSet()); + /** + * Server to regions assignment map. + * Contains the set of regions currently assigned to a given server. + */ + private final SortedMap> servers = + new TreeMap>(); + + /** + * Region to server assignment map. + * Contains the server a given region is currently assigned to. + * This object should be used for all synchronization around servers/regions. + */ + private final SortedMap regions = + new TreeMap(); + + /** * Constructs a new assignment manager. * *

        This manager must be started with {@link #start()}. @@ -63,88 +113,214 @@ * @param watcher zookeeper watcher * @param status master status */ - public AssignmentManager(ZooKeeperWatcher watcher, MasterStatus status, - ServerManager serverManager, RegionManager regionManager) { + public AssignmentManager(ZooKeeperWatcher watcher, MasterController master, + ServerManager serverManager, CatalogTracker catalogTracker) { super(watcher); - this.status = status; + this.master = master; this.serverManager = serverManager; - this.regionManager = regionManager; - serverName = status.getHServerAddress().toString(); + this.catalogTracker = catalogTracker; + Configuration conf = master.getConfiguration(); + this.timeoutMonitor = new TimeoutMonitor( + conf.getInt("hbase.master.assignment.timeoutmonitor.period", 30000), + master.getClosed(), + conf.getInt("hbase.master.assignment.timeoutmonitor.timeout", 15000)); + Threads.setDaemonThreadRunning(timeoutMonitor, + master.getServerName() + ".timeoutMonitor"); } /** - * Starts the assignment manager. - * - *

        This includes registering itself with ZooKeeper and handling - * the initial state of whatever unassigned nodes already exist. + * Cluster startup. Reset all unassigned nodes and assign all user regions. + * @throws IOException * @throws KeeperException */ - public void start() throws KeeperException { - watcher.registerListener(this); - if(status.isClusterStartup()) { - processStartup(); - } else { - processFailover(); - } - } - - public synchronized void processStartup() throws KeeperException { + void processStartup() throws IOException, KeeperException { + // Cleanup any existing ZK nodes and start watching ZKAssign.deleteAllNodes(watcher); - ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.assignmentZNode); + ZKUtil.listChildrenAndWatchForNewChildren(watcher, + watcher.assignmentZNode); + // Assign all existing user regions out + assignAllUserRegions(); } /** - * Handle failover. + * Handle failover. Restore state from META and ZK. Handle any regions in + * transition. * @throws KeeperException + * @throws IOException */ - public synchronized void processFailover() throws KeeperException { + void processFailover() throws KeeperException, IOException { + // Scan META to build list of existing regions, servers, and assignment + rebuildUserRegions(); + // Pickup any disabled tables + rebuildDisabledTables(); + // Check existing regions in transition List nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.assignmentZNode); if(nodes.isEmpty()) { - LOG.info("No regions in transition in ZK, nothing to do for failover"); + LOG.info("No regions in transition in ZK to process on failover"); return; } LOG.info("Failed-over master needs to process " + nodes.size() + " regions in transition"); for(String regionName : nodes) { RegionTransitionData data = ZKAssign.getData(watcher, regionName); + HRegionInfo regionInfo = + MetaReader.getRegion(catalogTracker, data.getRegionName()).getFirst(); + String encodedName = regionInfo.getEncodedName(); switch(data.getEventType()) { - case M2ZK_REGION_OFFLINE: - // TODO: Generate new assignment and send OPEN RPC - break; case RS2ZK_REGION_CLOSING: - // TODO: Only need to deal with timeouts. + // Just insert region into RIT + // If this never updates the timeout will trigger new assignment + regionsInTransition.put(encodedName, + new RegionState(regionInfo, RegionState.State.CLOSING, + data.getStamp())); break; + case RS2ZK_REGION_CLOSED: - // TODO: Generate new assignment and send OPEN RPC + // Region is closed, insert into RIT and handle it + regionsInTransition.put(encodedName, + new RegionState(regionInfo, RegionState.State.CLOSED, + data.getStamp())); + new ClosedRegionHandler(master, this, data, regionInfo).execute(); break; + case RS2ZK_REGION_OPENING: - // TODO: Only need to deal with timeouts. + // Just insert region into RIT + // If this never updates the timeout will trigger new assignment + regionsInTransition.put(encodedName, + new RegionState(regionInfo, RegionState.State.OPENING, + data.getStamp())); break; + case RS2ZK_REGION_OPENED: - // TODO: Delete the node from ZK. Region successfully opened but not - // acknowledged. + // Region is opened, insert into RIT and handle it + regionsInTransition.put(encodedName, + new RegionState(regionInfo, RegionState.State.OPENING, + data.getStamp())); + new OpenedRegionHandler(master, this, data, regionInfo, + serverManager.getServerInfo(data.getServerName())).execute(); break; } } } - private synchronized void handleRegion(RegionTransitionData data) { - switch(data.getEventType()) { - case RS2ZK_REGION_CLOSED: - new MasterCloseRegionHandler(data.getEventType(), serverManager, - serverName, data.getRegionName(), data.getBytes()) - .submit(); - break; - case RS2ZK_REGION_OPENED: - case RS2ZK_REGION_OPENING: - new MasterOpenRegionHandler(data.getEventType(), serverManager, - serverName, data.getRegionName(), data.getBytes()) - .submit(); - break; + /** + * Gets the region info for the region with the specified encoded name. + *

        + * Currently this does a full scan of the regions map looking for a region + * with the specified encoded name. + *

        + * Returns null if none found. + * @param regionName + * @return + * @deprecated should be able to remove this now? + */ + @Deprecated + private HRegionInfo getRegionInfoFromEncoded(String encodedName) { + for(HRegionInfo regionInfo : regions.keySet()) { + if(regionInfo.getEncodedName().equals(encodedName)) { + return regionInfo; + } } + return null; } + /** + * Handles various states an unassigned node can be in. + *

        + * Method is called when a state change is suspected for an unassigned node. + *

        + * This deals with skipped transitions (we got a CLOSED but didn't see CLOSING + * yet). + * @param data + */ + private void handleRegion(RegionTransitionData data) { + synchronized(regionsInTransition) { + // Verify this is a known server + if(!serverManager.isServerOnline(data.getServerName())) { + LOG.warn("Attempted to handle region transition for server " + + data.getServerName() + " but server is not online"); + return; + } + String encodedName = HRegionInfo.encodeRegionName(data.getRegionName()); + LOG.debug("Attempting to handle region transition for server " + + data.getServerName() + " and region " + encodedName); + RegionState regionState = regionsInTransition.get(encodedName); + switch(data.getEventType()) { + + case RS2ZK_REGION_CLOSING: + // Should see CLOSING after we have asked it to CLOSE or additional + // times after already being in state of CLOSING + if(regionState == null || + (!regionState.isPendingClose() && !regionState.isClosing())) { + LOG.warn("Received CLOSING for region " + encodedName + + " from server " + data.getServerName() + " but region was in " + + " the state " + regionState + " and not " + + "in expected PENDING_CLOSE or CLOSING states"); + return; + } + // Transition to CLOSING (or update stamp if already CLOSING) + regionState.update(RegionState.State.CLOSING, data.getStamp()); + break; + + case RS2ZK_REGION_CLOSED: + // Should see CLOSED after CLOSING but possible after PENDING_CLOSE + if(regionState == null || + (!regionState.isPendingClose() && !regionState.isClosing())) { + LOG.warn("Received CLOSED for region " + encodedName + + " from server " + data.getServerName() + " but region was in " + + " the state " + regionState + " and not " + + "in expected PENDING_CLOSE or CLOSING states"); + return; + } + // Handle CLOSED by assigning elsewhere or stopping if a disable + new ClosedRegionHandler(master, this, data, regionState.getRegion()) + .submit(); + break; + + case RS2ZK_REGION_OPENING: + // Should see OPENING after we have asked it to OPEN or additional + // times after already being in state of OPENING + if(regionState == null || + (!regionState.isPendingOpen() && !regionState.isOpening())) { + LOG.warn("Received OPENING for region " + encodedName + + " from server " + data.getServerName() + " but region was in " + + " the state " + regionState + " and not " + + "in expected PENDING_OPEN or OPENING states"); + return; + } + // Transition to OPENING (or update stamp if already OPENING) + regionState.update(RegionState.State.OPENING, data.getStamp()); + break; + + case RS2ZK_REGION_OPENED: + // Should see OPENED after OPENING but possible after PENDING_OPEN + if(regionState == null || + (!regionState.isPendingOpen() && !regionState.isOpening())) { + LOG.warn("Received OPENED for region " + encodedName + + " from server " + data.getServerName() + " but region was in " + + " the state " + regionState + " and not " + + "in expected PENDING_OPEN or OPENING states"); + return; + } + // If this is a catalog table, update catalog manager accordingly + // Moving root and meta editing over to RS who does the opening + LOG.debug("Processing OPENED for region " + regionState.getRegion() + + " which isMeta[" + regionState.getRegion().isMetaRegion() + "] " + + " isRoot[" + regionState.getRegion().isRootRegion() + "]"); + + // Used to have updating of root/meta locations here but it's + // automatic in CatalogTracker now + + // Handle OPENED by removing from transition and deleted zk node + new OpenedRegionHandler(master, this, data, regionState.getRegion(), + serverManager.getServerInfo(data.getServerName())) + .submit(); + break; + } + } + } + // ZooKeeper events /** @@ -160,17 +336,19 @@ * */ @Override - public synchronized void nodeCreated(String path) { + public void nodeCreated(String path) { if(path.startsWith(watcher.assignmentZNode)) { - try { - RegionTransitionData data = ZKAssign.getData(watcher, path); - if(data == null) { - return; + synchronized(regionsInTransition) { + try { + RegionTransitionData data = ZKAssign.getData(watcher, path); + if(data == null) { + return; + } + handleRegion(data); + } catch (KeeperException e) { + LOG.error("Unexpected ZK exception reading unassigned node data", e); + master.abort(); } - handleRegion(data); - } catch (KeeperException e) { - LOG.error("Unexpected ZK exception reading unassigned node data", e); - status.abort(); } } } @@ -188,17 +366,19 @@ * */ @Override - public synchronized void nodeDataChanged(String path) { + public void nodeDataChanged(String path) { if(path.startsWith(watcher.assignmentZNode)) { - try { - RegionTransitionData data = ZKAssign.getData(watcher, path); - if(data == null) { - return; + synchronized(regionsInTransition) { + try { + RegionTransitionData data = ZKAssign.getData(watcher, path); + if(data == null) { + return; + } + handleRegion(data); + } catch (KeeperException e) { + LOG.error("Unexpected ZK exception reading unassigned node data", e); + master.abort(); } - handleRegion(data); - } catch (KeeperException e) { - LOG.error("Unexpected ZK exception reading unassigned node data", e); - status.abort(); } } } @@ -217,19 +397,589 @@ * */ @Override - public synchronized void nodeChildrenChanged(String path) { + public void nodeChildrenChanged(String path) { if(path.equals(watcher.assignmentZNode)) { + synchronized(regionsInTransition) { + try { + List newNodes = ZKUtil.watchAndGetNewChildren(watcher, + watcher.assignmentZNode); + for(NodeAndData newNode : newNodes) { + LOG.debug("Handling new unassigned node: " + newNode); + handleRegion(RegionTransitionData.fromBytes(newNode.getData())); + } + } catch(KeeperException e) { + LOG.error("Unexpected ZK exception reading unassigned children", e); + master.abort(); + } + } + } + } + + /** + * Marks the region as online. Removes it from regions in transition and + * updates the in-memory assignment information. + *

        + * Used when a region has been successfully opened on a region server. + * @param regionInfo + * @param serverInfo + */ + public void regionOnline(HRegionInfo regionInfo, HServerInfo serverInfo) { + synchronized(regionsInTransition) { + regionsInTransition.remove(regionInfo.getEncodedName()); + regionsInTransition.notifyAll(); + } + synchronized(regions) { + regions.put(regionInfo, serverInfo); + Set regionSet = servers.get(serverInfo); + if(regionSet == null) { + regionSet = new TreeSet(); + servers.put(serverInfo, regionSet); + } + regionSet.add(regionInfo); + } + } + + /** + * Marks the region as offline. Removes it from regions in transition and + * removes in-memory assignment information. + *

        + * Used when a region has been closed and should remain closed. + * @param regionInfo + * @param serverInfo + */ + public void regionOffline(HRegionInfo regionInfo) { + synchronized(regionsInTransition) { + regionsInTransition.remove(regionInfo.getEncodedName()); + regionsInTransition.notifyAll(); + } + synchronized(regions) { + HServerInfo serverInfo = regions.remove(regionInfo); + Set serverRegions = servers.get(serverInfo); + serverRegions.remove(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(regions) { + HServerInfo serverInfo = regions.remove(regionInfo); + Set serverRegions = servers.get(serverInfo); + serverRegions.remove(regionInfo); + } + } + + // Assignment methods + + /** + * Assigns the specified region. + *

        + * If a RegionPlan is available with a valid destination then it will be used + * to determine what server region is assigned to. If no RegionPlan is + * available, region will be assigned to a random available server. + *

        + * Updates the RegionState and sends the OPEN RPC. + *

        + * This will only succeed if the region is in transition and in a CLOSED or + * OFFLINE state or not in transition (in-memory not zk). If the in-memory + * checks pass, the zk node is forced to OFFLINE before assigning. + * + * @param regionName server to be assigned + */ + public void assign(HRegionInfo region) { + LOG.debug("Starting assignment for region " + region); + // Grab the state of this region and synchronize on it + String regionName = region.getEncodedName(); + RegionState state; + synchronized(regionsInTransition) { + state = regionsInTransition.get(regionName); + if(state == null) { + state = new RegionState(region, RegionState.State.OFFLINE); + regionsInTransition.put(regionName, state); + } + } + synchronized(state) { + if(!state.isClosed() && !state.isOffline()) { + LOG.info("Attempting to assign region but it is in transition and in " + + "an unexpected state:" + state); + return; + } else { + state.update(RegionState.State.OFFLINE); + } try { - List newNodes = ZKUtil.watchAndGetNewChildren(watcher, - watcher.assignmentZNode); - for(NodeAndData newNode : newNodes) { - LOG.debug("Handling new unassigned node: " + newNode); - handleRegion(RegionTransitionData.fromBytes(newNode.getData())); + if(!ZKAssign.createOrForceNodeOffline(master.getZooKeeper(), region, + master.getServerName())) { + LOG.warn("Attempted to create/force node into OFFLINE state before " + + "completing assignment but failed to do so"); + return; } - } catch(KeeperException e) { - LOG.error("Unexpected ZK exception reading unassigned children", e); - status.abort(); + } catch (KeeperException e) { + LOG.error("Unexpected ZK exception creating/setting node OFFLINE", e); + master.abort(); + return; } + // Pickup existing plan or make a new one + RegionPlan plan; + synchronized(regionPlans) { + plan = regionPlans.get(regionName); + if(plan == null) { + LOG.debug("No previous transition plan for " + regionName + + " so generating a random one from " + serverManager.numServers() + + " ( " + serverManager.getOnlineServers().size() + ") available servers"); + plan = new RegionPlan(regionName, null, + LoadBalancer.randomAssignment(serverManager.getOnlineServersList())); + regionPlans.put(regionName, plan); + } + } + // Transition RegionState to PENDING_OPEN and send OPEN RPC + state.update(RegionState.State.PENDING_OPEN); + serverManager.sendRegionOpen(plan.getDestination(), state.getRegion()); } } + + /** + * Unassigns the specified region. + *

        + * Updates the RegionState and sends the OPEN RPC. + *

        + * If a RegionPlan is already set, it will remain. If this is being used + * to disable a table, be sure to use {@link #disableTable(String)} to ensure + * regions are not onlined after being closed. + * + * @param regionName server to be unassigned + */ + public void unassign(HRegionInfo region) { + LOG.debug("Starting unassignment of region " + region + " (offlining)"); + // Check if this region is currently assigned + if (!regions.containsKey(region)) { + LOG.debug("Attempted to unassign region " + region + " but it is not " + + "currently assigned anywhere"); + return; + } + String regionName = region.getEncodedName(); + // Grab the state of this region and synchronize on it + RegionState state; + synchronized(regionsInTransition) { + state = regionsInTransition.get(regionName); + if(state == null) { + state = new RegionState(region, RegionState.State.PENDING_CLOSE); + regionsInTransition.put(regionName, state); + } else { + LOG.debug("Attempting to unassign region " + region + " but it is " + + "already in transition (" + state.getState() + ")"); + return; + } + } + // Send OPEN RPC + try { + serverManager.sendRegionClose(regions.get(region), state.getRegion()); + } catch (NotServingRegionException e) { + LOG.warn("Attempted to close region " + region + " but got an NSRE", e); + } + } + + /** + * Waits until the specified region has completed assignment. + *

        + * If the region is already assigned, returns immediately. Otherwise, method + * blocks until the region is assigned. + * @param regionInfo region to wait on assignment for + * @throws InterruptedException + */ + public void waitForAssignment(HRegionInfo regionInfo) + throws InterruptedException { + synchronized(regions) { + while(!regions.containsKey(regionInfo)) { + regions.wait(); + } + } + } + + /** + * Assigns the ROOT region. + *

        + * Assumes that ROOT is currently closed and is not being actively served by + * any RegionServer. + *

        + * Forcibly unsets the current root region location in ZooKeeper and assigns + * ROOT to a random RegionServer. + */ + public void assignRoot() { + // Force assignment to a random server + assign(HRegionInfo.ROOT_REGIONINFO); + } + + /** + * Assigns the META region. + *

        + * Assumes that META is currently closed and is not being actively served by + * any RegionServer. + *

        + * Forcibly assigns META to a random RegionServer. + */ + public void assignMeta() { + // Force assignment to a random server + assign(HRegionInfo.FIRST_META_REGIONINFO); + } + + /** + * Assigns all user regions, if any exist. Used during cluster startup. + *

        + * This is a synchronous call and will return once every region has been + * assigned. If anything fails, an exception is thrown and the cluster + * should be shutdown. + */ + public void assignAllUserRegions() throws IOException { + // First experiment at synchronous assignment + // Simpler because just wait for no regions in transition + + // Scan META for all user regions + List allRegions = MetaScanner.listAllRegions( + master.getConfiguration()); + if(allRegions == null || allRegions.isEmpty()) { + return; + } + + // Get all available servers + List servers = serverManager.getOnlineServersList(); + + LOG.info("Assigning " + allRegions.size() + " across " + servers.size() + + " servers"); + + // Generate a cluster startup region placement plan + Map> bulkPlan = + LoadBalancer.bulkAssignment(allRegions, servers); + + // For each server, create OFFLINE nodes and send OPEN RPCs + for(Map.Entry> entry : bulkPlan.entrySet()) { + HServerInfo server = entry.getKey(); + List regions = entry.getValue(); + LOG.debug("Assigning " + regions.size() + " regions to " + server); + for(HRegionInfo region : regions) { + LOG.debug("Assigning " + region + " to " + server); + String regionName = region.getEncodedName(); + RegionPlan plan = new RegionPlan(regionName, null,server); + regionPlans.put(regionName, plan); + assign(region); + } + } + + // Wait for no regions to be in transition + try { + waitUntilNoRegionsInTransition(); + } catch (InterruptedException e) { + LOG.error("Interrupted waiting for regions to be assigned", e); + throw new IOException(e); + } + + LOG.info("\n\nAll user regions have been assigned"); + } + + private void rebuildUserRegions() throws IOException { + Map allRegions = + MetaReader.fullScan(catalogTracker); + for(Map.Entry region : allRegions.entrySet()) { + HServerAddress regionLocation = region.getValue(); + HRegionInfo regionInfo = region.getKey(); + if(regionLocation == null) { + regions.put(regionInfo, null); + continue; + } + HServerInfo serverInfo = serverManager.getHServerInfo(regionLocation); + regions.put(regionInfo, serverInfo); + Set regionSet = servers.get(serverInfo); + if(regionSet == null) { + regionSet = new TreeSet(); + servers.put(serverInfo, regionSet); + } + regionSet.add(regionInfo); + } + } + + /** + * Blocks until there are no regions in transition. It is possible that there + * are regions in transition immediately after this returns but guarantees + * that if it returns without an exception that there was a period of time + * with no regions in transition from the point-of-view of the in-memory + * state of the Master. + * @throws InterruptedException + */ + public void waitUntilNoRegionsInTransition() throws InterruptedException { + synchronized(regionsInTransition) { + while(regionsInTransition.size() > 0) { + regionsInTransition.wait(); + } + } + } + + /** + * Gets the map of regions currently in transition. + * @return + */ + public Map getRegionsInTransition() { + return regionsInTransition; + } + + /** + * Checks if the specified table has been disabled by the user. + * @param tableName + * @return + */ + public boolean isTableDisabled(String tableName) { + synchronized(disabledTables) { + return disabledTables.contains(tableName); + } + } + + /** + * Checks if the table of the specified region has been disabled by the user. + * @param regionName + * @return + */ + public boolean isTableOfRegionDisabled(byte [] regionName) { + return isTableDisabled(Bytes.toString( + HRegionInfo.getTableName(regionName))); + } + + /** + * Sets the specified table to be disabled. + * @param tableName table to be disabled + */ + public void disableTable(String tableName) { + synchronized(disabledTables) { + if(!isTableDisabled(tableName)) { + disabledTables.add(tableName); + try { + ZKTableDisable.disableTable(master.getZooKeeper(), tableName); + } catch (KeeperException e) { + LOG.warn("ZK error setting table as disabled", e); + } + } + } + } + + /** + * Unsets the specified table from being disabled. + *

        + * This operation only acts on the in-memory + * @param tableName table to be undisabled + */ + public void undisableTable(String tableName) { + synchronized(disabledTables) { + if(isTableDisabled(tableName)) { + disabledTables.remove(tableName); + try { + ZKTableDisable.undisableTable(master.getZooKeeper(), tableName); + } catch (KeeperException e) { + LOG.warn("ZK error setting table as disabled", e); + } + } + } + } + + /** + * Rebuild the set of disabled tables from zookeeper. Used during master + * failover. + */ + private void rebuildDisabledTables() { + synchronized(disabledTables) { + List disabledTables; + try { + disabledTables = ZKTableDisable.getDisabledTables(master.getZooKeeper()); + } catch (KeeperException e) { + LOG.warn("ZK error getting list of disabled tables", e); + return; + } + if(!disabledTables.isEmpty()) { + LOG.info("Rebuilt list of " + disabledTables.size() + " disabled " + + "tables from zookeeper"); + disabledTables.addAll(disabledTables); + } + } + } + + /** + * Gets the online regions of the specified table. + * @param tableName + * @return + */ + public List getRegionsOfTable(byte[] tableName) { + List tableRegions = new ArrayList(); + for(HRegionInfo regionInfo : regions.tailMap(new HRegionInfo( + new HTableDescriptor(tableName), null, null)).keySet()) { + if(Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) { + tableRegions.add(regionInfo); + } else { + break; + } + } + return tableRegions; + } + + /** + * Unsets the specified table as disabled (enables it). + */ + public class TimeoutMonitor extends Chore { + + private final int timeout; + + /** + * Creates a periodic monitor to check for time outs on region transition + * operations. This will deal with retries if for some reason something + * doesn't happen within the specified timeout. + * @param period + * @param stop + * @param timeout + */ + public TimeoutMonitor(final int period, final AtomicBoolean stop, + final int timeout) { + super("AssignmentTimeoutMonitor", period, stop); + this.timeout = timeout; + } + + @Override + protected void chore() { + synchronized(regionsInTransition) { + // Iterate all regions in transition checking for time outs + long now = System.currentTimeMillis(); + for(RegionState regionState : regionsInTransition.values()) { + if(regionState.getStamp() + timeout <= now) { + HRegionInfo regionInfo = regionState.getRegion(); + String regionName = regionInfo.getEncodedName(); + LOG.info("Region transition timed out for region " + regionName); + // Expired! Do a retry. + switch(regionState.getState()) { + case OFFLINE: + case CLOSED: + LOG.info("Region has been OFFLINE or CLOSED for too long, " + + "reassigning " + regionInfo.getRegionNameAsString()); + assign(regionState.getRegion()); + break; + case PENDING_OPEN: + case OPENING: + LOG.info("Region has been PENDING_OPEN or OPENING for too " + + "long, reassigning " + regionInfo.getRegionNameAsString()); + assign(regionState.getRegion()); + break; + case OPEN: + LOG.warn("Long-running region in OPEN state? This should " + + "not happen"); + break; + case PENDING_CLOSE: + case CLOSING: + LOG.info("Region has been PENDING_CLOSE or CLOSING for too " + + "long, resending close rpc"); + unassign(regionInfo); + break; + } + } + } + } + } + } + + public static class RegionState implements 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 + } + + private State state; + private long stamp; + + public RegionState() {} + + RegionState(HRegionInfo region, State state) { + this(region, state, System.currentTimeMillis()); + } + + RegionState(HRegionInfo region, State state, long stamp) { + this.region = region; + this.state = state; + this.stamp = stamp; + } + + public void update(State state, long stamp) { + this.state = state; + this.stamp = stamp; + } + + public void update(State state) { + this.state = state; + this.stamp = System.currentTimeMillis(); + } + + public State getState() { + return state; + } + + public long getStamp() { + return stamp; + } + + public HRegionInfo getRegion() { + return region; + } + + 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; + } + + @Override + public String toString() { + return "RegionState (" + region.getRegionNameAsString() + ") " + state + + " at time " + stamp; + } + + @Override + public void readFields(DataInput in) throws IOException { + region = new HRegionInfo(); + region.readFields(in); + state = State.valueOf(in.readUTF()); + stamp = in.readLong(); + } + + @Override + public void write(DataOutput out) throws IOException { + region.write(out); + out.writeUTF(state.name()); + out.writeLong(stamp); + } + } } Index: src/main/java/org/apache/hadoop/hbase/master/HMaster.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/HMaster.java (working copy) @@ -25,11 +25,6 @@ import java.lang.management.RuntimeMXBean; import java.lang.reflect.Constructor; import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -43,60 +38,57 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HMsg; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MiniZooKeeperCluster; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ServerConnection; import org.apache.hadoop.hbase.client.ServerConnectionManager; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.executor.HBaseExecutorService; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.executor.HBaseExecutorService.HBaseExecutorServiceType; import org.apache.hadoop.hbase.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion; import org.apache.hadoop.hbase.ipc.HBaseServer; import org.apache.hadoop.hbase.ipc.HMasterInterface; import org.apache.hadoop.hbase.ipc.HMasterRegionInterface; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.master.RegionServerOperationQueue.ProcessingResultCode; +import org.apache.hadoop.hbase.master.handler.DeleteTableHandler; +import org.apache.hadoop.hbase.master.handler.DisableTableHandler; +import org.apache.hadoop.hbase.master.handler.EnableTableHandler; +import org.apache.hadoop.hbase.master.handler.ModifyTableHandler; +import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler; +import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler; +import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler; import org.apache.hadoop.hbase.master.metrics.MasterMetrics; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; 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.VersionInfo; -import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; -import org.apache.hadoop.hbase.zookeeper.RootRegionTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.DNS; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.Watcher; -import com.google.common.collect.Lists; - /** * HMaster is the "master server" for HBase. An HBase cluster has one active * master. If many masters are started, all compete. Whichever wins goes on to @@ -108,7 +100,7 @@ * @see Watcher */ public class HMaster extends Thread -implements HMasterInterface, HMasterRegionInterface, MasterStatus { +implements HMasterInterface, HMasterRegionInterface, MasterController { // MASTER is name of the webapp and the attribute name used stuffing this //instance into web context. public static final String MASTER = "master"; @@ -129,14 +121,12 @@ // Reporting to track master metrics. private final MasterMetrics metrics; - // Our zk client. TODO: rename variable once we settle on naming - private ZooKeeperWatcher zooKeeperWrapper; + // Our zk client. + private ZooKeeperWatcher zooKeeper; // Manager and zk listener for master election private ActiveMasterManager activeMasterManager; // Cluster status zk tracker and local setter private ClusterStatusTracker clusterStatusTracker; - // Root region location tracker - private RootRegionTracker rootRegionTracker; // Region server tracker private RegionServerTracker regionServerTracker; @@ -152,20 +142,15 @@ private final ServerConnection connection; // server manager to deal with region server info private final ServerManager serverManager; - // region manager to deal with region specific stuff - private final RegionManager regionManager; // manager of assignment nodes in zookeeper private final AssignmentManager assignmentManager; + // manager of catalog regions + private final CatalogTracker catalogTracker; // True if this is the master that started the cluster. boolean isClusterStartup; - // TODO: the following should eventually be removed from here - private final RegionServerOperationQueue regionServerOperationQueue; - private long lastFragmentationQuery = -1L; - private Map fragmentation = null; - /** * Initializes the HMaster. The steps are as follows: * @@ -208,10 +193,10 @@ * an additional check if this master does not become primary on its * first attempt. */ - zooKeeperWrapper = - new ZooKeeperWatcher(conf, getHServerAddress().toString(), this); + zooKeeper = + new ZooKeeperWatcher(conf, MASTER + "-" + getHServerAddress(), this); isClusterStartup = 0 == - ZKUtil.getNumberOfChildren(zooKeeperWrapper, zooKeeperWrapper.rsZNode); + ZKUtil.getNumberOfChildren(zooKeeper, zooKeeper.rsZNode); /* * 3. Initialize master components. @@ -220,17 +205,16 @@ * metrics, queues, sleeper, etc... */ this.connection = ServerConnectionManager.getConnection(conf); - this.regionServerOperationQueue = new RegionServerOperationQueue(conf, closed); this.metrics = new MasterMetrics(this.getName()); - clusterStatusTracker = new ClusterStatusTracker(zooKeeperWrapper, this); - rootRegionTracker = new RootRegionTracker(zooKeeperWrapper, this); + clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this); fileSystemManager = new FileSystemManager(conf, this); - serverManager = new ServerManager(this, metrics, regionServerOperationQueue); - regionManager = new RegionManager(this, rootRegionTracker); - assignmentManager = new AssignmentManager(zooKeeperWrapper, this, - serverManager, regionManager); - regionServerTracker = new RegionServerTracker(zooKeeperWrapper, this, + serverManager = new ServerManager(this, metrics, fileSystemManager); + regionServerTracker = new RegionServerTracker(zooKeeper, this, serverManager); + catalogTracker = new CatalogTracker(zooKeeper, connection, this, + conf.getInt("hbase.master.catalog.timeout", 30000)); + assignmentManager = new AssignmentManager(zooKeeper, this, + serverManager, catalogTracker); // create a sleeper to sleep for a configured wait frequency int threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000); this.sleeper = new Sleeper(threadWakeFrequency, this.closed); @@ -246,13 +230,20 @@ * we do not succeed on our first attempt, this is no longer a cluster * startup. */ - activeMasterManager = new ActiveMasterManager(zooKeeperWrapper, address, + activeMasterManager = new ActiveMasterManager(zooKeeper, address, this); - zooKeeperWrapper.registerListener(activeMasterManager); + zooKeeper.registerListener(activeMasterManager); + zooKeeper.registerListener(assignmentManager); // Wait here until we are the active master activeMasterManager.blockUntilBecomingActiveMaster(); + // TODO: We should start everything here instead of before we become + // active master and some after. Requires change to RS side to not + // start until clusterStatus is up rather than master is available. + // We are the active master now. + regionServerTracker.start(); + catalogTracker.start(); clusterStatusTracker.setClusterUp(); LOG.info("Server has become the active/primary master. Address is " + @@ -272,56 +263,75 @@ @Override public void run() { try { - // If this is a fresh cluster start, make sure the root region exists. if(isClusterStartup()) { + // This is a cluster startup, you are the first master. // Initialize the filesystem, which does the following: - // - Creates the root hbase directory in the FS + // - Creates the root hbase directory in the FS if DNE + // - If fresh start, create first ROOT and META regions (bootstrap) // - Checks the FS to make sure the root directory is readable // - Creates the archive directory for logs fileSystemManager.initialize(); // Do any log splitting necessary // TODO: Should do this in background rather than block master startup + // TODO: Do we want to do this before/while/after RSs check in? + // It seems that this method looks at active RSs but happens + // concurrently with when we expect them to be checking in fileSystemManager.splitLogAfterStartup(); } - // TODO: fix the logic and naming for joinCluster() - joinCluster(); // start up all service threads. startServiceThreads(); - // assign the root region - regionManager.reassignRootRegion(); // set the master as opened this.closed.set(false); + // wait for minimum number of region servers to be up + serverManager.waitForMinServers(); + // assign the root region + assignmentManager.assignRoot(); + catalogTracker.waitForRoot(); + // assign the meta region + assignmentManager.assignMeta(); + catalogTracker.waitForMeta(); + // above check waits for general meta availability but this does not + // guarantee that the transition has completed + assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO); + // start assignment of user regions, startup or failure + if(isClusterStartup()) { + // Create or clear out unassigned node in ZK, read all regions from + // META and assign them out. + assignmentManager.processStartup(); + } else { + // Process existing unassigned nodes in ZK, read all regions from META, + // rebuild in-memory state. + assignmentManager.processFailover(); + } LOG.info("HMaster started on " + this.address.toString()); - while (!this.closed.get()) { // check if we should be shutting down if (this.shutdownRequested.get()) { - // The region servers won't all exit until we stop scanning the - // meta regions - this.regionManager.stopScanners(); if (this.serverManager.numServers() == 0) { startShutdown(); break; } else { LOG.debug("Waiting on " + - this.serverManager.getServersToServerInfo().keySet().toString()); + this.serverManager.getOnlineServers().keySet().toString()); } } - // process the operation, handle the result - ProcessingResultCode resultCode = regionServerOperationQueue.process(); - // If FAILED op processing, bad. Will exit. - if(resultCode == ProcessingResultCode.FAILED) { - break; - } - // If bad filesystem, exit - else if(resultCode == ProcessingResultCode.REQUEUED_BUT_PROBLEM) { - if (!fileSystemManager.checkFileSystem()) { - break; + // wait for an interruption + // TODO: something better? we need to check closed and shutdown? + synchronized(this.shutdownRequested) { + try { + this.shutdownRequested.wait(); + } catch(InterruptedException e) { + LOG.debug("Main thread interrupted", e); } } - // Continue run loop if conditions are PROCESSED, NOOP, REQUEUED + + // TODO: should be check file system like we used to? + if (!fileSystemManager.checkFileSystem()) { + break; + } + // Continue run loop } } catch (Throwable t) { LOG.fatal("Unhandled exception. Starting shutdown.", t); @@ -341,9 +351,8 @@ } } this.rpcServer.stop(); - this.regionManager.stop(); this.activeMasterManager.stop(); - this.zooKeeperWrapper.close(); + this.zooKeeper.close(); HBaseExecutorService.shutdown(); LOG.info("HMaster main thread exiting"); } @@ -416,10 +425,6 @@ return this.serverManager; } - public RegionManager getRegionManager() { - return this.regionManager; - } - public AtomicBoolean getShutdownRequested() { return this.shutdownRequested; } @@ -445,7 +450,7 @@ * @return the zookeeper wrapper */ public ZooKeeperWatcher getZooKeeperWatcher() { - return this.zooKeeperWrapper; + return this.zooKeeper; } /** * Get the HBase root dir - needed by master_jsp.java @@ -454,78 +459,7 @@ return fileSystemManager.getRootDir(); } - public RegionServerOperationQueue getRegionServerOperationQueue() { - return this.regionServerOperationQueue; - } - /* - * Joins cluster. Checks to see if this instance of HBase is fresh or the - * master was started following a failover. In the second case, it inspects - * the region server directory and gets their regions assignment. - */ - private void joinCluster() { - LOG.debug("Checking cluster state..."); - HServerAddress rootLocation = null; - List addresses = null; - try { - clusterStatusTracker.start(); - rootRegionTracker.start(); - regionServerTracker.start(); - rootLocation = rootRegionTracker.getRootRegionLocation(); - addresses = regionServerTracker.getOnlineServers(); - } catch(KeeperException e) { - LOG.fatal("Unexpected ZK exception initializing trackers", e); - abort(); - return; - } - // Check if this is a fresh start of the cluster - if (isClusterStartup) { - LOG.debug("Master fresh start, proceeding with normal startup"); - fileSystemManager.splitLogAfterStartup(); - return; - } - // Failover case. - LOG.info("Master failover, ZK inspection begins..."); - boolean isRootRegionAssigned = false; - Map assignedRegions = - new HashMap(); - // We must: - // - contact every region server to add them to the regionservers list - // - get their current regions assignment - // TODO: Run in parallel? - for (HServerAddress address : addresses) { - HRegionInfo[] regions = null; - try { - HRegionInterface hri = - this.connection.getHRegionConnection(address, false); - HServerInfo info = hri.getHServerInfo(); - LOG.debug("Inspection found server " + info.getServerName()); - this.serverManager.recordNewServer(info, true); - regions = hri.getRegionsAssignment(); - } catch (IOException e) { - LOG.error("Failed contacting " + address.toString(), e); - continue; - } - for (HRegionInfo r: regions) { - if (r.isRootRegion()) { - this.connection.setRootRegionLocation(new HRegionLocation(r, rootLocation)); - this.regionManager.setRootRegionLocation(rootLocation); - // Undo the unassign work in the RegionManager constructor - this.regionManager.removeRegion(r); - isRootRegionAssigned = true; - } else if (r.isMetaRegion()) { - MetaRegion m = new MetaRegion(new HServerAddress(address), r); - this.regionManager.addMetaRegionToScan(m); - } - assignedRegions.put(r.getRegionName(), r); - } - } - LOG.info("Inspection found " + assignedRegions.size() + " regions, " + - (isRootRegionAssigned ? "with -ROOT-" : "but -ROOT- was MIA")); - fileSystemManager.splitLogAfterStartup(); - } - - /* * Start up all services. If any of these threads gets an unhandled exception * then they just die with a logged message. This should be fine because * in general, we do not expect the master to get such unhandled exceptions @@ -534,22 +468,20 @@ */ private void startServiceThreads() { try { - // start the "close region" executor service - HBaseEventType.RS2ZK_REGION_CLOSED.startMasterExecutorService( - address.toString()); - // start the "open region" executor service - HBaseEventType.RS2ZK_REGION_OPENED.startMasterExecutorService(address.toString()); - // Start the assignment manager. Creates the unassigned node in ZK - // if it does not exist and handles regions in transition if a failed-over - // master. This is needed before RegionManager() constructor tries to - // assign the root region. - try { - assignmentManager.start(); - } catch (KeeperException e) { - throw new IOException(e); - } - // start the region manager - this.regionManager.start(); + // Start the executor service pools + HBaseExecutorServiceType.MASTER_OPEN_REGION.startExecutorService( + getServerName(), + conf.getInt("hbase.master.executor.openregion.threads", 5)); + HBaseExecutorServiceType.MASTER_CLOSE_REGION.startExecutorService( + getServerName(), + conf.getInt("hbase.master.executor.closeregion.threads", 5)); + HBaseExecutorServiceType.MASTER_SERVER_OPERATIONS.startExecutorService( + getServerName(), + conf.getInt("hbase.master.executor.serverops.threads", 5)); + HBaseExecutorServiceType.MASTER_TABLE_OPERATIONS.startExecutorService( + getServerName(), + conf.getInt("hbase.master.executor.tableops.threads", 5)); + // Put up info server. int port = this.conf.getInt("hbase.master.info.port", 60010); if (port >= 0) { @@ -581,10 +513,8 @@ /* * Start shutting down the master */ - public void startShutdown() { + void startShutdown() { setClosed(); - this.regionManager.stopScanners(); - this.regionServerOperationQueue.shutdown(); this.serverManager.notifyServers(); } @@ -641,9 +571,12 @@ return !this.closed.get(); } - public void shutdown() { + public void requestShutdown() { LOG.info("Cluster shutdown requested. Starting to quiesce servers"); - this.shutdownRequested.set(true); + synchronized(shutdownRequested) { + shutdownRequested.set(true); + shutdownRequested.notifyAll(); + } try { clusterStatusTracker.setClusterDown(); } catch (KeeperException e) { @@ -653,6 +586,12 @@ public void createTable(HTableDescriptor desc, byte [][] splitKeys) throws IOException { + createTable(desc, splitKeys, false); + } + + public void createTable(HTableDescriptor desc, byte [][] splitKeys, + boolean sync) + throws IOException { if (!isMasterRunning()) { throw new MasterNotRunningException(); } @@ -670,155 +609,123 @@ startKey = endKey; } } - int numRetries = conf.getInt("hbase.client.retries.number", 2); - for (int tries = 0; tries < numRetries; tries++) { - try { - // We can not create a table unless meta regions have already been - // assigned and scanned. - if (!this.regionManager.areAllMetaRegionsOnline()) { - throw new NotAllMetaRegionsOnlineException(); - } - if (!this.serverManager.canAssignUserRegions()) { - throw new IOException("not enough servers to create table yet"); - } - createTable(newRegions); - LOG.info("created table " + desc.getNameAsString()); - break; - } catch (TableExistsException e) { - throw e; - } catch (IOException e) { - if (tries == numRetries - 1) { - throw RemoteExceptionHandler.checkIOException(e); - } - this.sleeper.sleep(); + int timeout = conf.getInt("hbase.client.catalog.timeout", 10000); + // Need META availability to create a table + try { + if(catalogTracker.waitForMeta(timeout) == null) { + throw new NotAllMetaRegionsOnlineException(); } + } catch (InterruptedException e) { + LOG.warn("Interrupted waiting for meta availability", e); + throw new IOException(e); } + createTable(newRegions, sync); } - private synchronized void createTable(final HRegionInfo [] newRegions) + private synchronized void createTable(final HRegionInfo [] newRegions, + boolean sync) throws IOException { String tableName = newRegions[0].getTableDesc().getNameAsString(); - // 1. Check to see if table already exists. Get meta region where - // table would sit should it exist. Open scanner on it. If a region - // for the table we want to create already exists, then table already - // created. Throw already-exists exception. - MetaRegion m = regionManager.getFirstMetaRegionForRegion(newRegions[0]); - byte [] metaRegionName = m.getRegionName(); - HRegionInterface srvr = this.connection.getHRegionConnection(m.getServer()); - byte[] firstRowInTable = Bytes.toBytes(tableName + ",,"); - Scan scan = new Scan(firstRowInTable); - scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - long scannerid = srvr.openScanner(metaRegionName, scan); - try { - Result data = srvr.next(scannerid); - if (data != null && data.size() > 0) { - HRegionInfo info = Writables.getHRegionInfo( - data.getValue(HConstants.CATALOG_FAMILY, - HConstants.REGIONINFO_QUALIFIER)); - if (info.getTableDesc().getNameAsString().equals(tableName)) { - // A region for this table already exists. Ergo table exists. - throw new TableExistsException(tableName); - } - } - } finally { - srvr.close(scannerid); + if(MetaReader.tableExists(catalogTracker, tableName)) { + throw new TableExistsException(tableName); } for(HRegionInfo newRegion : newRegions) { - regionManager.createRegion(newRegion, srvr, metaRegionName); + // 1. Create HRegion + HRegion region = HRegion.createHRegion(newRegion, + fileSystemManager.getRootDir(), conf); + + // 2. Insert into META + MetaEditor.addRegionToMeta(catalogTracker, region.getRegionInfo()); + + // 3. Close the new region to flush to disk. Close log file too. + region.close(); + region.getLog().closeAndDelete(); + + // 4. Trigger immediate assignment of this region + assignmentManager.assign(region.getRegionInfo()); } + + // 5. If sync, wait for assignment of regions + if(sync) { + LOG.debug("Waiting for " + newRegions.length + " region(s) to be " + + "assigned before returning"); + for(HRegionInfo regionInfo : newRegions) { + try { + assignmentManager.waitForAssignment(regionInfo); + } catch (InterruptedException e) { + LOG.info("Interrupted waiting for region to be assigned during " + + "create table call"); + return; + } + } + } } + private boolean isCatalogTable(final byte [] tableName) { + return Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) || + Bytes.equals(tableName, HConstants.META_TABLE_NAME); + } + + // TODO: Sync or async on this stuff? + // Right now this will swallow exceptions either way, might need + // process() which throws nothing but execute() which throws IOE so + // synchronous stuff can throw exceptions? + public void deleteTable(final byte [] tableName) throws IOException { - if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { - throw new IOException("Can't delete root table"); + if (isCatalogTable(tableName)) { + throw new IOException("Can't delete catalog tables"); } - new TableDelete(this, tableName).process(); + // + new DeleteTableHandler(tableName, this, catalogTracker, fileSystemManager) + .execute(); LOG.info("deleted table: " + Bytes.toString(tableName)); } public void addColumn(byte [] tableName, HColumnDescriptor column) throws IOException { - new AddColumn(this, tableName, column).process(); + if (isCatalogTable(tableName)) { + throw new IOException("Can't modify catalog tables"); + } + new TableAddFamilyHandler(tableName, column, this, catalogTracker, + fileSystemManager).execute(); } public void modifyColumn(byte [] tableName, byte [] columnName, HColumnDescriptor descriptor) throws IOException { - new ModifyColumn(this, tableName, columnName, descriptor).process(); + if (isCatalogTable(tableName)) { + throw new IOException("Can't modify catalog tables"); + } + new TableModifyFamilyHandler(tableName, descriptor, this, catalogTracker, + fileSystemManager).execute(); } public void deleteColumn(final byte [] tableName, final byte [] c) throws IOException { - new DeleteColumn(this, tableName, KeyValue.parseColumn(c)[0]).process(); + if (isCatalogTable(tableName)) { + throw new IOException("Can't modify catalog tables"); + } + new TableDeleteFamilyHandler(tableName, c, this, catalogTracker, + fileSystemManager).execute(); } public void enableTable(final byte [] tableName) throws IOException { - if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { - throw new IOException("Can't enable root table"); + if (isCatalogTable(tableName)) { + throw new IOException("Can't enable catalog tables"); } - new ChangeTableState(this, tableName, true).process(); + new EnableTableHandler(this, tableName, catalogTracker, assignmentManager) + .execute(); } public void disableTable(final byte [] tableName) throws IOException { - if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) { - throw new IOException("Can't disable root table"); + if (isCatalogTable(tableName)) { + throw new IOException("Can't disable catalog tables"); } - new ChangeTableState(this, tableName, false).process(); + new DisableTableHandler(this, tableName, catalogTracker, assignmentManager) + .execute(); } /** - * Get a list of the regions for a given table. The pairs may have - * null for their second element in the case that they are not - * currently deployed. - * TODO: Redo so this method does not duplicate code with subsequent methods. - */ - List> getTableRegions( - final byte [] tableName) - throws IOException { - final ArrayList> result = - Lists.newArrayList(); - MetaScannerVisitor visitor = - new MetaScannerVisitor() { - @Override - public boolean processRow(Result data) throws IOException { - if (data == null || data.size() <= 0) { - return true; - } - Pair pair = - metaRowToRegionPair(data); - if (pair == null) { - return false; - } - if (!Bytes.equals(pair.getFirst().getTableDesc().getName(), - tableName)) { - return false; - } - result.add(pair); - return true; - } - }; - - MetaScanner.metaScan(conf, visitor, tableName); - return result; - } - - private Pair metaRowToRegionPair( - Result data) throws IOException { - HRegionInfo info = Writables.getHRegionInfo( - data.getValue(HConstants.CATALOG_FAMILY, - HConstants.REGIONINFO_QUALIFIER)); - final byte[] value = data.getValue(HConstants.CATALOG_FAMILY, - HConstants.SERVER_QUALIFIER); - if (value != null && value.length > 0) { - HServerAddress server = new HServerAddress(Bytes.toString(value)); - return new Pair(info, server); - } else { - //undeployed - return new Pair(info, null); - } - } - - /** * Return the region and current deployment for the region containing * the given row. If the region cannot be found, returns null. If it * is found, but not currently deployed, the second element of the pair @@ -838,7 +745,7 @@ return true; } Pair pair = - metaRowToRegionPair(data); + MetaReader.metaRowToRegionPair(data); if (pair == null) { return false; } @@ -855,155 +762,23 @@ return result.get(); } - Pair getTableRegionFromName( - final byte [] regionName) + @Override + public void modifyTable(final byte[] tableName, HTableDescriptor htd) throws IOException { - byte [] tableName = HRegionInfo.parseRegionName(regionName)[0]; - - Set regions = regionManager.getMetaRegionsForTable(tableName); - for (MetaRegion m: regions) { - byte [] metaRegionName = m.getRegionName(); - HRegionInterface srvr = connection.getHRegionConnection(m.getServer()); - Get get = new Get(regionName); - get.addColumn(HConstants.CATALOG_FAMILY, - HConstants.REGIONINFO_QUALIFIER); - get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); - Result data = srvr.get(metaRegionName, get); - if(data == null || data.size() <= 0) { - continue; - } - return metaRowToRegionPair(data); - } - return null; + LOG.info("modifyTable(SET_HTD): " + htd); + new ModifyTableHandler(tableName, this, catalogTracker, fileSystemManager) + .submit(); } /** - * Get row from meta table. - * @param row - * @param family - * @return Result - * @throws IOException - */ - protected Result getFromMETA(final byte [] row, final byte [] family) - throws IOException { - MetaRegion meta = this.regionManager.getMetaRegionForRow(row); - HRegionInterface srvr = getMETAServer(meta); - Get get = new Get(row); - get.addFamily(family); - return srvr.get(meta.getRegionName(), get); - } - - /* - * @param meta - * @return Server connection to meta .META. region. - * @throws IOException - */ - private HRegionInterface getMETAServer(final MetaRegion meta) - throws IOException { - return this.connection.getHRegionConnection(meta.getServer()); - } - - public void modifyTable(final byte[] tableName, HConstants.Modify op, - Writable[] args) - throws IOException { - switch (op) { - case TABLE_SET_HTD: - if (args == null || args.length < 1 || - !(args[0] instanceof HTableDescriptor)) { - throw new IOException("SET_HTD request requires an HTableDescriptor"); - } - HTableDescriptor htd = (HTableDescriptor) args[0]; - LOG.info("modifyTable(SET_HTD): " + htd); - new ModifyTableMeta(this, tableName, htd).process(); - break; - - case TABLE_SPLIT: - case TABLE_COMPACT: - case TABLE_MAJOR_COMPACT: - case TABLE_FLUSH: - if (args != null && args.length > 0) { - if (!(args[0] instanceof ImmutableBytesWritable)) { - throw new IOException( - "request argument must be ImmutableBytesWritable"); - } - Pair pair = null; - if(tableName == null) { - byte [] regionName = ((ImmutableBytesWritable)args[0]).get(); - pair = getTableRegionFromName(regionName); - } else { - byte [] rowKey = ((ImmutableBytesWritable)args[0]).get(); - pair = getTableRegionForRow(tableName, rowKey); - } - if (pair != null && pair.getSecond() != null) { - this.regionManager.startAction(pair.getFirst().getRegionName(), - pair.getFirst(), pair.getSecond(), op); - } - } else { - for (Pair pair: getTableRegions(tableName)) { - if (pair.getSecond() == null) { - continue; // undeployed - } - this.regionManager.startAction(pair.getFirst().getRegionName(), - pair.getFirst(), pair.getSecond(), op); - } - } - break; - - case CLOSE_REGION: - if (args == null || args.length < 1 || args.length > 2) { - throw new IOException("Requires at least a region name; " + - "or cannot have more than region name and servername"); - } - // Arguments are regionname and an optional server name. - byte [] regionname = ((ImmutableBytesWritable)args[0]).get(); - LOG.debug("Attempting to close region: " + Bytes.toStringBinary(regionname)); - String hostnameAndPort = null; - if (args.length == 2) { - hostnameAndPort = Bytes.toString(((ImmutableBytesWritable)args[1]).get()); - } - // Need hri - Result rr = getFromMETA(regionname, HConstants.CATALOG_FAMILY); - HRegionInfo hri = RegionManager.getHRegionInfo(rr.getRow(), rr); - if (hostnameAndPort == null) { - // Get server from the .META. if it wasn't passed as argument - hostnameAndPort = - Bytes.toString(rr.getValue(HConstants.CATALOG_FAMILY, - HConstants.SERVER_QUALIFIER)); - } - // Take region out of the intransistions in case it got stuck there doing - // an open or whatever. - this.regionManager.clearFromInTransition(regionname); - // If hostnameAndPort is still null, then none, exit. - if (hostnameAndPort == null) { - break; - } - long startCode = - Bytes.toLong(rr.getValue(HConstants.CATALOG_FAMILY, - HConstants.STARTCODE_QUALIFIER)); - String name = HServerInfo.getServerName(hostnameAndPort, startCode); - LOG.info("Marking " + hri.getRegionNameAsString() + - " as closing on " + name + "; cleaning SERVER + STARTCODE; " + - "master will tell regionserver to close region on next heartbeat"); - this.regionManager.setClosing(name, hri, hri.isOffline()); - MetaRegion meta = this.regionManager.getMetaRegionForRow(regionname); - HRegionInterface srvr = getMETAServer(meta); - HRegion.cleanRegionInMETA(srvr, meta.getRegionName(), hri); - break; - - default: - throw new IOException("unsupported modifyTable op " + op); - } - } - - /** * @return cluster status */ public ClusterStatus getClusterStatus() { ClusterStatus status = new ClusterStatus(); status.setHBaseVersion(VersionInfo.getVersion()); - status.setServerInfo(serverManager.getServersToServerInfo().values()); + status.setServerInfo(serverManager.getOnlineServers().values()); status.setDeadServers(serverManager.getDeadServers()); - status.setRegionsInTransition(this.regionManager.getRegionsInTransition()); + status.setRegionsInTransition(assignmentManager.getRegionsInTransition()); return status; } @@ -1155,17 +930,6 @@ } } - public Map getTableFragmentation() throws IOException { - long now = System.currentTimeMillis(); - // only check every two minutes by default - int check = this.conf.getInt("hbase.master.fragmentation.check.frequency", 2 * 60 * 1000); - if (lastFragmentationQuery == -1 || now - lastFragmentationQuery > check) { - fragmentation = FSUtils.getTableFragmentation(this); - lastFragmentationQuery = now; - } - return fragmentation; - } - /** * Main program * @param args @@ -1181,6 +945,30 @@ @Override public ZooKeeperWatcher getZooKeeper() { - return zooKeeperWrapper; + return zooKeeper; } + + @Override + public String getServerName() { + return address.toString(); + } + + @Override + public long getTimeout() { + // TODO: use configuration + return 5000; + } + + public CatalogTracker getCatalogTracker() { + return catalogTracker; + } + + @Override + public void shutdown() { + requestShutdown(); + } + + public void assignRegion(HRegionInfo hri) { + assignmentManager.assign(hri); + } } Index: src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java (working copy) @@ -1,78 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableNotDisabledException; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Writables; - -import java.io.IOException; - -/** Instantiated to modify table descriptor metadata */ -class ModifyTableMeta extends TableOperation { - - private static Log LOG = LogFactory.getLog(ModifyTableMeta.class); - - private HTableDescriptor desc; - - ModifyTableMeta(final HMaster master, final byte [] tableName, - HTableDescriptor desc) - throws IOException { - super(master, tableName); - this.desc = desc; - LOG.debug("modifying " + Bytes.toString(tableName) + ": " + - desc.toString()); - } - - protected void updateRegionInfo(HRegionInterface server, byte [] regionName, - HRegionInfo i) - throws IOException { - Put put = new Put(i.getRegionName()); - put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, Writables.getBytes(i)); - server.put(regionName, put); - LOG.debug("updated HTableDescriptor for region " + i.getRegionNameAsString()); - } - - @Override - protected void processScanItem(String serverName, - final HRegionInfo info) throws IOException { - if (isEnabled(info)) { - throw new TableNotDisabledException(Bytes.toString(tableName)); - } - } - - @Override - protected void postProcessMeta(MetaRegion m, HRegionInterface server) - throws IOException { - for (HRegionInfo i: unservedRegions) { - i.setTableDesc(desc); - updateRegionInfo(server, m.getRegionName(), i); - } - // kick off a meta scan right away - master.getRegionManager().metaScannerThread.triggerNow(); - } -} Index: src/main/java/org/apache/hadoop/hbase/master/AddColumn.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/AddColumn.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/AddColumn.java (working copy) @@ -1,50 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.ipc.HRegionInterface; - -import java.io.IOException; - -/** Instantiated to add a column family to a table */ -class AddColumn extends ColumnOperation { - private final HColumnDescriptor newColumn; - - AddColumn(final HMaster master, final byte [] tableName, - final HColumnDescriptor newColumn) - throws IOException { - super(master, tableName); - this.newColumn = newColumn; - } - - @Override - protected void postProcessMeta(MetaRegion m, HRegionInterface server) - throws IOException { - for (HRegionInfo i: unservedRegions) { - // All we need to do to add a column is add it to the table descriptor. - // When the region is brought on-line, it will find the column missing - // and create it. - i.getTableDesc().addFamily(newColumn); - updateRegionInfo(server, m.getRegionName(), i); - } - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java (working copy) @@ -59,7 +59,7 @@ // Number of seconds between each run of the load balancer private final long balancerPeriod; - private Random rand; + private static final Random rand = new Random(); /** * Instantiate the load balancer with the specified configuration. @@ -70,7 +70,6 @@ */ public LoadBalancer(Configuration conf) { balancerPeriod = conf.getLong("hbase.balancer.period", 300000); - rand = new Random(); } /** @@ -197,7 +196,8 @@ List regions = server.getValue(); int numToOffload = Math.min(regionCount - max, regions.size()); for(int i=0; i> bulkAssignment( + public static Map> bulkAssignment( List regions, List servers) { if(regions.size() == 0 || servers.size() == 0) { return null; @@ -499,7 +500,7 @@ * @param servers * @return map of regions to the server it should be assigned to */ - public Map immediateAssignment( + public static Map immediateAssignment( List regions, List servers) { Map assignments = new TreeMap(); @@ -509,6 +510,14 @@ return assignments; } + public static HServerInfo randomAssignment(List servers) { + if(servers == null || servers.isEmpty()) { + LOG.warn("Wanted to do random assignment but no servers to assign to"); + return null; + } + return servers.get(rand.nextInt(servers.size())); + } + /** * Stores the plan for the move of an individual region. * @@ -521,7 +530,7 @@ */ public static class RegionPlan implements Comparable { - private final HRegionInfo region; + private final String regionName; private final HServerInfo source; private HServerInfo dest; @@ -536,8 +545,8 @@ * @param source regionserver region should be moved from * @param dest regionserver region should be moved to */ - public RegionPlan(HRegionInfo region, HServerInfo source, HServerInfo dest) { - this.region = region; + public RegionPlan(String regionName, HServerInfo source, HServerInfo dest) { + this.regionName = regionName; this.source = source; this.dest = dest; } @@ -566,11 +575,11 @@ } /** - * Get the region information for the region this plan is for. - * @return region info + * Get the region name for the region this plan is for. + * @return region name */ - public HRegionInfo getRegionInfo() { - return region; + public String getRegionName() { + return regionName; } /** @@ -579,7 +588,7 @@ */ @Override public int compareTo(RegionPlan o) { - return getRegionInfo().compareTo(o.getRegionInfo()); + return regionName.compareTo(o.getRegionName()); } } } Index: src/main/java/org/apache/hadoop/hbase/master/ProcessRegionClose.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ProcessRegionClose.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ProcessRegionClose.java (working copy) @@ -1,109 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.regionserver.HRegion; - -import java.io.IOException; - -/** - * ProcessRegionClose is the way we do post-processing on a closed region. We - * only spawn one of these asynchronous tasks when the region needs to be - * either offlined or deleted. We used to create one of these tasks whenever - * a region was closed, but since closing a region that isn't being offlined - * or deleted doesn't actually require post processing, it's no longer - * necessary. - */ -public class ProcessRegionClose extends ProcessRegionStatusChange { - protected final boolean offlineRegion; - protected final boolean reassignRegion; - - /** - * @param masterStatus - * @param regionInfo Region to operate on - * @param offlineRegion if true, set the region to offline in meta - * @param reassignRegion if true, region is to be reassigned - */ - public ProcessRegionClose(MasterStatus masterStatus, HRegionInfo regionInfo, - boolean offlineRegion, boolean reassignRegion) { - - super(masterStatus, regionInfo); - this.offlineRegion = offlineRegion; - this.reassignRegion = reassignRegion; - } - - @Override - public String toString() { - return "ProcessRegionClose of " + this.regionInfo.getRegionNameAsString() + - ", " + this.offlineRegion + ", reassign: " + this.reassignRegion; - } - - @Override - protected boolean process() throws IOException { - if (!metaRegionAvailable()) { - // We can't proceed unless the meta region we are going to update - // is online. metaRegionAvailable() has put this operation on the - // delayedToDoQueue, so return true so the operation is not put - // back on the toDoQueue - return true; - } - Boolean result = null; - if (offlineRegion || reassignRegion) { - result = - new RetryableMetaOperation(getMetaRegion(), this.masterStatus) { - public Boolean call() throws IOException { - - - // We can't proceed unless the meta region we are going to update - // is online. metaRegionAvailable() will put this operation on the - // delayedToDoQueue, so return true so the operation is not put - // back on the toDoQueue - - if (metaRegionAvailable()) { - if(offlineRegion) { - // offline the region in meta and then remove it from the - // set of regions in transition - HRegion.offlineRegionInMETA(server, metaRegionName, - regionInfo); - masterStatus.getRegionManager().removeRegion(regionInfo); - LOG.info("region closed: " + regionInfo.getRegionNameAsString()); - } else { - // we are reassigning the region eventually, so set it unassigned - // and remove the server info - HRegion.cleanRegionInMETA(server, metaRegionName, - regionInfo); - masterStatus.getRegionManager().setUnassigned(regionInfo, false); - LOG.info("region set as unassigned: " + regionInfo.getRegionNameAsString()); - } - } - return true; - } - }.doWithRetries(); - result = result == null ? true : result; - - } else { - LOG.info("Region was neither offlined, or asked to be reassigned, what gives: " + - regionInfo.getRegionNameAsString()); - } - - return result == null ? true : result; - } -} Index: src/main/java/org/apache/hadoop/hbase/master/RegionManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/RegionManager.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/RegionManager.java (working copy) @@ -1,1781 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -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.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.HServerLoad; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.hbase.zookeeper.RootRegionTracker; -import org.apache.hadoop.hbase.zookeeper.ZKAssign; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.zookeeper.KeeperException; - -/** - * Class to manage assigning regions to servers, state of root and meta, etc. - */ -public class RegionManager { - protected static final Log LOG = LogFactory.getLog(RegionManager.class); - - private static final boolean ABORT_ON_ZK_ERROR = false; - - private AtomicReference rootRegionLocation = - new AtomicReference(null); - - private final RootScanner rootScannerThread; - final MetaScanner metaScannerThread; - - /** Set by root scanner to indicate the number of meta regions */ - private final AtomicInteger numberOfMetaRegions = new AtomicInteger(); - - /** These are the online meta regions */ - private final NavigableMap onlineMetaRegions = - new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR); - - private static final byte[] OVERLOADED = Bytes.toBytes("Overloaded"); - - private static final byte [] META_REGION_PREFIX = Bytes.toBytes(".META.,"); - - private static int threadWakeFrequency; - - /** - * Map of region name to RegionState for regions that are in transition such as - * - * unassigned -> pendingOpen -> open - * closing -> pendingClose -> closed; if (closed && !offline) -> unassigned - * - * At the end of a transition, removeRegion is used to remove the region from - * the map (since it is no longer in transition) - * - * Note: Needs to be SortedMap so we can specify a comparator - * - * @see RegionState inner-class below - */ - final SortedMap regionsInTransition = - Collections.synchronizedSortedMap(new TreeMap()); - - // regions in transition are also recorded in ZK using the zk watcher - final ZooKeeperWatcher zooKeeper; - - // How many regions to assign a server at a time. - private final int maxAssignInOneGo; - - final MasterStatus masterStatus; - private final LoadBalancer loadBalancer; - final RootRegionTracker rootRegionTracker; - - /** Set of regions to split. */ - private final SortedMap> - regionsToSplit = Collections.synchronizedSortedMap( - new TreeMap> - (Bytes.BYTES_COMPARATOR)); - /** Set of regions to compact. */ - private final SortedMap> - regionsToCompact = Collections.synchronizedSortedMap( - new TreeMap> - (Bytes.BYTES_COMPARATOR)); - /** Set of regions to major compact. */ - private final SortedMap> - regionsToMajorCompact = Collections.synchronizedSortedMap( - new TreeMap> - (Bytes.BYTES_COMPARATOR)); - /** Set of regions to flush. */ - private final SortedMap> - regionsToFlush = Collections.synchronizedSortedMap( - new TreeMap> - (Bytes.BYTES_COMPARATOR)); - private final int zooKeeperNumRetries; - private final int zooKeeperPause; - - RegionManager(MasterStatus masterStatus, RootRegionTracker rootRegionTracker) - throws IOException { - Configuration conf = masterStatus.getConfiguration(); - - this.masterStatus = masterStatus; - this.rootRegionTracker = rootRegionTracker; - threadWakeFrequency = - masterStatus.getConfiguration().getInt( - HConstants.THREAD_WAKE_FREQUENCY, - HConstants.DEFAULT_THREAD_WAKE_FREQUENCY); - this.zooKeeper = masterStatus.getZooKeeper(); - this.maxAssignInOneGo = conf.getInt("hbase.regions.percheckin", 10); - this.loadBalancer = new LoadBalancer(conf); - - // The root region - rootScannerThread = new RootScanner(masterStatus); - - // Scans the meta table - metaScannerThread = new MetaScanner(masterStatus); - - zooKeeperNumRetries = conf.getInt(HConstants.ZOOKEEPER_RETRIES, - HConstants.DEFAULT_ZOOKEEPER_RETRIES); - zooKeeperPause = conf.getInt(HConstants.ZOOKEEPER_PAUSE, - HConstants.DEFAULT_ZOOKEEPER_PAUSE); - } - - void start() { - Threads.setDaemonThreadRunning(rootScannerThread, - "RegionManager.rootScanner"); - Threads.setDaemonThreadRunning(metaScannerThread, - "RegionManager.metaScanner"); - } - - void unsetRootRegion() { - synchronized (regionsInTransition) { - rootRegionLocation.set(null); - regionsInTransition.remove( - HRegionInfo.ROOT_REGIONINFO.getRegionNameAsString()); - LOG.info("-ROOT- region unset (but not set to be reassigned)"); - } - } - - public void reassignRootRegion() { - unsetRootRegion(); - if (!masterStatus.getShutdownRequested().get()) { - synchronized (regionsInTransition) { - String regionName = HRegionInfo.ROOT_REGIONINFO.getRegionNameAsString(); - try { - ZKAssign.createNodeOffline(zooKeeper, - HRegionInfo.ROOT_REGIONINFO.getEncodedName(), HMaster.MASTER); - } catch (KeeperException e) { - LOG.error("Unexpected ZK exception creating offline node when " + - "trying to reassign root region", e); - if(ABORT_ON_ZK_ERROR) masterStatus.abort(); - } - LOG.debug("Created UNASSIGNED zNode " + regionName + " in state " + HBaseEventType.M2ZK_REGION_OFFLINE); - RegionState s = new RegionState(HRegionInfo.ROOT_REGIONINFO, RegionState.State.UNASSIGNED); - regionsInTransition.put(regionName, s); - LOG.info("ROOT inserted into regionsInTransition"); - } - } - } - - /* - * Assigns regions to region servers attempting to balance the load across - * all region servers. Note that no synchronization is necessary as the caller - * (ServerManager.processMsgs) already owns the monitor for the RegionManager. - * - * @param info - * @param mostLoadedRegions - * @param returnMsgs - */ - void assignRegions(HServerInfo info, HRegionInfo[] mostLoadedRegions, - ArrayList returnMsgs) { - HServerLoad thisServersLoad = info.getLoad(); - boolean isSingleServer = this.masterStatus.getServerManager().numServers() == 1; - - // figure out what regions need to be assigned and aren't currently being - // worked on elsewhere. - Set regionsToAssign = - regionsAwaitingAssignment(info.getServerAddress(), isSingleServer); - if (regionsToAssign.size() == 0) { - // There are no regions waiting to be assigned. - this.loadBalancer.loadBalancing(info, mostLoadedRegions, returnMsgs); - } else { - // if there's only one server, just give it all the regions - if (isSingleServer) { - assignRegionsToOneServer(regionsToAssign, info, returnMsgs); - } else { - // otherwise, give this server a few regions taking into account the - // load of all the other servers. - assignRegionsToMultipleServers(thisServersLoad, regionsToAssign, - info, returnMsgs); - } - } - } - - /* - * Make region assignments taking into account multiple servers' loads. - * - * Note that no synchronization is needed while we iterate over - * regionsInTransition because this method is only called by assignRegions - * whose caller owns the monitor for RegionManager - * - * TODO: This code is unintelligible. REWRITE. Add TESTS! St.Ack 09/30/2009 - * @param thisServersLoad - * @param regionsToAssign - * @param info - * @param returnMsgs - */ - private void assignRegionsToMultipleServers(final HServerLoad thisServersLoad, - final Set regionsToAssign, final HServerInfo info, - final ArrayList returnMsgs) { - boolean isMetaAssign = false; - for (RegionState s : regionsToAssign) { - if (s.getRegionInfo().isMetaRegion()) { - isMetaAssign = true; - } - } - int nRegionsToAssign = regionsToAssign.size(); - int otherServersRegionsCount = - regionsToGiveOtherServers(nRegionsToAssign, thisServersLoad); - nRegionsToAssign -= otherServersRegionsCount; - if (nRegionsToAssign > 0 || isMetaAssign) { - LOG.debug("Assigning for " + info + ": total nregions to assign=" + - nRegionsToAssign + ", regions to give other servers than this=" + - otherServersRegionsCount + ", isMetaAssign=" + isMetaAssign); - - // See how many we can assign before this server becomes more heavily - // loaded than the next most heavily loaded server. - HServerLoad heavierLoad = new HServerLoad(); - int nservers = computeNextHeaviestLoad(thisServersLoad, heavierLoad); - int nregions = 0; - // Advance past any less-loaded servers - for (HServerLoad load = new HServerLoad(thisServersLoad); - load.compareTo(heavierLoad) <= 0 && nregions < nRegionsToAssign; - load.setNumberOfRegions(load.getNumberOfRegions() + 1), nregions++) { - // continue; - } - if (nregions < nRegionsToAssign) { - // There are some more heavily loaded servers - // but we can't assign all the regions to this server. - if (nservers > 0) { - // There are other servers that can share the load. - // Split regions that need assignment across the servers. - nregions = (int) Math.ceil((1.0 * nRegionsToAssign)/(1.0 * nservers)); - } else { - // No other servers with same load. - // Split regions over all available servers - nregions = (int) Math.ceil((1.0 * nRegionsToAssign)/ - (1.0 * masterStatus.getServerManager().numServers())); - } - } else { - // Assign all regions to this server - nregions = nRegionsToAssign; - } - LOG.debug("Assigning " + info + " " + nregions + " regions"); - assignRegions(regionsToAssign, nregions, info, returnMsgs); - } - } - - /* - * Assign nregions regions. - * @param regionsToAssign - * @param nregions - * @param info - * @param returnMsgs - */ - private void assignRegions(final Set regionsToAssign, - final int nregions, final HServerInfo info, - final ArrayList returnMsgs) { - int count = nregions; - if (count > this.maxAssignInOneGo) { - count = this.maxAssignInOneGo; - } - for (RegionState s: regionsToAssign) { - doRegionAssignment(s, info, returnMsgs); - if (--count <= 0) { - break; - } - } - } - - /* - * Assign all to the only server. An unlikely case but still possible. - * - * Note that no synchronization is needed on regionsInTransition while - * iterating on it because the only caller is assignRegions whose caller owns - * the monitor for RegionManager - * - * @param regionsToAssign - * @param serverName - * @param returnMsgs - */ - private void assignRegionsToOneServer(final Set regionsToAssign, - final HServerInfo info, final ArrayList returnMsgs) { - for (RegionState s: regionsToAssign) { - doRegionAssignment(s, info, returnMsgs); - } - } - - /* - * Do single region assignment. - * @param rs - * @param sinfo - * @param returnMsgs - */ - private void doRegionAssignment(final RegionState rs, - final HServerInfo sinfo, final ArrayList returnMsgs) { - String regionName = rs.getRegionInfo().getRegionNameAsString(); - LOG.info("Assigning region " + regionName + " to " + sinfo.getServerName()); - rs.setPendingOpen(sinfo.getServerName()); - synchronized (this.regionsInTransition) { - try { - ZKAssign.createNodeOffline(zooKeeper, - rs.getRegionInfo().getEncodedName(), HMaster.MASTER); - } catch (KeeperException e) { - LOG.error("Unexpected ZK exception creating offline node when " + - "trying to create offline node for region", e); - if(ABORT_ON_ZK_ERROR) masterStatus.abort(); - } - LOG.debug("Created UNASSIGNED zNode " + regionName + " in state " + HBaseEventType.M2ZK_REGION_OFFLINE); - this.regionsInTransition.put(regionName, rs); - } - - returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, rs.getRegionInfo())); - } - - /* - * @param nRegionsToAssign - * @param thisServersLoad - * @return How many regions should go to servers other than this one; i.e. - * more lightly loaded servers - */ - private int regionsToGiveOtherServers(final int numUnassignedRegions, - final HServerLoad thisServersLoad) { - SortedMap> lightServers = - new TreeMap>(); - this.masterStatus.getServerManager().getLightServers(thisServersLoad, lightServers); - // Examine the list of servers that are more lightly loaded than this one. - // Pretend that we will assign regions to these more lightly loaded servers - // until they reach load equal with ours. Then, see how many regions are left - // unassigned. That is how many regions we should assign to this server. - int nRegions = 0; - for (Map.Entry> e: lightServers.entrySet()) { - HServerLoad lightLoad = new HServerLoad(e.getKey()); - do { - lightLoad.setNumberOfRegions(lightLoad.getNumberOfRegions() + 1); - nRegions += 1; - } while (lightLoad.compareTo(thisServersLoad) <= 0 - && nRegions < numUnassignedRegions); - nRegions *= e.getValue().size(); - if (nRegions >= numUnassignedRegions) { - break; - } - } - return nRegions; - } - - /* - * Get the set of regions that should be assignable in this pass. - * - * Note that no synchronization on regionsInTransition is needed because the - * only caller (assignRegions, whose caller is ServerManager.processMsgs) owns - * the monitor for RegionManager - */ - private Set regionsAwaitingAssignment(HServerAddress addr, - boolean isSingleServer) { - // set of regions we want to assign to this server - Set regionsToAssign = new HashSet(); - - boolean isMetaServer = isMetaServer(addr); - RegionState rootState = null; - // Handle if root is unassigned... only assign root if root is offline. - synchronized (this.regionsInTransition) { - rootState = regionsInTransition.get(HRegionInfo.ROOT_REGIONINFO.getRegionNameAsString()); - } - if (rootState != null && rootState.isUnassigned()) { - // make sure root isnt assigned here first. - // if so return 'empty list' - // by definition there is no way this could be a ROOT region (since it's - // unassigned) so just make sure it isn't hosting META regions (unless - // it's the only server left). - if (!isMetaServer || isSingleServer) { - regionsToAssign.add(rootState); - } - return regionsToAssign; - } - - // Look over the set of regions that aren't currently assigned to - // determine which we should assign to this server. - boolean reassigningMetas = numberOfMetaRegions.get() != onlineMetaRegions.size(); - boolean isMetaOrRoot = isMetaServer || isRootServer(addr); - if (reassigningMetas && isMetaOrRoot && !isSingleServer) { - return regionsToAssign; // dont assign anything to this server. - } - synchronized (this.regionsInTransition) { - for (RegionState s: regionsInTransition.values()) { - HRegionInfo i = s.getRegionInfo(); - if (i == null) { - continue; - } - if (reassigningMetas && - !i.isMetaRegion()) { - // Can't assign user regions until all meta regions have been assigned - // and are on-line - continue; - } - if (!i.isMetaRegion() && - !masterStatus.getServerManager().canAssignUserRegions()) { - LOG.debug("user region " + i.getRegionNameAsString() + - " is in transition but not enough servers yet"); - continue; - } - if (s.isUnassigned()) { - regionsToAssign.add(s); - } - } - } - return regionsToAssign; - } - - /* - * Figure out the load that is next highest amongst all regionservers. Also, - * return how many servers exist at that load. - */ - private int computeNextHeaviestLoad(HServerLoad referenceLoad, - HServerLoad heavierLoad) { - - SortedMap> heavyServers = - new TreeMap>(); - synchronized (masterStatus.getServerManager().getLoadToServers()) { - heavyServers.putAll( - masterStatus.getServerManager().getLoadToServers().tailMap(referenceLoad)); - } - int nservers = 0; - for (Map.Entry> e : heavyServers.entrySet()) { - Set servers = e.getValue(); - nservers += servers.size(); - if (e.getKey().compareTo(referenceLoad) == 0) { - // This is the load factor of the server we are considering - nservers -= 1; - continue; - } - - // If we get here, we are at the first load entry that is a - // heavier load than the server we are considering - heavierLoad.setNumberOfRequests(e.getKey().getNumberOfRequests()); - heavierLoad.setNumberOfRegions(e.getKey().getNumberOfRegions()); - break; - } - return nservers; - } - - /* - * The server checking in right now is overloaded. We will tell it to close - * some or all of its most loaded regions, allowing it to reduce its load. - * The closed regions will then get picked up by other underloaded machines. - * - * Note that no synchronization is needed because the only caller - * (assignRegions) whose caller owns the monitor for RegionManager - */ - void unassignSomeRegions(final HServerInfo info, - int numRegionsToClose, final HRegionInfo[] mostLoadedRegions, - ArrayList returnMsgs) { - LOG.debug("Unassigning " + numRegionsToClose + " regions from " + - info.getServerName()); - int regionIdx = 0; - int regionsClosed = 0; - int skipped = 0; - while (regionsClosed < numRegionsToClose && - regionIdx < mostLoadedRegions.length) { - HRegionInfo currentRegion = mostLoadedRegions[regionIdx]; - regionIdx++; - // skip the region if it's meta or root - if (currentRegion.isRootRegion() || currentRegion.isMetaTable()) { - continue; - } - final String regionName = currentRegion.getRegionNameAsString(); - if (regionIsInTransition(regionName)) { - skipped++; - continue; - } - if (LOG.isDebugEnabled()) { - LOG.debug("Going to close region " + regionName); - } - // make a message to close the region - returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE, currentRegion, - OVERLOADED)); - // mark the region as closing - setClosing(info.getServerName(), currentRegion, false); - setPendingClose(regionName); - // increment the count of regions we've marked - regionsClosed++; - } - LOG.info("Skipped assigning " + skipped + " region(s) to " + - info.getServerName() + "because already in transition"); - } - - /* - * PathFilter that accepts hbase tables only. - */ - static class TableDirFilter implements PathFilter { - public boolean accept(final Path path) { - // skip the region servers' log dirs && version file - // HBASE-1112 want to separate the log dirs from table's data dirs by a - // special character. - final String pathname = path.getName(); - return (!pathname.equals(HConstants.HREGION_LOGDIR_NAME) - && !pathname.equals(HConstants.VERSION_FILE_NAME)); - } - - } - - /* - * PathFilter that accepts all but compaction.dir names. - */ - static class RegionDirFilter implements PathFilter { - public boolean accept(Path path) { - return !path.getName().equals(HConstants.HREGION_COMPACTIONDIR_NAME); - } - } - - /** - * @return the rough number of the regions on fs - * Note: this method simply counts the regions on fs by accumulating all the dirs - * in each table dir (${HBASE_ROOT}/$TABLE) and skipping logfiles, compaction dirs. - * @throws IOException - */ - public int countRegionsOnFS() throws IOException { - int regions = 0; - FileStatus [] tableDirs = - masterStatus.getFileSystemManager().getFileSystem().listStatus( - this.masterStatus.getFileSystemManager().getRootDir(), new TableDirFilter()); - FileStatus[] regionDirs; - RegionDirFilter rdf = new RegionDirFilter(); - for(FileStatus tabledir : tableDirs) { - if(tabledir.isDir()) { - regionDirs = - masterStatus.getFileSystemManager().getFileSystem().listStatus( - tabledir.getPath(), rdf); - regions += regionDirs.length; - } - } - return regions; - } - - /** - * @return Read-only map of online regions. - */ - public Map getOnlineMetaRegions() { - synchronized (onlineMetaRegions) { - return Collections.unmodifiableMap(onlineMetaRegions); - } - } - - public boolean metaRegionsInTransition() { - synchronized (onlineMetaRegions) { - for (MetaRegion metaRegion : onlineMetaRegions.values()) { - String regionName = Bytes.toString(metaRegion.getRegionName()); - if (regionIsInTransition(regionName)) { - return true; - } - } - } - return false; - } - - /** - * Return a map of the regions in transition on a server. - * Returned map entries are region name -> RegionState - */ - Map getRegionsInTransitionOnServer(String serverName) { - Map ret = new HashMap(); - synchronized (regionsInTransition) { - for (Map.Entry entry : regionsInTransition.entrySet()) { - RegionState rs = entry.getValue(); - if (serverName.equals(rs.getServerName())) { - ret.put(entry.getKey(), rs); - } - } - } - return ret; - } - - /** - * Stop the root and meta scanners so that the region servers serving meta - * regions can shut down. - */ - public void stopScanners() { - this.rootScannerThread.interruptAndStop(); - this.metaScannerThread.interruptAndStop(); - } - - /** Stop the region assigner */ - public void stop() { - try { - if (rootScannerThread.isAlive()) { - rootScannerThread.join(); // Wait for the root scanner to finish. - } - } catch (Exception iex) { - LOG.warn("root scanner", iex); - } - try { - if (metaScannerThread.isAlive()) { - metaScannerThread.join(); // Wait for meta scanner to finish. - } - } catch(Exception iex) { - LOG.warn("meta scanner", iex); - } - try { - ZKUtil.deleteChildrenRecursively(zooKeeper, zooKeeper.rsZNode); - } catch (KeeperException e) { - LOG.error("Unable to delete RS nodes during shutdown", e); - } - } - - /** - * Block until meta regions are online or we're shutting down. - * @return true if we found meta regions, false if we're closing. - */ - public boolean areAllMetaRegionsOnline() { - synchronized (onlineMetaRegions) { - return (rootRegionLocation.get() != null && - numberOfMetaRegions.get() == onlineMetaRegions.size()); - } - } - - /** - * Search our map of online meta regions to find the first meta region that - * should contain a pointer to newRegion. - * @param newRegion - * @return MetaRegion where the newRegion should live - */ - public MetaRegion getFirstMetaRegionForRegion(HRegionInfo newRegion) { - synchronized (onlineMetaRegions) { - if (onlineMetaRegions.size() == 0) { - return null; - } else if (onlineMetaRegions.size() == 1) { - return onlineMetaRegions.get(onlineMetaRegions.firstKey()); - } else { - if (onlineMetaRegions.containsKey(newRegion.getRegionName())) { - return onlineMetaRegions.get(newRegion.getRegionName()); - } - return onlineMetaRegions.get(onlineMetaRegions.headMap( - newRegion.getRegionName()).lastKey()); - } - } - } - - /** - * Get a set of all the meta regions that contain info about a given table. - * @param tableName Table you need to know all the meta regions for - * @return set of MetaRegion objects that contain the table - * @throws NotAllMetaRegionsOnlineException - */ - public Set getMetaRegionsForTable(byte [] tableName) - throws NotAllMetaRegionsOnlineException { - byte [] firstMetaRegion = null; - Set metaRegions = new HashSet(); - if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) { - if (rootRegionLocation.get() == null) { - throw new NotAllMetaRegionsOnlineException( - Bytes.toString(HConstants.ROOT_TABLE_NAME)); - } - metaRegions.add(new MetaRegion(rootRegionLocation.get(), - HRegionInfo.ROOT_REGIONINFO)); - } else { - if (!areAllMetaRegionsOnline()) { - throw new NotAllMetaRegionsOnlineException(); - } - synchronized (onlineMetaRegions) { - if (onlineMetaRegions.size() == 1) { - firstMetaRegion = onlineMetaRegions.firstKey(); - } else if (onlineMetaRegions.containsKey(tableName)) { - firstMetaRegion = tableName; - } else { - firstMetaRegion = onlineMetaRegions.headMap(tableName).lastKey(); - } - metaRegions.addAll(onlineMetaRegions.tailMap(firstMetaRegion).values()); - } - } - return metaRegions; - } - - /** - * Get metaregion that would host passed in row. - * @param row Row need to know all the meta regions for - * @return MetaRegion for passed row. - * @throws NotAllMetaRegionsOnlineException - */ - public MetaRegion getMetaRegionForRow(final byte [] row) - throws NotAllMetaRegionsOnlineException { - if (!areAllMetaRegionsOnline()) { - throw new NotAllMetaRegionsOnlineException(); - } - // Row might be in -ROOT- table. If so, return -ROOT- region. - int prefixlen = META_REGION_PREFIX.length; - if (row.length > prefixlen && - Bytes.compareTo(META_REGION_PREFIX, 0, prefixlen, row, 0, prefixlen) == 0) { - return new MetaRegion(this.masterStatus.getRegionManager().getRootRegionLocation(), - HRegionInfo.ROOT_REGIONINFO); - } - return this.onlineMetaRegions.floorEntry(row).getValue(); - } - - /** - * Create a new HRegion, put a row for it into META (or ROOT), and mark the - * new region unassigned so that it will get assigned to a region server. - * @param newRegion HRegionInfo for the region to create - * @param server server hosting the META (or ROOT) region where the new - * region needs to be noted - * @param metaRegionName name of the meta region where new region is to be - * written - * @throws IOException - */ - public void createRegion(HRegionInfo newRegion, HRegionInterface server, - byte [] metaRegionName) - throws IOException { - // 2. Create the HRegion - HRegion region = HRegion.createHRegion(newRegion, this.masterStatus.getFileSystemManager().getRootDir(), - masterStatus.getConfiguration()); - - // 3. Insert into meta - HRegionInfo info = region.getRegionInfo(); - byte [] regionName = region.getRegionName(); - - Put put = new Put(regionName); - put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, - Writables.getBytes(info)); - server.put(metaRegionName, put); - - // 4. Close the new region to flush it to disk. Close its log file too. - region.close(); - region.getLog().closeAndDelete(); - - // 5. Get it assigned to a server - setUnassigned(info, true); - } - - /** - * Set a MetaRegion as online. - * @param metaRegion - */ - public void putMetaRegionOnline(MetaRegion metaRegion) { - onlineMetaRegions.put(metaRegion.getStartKey(), metaRegion); - } - - /** - * Get a list of online MetaRegions - * @return list of MetaRegion objects - */ - public List getListOfOnlineMetaRegions() { - List regions; - synchronized(onlineMetaRegions) { - regions = new ArrayList(onlineMetaRegions.values()); - } - return regions; - } - - /** - * Count of online meta regions - * @return count of online meta regions - */ - public int numOnlineMetaRegions() { - return onlineMetaRegions.size(); - } - - /** - * Check if a meta region is online by its name - * @param startKey name of the meta region to check - * @return true if the region is online, false otherwise - */ - public boolean isMetaRegionOnline(byte [] startKey) { - return onlineMetaRegions.containsKey(startKey); - } - - /** - * Set an online MetaRegion offline - remove it from the map. - * @param startKey Startkey to use finding region to remove. - * @return the MetaRegion that was taken offline. - */ - public MetaRegion offlineMetaRegionWithStartKey(byte [] startKey) { - LOG.info("META region whose startkey is " + Bytes.toString(startKey) + - " removed from onlineMetaRegions"); - return onlineMetaRegions.remove(startKey); - } - - public boolean isRootServer(HServerAddress server) { - return this.masterStatus.getRegionManager().getRootRegionLocation() != null && - server.equals(masterStatus.getRegionManager().getRootRegionLocation()); - } - - /** - * Returns the list of byte[] start-keys for any .META. regions hosted - * on the indicated server. - * - * @param server server address - * @return list of meta region start-keys. - */ - public List listMetaRegionsForServer(HServerAddress server) { - List metas = new ArrayList(); - for ( MetaRegion region : onlineMetaRegions.values() ) { - if (server.equals(region.getServer())) { - metas.add(region.getStartKey()); - } - } - return metas; - } - - /** - * Does this server have any META regions open on it, or any meta - * regions being assigned to it? - * - * @param server Server IP:port - * @return true if server has meta region assigned - */ - public boolean isMetaServer(HServerAddress server) { - for ( MetaRegion region : onlineMetaRegions.values() ) { - if (server.equals(region.getServer())) { - return true; - } - } - - // This might be expensive, but we need to make sure we dont - // get double assignment to the same regionserver. - synchronized(regionsInTransition) { - for (RegionState s : regionsInTransition.values()) { - if (s.getRegionInfo().isMetaRegion() - && !s.isUnassigned() - && s.getServerName() != null - && s.getServerName().equals(server.toString())) { - // TODO this code appears to be entirely broken, since - // server.toString() has no start code, but s.getServerName() - // does! - LOG.fatal("I DONT BELIEVE YOU WILL EVER SEE THIS!"); - // Has an outstanding meta region to be assigned. - return true; - } - } - } - return false; - } - - /** - * Is this server assigned to transition the ROOT table. HBASE-1928 - * - * @param server Server - * @return true if server is transitioning the ROOT table - */ - public boolean isRootInTransitionOnThisServer(final String server) { - synchronized (this.regionsInTransition) { - for (RegionState s : regionsInTransition.values()) { - if (s.getRegionInfo().isRootRegion() - && !s.isUnassigned() - && s.getServerName() != null - && s.getServerName().equals(server)) { - // Has an outstanding root region to be assigned. - return true; - } - } - } - return false; - } - - /** - * Is this server assigned to transition a META table. HBASE-1928 - * - * @param server Server - * @return if this server was transitioning a META table then a not null HRegionInfo pointing to it - */ - public HRegionInfo getMetaServerRegionInfo(final String server) { - synchronized (this.regionsInTransition) { - for (RegionState s : regionsInTransition.values()) { - if (s.getRegionInfo().isMetaRegion() - && !s.isUnassigned() - && s.getServerName() != null - && s.getServerName().equals(server)) { - // Has an outstanding meta region to be assigned. - return s.getRegionInfo(); - } - } - } - return null; - } - - /** - * Call to take this metaserver offline for immediate reassignment. Used only - * when we know a region has shut down cleanly. - * - * A meta server is a server that hosts either -ROOT- or any .META. regions. - * - * If you are considering a unclean shutdown potentially, use ProcessServerShutdown which - * calls other methods to immediately unassign root/meta but delay the reassign until the - * log has been split. - * - * @param server the server that went down - * @return true if this was in fact a meta server, false if it did not carry meta regions. - */ - public synchronized boolean offlineMetaServer(HServerAddress server) { - boolean hasMeta = false; - - // check to see if ROOT and/or .META. are on this server, reassign them. - // use master.getRootRegionLocation. - if (masterStatus.getRegionManager().getRootRegionLocation() != null && - server.equals(masterStatus.getRegionManager().getRootRegionLocation())) { - LOG.info("Offlined ROOT server: " + server); - reassignRootRegion(); - hasMeta = true; - } - // AND - for ( MetaRegion region : onlineMetaRegions.values() ) { - if (server.equals(region.getServer())) { - LOG.info("Offlining META region: " + region); - offlineMetaRegionWithStartKey(region.getStartKey()); - // Set for reassignment. - setUnassigned(region.getRegionInfo(), true); - hasMeta = true; - } - } - return hasMeta; - } - - /** - * Remove a region from the region state map. - * - * @param info - */ - public void removeRegion(HRegionInfo info) { - synchronized (this.regionsInTransition) { - this.regionsInTransition.remove(info.getRegionNameAsString()); - } - } - - /** - * @param regionName - * @return true if the named region is in a transition state - */ - public boolean regionIsInTransition(String regionName) { - synchronized (this.regionsInTransition) { - return regionsInTransition.containsKey(regionName); - } - } - - /** - * @param regionName - * @return true if the region is unassigned, pendingOpen or open - */ - public boolean regionIsOpening(String regionName) { - synchronized (this.regionsInTransition) { - RegionState state = regionsInTransition.get(regionName); - if (state != null) { - return state.isOpening(); - } - } - return false; - } - - /** - * Set a region to unassigned - * @param info Region to set unassigned - * @param force if true mark region unassigned whatever its current state - */ - public void setUnassigned(HRegionInfo info, boolean force) { - RegionState s = null; - synchronized(this.regionsInTransition) { - s = regionsInTransition.get(info.getRegionNameAsString()); - if (s == null) { - try { - ZKAssign.createNodeOffline(zooKeeper, - info.getEncodedName(), HMaster.MASTER); - } catch (KeeperException e) { - LOG.error("Unexpected ZK exception creating offline node when " + - "trying to reassign root region", e); - if(ABORT_ON_ZK_ERROR) masterStatus.abort(); - } - LOG.debug("Created UNASSIGNED zNode " + info.getRegionNameAsString() + " in state " + HBaseEventType.M2ZK_REGION_OFFLINE); - s = new RegionState(info, RegionState.State.UNASSIGNED); - regionsInTransition.put(info.getRegionNameAsString(), s); - } - } - if (force || (!s.isPendingOpen() && !s.isOpen())) { - s.setUnassigned(); - } - } - - /** - * Check if a region is on the unassigned list - * @param info HRegionInfo to check for - * @return true if on the unassigned list, false if it isn't. Note that this - * means a region could not be on the unassigned list AND not be assigned, if - * it happens to be between states. - */ - public boolean isUnassigned(HRegionInfo info) { - synchronized (regionsInTransition) { - RegionState s = regionsInTransition.get(info.getRegionNameAsString()); - if (s != null) { - return s.isUnassigned(); - } - } - return false; - } - - /** - * Check if a region has been assigned and we're waiting for a response from - * the region server. - * - * @param regionName name of the region - * @return true if open, false otherwise - */ - public boolean isPendingOpen(String regionName) { - synchronized (regionsInTransition) { - RegionState s = regionsInTransition.get(regionName); - if (s != null) { - return s.isPendingOpen(); - } - } - return false; - } - - /** - * Region has been assigned to a server and the server has told us it is open - * @param regionName - */ - public void setOpen(String regionName) { - synchronized (regionsInTransition) { - RegionState s = regionsInTransition.get(regionName); - if (s != null) { - s.setOpen(); - } - } - } - - /** - * @param regionName - * @return true if region is marked to be offlined. - */ - public boolean isOfflined(String regionName) { - synchronized (regionsInTransition) { - RegionState s = regionsInTransition.get(regionName); - if (s != null) { - return s.isOfflined(); - } - } - return false; - } - - /** - * Mark a region as closing - * @param serverName - * @param regionInfo - * @param setOffline - */ - public void setClosing(String serverName, final HRegionInfo regionInfo, - final boolean setOffline) { - synchronized (this.regionsInTransition) { - RegionState s = - this.regionsInTransition.get(regionInfo.getRegionNameAsString()); - if (s == null) { - s = new RegionState(regionInfo, RegionState.State.CLOSING); - } - // If region was asked to open before getting here, we could be taking - // the wrong server name - if(s.isPendingOpen()) { - serverName = s.getServerName(); - } - s.setClosing(serverName, setOffline); - this.regionsInTransition.put(regionInfo.getRegionNameAsString(), s); - } - } - - /** - * Remove the map of region names to region infos waiting to be offlined for a - * given server - * - * @param serverName - * @return set of infos to close - */ - public Set getMarkedToClose(String serverName) { - Set result = new HashSet(); - synchronized (regionsInTransition) { - for (RegionState s: regionsInTransition.values()) { - if (s.isClosing() && !s.isPendingClose() && !s.isClosed() && - s.getServerName().compareTo(serverName) == 0) { - result.add(s.getRegionInfo()); - } - } - } - return result; - } - - /** - * Called when we have told a region server to close the region - * - * @param regionName - */ - public void setPendingClose(String regionName) { - synchronized (regionsInTransition) { - RegionState s = regionsInTransition.get(regionName); - if (s != null) { - s.setPendingClose(); - } - } - } - - /** - * @param regionName - */ - public void setClosed(String regionName) { - synchronized (regionsInTransition) { - RegionState s = regionsInTransition.get(regionName); - if (s != null) { - s.setClosed(); - } - } - } - /** - * Add a meta region to the scan queue - * @param m MetaRegion that needs to get scanned - */ - public void addMetaRegionToScan(MetaRegion m) { - metaScannerThread.addMetaRegionToScan(m); - } - - /** - * Check if the initial root scan has been completed. - * @return true if scan completed, false otherwise - */ - public boolean isInitialRootScanComplete() { - return rootScannerThread.isInitialScanComplete(); - } - - /** - * Check if the initial meta scan has been completed. - * @return true if meta completed, false otherwise - */ - public boolean isInitialMetaScanComplete() { - return metaScannerThread.isInitialScanComplete(); - } - - /** - * Get the root region location. - * @return HServerAddress describing root region server. - */ - public HServerAddress getRootRegionLocation() { - return rootRegionLocation.get(); - } - - /** - * Block until either the root region location is available or we're shutting - * down. - */ - public void waitForRootRegionLocation() { - synchronized (rootRegionLocation) { - while (!masterStatus.getShutdownRequested().get() && - !masterStatus.isClosed() && rootRegionLocation.get() == null) { - // rootRegionLocation will be filled in when we get an 'open region' - // regionServerReport message from the HRegionServer that has been - // allocated the ROOT region below. - try { - // Cycle rather than hold here in case master is closed meantime. - rootRegionLocation.wait(threadWakeFrequency); - } catch (InterruptedException e) { - // continue - } - } - } - } - - /** - * Return the number of meta regions. - * @return number of meta regions - */ - public int numMetaRegions() { - return numberOfMetaRegions.get(); - } - - /** - * Bump the count of meta regions up one - */ - public void incrementNumMetaRegions() { - numberOfMetaRegions.incrementAndGet(); - } - - private long getPauseTime(int tries) { - int attempt = tries; - if (attempt >= HConstants.RETRY_BACKOFF.length) { - attempt = HConstants.RETRY_BACKOFF.length - 1; - } - return this.zooKeeperPause * HConstants.RETRY_BACKOFF[attempt]; - } - - private void sleep(int attempt) { - try { - Thread.sleep(getPauseTime(attempt)); - } catch (InterruptedException e) { - // continue - } - } - - private void writeRootRegionLocationToZooKeeper(HServerAddress address) { - for (int attempt = 0; attempt < zooKeeperNumRetries; ++attempt) { - try { - rootRegionTracker.setRootRegionLocation(address); - return; - } catch (KeeperException e) { - LOG.info("ZK exception writing root region location", e); - sleep(attempt); - } - } - - LOG.error("Failed to write root region location to ZooKeeper after " + - zooKeeperNumRetries + " retries, shutting down"); - - this.masterStatus.shutdown(); - } - - /** - * Set the root region location. - * @param address Address of the region server where the root lives - */ - public void setRootRegionLocation(HServerAddress address) { - writeRootRegionLocationToZooKeeper(address); - synchronized (rootRegionLocation) { - // the root region has been assigned, remove it from transition in ZK - try { - ZKAssign.deleteOpenedNode(zooKeeper, - HRegionInfo.ROOT_REGIONINFO.getEncodedName()); - } catch (KeeperException e) { - LOG.error("Exception deleting root region unassigned node", e); - if(ABORT_ON_ZK_ERROR) masterStatus.abort(); - } - rootRegionLocation.set(new HServerAddress(address)); - rootRegionLocation.notifyAll(); - } - } - - /** - * Set the number of meta regions. - * @param num Number of meta regions - */ - public void setNumMetaRegions(int num) { - numberOfMetaRegions.set(num); - } - - /** - * @param regionName - * @param info - * @param server - * @param op - */ - public void startAction(byte[] regionName, HRegionInfo info, - HServerAddress server, HConstants.Modify op) { - if (LOG.isDebugEnabled()) { - LOG.debug("Adding operation " + op + " from tasklist"); - } - switch (op) { - case TABLE_SPLIT: - startAction(regionName, info, server, this.regionsToSplit); - break; - case TABLE_COMPACT: - startAction(regionName, info, server, this.regionsToCompact); - break; - case TABLE_MAJOR_COMPACT: - startAction(regionName, info, server, this.regionsToMajorCompact); - break; - case TABLE_FLUSH: - startAction(regionName, info, server, this.regionsToFlush); - break; - default: - throw new IllegalArgumentException("illegal table action " + op); - } - } - - private void startAction(final byte[] regionName, final HRegionInfo info, - final HServerAddress server, - final SortedMap> map) { - map.put(regionName, new Pair(info, server)); - } - - /** - * @param regionName - * @param op - */ - public void endAction(byte[] regionName, HConstants.Modify op) { - if (LOG.isDebugEnabled()) { - LOG.debug("Removing operation " + op + " from tasklist"); - } - switch (op) { - case TABLE_SPLIT: - this.regionsToSplit.remove(regionName); - break; - case TABLE_COMPACT: - this.regionsToCompact.remove(regionName); - break; - case TABLE_MAJOR_COMPACT: - this.regionsToMajorCompact.remove(regionName); - break; - case TABLE_FLUSH: - this.regionsToFlush.remove(regionName); - break; - default: - throw new IllegalArgumentException("illegal table action " + op); - } - } - - /** - * @param regionName - */ - public void endActions(byte[] regionName) { - regionsToSplit.remove(regionName); - regionsToCompact.remove(regionName); - } - - /** - * Send messages to the given region server asking it to split any - * regions in 'regionsToSplit', etc. - * @param serverInfo - * @param returnMsgs - */ - public void applyActions(HServerInfo serverInfo, ArrayList returnMsgs) { - applyActions(serverInfo, returnMsgs, this.regionsToCompact, - HMsg.Type.MSG_REGION_COMPACT); - applyActions(serverInfo, returnMsgs, this.regionsToSplit, - HMsg.Type.MSG_REGION_SPLIT); - applyActions(serverInfo, returnMsgs, this.regionsToFlush, - HMsg.Type.MSG_REGION_FLUSH); - applyActions(serverInfo, returnMsgs, this.regionsToMajorCompact, - HMsg.Type.MSG_REGION_MAJOR_COMPACT); - } - - private void applyActions(final HServerInfo serverInfo, - final ArrayList returnMsgs, - final SortedMap> map, - final HMsg.Type msg) { - HServerAddress addr = serverInfo.getServerAddress(); - synchronized (map) { - Iterator> i = map.values().iterator(); - while (i.hasNext()) { - Pair pair = i.next(); - if (addr.equals(pair.getSecond())) { - if (LOG.isDebugEnabled()) { - LOG.debug("Sending " + msg + " " + pair.getFirst() + " to " + addr); - } - returnMsgs.add(new HMsg(msg, pair.getFirst())); - i.remove(); - } - } - } - } - - /** - * Class to balance region servers load. - * It keeps Region Servers load in slop range by unassigning Regions - * from most loaded servers. - * - * Equilibrium is reached when load of all serves are in slop range - * [avgLoadMinusSlop, avgLoadPlusSlop], where - * avgLoadPlusSlop = Math.ceil(avgLoad * (1 + this.slop)), and - * avgLoadMinusSlop = Math.floor(avgLoad * (1 - this.slop)) - 1. - */ - private class LoadBalancer { - private float slop; // hbase.regions.slop - private final int maxRegToClose; // hbase.regions.close.max - - LoadBalancer(Configuration conf) { - this.slop = conf.getFloat("hbase.regions.slop", (float)0.3); - if (this.slop <= 0) { - this.slop = 1; - } - //maxRegToClose to constrain balance closing per one iteration - // -1 to turn off - // TODO: change default in HBASE-862, need a suggestion - this.maxRegToClose = conf.getInt("hbase.regions.close.max", -1); - } - - /** - * Balance server load by unassigning some regions. - * - * @param info - server info - * @param mostLoadedRegions - array of most loaded regions - * @param returnMsgs - array of return massages - */ - void loadBalancing(HServerInfo info, HRegionInfo[] mostLoadedRegions, - ArrayList returnMsgs) { - HServerLoad servLoad = info.getLoad(); - double avg = masterStatus.getServerManager().getAverageLoad(); - - // nothing to balance if server load not more then average load - if(servLoad.getLoad() <= Math.ceil(avg) || avg <= 2.0) { - return; - } - - // check if current server is overloaded - int numRegionsToClose = balanceFromOverloaded(info.getServerName(), - servLoad, avg); - - // check if we can unload server by low loaded servers - if(numRegionsToClose <= 0) { - numRegionsToClose = balanceToLowloaded(info.getServerName(), servLoad, - avg); - } - - if(maxRegToClose > 0) { - numRegionsToClose = Math.min(numRegionsToClose, maxRegToClose); - } - - if(numRegionsToClose > 0) { - unassignSomeRegions(info, numRegionsToClose, mostLoadedRegions, - returnMsgs); - } - } - - /* - * Check if server load is not overloaded (with load > avgLoadPlusSlop). - * @return number of regions to unassign. - */ - private int balanceFromOverloaded(final String serverName, - HServerLoad srvLoad, double avgLoad) { - int avgLoadPlusSlop = (int)Math.ceil(avgLoad * (1 + this.slop)); - int numSrvRegs = srvLoad.getNumberOfRegions(); - if (numSrvRegs > avgLoadPlusSlop) { - if (LOG.isDebugEnabled()) { - LOG.debug("Server " + serverName + " is carrying more than its fair " + - "share of regions: " + - "load=" + numSrvRegs + ", avg=" + avgLoad + ", slop=" + this.slop); - } - return numSrvRegs - (int)Math.ceil(avgLoad); - } - return 0; - } - - /* - * Check if server is most loaded and can be unloaded to - * low loaded servers (with load < avgLoadMinusSlop). - * @return number of regions to unassign. - */ - private int balanceToLowloaded(String srvName, HServerLoad srvLoad, - double avgLoad) { - - SortedMap> loadToServers = - masterStatus.getServerManager().getLoadToServers(); - // check if server most loaded - if (!loadToServers.get(loadToServers.lastKey()).contains(srvName)) { - return 0; - } - - // this server is most loaded, we will try to unload it by lowest - // loaded servers - int avgLoadMinusSlop = (int)Math.floor(avgLoad * (1 - this.slop)) - 1; - int lowestLoad = loadToServers.firstKey().getNumberOfRegions(); - - if(lowestLoad >= avgLoadMinusSlop) { - return 0; // there is no low loaded servers - } - - int lowSrvCount = loadToServers.get(loadToServers.firstKey()).size(); - int numRegionsToClose = 0; - - int numSrvRegs = srvLoad.getNumberOfRegions(); - int numMoveToLowLoaded = (avgLoadMinusSlop - lowestLoad) * lowSrvCount; - numRegionsToClose = numSrvRegs - (int)Math.ceil(avgLoad); - numRegionsToClose = Math.min(numRegionsToClose, numMoveToLowLoaded); - if (LOG.isDebugEnabled()) { - LOG.debug("Server(s) are carrying only " + lowestLoad + " regions. " + - "Server " + srvName + " is most loaded (" + numSrvRegs + - "). Shedding " + numRegionsToClose + " regions to pass to " + - " least loaded (numMoveToLowLoaded=" + numMoveToLowLoaded +")"); - } - return numRegionsToClose; - } - } - - /** - * @return Snapshot of regionsintransition as a sorted Map. - */ - NavigableMap getRegionsInTransition() { - NavigableMap result = new TreeMap(); - synchronized (this.regionsInTransition) { - if (this.regionsInTransition.isEmpty()) { - return result; - } - for (Map.Entry e: this.regionsInTransition.entrySet()) { - result.put(e.getKey(), e.getValue().toString()); - } - } - return result; - } - - /** - * @param regionname Name to clear from regions in transistion. - * @return True if we removed an element for the passed regionname. - */ - boolean clearFromInTransition(final byte [] regionname) { - boolean result = false; - synchronized (this.regionsInTransition) { - if (this.regionsInTransition.isEmpty()) { - return result; - } - for (Map.Entry e: this.regionsInTransition.entrySet()) { - if (Bytes.equals(regionname, e.getValue().getRegionName())) { - this.regionsInTransition.remove(e.getKey()); - LOG.debug("Removed " + e.getKey() + ", " + e.getValue()); - result = true; - break; - } - } - } - return result; - } - - /* - * State of a Region as it transitions from closed to open, etc. See - * note on regionsInTransition data member above for listing of state - * transitions. - */ - static class RegionState implements Comparable { - private final HRegionInfo regionInfo; - - enum State { - UNASSIGNED, // awaiting a server to be assigned - PENDING_OPEN, // told a server to open, hasn't opened yet - OPEN, // has been opened on RS, but not yet marked in META/ROOT - CLOSING, // a msg has been enqueued to close ths region, but not delivered to RS yet - PENDING_CLOSE, // msg has been delivered to RS to close this region - CLOSED // region has been closed but not yet marked in meta - - } - - private State state; - - private boolean isOfflined; - - /* Set when region is assigned or closing */ - private String serverName = null; - - /* Constructor */ - RegionState(HRegionInfo info, State state) { - this.regionInfo = info; - this.state = state; - } - - synchronized HRegionInfo getRegionInfo() { - return this.regionInfo; - } - - synchronized byte [] getRegionName() { - return this.regionInfo.getRegionName(); - } - - /* - * @return Server this region was assigned to - */ - synchronized String getServerName() { - return this.serverName; - } - - /* - * @return true if the region is being opened - */ - synchronized boolean isOpening() { - return state == State.UNASSIGNED || - state == State.PENDING_OPEN || - state == State.OPEN; - } - - /* - * @return true if region is unassigned - */ - synchronized boolean isUnassigned() { - return state == State.UNASSIGNED; - } - - /* - * Note: callers of this method (reassignRootRegion, - * regionsAwaitingAssignment, setUnassigned) ensure that this method is not - * called unless it is safe to do so. - */ - synchronized void setUnassigned() { - state = State.UNASSIGNED; - this.serverName = null; - } - - synchronized boolean isPendingOpen() { - return state == State.PENDING_OPEN; - } - - /* - * @param serverName Server region was assigned to. - */ - synchronized void setPendingOpen(final String serverName) { - if (state != State.UNASSIGNED) { - LOG.warn("Cannot assign a region that is not currently unassigned. " + - "FIX!! State: " + toString()); - } - state = State.PENDING_OPEN; - this.serverName = serverName; - } - - synchronized boolean isOpen() { - return state == State.OPEN; - } - - synchronized void setOpen() { - if (state != State.PENDING_OPEN) { - LOG.warn("Cannot set a region as open if it has not been pending. " + - "FIX!! State: " + toString()); - } - state = State.OPEN; - } - - synchronized boolean isClosing() { - return state == State.CLOSING; - } - - synchronized void setClosing(String serverName, boolean setOffline) { - state = State.CLOSING; - this.serverName = serverName; - this.isOfflined = setOffline; - } - - synchronized boolean isPendingClose() { - return state == State.PENDING_CLOSE; - } - - synchronized void setPendingClose() { - if (state != State.CLOSING) { - LOG.warn("Cannot set a region as pending close if it has not been " + - "closing. FIX!! State: " + toString()); - } - state = State.PENDING_CLOSE; - } - - synchronized boolean isClosed() { - return state == State.CLOSED; - } - - synchronized void setClosed() { - if (state != State.PENDING_CLOSE && - state != State.PENDING_OPEN && - state != State.CLOSING) { - throw new IllegalStateException( - "Cannot set a region to be closed if it was not already marked as" + - " pending close, pending open or closing. State: " + this); - } - state = State.CLOSED; - } - - synchronized boolean isOfflined() { - return (state == State.CLOSING || - state == State.PENDING_CLOSE) && isOfflined; - } - - @Override - public synchronized String toString() { - return ("name=" + Bytes.toString(getRegionName()) + - ", state=" + this.state); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - return this.compareTo((RegionState) o) == 0; - } - - @Override - public int hashCode() { - return Bytes.toString(getRegionName()).hashCode(); - } - - public int compareTo(RegionState o) { - if (o == null) { - return 1; - } - return Bytes.compareTo(getRegionName(), o.getRegionName()); - } - } - - /* - * When we find rows in a meta region that has an empty HRegionInfo, we - * clean them up here. - * - * @param s connection to server serving meta region - * @param metaRegionName name of the meta region we scanned - * @param emptyRows the row keys that had empty HRegionInfos - */ - public static void deleteEmptyMetaRows(HRegionInterface s, - byte [] metaRegionName, - List emptyRows) { - for (byte [] regionName: emptyRows) { - try { - HRegion.removeRegionFromMETA(s, metaRegionName, regionName); - LOG.warn("Removed region: " + Bytes.toString(regionName) + - " from meta region: " + - Bytes.toString(metaRegionName) + " because HRegionInfo was empty"); - } catch (IOException e) { - LOG.error("deleting region: " + Bytes.toString(regionName) + - " from meta region: " + Bytes.toString(metaRegionName), e); - } - } - } - - // TODO ryan rework this function - /* - * Get HRegionInfo from passed META map of row values. - * Returns null if none found (and logs fact that expected COL_REGIONINFO - * was missing). Utility method used by scanners of META tables. - * @param row name of the row - * @param map Map to do lookup in. - * @return Null or found HRegionInfo. - * @throws IOException - */ - public static HRegionInfo getHRegionInfo(final byte [] row, final Result res) - throws IOException { - byte[] regioninfo = res.getValue(HConstants.CATALOG_FAMILY, - HConstants.REGIONINFO_QUALIFIER); - if (regioninfo == null) { - StringBuilder sb = new StringBuilder(); - NavigableMap infoMap = - res.getFamilyMap(HConstants.CATALOG_FAMILY); - for (byte [] e: infoMap.keySet()) { - if (sb.length() > 0) { - sb.append(", "); - } - sb.append(Bytes.toString(HConstants.CATALOG_FAMILY) + ":" - + Bytes.toString(e)); - } - LOG.warn(Bytes.toString(HConstants.CATALOG_FAMILY) + ":" + - Bytes.toString(HConstants.REGIONINFO_QUALIFIER) - + " is empty for row: " + Bytes.toString(row) + "; has keys: " - + sb.toString()); - return null; - } - return Writables.getHRegionInfo(regioninfo); - } -} Index: src/main/java/org/apache/hadoop/hbase/master/ColumnOperation.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ColumnOperation.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ColumnOperation.java (working copy) @@ -1,58 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.TableNotDisabledException; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.util.Writables; - -import java.io.IOException; - -abstract class ColumnOperation extends TableOperation { - private final Log LOG = LogFactory.getLog(this.getClass()); - - protected ColumnOperation(final HMaster master, final byte [] tableName) - throws IOException { - super(master, tableName); - } - - @Override - protected void processScanItem(String serverName, final HRegionInfo info) - throws IOException { - if (isEnabled(info)) { - throw new TableNotDisabledException(tableName); - } - } - - protected void updateRegionInfo(HRegionInterface server, byte [] regionName, - HRegionInfo i) throws IOException { - Put put = new Put(i.getRegionName()); - put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, Writables.getBytes(i)); - server.put(regionName, put); - if (LOG.isDebugEnabled()) { - LOG.debug("updated columns in row: " + i.getRegionNameAsString()); - } - } -} Index: src/main/java/org/apache/hadoop/hbase/master/FileSystemManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/FileSystemManager.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/FileSystemManager.java (working copy) @@ -34,13 +34,14 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; /** - * This class abstract a bunch of operations the HMaster needs to interact with - * the underlying file system, including splitting log files, checking file + * This class abstract a bunch of operations the HMaster needs to interact with + * the underlying file system, including splitting log files, checking file * system status, etc. */ public class FileSystemManager { @@ -48,7 +49,7 @@ // HBase configuration Configuration conf; // master status - MasterStatus masterStatus; + MasterController masterStatus; // Keep around for convenience. private final FileSystem fs; // Is the fileystem ok? @@ -59,8 +60,8 @@ private final Path rootdir; // create the split log lock final Lock splitLogLock = new ReentrantLock(); - - public FileSystemManager(Configuration conf, MasterStatus masterStatus) throws IOException { + + public FileSystemManager(Configuration conf, MasterController masterStatus) throws IOException { this.conf = conf; this.masterStatus = masterStatus; // Set filesystem to be that of this.rootdir else we get complaints about @@ -105,7 +106,7 @@ public Path getOldLogDir() { return this.oldLogDir; } - + /** * Checks to see if the file system is still accessible. * If not, sets closed @@ -123,7 +124,7 @@ } return this.fsOk; } - + /** * @return HBase root dir. * @throws IOException @@ -131,20 +132,22 @@ public Path getRootDir() { return this.rootdir; } - + public Lock getSplitLogLock() { return splitLogLock; } - - /* + + /** * Inspect the log directory to recover any log file without - * ad active region server. + * an active region server. */ public void splitLogAfterStartup() { Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME); try { - if (!this.fs.exists(logsDirPath)) return; + if (!this.fs.exists(logsDirPath)) { + return; + } } catch (IOException e) { throw new RuntimeException("Could exists for " + logsDirPath, e); } @@ -179,8 +182,8 @@ } } } - - /* + + /** * Get the rootdir. Make sure its wholesome and exists before returning. * @param rd * @param conf @@ -238,7 +241,7 @@ } } - /* + /** * @param hri Set all family block caching to b * @param b */ @@ -250,4 +253,29 @@ } } } + + public void deleteRegion(HRegionInfo region) throws IOException { + fs.delete(HRegion.getRegionDir(rootdir, region), true); + } + + public void deleteTable(byte[] tableName) throws IOException { + fs.delete(new Path(rootdir, Bytes.toString(tableName)), true); + } + + public void updateRegionInfo(HRegionInfo region) { + // TODO implement this. i think this is currently broken in trunk i don't + // see this getting updated. + // @see HRegion.checkRegioninfoOnFilesystem() + } + + public void addFamily(HRegionInfo region, byte[] familyName) { + // TODO Looks like the family directory is just created on the first flush? + } + + public void deleteFamily(HRegionInfo region, byte[] familyName) + throws IOException { + fs.delete(Store.getStoreHomedir( + new Path(rootdir, region.getTableDesc().getNameAsString()), + region.getEncodedName(), familyName), true); + } } Index: src/main/java/org/apache/hadoop/hbase/master/DeleteColumn.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/DeleteColumn.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/DeleteColumn.java (working copy) @@ -1,54 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.regionserver.Store; - -import java.io.IOException; - -/** Instantiated to remove a column family from a table */ -class DeleteColumn extends ColumnOperation { - private final byte [] columnName; - - DeleteColumn(final HMaster master, final byte [] tableName, - final byte [] columnName) - throws IOException { - super(master, tableName); - this.columnName = columnName; - } - - @Override - protected void postProcessMeta(MetaRegion m, HRegionInterface server) - throws IOException { - for (HRegionInfo i: unservedRegions) { - i.getTableDesc().removeFamily(columnName); - updateRegionInfo(server, m.getRegionName(), i); - // Delete the directories used by the column - Path tabledir = - new Path(this.master.getFileSystemManager().getRootDir(), i.getTableDesc().getNameAsString()); - this.master.getFileSystemManager().getFileSystem(). - delete(Store.getStoreHomedir(tabledir, i.getEncodedName(), - this.columnName), true); - } - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java (working copy) @@ -1,600 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.hbase.Chore; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.UnknownScannerException; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.BooleanWritable; -import org.apache.hadoop.ipc.RemoteException; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - - -/** - * Base HRegion scanner class. Holds utilty common to ROOT and - * META HRegion scanners. - * - *

        How do we know if all regions are assigned? After the initial scan of - * the ROOT and META regions, all regions known at - * that time will have been or are in the process of being assigned.

        - * - *

        When a region is split the region server notifies the master of the - * split and the new regions are assigned. But suppose the master loses the - * split message? We need to periodically rescan the ROOT and - * META regions. - *

          - *
        • If we rescan, any regions that are new but not assigned will have - * no server info. Any regions that are not being served by the same - * server will get re-assigned.
        • - * - *
        • Thus a periodic rescan of the root region will find any new - * META regions where we missed the META split - * message or we failed to detect a server death and consequently need to - * assign the region to a new server.
        • - * - *
        • if we keep track of all the known META regions, then - * we can rescan them periodically. If we do this then we can detect any - * regions for which we missed a region split message.
        • - *
        - * - * Thus just keeping track of all the META regions permits - * periodic rescanning which will detect unassigned regions (new or - * otherwise) without the need to keep track of every region.

        - * - *

        So the ROOT region scanner needs to wake up: - *

          - *
        1. when the master receives notification that the ROOT - * region has been opened.
        2. - *
        3. periodically after the first scan
        4. - *
        - * - * The META scanner needs to wake up: - *
          - *
        1. when a META region comes on line
        2. - * periodically to rescan the online META regions - *
        - * - *

        A META region is not 'online' until it has been scanned - * once. - */ -abstract class BaseScanner extends Chore { - static final Log LOG = LogFactory.getLog(BaseScanner.class.getName()); - // These are names of new columns in a meta region offlined parent row. They - // are added by the metascanner after we verify that split daughter made it - // in. Their value is 'true' if present. - private static final byte[] SPLITA_CHECKED = - Bytes.toBytes(Bytes.toString(HConstants.SPLITA_QUALIFIER) + "_checked"); - private static final byte[] SPLITB_CHECKED = - Bytes.toBytes(Bytes.toString(HConstants.SPLITB_QUALIFIER) + "_checked"); - // Make the 'true' Writable once only. - private static byte[] TRUE_WRITABLE_AS_BYTES; - static { - try { - TRUE_WRITABLE_AS_BYTES = Writables.getBytes(new BooleanWritable(true)); - } catch (IOException e) { - e.printStackTrace(); - } - } - private final boolean rootRegion; - protected final MasterStatus masterStatus; - - protected boolean initialScanComplete; - - protected abstract boolean initialScan(); - protected abstract void maintenanceScan(); - - // will use this variable to synchronize and make sure we aren't interrupted - // mid-scan - final Object scannerLock = new Object(); - - BaseScanner(final MasterStatus masterStatus, final boolean rootRegion, - final AtomicBoolean stop) { - super("Scanner for " + (rootRegion ? "-ROOT-":".META.") + " table", - masterStatus.getConfiguration(). - getInt("hbase.master.meta.thread.rescanfrequency", 60 * 1000), stop); - this.rootRegion = rootRegion; - this.masterStatus = masterStatus; - this.initialScanComplete = false; - } - - /** @return true if initial scan completed successfully */ - public boolean isInitialScanComplete() { - return initialScanComplete; - } - - @Override - protected boolean initialChore() { - return initialScan(); - } - - @Override - protected void chore() { - maintenanceScan(); - } - - /** - * @param region Region to scan - * @throws IOException - */ - protected void scanRegion(final MetaRegion region) throws IOException { - HRegionInterface regionServer = null; - long scannerId = -1L; - LOG.info(Thread.currentThread().getName() + " scanning meta region " + - region.toString()); - - // Array to hold list of split parents found. Scan adds to list. After - // scan we go check if parents can be removed and that their daughters - // are in place. - Map splitParents = new HashMap(); - List emptyRows = new ArrayList(); - int rows = 0; - try { - regionServer = - this.masterStatus.getServerConnection().getHRegionConnection(region.getServer()); - Scan s = new Scan().addFamily(HConstants.CATALOG_FAMILY); - // Make this scan do a row at a time otherwise, data can be stale. - s.setCaching(1); - scannerId = regionServer.openScanner(region.getRegionName(), s); - while (true) { - Result values = regionServer.next(scannerId); - if (values == null || values.size() == 0) { - break; - } - HRegionInfo info = RegionManager.getHRegionInfo(values.getRow(), values); - if (info == null) { - emptyRows.add(values.getRow()); - continue; - } - String serverAddress = getServerAddress(values); - long startCode = getStartCode(values); - - // Note Region has been assigned. - checkAssigned(regionServer, region, info, serverAddress, startCode); - if (isSplitParent(info)) { - splitParents.put(info, values); - } - rows += 1; - } - if (rootRegion) { - this.masterStatus.getRegionManager().setNumMetaRegions(rows); - } - } catch (IOException e) { - if (e instanceof RemoteException) { - e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e); - if (e instanceof UnknownScannerException) { - // Reset scannerId so we do not try closing a scanner the other side - // has lost account of: prevents duplicated stack trace out of the - // below close in the finally. - scannerId = -1L; - } - } - throw e; - } finally { - try { - if (scannerId != -1L && regionServer != null) { - regionServer.close(scannerId); - } - } catch (IOException e) { - LOG.error("Closing scanner", - RemoteExceptionHandler.checkIOException(e)); - } - } - - // Scan is finished. - - // First clean up any meta region rows which had null HRegionInfos - if (emptyRows.size() > 0) { - LOG.warn("Found " + emptyRows.size() + " rows with empty HRegionInfo " + - "while scanning meta region " + Bytes.toString(region.getRegionName())); - RegionManager.deleteEmptyMetaRows(regionServer, region.getRegionName(), - emptyRows); - } - - // Take a look at split parents to see if any we can clean up any and to - // make sure that daughter regions are in place. - if (splitParents.size() > 0) { - for (Map.Entry e : splitParents.entrySet()) { - HRegionInfo hri = e.getKey(); - cleanupAndVerifySplits(region.getRegionName(), regionServer, - hri, e.getValue()); - } - } - LOG.info(Thread.currentThread().getName() + " scan of " + rows + - " row(s) of meta region " + region.toString() + " complete"); - } - - /* - * @param r - * @return Empty String or server address found in r - */ - static String getServerAddress(final Result r) { - final byte[] val = r.getValue(HConstants.CATALOG_FAMILY, - HConstants.SERVER_QUALIFIER); - return val == null || val.length <= 0 ? "" : Bytes.toString(val); - } - - /* - * @param r - * @return Return 0L or server startcode found in r - */ - static long getStartCode(final Result r) { - final byte[] val = r.getValue(HConstants.CATALOG_FAMILY, - HConstants.STARTCODE_QUALIFIER); - return val == null || val.length <= 0 ? 0L : Bytes.toLong(val); - } - - /* - * @param info Region to check. - * @return True if this is a split parent. - */ - private boolean isSplitParent(final HRegionInfo info) { - if (!info.isSplit()) { - return false; - } - if (!info.isOffline()) { - LOG.warn("Region is split but not offline: " + - info.getRegionNameAsString()); - } - return true; - } - - /* - * If daughters no longer hold reference to the parents, delete the parent. - * If the parent is lone without daughter splits AND there are references in - * the filesystem, then a daughters was not added to .META. -- must have been - * a crash before their addition. Add them here. - * @param metaRegionName Meta region name: e.g. .META.,,1 - * @param server HRegionInterface of meta server to talk to - * @param parent HRegionInfo of split offlined parent - * @param rowContent Content of parent row in - * metaRegionName - * @return True if we removed parent from meta table and from - * the filesystem. - * @throws IOException - */ - private boolean cleanupAndVerifySplits(final byte [] metaRegionName, - final HRegionInterface srvr, final HRegionInfo parent, - Result rowContent) - throws IOException { - boolean result = false; - // Run checks on each daughter split. - boolean hasReferencesA = checkDaughter(metaRegionName, srvr, - parent, rowContent, HConstants.SPLITA_QUALIFIER); - boolean hasReferencesB = checkDaughter(metaRegionName, srvr, - parent, rowContent, HConstants.SPLITB_QUALIFIER); - if (!hasReferencesA && !hasReferencesB) { - LOG.info("Deleting region " + parent.getRegionNameAsString() + - " (encoded=" + parent.getEncodedName() + - ") because daughter splits no longer hold references"); - HRegion.deleteRegion(masterStatus.getFileSystemManager().getFileSystem(), - this.masterStatus.getFileSystemManager().getRootDir(), parent); - HRegion.removeRegionFromMETA(srvr, metaRegionName, - parent.getRegionName()); - result = true; - } - return result; - } - - - /* - * See if the passed daughter has references in the filesystem to the parent - * and if not, remove the note of daughter region in the parent row: its - * column info:splitA or info:splitB. Also make sure that daughter row is - * present in the .META. and mark the parent row when confirmed so we don't - * keep checking. The mark will be info:splitA_checked and its value will be - * a true BooleanWritable. - * @param metaRegionName - * @param srvr - * @param parent - * @param rowContent - * @param qualifier - * @return True if this daughter still has references to the parent. - * @throws IOException - */ - private boolean checkDaughter(final byte [] metaRegionName, - final HRegionInterface srvr, final HRegionInfo parent, - final Result rowContent, final byte [] qualifier) - throws IOException { - HRegionInfo hri = getDaughterRegionInfo(rowContent, qualifier); - boolean references = hasReferences(metaRegionName, srvr, parent, rowContent, - hri, qualifier); - // Return if no references. - if (!references) return references; - if (!verifyDaughterRowPresent(rowContent, qualifier, srvr, metaRegionName, - hri, parent)) { - // If we got here, then the parent row does not yet have the - // "daughter row verified present" marker present. Add it. - addDaughterRowChecked(metaRegionName, srvr, parent.getRegionName(), hri, - qualifier); - } - return references; - } - - /* - * Check the daughter of parent is present in meta table. If not there, - * add it. - * @param rowContent - * @param daughter - * @param srvr - * @param metaRegionName - * @param daughterHRI - * @throws IOException - * @return True, if parent row has marker for "daughter row verified present" - * else, false (and will do fixup adding daughter if daughter not present). - */ - private boolean verifyDaughterRowPresent(final Result rowContent, - final byte [] daughter, final HRegionInterface srvr, - final byte [] metaRegionName, - final HRegionInfo daughterHRI, final HRegionInfo parent) - throws IOException { - // See if the 'checked' column is in parent. If so, we're done. - boolean present = getDaughterRowChecked(rowContent, daughter); - if (present) return present; - // Parent is not carrying the splitA_checked/splitB_checked so this must - // be the first time through here checking splitA/splitB are in metatable. - byte [] daughterRowKey = daughterHRI.getRegionName(); - Get g = new Get(daughterRowKey); - g.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - Result r = srvr.get(metaRegionName, g); - if (r == null || r.isEmpty()) { - // Daughter row not present. Fixup kicks in. Insert it. - LOG.warn("Fixup broke split: Add missing split daughter to meta," + - " daughter=" + daughterHRI.toString() + ", parent=" + parent.toString()); - Put p = new Put(daughterRowKey); - p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, - Writables.getBytes(daughterHRI)); - srvr.put(metaRegionName, p); - } - return present; - } - - /* - * Add to parent a marker that we verified the daughter exists. - * @param metaRegionName - * @param srvr - * @param parent - * @param split - * @param daughter - * @throws IOException - */ - private void addDaughterRowChecked(final byte [] metaRegionName, - final HRegionInterface srvr, final byte [] parent, - final HRegionInfo split, final byte [] daughter) - throws IOException { - Put p = new Put(parent); - p.add(HConstants.CATALOG_FAMILY, getNameOfVerifiedDaughterColumn(daughter), - TRUE_WRITABLE_AS_BYTES); - srvr.put(metaRegionName, p); - } - - /* - * @param rowContent - * @param which - * @return True if the daughter row has already been verified present in - * metatable. - * @throws IOException - */ - private boolean getDaughterRowChecked(final Result rowContent, - final byte[] which) - throws IOException { - final byte[] b = rowContent.getValue(HConstants.CATALOG_FAMILY, - getNameOfVerifiedDaughterColumn(which)); - BooleanWritable bw = null; - if (b != null && b.length > 0) { - bw = (BooleanWritable)Writables.getWritable(b, new BooleanWritable()); - } - return bw == null? false: bw.get(); - } - - /* - * @param daughter - * @return Returns splitA_checked or splitB_checked dependent on what - * daughter is. - */ - private static byte [] getNameOfVerifiedDaughterColumn(final byte [] daughter) { - return (Bytes.equals(HConstants.SPLITA_QUALIFIER, daughter) - ? SPLITA_CHECKED : SPLITB_CHECKED); - } - - /* - * Get daughter HRegionInfo out of parent info:splitA/info:splitB columns. - * @param rowContent - * @param which Whether "info:splitA" or "info:splitB" column - * @return Deserialized content of the info:splitA or info:splitB as a - * HRegionInfo - * @throws IOException - */ - private HRegionInfo getDaughterRegionInfo(final Result rowContent, - final byte [] which) - throws IOException { - return Writables.getHRegionInfoOrNull( - rowContent.getValue(HConstants.CATALOG_FAMILY, which)); - } - - /* - * Remove mention of daughter from parent row. - * parent row. - * @param metaRegionName - * @param srvr - * @param parent - * @param split - * @param qualifier - * @throws IOException - */ - private void removeDaughterFromParent(final byte [] metaRegionName, - final HRegionInterface srvr, final HRegionInfo parent, - final HRegionInfo split, final byte [] qualifier) - throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug(split.getRegionNameAsString() + "/" + split.getEncodedName() + - " no longer has references to " + parent.getRegionNameAsString()); - } - Delete delete = new Delete(parent.getRegionName()); - delete.deleteColumns(HConstants.CATALOG_FAMILY, qualifier); - srvr.delete(metaRegionName, delete); - } - - /* - * Checks if a daughter region -- either splitA or splitB -- still holds - * references to parent. If not, removes reference to the split from - * the parent meta region row so we don't check it any more. - * @param metaRegionName Name of meta region to look in. - * @param srvr Where region resides. - * @param parent Parent region name. - * @param rowContent Keyed content of the parent row in meta region. - * @param split Which column family. - * @param qualifier Which of the daughters to look at, splitA or splitB. - * @return True if still has references to parent. - * @throws IOException - */ - private boolean hasReferences(final byte [] metaRegionName, - final HRegionInterface srvr, final HRegionInfo parent, - Result rowContent, final HRegionInfo split, byte [] qualifier) - throws IOException { - boolean result = false; - if (split == null) { - return result; - } - Path tabledir = - new Path(this.masterStatus.getFileSystemManager().getRootDir(), split.getTableDesc().getNameAsString()); - for (HColumnDescriptor family: split.getTableDesc().getFamilies()) { - Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(), - family.getName()); - if (!masterStatus.getFileSystemManager().getFileSystem().exists(p)) continue; - // Look for reference files. Call listStatus with an anonymous - // instance of PathFilter. - FileStatus [] ps = - masterStatus.getFileSystemManager().getFileSystem().listStatus(p, new PathFilter () { - public boolean accept(Path path) { - return StoreFile.isReference(path); - } - } - ); - - if (ps != null && ps.length > 0) { - result = true; - break; - } - } - if (!result) { - removeDaughterFromParent(metaRegionName, srvr, parent, split, qualifier); - } - return result; - } - - /* - * Check the passed region is assigned. If not, add to unassigned. - * @param regionServer - * @param meta - * @param info - * @param hostnameAndPort hostname ':' port as it comes out of .META. - * @param startCode - * @throws IOException - */ - protected void checkAssigned(final HRegionInterface regionServer, - final MetaRegion meta, final HRegionInfo info, - final String hostnameAndPort, final long startCode) - throws IOException { - String serverName = null; - String sa = hostnameAndPort; - long sc = startCode; - if (sa == null || sa.length() <= 0) { - // Scans are sloppy. They cache a row internally so may have data that - // is a little stale. Make sure that for sure this serverAddress is null. - // We are trying to avoid double-assignments. See hbase-1784. - Get g = new Get(info.getRegionName()); - g.addFamily(HConstants.CATALOG_FAMILY); - Result r = regionServer.get(meta.getRegionName(), g); - if (r != null && !r.isEmpty()) { - sa = getServerAddress(r); - sc = getStartCode(r); - } - } - if (sa != null && sa.length() > 0) { - serverName = HServerInfo.getServerName(sa, sc); - } - HServerInfo storedInfo = null; - synchronized (this.masterStatus.getRegionManager()) { - /* We don't assign regions that are offline, in transition or were on - * a dead server. Regions that were on a dead server will get reassigned - * by ProcessServerShutdown - */ - if (info.isOffline() || - this.masterStatus.getRegionManager().regionIsInTransition(info.getRegionNameAsString()) || - (serverName != null && this.masterStatus.getServerManager().isDead(serverName))) { - return; - } - if (serverName != null) { - storedInfo = this.masterStatus.getServerManager().getServerInfo(serverName); - } - - // If we can't find the HServerInfo, then add it to the list of - // unassigned regions. - if (storedInfo == null) { - // The current assignment is invalid - if (LOG.isDebugEnabled()) { - LOG.debug("Current assignment of " + info.getRegionNameAsString() + - " is not valid; " + " serverAddress=" + sa + - ", startCode=" + sc + " unknown."); - } - // Now get the region assigned - this.masterStatus.getRegionManager().setUnassigned(info, true); - } - } - } - - /** - * Interrupt thread regardless of what it's doing - */ - public void interruptAndStop() { - synchronized(scannerLock){ - if (isAlive()) { - super.interrupt(); - LOG.info("Interrupted"); - } - } - } -} Index: src/main/java/org/apache/hadoop/hbase/master/MetaRegion.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/MetaRegion.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/MetaRegion.java (working copy) @@ -1,96 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.util.Bytes; - - -/** Describes a meta region and its server */ -public class MetaRegion implements Comparable { - private final HServerAddress server; - private HRegionInfo regionInfo; - - MetaRegion(final HServerAddress server, HRegionInfo regionInfo) { - if (server == null) { - throw new IllegalArgumentException("server cannot be null"); - } - this.server = server; - if (regionInfo == null) { - throw new IllegalArgumentException("regionInfo cannot be null"); - } - this.regionInfo = regionInfo; - } - - @Override - public String toString() { - return "{server: " + this.server.toString() + ", regionname: " + - regionInfo.getRegionNameAsString() + ", startKey: <" + - Bytes.toString(regionInfo.getStartKey()) + ">}"; - } - - /** @return the regionName */ - public byte [] getRegionName() { - return regionInfo.getRegionName(); - } - - /** @return the server */ - public HServerAddress getServer() { - return server; - } - - /** @return the startKey */ - public byte [] getStartKey() { - return regionInfo.getStartKey(); - } - - - /** @return the endKey */ - public byte [] getEndKey() { - return regionInfo.getEndKey(); - } - - - public HRegionInfo getRegionInfo() { - return regionInfo; - } - - @Override - public boolean equals(Object o) { - return o instanceof MetaRegion && this.compareTo((MetaRegion)o) == 0; - } - - @Override - public int hashCode() { - return regionInfo.hashCode(); - } - - // Comparable - - public int compareTo(MetaRegion other) { - int cmp = regionInfo.compareTo(other.regionInfo); - if(cmp == 0) { - // Might be on different host? - cmp = this.server.compareTo(other.server); - } - return cmp; - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java (working copy) @@ -1,85 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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 org.apache.hadoop.hbase.HRegionInfo; - -/** - * Abstract class that performs common operations for - * @see ProcessRegionClose and @see ProcessRegionOpen - */ -abstract class ProcessRegionStatusChange extends RegionServerOperation { - protected final boolean isMetaTable; - protected final HRegionInfo regionInfo; - @SuppressWarnings({"FieldCanBeLocal"}) - private volatile MetaRegion metaRegion = null; - protected volatile byte[] metaRegionName = null; - - /** - * @param masterStatus the master - * @param regionInfo region info - */ - public ProcessRegionStatusChange(MasterStatus masterStatus, HRegionInfo regionInfo) { - super(masterStatus); - this.regionInfo = regionInfo; - this.isMetaTable = regionInfo.isMetaTable(); - } - - protected boolean metaRegionAvailable() { - boolean available = true; - if (isMetaTable) { - // This operation is for the meta table - if (!rootAvailable()) { - requeue(); - // But we can't proceed unless the root region is available - available = false; - } - } else { - if (!masterStatus.getRegionManager().isInitialRootScanComplete() || - !metaTableAvailable()) { - // The root region has not been scanned or the meta table is not - // available so we can't proceed. - // Put the operation on the delayedToDoQueue - requeue(); - available = false; - } - } - return available; - } - - protected MetaRegion getMetaRegion() { - if (isMetaTable) { - this.metaRegionName = HRegionInfo.ROOT_REGIONINFO.getRegionName(); - this.metaRegion = new MetaRegion(masterStatus.getRegionManager().getRootRegionLocation(), - HRegionInfo.ROOT_REGIONINFO); - } else { - this.metaRegion = - masterStatus.getRegionManager().getFirstMetaRegionForRegion(regionInfo); - if (this.metaRegion != null) { - this.metaRegionName = this.metaRegion.getRegionName(); - } - } - return this.metaRegion; - } - - public HRegionInfo getRegionInfo() { - return regionInfo; - } -} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (revision 0) @@ -0,0 +1,312 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.executor; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.ServerController; +import org.apache.hadoop.hbase.executor.HBaseExecutorService.HBaseExecutorServiceType; + + +/** + * Abstract base class for all HBase event handlers. Subclasses should + * implement the process() method where the actual handling of the event + * happens. + *

        + * HBaseEventType is a list of ALL events (which also corresponds to messages - + * either internal to one component or between components). The event type + * names specify the component from which the event originated, and the + * component which is supposed to handle it. + *

        + * Listeners can listen to all the events by implementing the interface + * HBaseEventHandlerListener, and by registering themselves as a listener. They + * will be called back before and after the process of every event. + */ +public abstract class EventHandler implements Runnable, Comparable { + private static final Log LOG = LogFactory.getLog(EventHandler.class); + // type of event this object represents + protected EventType eventType; + // server controller + protected ServerController server; + // listeners that are called before and after an event is processed + protected static List eventHandlerListeners = + Collections.synchronizedList(new ArrayList()); + // sequence id generator for default FIFO ordering of events + protected static AtomicLong seqids = new AtomicLong(0); + // sequence id for this event + protected long seqid; + + /** + * This interface provides hooks to listen to various events received by the + * queue. A class implementing this can listen to the updates by calling + * registerListener and stop receiving updates by calling unregisterListener + */ + public interface EventHandlerListener { + /** + * Called before any event is processed + */ + public void beforeProcess(EventHandler event); + /** + * Called after any event is processed + */ + public void afterProcess(EventHandler event); + } + + /** + * These are a list of HBase events that can be handled by the various + * HBaseExecutorService's. All the events are serialized as byte values. + */ + public enum EventType { + // Messages originating from RS (NOTE: there is NO direct communication from + // RS to Master). These are a result of RS updates into ZK. + RS2ZK_REGION_CLOSING (1), // RS is in process of closing a region + RS2ZK_REGION_CLOSED (2), // RS has finished closing a region + RS2ZK_REGION_OPENING (3), // RS is in process of opening a region + RS2ZK_REGION_OPENED (4), // RS has finished opening a region + + // Messages originating from Master to RS + M2RS_OPEN_REGION (20), // Master asking RS to open a region + M2RS_OPEN_ROOT (21), // Master asking RS to open root + M2RS_OPEN_META (22), // Master asking RS to open meta + M2RS_CLOSE_REGION (23), // Master asking RS to close a region + M2RS_CLOSE_ROOT (24), // Master asking RS to close root + M2RS_CLOSE_META (25), // Master asking RS to close meta + + // Messages originating from Client to Master + C2M_DELETE_TABLE (40), // Client asking Master to delete a table + C2M_DISABLE_TABLE (41), // Client asking Master to disable a table + C2M_ENABLE_TABLE (42), // Client asking Master to enable a table + C2M_MODIFY_TABLE (43), // Client asking Master to modify a table + C2M_ADD_FAMILY (44), // Client asking Master to add family to table + C2M_DELETE_FAMILY (45), // Client asking Master to delete family of table + C2M_MODIFY_FAMILY (46), // Client asking Master to modify family of table + + // Updates from master to ZK. This is done by the master and there is + // nothing to process by either Master or RS + M2ZK_REGION_OFFLINE (50), // Master adds this region as offline in ZK + + // Master controlled events to be executed on the master + M_SERVER_SHUTDOWN (70); // Master is processing shutdown of a RS + + /** + * Returns the executor service type (the thread pool instance) for this + * event type. Every type must be handled here. Multiple types map to + * Called by the HMaster. Returns a name of the executor service given an + * event type. Every event type has an entry - if the event should not be + * handled just add the NONE executor. + * @return name of the executor service + */ + public HBaseExecutorServiceType getExecutorServiceType() { + switch(this) { + + // Master executor services + + case RS2ZK_REGION_CLOSED: + return HBaseExecutorServiceType.MASTER_CLOSE_REGION; + + case RS2ZK_REGION_OPENED: + return HBaseExecutorServiceType.MASTER_OPEN_REGION; + + case M_SERVER_SHUTDOWN: + return HBaseExecutorServiceType.MASTER_SERVER_OPERATIONS; + + case C2M_DELETE_TABLE: + case C2M_DISABLE_TABLE: + case C2M_ENABLE_TABLE: + case C2M_MODIFY_TABLE: + return HBaseExecutorServiceType.MASTER_TABLE_OPERATIONS; + + // RegionServer executor services + + case M2RS_OPEN_REGION: + return HBaseExecutorServiceType.RS_OPEN_REGION; + + case M2RS_OPEN_ROOT: + return HBaseExecutorServiceType.RS_OPEN_ROOT; + + case M2RS_OPEN_META: + return HBaseExecutorServiceType.RS_OPEN_META; + + case M2RS_CLOSE_REGION: + return HBaseExecutorServiceType.RS_CLOSE_REGION; + + case M2RS_CLOSE_ROOT: + return HBaseExecutorServiceType.RS_CLOSE_ROOT; + + case M2RS_CLOSE_META: + return HBaseExecutorServiceType.RS_CLOSE_META; + + default: + throw new RuntimeException("Unhandled event type " + this.name()); + } + } + + /** + * Start the executor service that handles the passed in event type. The + * server that starts these event executor services wants to handle these + * event types. + */ + public void startExecutorService(String serverName, int maxThreads) { + getExecutorServiceType().startExecutorService(serverName, maxThreads); + } + + EventType(int value) {} + + @Override + public String toString() { + switch(this) { + case RS2ZK_REGION_CLOSED: return "CLOSED"; + case RS2ZK_REGION_CLOSING: return "CLOSING"; + case RS2ZK_REGION_OPENED: return "OPENED"; + case RS2ZK_REGION_OPENING: return "OPENING"; + case M2ZK_REGION_OFFLINE: return "OFFLINE"; + default: return this.name(); + } + } + } + + /** + * Default base class constructor. + */ + public EventHandler(ServerController server, EventType eventType) { + this.server = server; + this.eventType = eventType; + seqid = seqids.incrementAndGet(); + } + + /** + * This is a wrapper around process, used to update listeners before and after + * events are processed. + */ + public void run() { + // fire all beforeProcess listeners + for(EventHandlerListener listener : eventHandlerListeners) { + listener.beforeProcess(this); + } + + // call the main process function + try { + process(); + } catch(Throwable t) { + LOG.error("Caught throwable while processing event " + eventType, t); + } + + // fire all afterProcess listeners + for(EventHandlerListener listener : eventHandlerListeners) { + LOG.debug("Firing " + listener.getClass().getName() + + ".afterProcess event listener for event " + eventType); + listener.afterProcess(this); + } + } + + /** + * This method is the main processing loop to be implemented by the various + * subclasses. + */ + public abstract void process(); + + /** + * Subscribe to updates before and after processing events + */ + public static void registerListener(EventHandlerListener listener) { + eventHandlerListeners.add(listener); + } + + /** + * Stop receiving updates before and after processing events + */ + public static void unregisterListener(EventHandlerListener listener) { + eventHandlerListeners.remove(listener); + } + + /** + * Return the name for this event type. + * @return + */ + public HBaseExecutorServiceType getEventHandlerName() { + return eventType.getExecutorServiceType(); + } + + /** + * Return the event type + * @return + */ + public EventType getEventType() { + return eventType; + } + + /** + * Submits this event object to the correct executor service. This is causes + * this object to get executed by the correct ExecutorService. + */ + public void submit() { + HBaseExecutorServiceType serviceType = getEventHandlerName(); + if(serviceType == null) { + throw new RuntimeException("Event " + eventType + " not handled on " + + "this server " + server.getServerName()); + } + serviceType.getExecutor(server.getServerName()).submit(this); + } + + + /** + * Get the priority level for this handler instance. This uses natural + * ordering so lower numbers are higher priority. + *

        + * Lowest priority is Integer.MAX_VALUE. Highest priority is 0. + *

        + * Subclasses should override this method to allow prioritizing handlers. + *

        + * Handlers with the same priority are handled in FIFO order. + *

        + * @return Integer.MAX_VALUE by default, override to set higher priorities + */ + public int getPriority() { + return Integer.MAX_VALUE; + } + + /** + * Default prioritized runnable comparator which implements a FIFO ordering. + *

        + * Subclasses should not override this. Instead, if they want to implement + * priority beyond FIFO, they should override {@link #getPriority()}. + */ + @Override + public int compareTo(Runnable o) { + EventHandler eh = (EventHandler)o; + if(getPriority() != eh.getPriority()) { + return (getPriority() < eh.getPriority()) ? -1 : 1; + } + return (this.seqid < eh.seqid) ? -1 : 1; + } + + /** + * Executes this event object in the caller's thread. This is a synchronous + * way of executing the event. + */ + public void execute() { + this.run(); + } +} Index: src/main/java/org/apache/hadoop/hbase/executor/HBaseEventHandler.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/executor/HBaseEventHandler.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/executor/HBaseEventHandler.java (working copy) @@ -1,289 +0,0 @@ -/** - * Copyright 2010 The Apache Software Foundation - * - * 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.executor; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.executor.HBaseExecutorService.HBaseExecutorServiceType; - - -/** - * Abstract base class for all HBase event handlers. Subclasses should - * implement the process() method where the actual handling of the event - * happens. - * - * HBaseEventType is a list of ALL events (which also corresponds to messages - - * either internal to one component or between components). The event type - * names specify the component from which the event originated, and the - * component which is supposed to handle it. - * - * Listeners can listen to all the events by implementing the interface - * HBaseEventHandlerListener, and by registering themselves as a listener. They - * will be called back before and after the process of every event. - * - * TODO: Rename HBaseEvent and HBaseEventType to EventHandler and EventType - * after ZK refactor as it currently would clash with EventType from ZK and - * make the code very confusing. - */ -public abstract class HBaseEventHandler implements Runnable -{ - private static final Log LOG = LogFactory.getLog(HBaseEventHandler.class); - // type of event this object represents - protected HBaseEventType eventType = HBaseEventType.NONE; - // is this a region server or master? - protected boolean isRegionServer; - // name of the server - this is needed for naming executors in case of tests - // where region servers may be co-located. - protected String serverName; - // listeners that are called before and after an event is processed - protected static List eventHandlerListeners = - Collections.synchronizedList(new ArrayList()); - - /** - * This interface provides hooks to listen to various events received by the - * queue. A class implementing this can listen to the updates by calling - * registerListener and stop receiving updates by calling unregisterListener - */ - public interface HBaseEventHandlerListener { - /** - * Called before any event is processed - */ - public void beforeProcess(HBaseEventHandler event); - /** - * Called after any event is processed - */ - public void afterProcess(HBaseEventHandler event); - } - - /** - * These are a list of HBase events that can be handled by the various - * HBaseExecutorService's. All the events are serialized as byte values. - */ - public enum HBaseEventType { - NONE (-1), - // Messages originating from RS (NOTE: there is NO direct communication from - // RS to Master). These are a result of RS updates into ZK. - RS2ZK_REGION_CLOSING (1), // RS is in process of closing a region - RS2ZK_REGION_CLOSED (2), // RS has finished closing a region - RS2ZK_REGION_OPENING (3), // RS is in process of opening a region - RS2ZK_REGION_OPENED (4), // RS has finished opening a region - - // Updates from master to ZK. This is done by the master and there is - // nothing to process by either Master or RS - M2ZK_REGION_OFFLINE (50); // Master adds this region as offline in ZK - - private final byte value; - - /** - * Called by the HMaster. Returns a name of the executor service given an - * event type. Every event type has en entry - if the event should not be - * handled just add the NONE executor. - * @return name of the executor service - */ - public HBaseExecutorServiceType getMasterExecutorForEvent() { - HBaseExecutorServiceType executorServiceType = null; - switch(this) { - - case RS2ZK_REGION_CLOSING: - case RS2ZK_REGION_CLOSED: - executorServiceType = HBaseExecutorServiceType.MASTER_CLOSEREGION; - break; - - case RS2ZK_REGION_OPENING: - case RS2ZK_REGION_OPENED: - executorServiceType = HBaseExecutorServiceType.MASTER_OPENREGION; - break; - - case M2ZK_REGION_OFFLINE: - executorServiceType = HBaseExecutorServiceType.NONE; - break; - - default: - throw new RuntimeException("Unhandled event type in the master."); - } - - return executorServiceType; - } - - /** - * Called by the RegionServer. Returns a name of the executor service given an - * event type. Every event type has en entry - if the event should not be - * handled just return a null executor name. - * @return name of the event service - */ - public static String getRSExecutorForEvent(String serverName) { - throw new RuntimeException("Unsupported operation."); - } - - /** - * Start the executor service that handles the passed in event type. The - * server that starts these event executor services wants to handle these - * event types. - */ - public void startMasterExecutorService(String serverName) { - HBaseExecutorServiceType serviceType = getMasterExecutorForEvent(); - if(serviceType == HBaseExecutorServiceType.NONE) { - throw new RuntimeException("Event type " + toString() + " not handled on master."); - } - serviceType.startExecutorService(serverName); - } - - public static void startRSExecutorService() { - - } - - HBaseEventType(int intValue) { - this.value = (byte)intValue; - } - - public byte getByteValue() { - return value; - } - - @Override - public String toString() { - switch(this) { - case RS2ZK_REGION_CLOSED: return "CLOSED"; - case RS2ZK_REGION_CLOSING: return "CLOSING"; - case RS2ZK_REGION_OPENED: return "OPENED"; - case RS2ZK_REGION_OPENING: return "OPENING"; - case M2ZK_REGION_OFFLINE: return "OFFLINE"; - default: return this.name(); - } - } - - public static HBaseEventType fromByte(byte value) { - switch(value) { - case -1: return HBaseEventType.NONE; - case 1 : return HBaseEventType.RS2ZK_REGION_CLOSING; - case 2 : return HBaseEventType.RS2ZK_REGION_CLOSED; - case 3 : return HBaseEventType.RS2ZK_REGION_OPENING; - case 4 : return HBaseEventType.RS2ZK_REGION_OPENED; - case 50: return HBaseEventType.M2ZK_REGION_OFFLINE; - - default: - throw new RuntimeException("Invalid byte value for conversion to HBaseEventType"); - } - } - } - - /** - * Default base class constructor. - * - * TODO: isRegionServer and serverName will go away once we do the HMaster - * refactor. We will end up passing a ServerStatus which should tell us both - * the name and if it is a RS or master. - */ - public HBaseEventHandler(boolean isRegionServer, String serverName, HBaseEventType eventType) { - this.isRegionServer = isRegionServer; - this.eventType = eventType; - this.serverName = serverName; - } - - /** - * This is a wrapper around process, used to update listeners before and after - * events are processed. - */ - public void run() { - // fire all beforeProcess listeners - for(HBaseEventHandlerListener listener : eventHandlerListeners) { - listener.beforeProcess(this); - } - - // call the main process function - try { - process(); - } catch(Throwable t) { - LOG.error("Caught throwable while processing event " + eventType, t); - } - - // fire all afterProcess listeners - for(HBaseEventHandlerListener listener : eventHandlerListeners) { - LOG.debug("Firing " + listener.getClass().getName() + - ".afterProcess event listener for event " + eventType); - listener.afterProcess(this); - } - } - - /** - * This method is the main processing loop to be implemented by the various - * subclasses. - */ - public abstract void process(); - - /** - * Subscribe to updates before and after processing events - */ - public static void registerListener(HBaseEventHandlerListener listener) { - eventHandlerListeners.add(listener); - } - - /** - * Stop receiving updates before and after processing events - */ - public static void unregisterListener(HBaseEventHandlerListener listener) { - eventHandlerListeners.remove(listener); - } - - public boolean isRegionServer() { - return isRegionServer; - } - - /** - * Return the name for this event type. - * @return - */ - public HBaseExecutorServiceType getEventHandlerName() { - // TODO: check for isRegionServer here - return eventType.getMasterExecutorForEvent(); - } - - /** - * Return the event type - * @return - */ - public HBaseEventType getHBEvent() { - return eventType; - } - - /** - * Submits this event object to the correct executor service. This is causes - * this object to get executed by the correct ExecutorService. - */ - public void submit() { - HBaseExecutorServiceType serviceType = getEventHandlerName(); - if(serviceType == null) { - throw new RuntimeException("Event " + eventType + " not handled on this server " + serverName); - } - serviceType.getExecutor(serverName).submit(this); - } - - /** - * Executes this event object in the caller's thread. This is a synchronous - * way of executing the event. - */ - public void execute() { - this.run(); - } -} Index: src/main/java/org/apache/hadoop/hbase/executor/HBaseExecutorService.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/executor/HBaseExecutorService.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/executor/HBaseExecutorService.java (working copy) @@ -24,7 +24,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -43,88 +43,77 @@ * HBEventHandler class and create an event type that submits to this service. * */ -public class HBaseExecutorService -{ +public class HBaseExecutorService { private static final Log LOG = LogFactory.getLog(HBaseExecutorService.class); // default number of threads in the pool private int corePoolSize = 1; - // max number of threads - maximum concurrency - private int maximumPoolSize = 5; // how long to retain excess threads private long keepAliveTimeInMillis = 1000; // the thread pool executor that services the requests ThreadPoolExecutor threadPoolExecutor; // work queue to use - unbounded queue - BlockingQueue workQueue = new LinkedBlockingQueue(); + BlockingQueue workQueue = new PriorityBlockingQueue(); // name for this executor service String name; // hold the all the executors created in a map addressable by their names static Map executorServicesMap = Collections.synchronizedMap(new HashMap()); - /** - * The following is a list of names for the various executor services in both + * The following is a list of names for the various executor services in both * the master and the region server. */ public enum HBaseExecutorServiceType { - NONE (-1), - MASTER_CLOSEREGION (1), - MASTER_OPENREGION (2); - - private final int value; - - HBaseExecutorServiceType(int intValue) { - this.value = intValue; - } - - public void startExecutorService(String serverName) { - // if this is NONE then there is no executor to start - if(value == NONE.value) { - throw new RuntimeException("Cannot start NONE executor type."); - } + + // Master executor services + MASTER_CLOSE_REGION (1), + MASTER_OPEN_REGION (2), + MASTER_SERVER_OPERATIONS (3), + MASTER_TABLE_OPERATIONS (4), + + // RegionServer executor services + RS_OPEN_REGION (20), + RS_OPEN_ROOT (21), + RS_OPEN_META (22), + RS_CLOSE_REGION (23), + RS_CLOSE_ROOT (24), + RS_CLOSE_META (25); + + HBaseExecutorServiceType(int value) {} + + public void startExecutorService(String serverName, int maxThreads) { String name = getExecutorName(serverName); if(HBaseExecutorService.isExecutorServiceRunning(name)) { LOG.debug("Executor service " + toString() + " already running on " + serverName); return; } LOG.debug("Starting executor service [" + name + "]"); - HBaseExecutorService.startExecutorService(name); + HBaseExecutorService.startExecutorService(name, maxThreads); } - + public HBaseExecutorService getExecutor(String serverName) { - // if this is NONE then there is no executor - if(value == NONE.value) { - return null; - } return HBaseExecutorService.getExecutorService(getExecutorName(serverName)); } - + public String getExecutorName(String serverName) { - // if this is NONE then there is no executor - if(value == NONE.value) { - return null; - } return (this.toString() + "-" + serverName); } } - - /** * Start an executor service with a given name. If there was a service already * started with the same name, this throws a RuntimeException. * @param name Name of the service to start. */ - public static void startExecutorService(String name) { + public static void startExecutorService(String name, int maxThreads) { if(executorServicesMap.get(name) != null) { throw new RuntimeException("An executor service with the name " + name + " is already running!"); } - HBaseExecutorService hbes = new HBaseExecutorService(name); + HBaseExecutorService hbes = new HBaseExecutorService(name, maxThreads); executorServicesMap.put(name, hbes); LOG.debug("Starting executor service: " + name); } - + public static boolean isExecutorServiceRunning(String name) { return (executorServicesMap.containsKey(name)); } @@ -140,7 +129,7 @@ } return executor; } - + public static void shutdown() { for(Entry entry : executorServicesMap.entrySet()) { entry.getValue().threadPoolExecutor.shutdown(); @@ -148,16 +137,11 @@ executorServicesMap.clear(); } - protected HBaseExecutorService(String name) { + protected HBaseExecutorService(String name, int maxThreads) { this.name = name; // create the thread pool executor - threadPoolExecutor = new ThreadPoolExecutor( - corePoolSize, - maximumPoolSize, - keepAliveTimeInMillis, - TimeUnit.MILLISECONDS, - workQueue - ); + threadPoolExecutor = new ThreadPoolExecutor(corePoolSize, maxThreads, + keepAliveTimeInMillis, TimeUnit.MILLISECONDS, workQueue); // name the threads for this threadpool threadPoolExecutor.setThreadFactory(new NamedThreadFactory(name)); } Index: src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/executor/RegionTransitionData.java (working copy) @@ -23,8 +23,8 @@ import java.io.DataOutput; import java.io.IOException; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType; +import org.apache.hadoop.hbase.executor.EventHandler.EventType; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Writable; @@ -36,20 +36,17 @@ * Type of transition event (offline, opening, opened, closing, closed). * Required. */ - private HBaseEventType eventType; + private EventType eventType; /** Region being transitioned. Required. */ - private String regionName; + private byte [] regionName; /** Server event originated from. Optional. */ private String serverName; /** Time the event was created. Required but automatically set. */ - private long timeStamp; + private long stamp; - /** Temporary. Holds payload used doing transitions via heartbeats. */ - private HMsg hmsg; // to be removed shortly once we stop using heartbeats - /** * Writable constructor. Do not use directly. */ @@ -68,12 +65,12 @@ * assignment. * *

        Since only the master uses this constructor, the type should always be - * {@link HBaseEventType#M2ZK_REGION_OFFLINE}. + * {@link EventType#M2ZK_REGION_OFFLINE}. * * @param eventType type of event * @param regionName name of region */ - public RegionTransitionData(HBaseEventType eventType, String regionName) { + public RegionTransitionData(EventType eventType, byte [] regionName) { this(eventType, regionName, null); } @@ -83,37 +80,20 @@ * *

        Used when the server name is known (a regionserver is setting it). * - *

        Valid types for this constructor are {@link HBaseEventType#RS2ZK_REGION_CLOSING}, - * {@link HBaseEventType#RS2ZK_REGION_CLOSED}, {@link HBaseEventType#RS2ZK_REGION_OPENING}, - * and {@link HBaseEventType#RS2ZK_REGION_OPENED}. + *

        Valid types for this constructor are {@link EventType#RS2ZK_REGION_CLOSING}, + * {@link EventType#RS2ZK_REGION_CLOSED}, {@link EventType#RS2ZK_REGION_OPENING}, + * and {@link EventType#RS2ZK_REGION_OPENED}. * * @param eventType type of event * @param regionName name of region * @param serverName name of server setting data */ - public RegionTransitionData(HBaseEventType eventType, String regionName, + public RegionTransitionData(EventType eventType, byte [] regionName, String serverName) { - this(eventType, regionName, serverName, null); - } - - /** - * Construct data for a fully-specified, old-format region transition event - * which uses HMsg/heartbeats. - * - * TODO: Remove this constructor once we stop using heartbeats. - * - * @param eventType - * @param regionName - * @param serverName - * @param hmsg - */ - public RegionTransitionData(HBaseEventType eventType, String regionName, - String serverName, HMsg hmsg) { this.eventType = eventType; - this.timeStamp = System.currentTimeMillis(); + this.stamp = System.currentTimeMillis(); this.regionName = regionName; this.serverName = serverName; - this.hmsg = hmsg; } /** @@ -121,25 +101,25 @@ * *

        One of: *

          - *
        • {@link HBaseEventType#M2ZK_REGION_OFFLINE} - *
        • {@link HBaseEventType#RS2ZK_REGION_CLOSING} - *
        • {@link HBaseEventType#RS2ZK_REGION_CLOSED} - *
        • {@link HBaseEventType#RS2ZK_REGION_OPENING} - *
        • {@link HBaseEventType#RS2ZK_REGION_OPENED} + *
        • {@link EventType#M2ZK_REGION_OFFLINE} + *
        • {@link EventType#RS2ZK_REGION_CLOSING} + *
        • {@link EventType#RS2ZK_REGION_CLOSED} + *
        • {@link EventType#RS2ZK_REGION_OPENING} + *
        • {@link EventType#RS2ZK_REGION_OPENED} *
        * @return type of region transition event */ - public HBaseEventType getEventType() { + public EventType getEventType() { return eventType; } /** - * Gets the encoded name of the region being transitioned. + * Gets the name of the region being transitioned. * *

        Region name is required so this never returns null. * @return region name */ - public String getRegionName() { + public byte [] getRegionName() { return regionName; } @@ -156,54 +136,39 @@ /** * Gets the timestamp when this event was created. * - * @return time event was created + * @return stamp event was created */ - public long getTimeStamp() { - return timeStamp; + public long getStamp() { + return stamp; } - /** - * Gets the {@link HMsg} payload of this region transition event. - * @return heartbeat payload - */ - public HMsg getHmsg() { - return hmsg; - } - @Override public void readFields(DataInput in) throws IOException { // the event type byte - eventType = HBaseEventType.fromByte(in.readByte()); + eventType = EventType.values()[in.readShort()]; // the timestamp - timeStamp = in.readLong(); + stamp = in.readLong(); // the encoded name of the region being transitioned - regionName = in.readUTF(); + regionName = Bytes.readByteArray(in); // remaining fields are optional so prefixed with boolean // the name of the regionserver sending the data if(in.readBoolean()) { serverName = in.readUTF(); + } else { + serverName = null; } - // hmsg - if(in.readBoolean()) { - hmsg = new HMsg(); - hmsg.readFields(in); - } } @Override public void write(DataOutput out) throws IOException { - out.writeByte(eventType.getByteValue()); + out.writeShort(eventType.ordinal()); out.writeLong(System.currentTimeMillis()); - out.writeUTF(regionName); + Bytes.writeByteArray(out, regionName); // remaining fields are optional so prefixed with boolean out.writeBoolean(serverName != null); if(serverName != null) { out.writeUTF(serverName); } - out.writeBoolean(hmsg != null); - if(hmsg != null) { - hmsg.write(out); - } } /** Index: src/main/java/org/apache/hadoop/hbase/catalog/RootLocationEditor.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/catalog/RootLocationEditor.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/catalog/RootLocationEditor.java (revision 0) @@ -0,0 +1,72 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.catalog; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.KeeperException; + +/** + * Makes changes to the location of -ROOT- in ZooKeeper. + */ +public class RootLocationEditor { + private static final Log LOG = LogFactory.getLog(RootLocationEditor.class); + + /** + * Deletes the location of -ROOT- in ZooKeeper. + * @param zookeeper zookeeper reference + * @throws KeeperException unexpected zookeeper exception + */ + public static void deleteRootLocation(ZooKeeperWatcher zookeeper) + throws KeeperException { + LOG.info("Unsetting ROOT region location in ZooKeeper"); + try { + // Just delete the node. Don't need any watches, only we will create it. + ZKUtil.deleteNode(zookeeper, zookeeper.rootServerZNode); + } catch(KeeperException.NoNodeException nne) { + // Has already been deleted + } + } + + /** + * Sets the location of -ROOT- in ZooKeeper to the + * specified server address. + * @param zookeeper zookeeper reference + * @param location server address hosting root + * @throws KeeperException unexpected zookeeper exception + */ + public static void setRootLocation(ZooKeeperWatcher zookeeper, + HServerAddress location) + throws KeeperException { + LOG.info("Setting ROOT region location in ZooKeeper as " + location); + try { + ZKUtil.createAndWatch(zookeeper, zookeeper.rootServerZNode, + Bytes.toBytes(location.toString())); + } catch(KeeperException.NodeExistsException nee) { + LOG.debug("ROOT region location already existed, updated location"); + ZKUtil.setData(zookeeper, zookeeper.rootServerZNode, + Bytes.toBytes(location.toString())); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java (revision 0) @@ -0,0 +1,333 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.catalog; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.client.ServerConnection; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.master.NotAllMetaRegionsOnlineException; +import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker; +import org.apache.hadoop.hbase.zookeeper.RootRegionTracker; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; + +/** + * Tracks the availability of the catalog tables -ROOT- and + * .META.. + *

        + * This class is "read-only" in that the locations of the catalog tables cannot + * be explicitly set. Instead, ZooKeeper is used to learn of the availability + * and location of ROOT. ROOT is used to learn of the location of META. If not + * available in ROOT, ZooKeeper is used to monitor for a new location of META. + */ +public class CatalogTracker { + private static final Log LOG = LogFactory.getLog(CatalogTracker.class); + + private final ServerConnection connection; + + private final ZooKeeperWatcher zookeeper; + + private final RootRegionTracker rootRegionTracker; + + private final MetaNodeTracker metaNodeTracker; + + private final AtomicBoolean metaAvailable = new AtomicBoolean(false); + private HServerAddress metaLocation; + + private final int defaultTimeout; + + public static final byte [] ROOT_REGION = + HRegionInfo.ROOT_REGIONINFO.getRegionName(); + + public static final byte [] META_REGION = + HRegionInfo.FIRST_META_REGIONINFO.getRegionName(); + + /** + * Constructs the catalog tracker. Find current state of catalog tables and + * begin active tracking by executing {@link #start()}. + * @param zookeeper zk reference + * @param connection server connection + * @param abortable if fatal exception + */ + public CatalogTracker(ZooKeeperWatcher zookeeper, ServerConnection connection, + Abortable abortable, int defaultTimeout) { + this.zookeeper = zookeeper; + this.connection = connection; + this.rootRegionTracker = new RootRegionTracker(zookeeper, abortable); + this.metaNodeTracker = new MetaNodeTracker(zookeeper, this); + this.defaultTimeout = defaultTimeout; + } + + /** + * Starts the catalog tracker. + *

        + * Determines current availability of catalog tables and ensures all further + * transitions of either region is tracked. + * @throws IOException + */ + public void start() throws IOException { + // Register listeners with zk + zookeeper.registerListener(rootRegionTracker); + zookeeper.registerListener(metaNodeTracker); + // Start root tracking + rootRegionTracker.start(); + // Determine meta assignment + getMetaServerConnection(true); + } + + /** + * Gets the current location for -ROOT- or null if location is + * not currently available. + * @return location of root, null if not available + */ + public HServerAddress getRootLocation() { + return rootRegionTracker.getRootRegionLocation(); + } + + /** + * Waits indefinitely for availability of -ROOT-. Used during + * cluster startup. + * @throws InterruptedException if interrupted while waiting + */ + public void waitForRoot() + throws InterruptedException { + rootRegionTracker.waitRootRegionLocation(0); + } + + /** + * Gets the current location for -ROOT- if available and waits + * for up to the specified timeout if not immediately available. Returns null + * if the timeout elapses before root is available. + * @param timeout maximum time to wait for root availability, in milliseconds + * @return location of root + * @throws InterruptedException if interrupted while waiting + * @throws NotAllMetaRegionsOnlineException if root not available before + * timeout + */ + public HServerAddress waitForRoot(long timeout) + throws InterruptedException, NotAllMetaRegionsOnlineException { + HServerAddress address = rootRegionTracker.waitRootRegionLocation(timeout); + if (address == null) { + throw new NotAllMetaRegionsOnlineException( + "Timed out (" + timeout + "ms)"); + } + return address; + } + + /** + * Gets a connection to the server hosting root, as reported by ZooKeeper, + * waiting up to the specified timeout for availability. + * @see #waitForRoot(long) for additional information + * @return connection to server hosting root + * @throws InterruptedException + * @throws NotAllMetaRegionsOnlineException if timed out waiting + * @throws IOException + */ + public HRegionInterface waitForRootServerConnection(long timeout) + throws InterruptedException, NotAllMetaRegionsOnlineException, IOException { + return getCachedConnection(waitForRoot(timeout)); + } + + /** + * Gets a connection to the server hosting root, as reported by ZooKeeper, + * waiting for the default timeout specified on instantiation. + * @see #waitForRoot(long) for additional information + * @return connection to server hosting root + * @throws NotAllMetaRegionsOnlineException if timed out waiting + * @throws IOException + */ + public HRegionInterface waitForRootServerConnectionDefault() + throws NotAllMetaRegionsOnlineException, IOException { + try { + return getCachedConnection(waitForRoot(defaultTimeout)); + } catch (InterruptedException e) { + throw new NotAllMetaRegionsOnlineException("Interrupted"); + } + } + + /** + * Gets a connection to the server hosting root, as reported by ZooKeeper, + * if available. Returns null if no location is immediately available. + * @return connection to server hosting root, null if not available + * @throws IOException + */ + private HRegionInterface getRootServerConnection() + throws IOException { + HServerAddress address = rootRegionTracker.getRootRegionLocation(); + if (address == null) { + return null; + } + return getCachedConnection(address); + } + + /** + * Gets a connection to the server currently hosting .META. or + * null if location is not currently available. + *

        + * If a location is known, a connection to the cached location is returned. + * If refresh is true, the cached connection is verified first before + * returning. If the connection is not valid, it is reset and rechecked. + *

        + * If no location for meta is currently known, method checks ROOT for a new + * location, verifies META is currently there, and returns a cached connection + * to the server hosting META. + * + * @return connection to server hosting meta, null if location not available + * @throws IOException + */ + private HRegionInterface getMetaServerConnection(boolean refresh) + throws IOException { + synchronized(metaAvailable) { + if(metaAvailable.get()) { + HRegionInterface current = getCachedConnection(metaLocation); + if(!refresh) { + return current; + } + if(verifyRegionLocation(current, META_REGION)) { + return current; + } + resetMetaLocation(); + } + HRegionInterface rootConnection = getRootServerConnection(); + if(rootConnection == null) { + return null; + } + HServerAddress newLocation = MetaReader.readMetaLocation(rootConnection); + if(newLocation == null) { + return null; + } + HRegionInterface newConnection = getCachedConnection(newLocation); + if(verifyRegionLocation(newConnection, META_REGION)) { + setMetaLocation(newLocation); + return newConnection; + } + return null; + } + } + + /** + * Waits indefinitely for availability of .META.. Used during + * cluster startup. + * @throws InterruptedException if interrupted while waiting + */ + public void waitForMeta() throws InterruptedException { + synchronized(metaAvailable) { + while(!metaAvailable.get()) { + metaAvailable.wait(); + } + } + } + + /** + * Gets the current location for .META. if available and waits + * for up to the specified timeout if not immediately available. Throws an + * exception if timed out waiting. + * @param timeout maximum time to wait for meta availability, in milliseconds + * @return location of meta + * @throws InterruptedException if interrupted while waiting + * @throws IOException unexpected exception connecting to meta server + * @throws NotAllMetaRegionsOnlineException if meta not available before + * timeout + */ + public HServerAddress waitForMeta(long timeout) + throws InterruptedException, IOException, NotAllMetaRegionsOnlineException { + long stop = System.currentTimeMillis() + timeout; + synchronized(metaAvailable) { + if(getMetaServerConnection(true) != null) { + return metaLocation; + } + while(!metaAvailable.get() && + (timeout == 0 || System.currentTimeMillis() < stop)) { + metaAvailable.wait(timeout); + } + if(getMetaServerConnection(true) == null) { + throw new NotAllMetaRegionsOnlineException( + "Timed out (" + timeout + "ms"); + } + return metaLocation; + } + } + + /** + * Gets a connection to the server hosting meta, as reported by ZooKeeper, + * waiting up to the specified timeout for availability. + * @see #waitForMeta(long) for additional information + * @return connection to server hosting meta + * @throws InterruptedException + * @throws NotAllMetaRegionsOnlineException if timed out waiting + * @throws IOException + */ + public HRegionInterface waitForMetaServerConnection(long timeout) + throws InterruptedException, NotAllMetaRegionsOnlineException, IOException { + return getCachedConnection(waitForMeta(timeout)); + } + + /** + * Gets a connection to the server hosting meta, as reported by ZooKeeper, + * waiting up to the specified timeout for availability. + * @see #waitForMeta(long) for additional information + * @return connection to server hosting meta + * @throws NotAllMetaRegionsOnlineException if timed out or interrupted + * @throws IOException + */ + public HRegionInterface waitForMetaServerConnectionDefault() + throws NotAllMetaRegionsOnlineException, IOException { + try { + return getCachedConnection(waitForMeta(defaultTimeout)); + } catch (InterruptedException e) { + throw new NotAllMetaRegionsOnlineException("Interrupted"); + } + } + + private void resetMetaLocation() { + LOG.info("Current cached META location is not valid, resetting"); + metaAvailable.set(false); + metaLocation = null; + } + + private void setMetaLocation(HServerAddress metaLocation) { + LOG.info("Found new META location, " + metaLocation); + metaAvailable.set(true); + this.metaLocation = metaLocation; + // no synchronization because these are private and already under lock + metaAvailable.notifyAll(); + } + + private HRegionInterface getCachedConnection(HServerAddress address) + throws IOException { + return connection.getHRegionConnection(address, false); + } + + private boolean verifyRegionLocation(HRegionInterface metaServer, + byte [] regionName) { + try { + return metaServer.getRegionInfo(regionName) != null; + } catch (NotServingRegionException e) { + return false; + } + } +} Index: src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java (revision 0) @@ -0,0 +1,261 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.catalog; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Writables; + +/** + * Reads region and assignment information from .META.. + *

        + * Uses the {@link CatalogTracker} to obtain locations and connections to + * catalogs. + */ +public class MetaReader { + + /** + * Performs a full scan of .META.. + *

        + * Returns a map of every region to it's currently assigned server, according + * to META. If the region does not have an assignment it will have a null + * value in the map. + * + * @return map of regions to their currently assigned server + * @throws IOException + */ + public static Map fullScan( + CatalogTracker catalogTracker) + throws IOException { + HRegionInterface metaServer = + catalogTracker.waitForMetaServerConnectionDefault(); + Map allRegions = + new TreeMap(); + Scan scan = new Scan(); + scan.addFamily(HConstants.CATALOG_FAMILY); + long scannerid = metaServer.openScanner( + HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), scan); + try { + Result data; + while((data = metaServer.next(scannerid)) != null) { + if (!data.isEmpty()) { + Pair region = + metaRowToRegionPair(data); + allRegions.put(region.getFirst(), region.getSecond()); + } + } + } finally { + metaServer.close(scannerid); + } + return allRegions; + } + + /** + * Reads the location of META from ROOT. + * @param metaServer connection to server hosting ROOT + * @return location of META in ROOT, null if not available + * @throws IOException + */ + public static HServerAddress readMetaLocation(HRegionInterface metaServer) + throws IOException { + return readLocation(metaServer, CatalogTracker.ROOT_REGION, + CatalogTracker.META_REGION); + } + + /** + * Reads the location of the specified region from META. + * @param catalogTracker + * @param regionName region to read location of + * @return location of region in META, null if not available + * @throws IOException + */ + public static HServerAddress readRegionLocation(CatalogTracker catalogTracker, + byte [] regionName) + throws IOException { + return readLocation(catalogTracker.waitForMetaServerConnectionDefault(), + CatalogTracker.META_REGION, regionName); + } + + private static HServerAddress readLocation(HRegionInterface metaServer, + byte [] catalogRegionName, byte [] regionName) + throws IOException { + Result r = metaServer.get(catalogRegionName, new Get(regionName).addColumn( + HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER)); + if(r == null || r.isEmpty()) { + return null; + } + byte [] value = + r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); + return new HServerAddress(Bytes.toString(value)); + } + + /** + * Gets the region info and assignment for the specified region from META. + * @param catalogTracker + * @param regionName + * @return region info and assignment from META, null if not available + * @throws IOException + */ + public static Pair getRegion( + CatalogTracker catalogTracker, byte [] regionName) + throws IOException { + Get get = new Get(regionName); + get.addFamily(HConstants.CATALOG_FAMILY); + Result r = catalogTracker.waitForMetaServerConnectionDefault().get( + CatalogTracker.META_REGION, get); + if(r == null || r.isEmpty()) { + return null; + } + return metaRowToRegionPair(r); + } + + public static Pair metaRowToRegionPair( + Result data) throws IOException { + HRegionInfo info = Writables.getHRegionInfo( + data.getValue(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)); + final byte[] value = data.getValue(HConstants.CATALOG_FAMILY, + HConstants.SERVER_QUALIFIER); + if (value != null && value.length > 0) { + HServerAddress server = new HServerAddress(Bytes.toString(value)); + return new Pair(info, server); + } else { + return new Pair(info, null); + } + } + + /** + * Checks if the specified table exists. Looks at the META table hosted on + * the specified server. + * @param metaServer server hosting meta + * @param tableName table to check + * @return true if the table exists in meta, false if not + * @throws IOException + */ + public static boolean tableExists(CatalogTracker catalogTracker, + String tableName) + throws IOException { + HRegionInterface metaServer = + catalogTracker.waitForMetaServerConnectionDefault(); + byte[] firstRowInTable = Bytes.toBytes(tableName + ",,"); + Scan scan = new Scan(firstRowInTable); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + long scannerid = metaServer.openScanner( + HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), scan); + try { + Result data = metaServer.next(scannerid); + if (data != null && data.size() > 0) { + HRegionInfo info = Writables.getHRegionInfo( + data.getValue(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)); + if (info.getTableDesc().getNameAsString().equals(tableName)) { + // A region for this table already exists. Ergo table exists. + return true; + } + } + return false; + } finally { + metaServer.close(scannerid); + } + } + + /** + * Gets all of the regions of the specified table from META. + * @param catalogTracker + * @param tableName + * @return + * @throws IOException + */ + public static List getTableRegions(CatalogTracker catalogTracker, + byte [] tableName) + throws IOException { + HRegionInterface metaServer = + catalogTracker.waitForMetaServerConnectionDefault(); + List regions = new ArrayList(); + String tableString = Bytes.toString(tableName); + byte[] firstRowInTable = Bytes.toBytes(tableString + ",,"); + Scan scan = new Scan(firstRowInTable); + scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); + long scannerid = metaServer.openScanner( + HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), scan); + try { + Result data; + while((data = metaServer.next(scannerid)) != null) { + if (data != null && data.size() > 0) { + HRegionInfo info = Writables.getHRegionInfo( + data.getValue(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)); + if (info.getTableDesc().getNameAsString().equals(tableString)) { + regions.add(info); + } else { + break; + } + } + } + return regions; + } finally { + metaServer.close(scannerid); + } + } + + public static List> + getTableRegionsAndLocations(CatalogTracker catalogTracker, String tableName) + throws IOException { + HRegionInterface metaServer = + catalogTracker.waitForMetaServerConnectionDefault(); + List> regions = + new ArrayList>(); + byte[] firstRowInTable = Bytes.toBytes(tableName + ",,"); + Scan scan = new Scan(firstRowInTable); + scan.addFamily(HConstants.CATALOG_FAMILY); + long scannerid = metaServer.openScanner( + HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), scan); + try { + Result data; + while((data = metaServer.next(scannerid)) != null) { + if (data != null && data.size() > 0) { + Pair region = metaRowToRegionPair(data); + if (region.getFirst().getTableDesc().getNameAsString().equals( + tableName)) { + regions.add(region); + } else { + break; + } + } + } + return regions; + } finally { + metaServer.close(scannerid); + } + } +} Index: src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/catalog/MetaEditor.java (revision 0) @@ -0,0 +1,158 @@ +/** + * Copyright 2010 The Apache Software Foundation + * + * 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.catalog; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerInfo; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; + +/** + * Writes region and assignment information to .META.. + *

        + * Uses the {@link CatalogTracker} to obtain locations and connections to + * catalogs. + */ +public class MetaEditor { + private static final Log LOG = LogFactory.getLog(MetaEditor.class); + + /** + * Adds a META row for the specified new region. + * @param info region information + * @throws IOException if problem connecting or updating meta + */ + public static void addRegionToMeta(CatalogTracker catalogTracker, + HRegionInfo regionInfo) + throws IOException { + Put put = new Put(regionInfo.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(regionInfo)); + catalogTracker.waitForMetaServerConnectionDefault().put( + CatalogTracker.META_REGION, put); + LOG.info("Added region " + regionInfo + " to META"); + } + + /** + * Updates the location of the specified META region in ROOT to be the + * specified server hostname and startcode. + *

        + * Uses passed catalog tracker to get a connection to the server hosting + * ROOT and makes edits to that region. + * + * @param catalogTracker catalog tracker + * @param regionInfo region to update location of + * @param serverInfo server the region is located on + * @throws IOException + */ + public static void updateMetaLocation(CatalogTracker catalogTracker, + HRegionInfo regionInfo, HServerInfo serverInfo) + throws IOException { + updateLocation(catalogTracker.waitForRootServerConnectionDefault(), + CatalogTracker.ROOT_REGION, regionInfo, serverInfo); + } + + /** + * Updates the location of the specified region in META to be the specified + * server hostname and startcode. + *

        + * Uses passed catalog tracker to get a connection to the server hosting + * META and makes edits to that region. + * + * @param catalogTracker catalog tracker + * @param regionInfo region to update location of + * @param serverInfo server the region is located on + * @throws IOException + */ + public static void updateRegionLocation(CatalogTracker catalogTracker, + HRegionInfo regionInfo, HServerInfo serverInfo) + throws IOException { + updateLocation(catalogTracker.waitForMetaServerConnectionDefault(), + CatalogTracker.META_REGION, regionInfo, serverInfo); + } + + /** + * Updates the location of the specified region to be the specified server. + *

        + * Connects to the specified server which should be hosting the specified + * catalog region name to perform the edit. + * + * @param server connection to server hosting catalog region + * @param catalogRegionName name of catalog region being updated + * @param regionInfo region to update location of + * @param serverInfo server the region is located on + * @throws IOException + */ + public static void updateLocation(HRegionInterface server, + byte [] catalogRegionName, HRegionInfo regionInfo, HServerInfo serverInfo) + throws IOException { + Put put = new Put(regionInfo.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, + Bytes.toBytes(serverInfo.getHostnamePort())); + put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, + Bytes.toBytes(serverInfo.getStartCode())); + server.put(catalogRegionName, put); + + LOG.info("Updated row " + regionInfo.getRegionNameAsString() + + " in region " + Bytes.toString(catalogRegionName) + " with " + + "server=" + serverInfo.getHostnamePort() + ", " + + "startcode=" + serverInfo.getStartCode()); + } + + /** + * Deletes the specified region from META. + * @param catalogTracker + * @param regionInfo region to be deleted from META + * @throws IOException + */ + public static void deleteRegion(CatalogTracker catalogTracker, + HRegionInfo regionInfo) + throws IOException { + Delete delete = new Delete(regionInfo.getRegionName()); + catalogTracker.waitForMetaServerConnectionDefault().delete( + CatalogTracker.META_REGION, delete); + + LOG.info("Deleted region " + regionInfo + " from META"); + } + + /** + * Updates the region information for the specified region in META. + * @param catalogTracker + * @param regionInfo region to be updated in META + * @throws IOException + */ + public static void updateRegionInfo(CatalogTracker catalogTracker, + HRegionInfo regionInfo) + throws IOException { + Put put = new Put(regionInfo.getRegionName()); + put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, + Writables.getBytes(regionInfo)); + catalogTracker.waitForMetaServerConnectionDefault().put( + CatalogTracker.META_REGION, put); + LOG.info("Updated region " + regionInfo + " in META"); + } +} Index: src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (working copy) @@ -19,14 +19,12 @@ */ package org.apache.hadoop.hbase.ipc; +import java.io.IOException; + import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.io.Writable; -import java.io.IOException; - /** * Clients interact with the HMasterInterface to gain access to meta-level * HBase functionality, like finding an HRegionServer and creating/destroying @@ -110,11 +108,10 @@ * Modify a table's metadata * * @param tableName table to modify - * @param op the operation to do - * @param args arguments for operation + * @param htd new descriptor for table * @throws IOException e */ - public void modifyTable(byte[] tableName, HConstants.Modify op, Writable[] args) + public void modifyTable(byte[] tableName, HTableDescriptor htd) throws IOException; /** Index: src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java (working copy) @@ -19,6 +19,9 @@ */ package org.apache.hadoop.hbase.ipc; +import java.io.IOException; +import java.util.List; + import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.NotServingRegionException; @@ -31,9 +34,6 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; -import java.io.IOException; -import java.util.List; - /** * Clients interact with HRegionServers using a handle to the HRegionInterface. * @@ -279,10 +279,64 @@ * @throws IOException e */ public MultiPutResponse multiPut(MultiPut puts) throws IOException; - + /** * Bulk load an HFile into an open region */ public void bulkLoadHFile(String hfilePath, byte[] regionName, byte[] familyName) throws IOException; + + // Master methods + + /** + * Opens the specified region. + * @param region region to open + */ + public void openRegion(final HRegionInfo region); + + /** + * Closes the specified region. + * @param region region to close + * @return true if closing region, false if not + */ + public boolean closeRegion(final HRegionInfo region) + throws NotServingRegionException; + + // Region administrative methods + + /** + * Flushes the MemStore of the specified region. + *

        + * This method is synchronous. + * @param regionInfo region to flush + * @throws NotServingRegionException + * @throws IOException + */ + void flushRegion(HRegionInfo regionInfo) + throws NotServingRegionException, IOException; + + /** + * Splits the specified region. + *

        + * This method currently flushes the region and then forces a compaction which + * will then trigger a split. The flush is done synchronously but the + * compaction is asynchronous. + * @param regionInfo region to split + * @throws NotServingRegionException + * @throws IOException + */ + void splitRegion(HRegionInfo regionInfo) + throws NotServingRegionException, IOException; + + /** + * Compacts the specified region. Performs a major compaction if specified. + *

        + * This method is asynchronous. + * @param regionInfo region to compact + * @param major true to force major compaction + * @throws NotServingRegionException + * @throws IOException + */ + void compactRegion(HRegionInfo regionInfo, boolean major) + throws NotServingRegionException, IOException; } Index: src/main/java/org/apache/hadoop/hbase/Abortable.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/Abortable.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/Abortable.java (working copy) @@ -21,11 +21,11 @@ /** * Interface to support the aborting of a given server or client. - * - *

        This is used primarily for ZooKeeper usage when we could get an unexpected + *

        + * This is used primarily for ZooKeeper usage when we could get an unexpected * and fatal exception, requiring an abort. - * - *

        Implemented by the Master, RegionServer, and TableServers (client). + *

        + * Implemented by the Master, RegionServer, and TableServers (client). */ public interface Abortable { /** Index: src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (working copy) @@ -20,6 +20,10 @@ package org.apache.hadoop.hbase.client; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -27,8 +31,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; -import java.io.IOException; - /** * Scanner class that contains the .META. table scanning logic * and uses a Retryable scanner. Provided visitors will be called @@ -170,6 +172,32 @@ } /** + * Lists all of the regions currently in META. + * @return + * @throws IOException + */ + public static List listAllRegions(Configuration conf) + throws IOException { + final List regions = new ArrayList(); + MetaScannerVisitor visitor = + new MetaScannerVisitor() { + @Override + public boolean processRow(Result result) throws IOException { + if (result == null || result.isEmpty()) { + return true; + } + HRegionInfo regionInfo = Writables.getHRegionInfo( + result.getValue(HConstants.CATALOG_FAMILY, + HConstants.REGIONINFO_QUALIFIER)); + regions.add(regionInfo); + return true; + } + }; + metaScan(conf, visitor); + return regions; + } + + /** * Visitor class called to process each row of the .META. table */ public interface MetaScannerVisitor { Index: src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (working copy) @@ -40,6 +40,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; @@ -50,7 +51,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.ServerController; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; @@ -185,7 +185,7 @@ } /* Encapsulates finding the servers for an HBase instance */ - static class TableServers implements ServerConnection, ServerController { + static class TableServers implements ServerConnection, Abortable { static final Log LOG = LogFactory.getLog(TableServers.class); private final Class serverInterfaceClass; private final long pause; @@ -262,7 +262,8 @@ // initialize zookeeper and master address manager getZooKeeperWatcher(); masterAddressManager = new MasterAddressManager(zooKeeper, this); - masterAddressManager.monitorMaster(); + zooKeeper.registerListener(masterAddressManager); + masterAddressManager.start(); this.master = null; this.masterChecked = false; @@ -581,6 +582,20 @@ return result; } + @Override + public HRegionLocation locateRegion(final byte [] regionName) + throws IOException { + // TODO implement. use old stuff or new stuff? + return null; + } + + @Override + public List locateRegions(final byte [] tableName) + throws IOException { + // TODO implement. use old stuff or new stuff? + return null; + } + public HRegionLocation locateRegion(final byte [] tableName, final byte [] row) throws IOException{ @@ -972,6 +987,7 @@ regionServer.getInetSocketAddress(), this.conf, this.maxRPCAttempts, this.rpcTimeout); } catch (RemoteException e) { + LOG.warn("Remove exception connecting to RS", e); throw RemoteExceptionHandler.decodeRemoteException(e); } this.servers.put(regionServer.toString(), server); @@ -1006,7 +1022,7 @@ return zooKeeper; } - /* + /** * Repeatedly try to find the root region in ZK * @return HRegionLocation for root region if found * @throws NoServerForRegionException - if the root region can not be @@ -1579,13 +1595,6 @@ } } - // ServerController implementation so that we can use ZooKeeperWatcher - // Our abort() call does the ZK reset() as was previously done when - // getting ZK expiration - // TODO: Maybe this is not right. Should there be a super-interface to - // ServerStatus/Controller that _just_ has the abort method? - // The only method that really makes no sense here is get address - @Override public void abort() { if(zooKeeper != null) { @@ -1593,25 +1602,5 @@ zooKeeper = null; } } - - @Override - public Configuration getConfiguration() { - return conf; - } - - @Override - public HServerAddress getHServerAddress() { - return null; - } - - @Override - public ZooKeeperWatcher getZooKeeper() { - try { - return getZooKeeperWatcher(); - } catch (IOException e) { - LOG.error("Problem getting zk watcher", e); - return null; - } - } } } Index: src/main/java/org/apache/hadoop/hbase/client/HConnection.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/client/HConnection.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/client/HConnection.java (working copy) @@ -136,7 +136,7 @@ * lives in, ignoring any value that might be in the cache. * @param tableName name of the table row is in * @param row row key you're trying to find the region of - * @return HRegionLocation that describes where to find the reigon in + * @return HRegionLocation that describes where to find the region in * question * @throws IOException if a remote or network exception occurs */ @@ -145,6 +145,25 @@ throws IOException; /** + * Gets the location of the region of regionName. + * @param regionName name of the region to locate + * @return HRegionLocation that describes where to find the region in + * question + * @throws IOException if a remote or network exception occurs + */ + public HRegionLocation locateRegion(final byte [] regionName) + throws IOException; + + /** + * Gets the locations of all regions in the specified table, tableName. + * @param tableName table to get regions of + * @return list of region locations for all regions of table + * @throws IOException + */ + public List locateRegions(byte[] tableName) + throws IOException; + + /** * Establishes a connection to the region server at the specified address. * @param regionServer - the server to connect to * @return proxy for HRegionServer @@ -199,7 +218,7 @@ * @throws IOException if a remote or network exception occurs * @throws RuntimeException other unspecified error */ - public T getRegionServerWithoutRetries(ServerCallable callable) + public T getRegionServerWithoutRetries(ServerCallable callable) throws IOException, RuntimeException; Index: src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (revision 979909) +++ src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (working copy) @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -642,12 +643,15 @@ * Asynchronous operation. * * @param regionname region name to close - * @param args Optional server name. Otherwise, we'll send close to the + * @param Optional server name. Otherwise, we'll send close to the * server registered in .META. * @throws IOException if a remote or network exception occurs */ public void closeRegion(final String regionname, final Object... args) throws IOException { + // TODO: reimplement this. i don't think we will be able to send a + // server name anymore as client does not have this, would have to + // use META or be passed an HServerAddress closeRegion(Bytes.toBytes(regionname), args); } @@ -693,10 +697,43 @@ * @throws IOException if a remote or network exception occurs */ public void flush(final byte [] tableNameOrRegionName) throws IOException { - modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_FLUSH); + // TODO: implement two new connection methods and change split/compact + boolean isTable = checkTableOrRegion(tableNameOrRegionName); + if(!isTable) { + HRegionLocation hrl = connection.locateRegion(tableNameOrRegionName); + HRegionInterface hri = + connection.getHRegionConnection(hrl.getServerAddress()); + hri.flushRegion(hrl.getRegionInfo()); + return; + } + List regionLocations = + connection.locateRegions(tableNameOrRegionName); + for(HRegionLocation hrl : regionLocations) { + HRegionInterface hri = + connection.getHRegionConnection(hrl.getServerAddress()); + hri.flushRegion(hrl.getRegionInfo()); + } } /** + * Checks if the specified table or region name is a table or region. Returns + * true if it is a table name and false if not (so likely a region name). + * @param tableNameOrRegionName + * @return true if table, false if not + * @throws IllegalArgumentException + * @throws ZooKeeperConnectionException + * @throws MasterNotRunningException + */ + private boolean checkTableOrRegion(final byte [] tableNameOrRegionName) + throws IllegalArgumentException, MasterNotRunningException, + ZooKeeperConnectionException { + if (tableNameOrRegionName == null) { + throw new IllegalArgumentException("Pass a table name or region name"); + } + return tableExists(tableNameOrRegionName); + } + + /** * Compact a table or an individual region. * Asynchronous operation. * @@ -825,7 +862,7 @@ } arr = new Writable[1]; arr[0] = (HTableDescriptor)args[0]; - getMaster().modifyTable(tableName, op, arr); +// getMaster().modifyTable(tableName, op, arr); break; case TABLE_COMPACT: @@ -845,7 +882,7 @@ "ImmutableBytesWritable"); } } - getMaster().modifyTable(tableName, op, arr); +// getMaster().modifyTable(tableName, op, arr); break; case CLOSE_REGION: @@ -867,7 +904,7 @@ "ImmutableBytesWritable, not " + args[i]); } } - getMaster().modifyTable(tableName, op, arr); +// getMaster().modifyTable(tableName, op, arr); break; default: