|
@@ -15,19 +15,33 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hbase;
|
|
|
|
|
|
+import java.io.ByteArrayOutputStream;
|
|
|
+import java.io.DataOutputStream;
|
|
|
+import java.io.IOException;
|
|
|
+import java.io.UnsupportedEncodingException;
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.Iterator;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Random;
|
|
|
+import java.util.SortedMap;
|
|
|
+import java.util.Timer;
|
|
|
+import java.util.TimerTask;
|
|
|
+import java.util.TreeMap;
|
|
|
+import java.util.TreeSet;
|
|
|
+import java.util.Vector;
|
|
|
+
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
-
|
|
|
-import org.apache.hadoop.conf.*;
|
|
|
-import org.apache.hadoop.io.*;
|
|
|
-import org.apache.hadoop.fs.*;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
-import org.apache.hadoop.ipc.*;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.io.BytesWritable;
|
|
|
+import org.apache.hadoop.io.DataInputBuffer;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
+import org.apache.hadoop.ipc.RPC;
|
|
|
+import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
|
|
|
-import java.io.*;
|
|
|
-import java.util.*;
|
|
|
-
|
|
|
/**
|
|
|
* HMaster is the "master server" for a HBase.
|
|
|
* There is only one HMaster for a single HBase deployment.
|
|
@@ -604,7 +618,8 @@ public class HMaster implements HConstants, HMasterInterface,
|
|
|
msgQueue.insertElementAt(op, 0);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+ letRegionServersShutdown();
|
|
|
+
|
|
|
/*
|
|
|
* Clean up and close up shop
|
|
|
*/
|
|
@@ -653,11 +668,34 @@ public class HMaster implements HConstants, HMasterInterface,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Wait on regionservers to report in. Then, they notice the HMaster
|
|
|
+ * is going down and will shut themselves down.
|
|
|
+ */
|
|
|
+ private void letRegionServersShutdown() {
|
|
|
+ long regionServerMsgInterval =
|
|
|
+ conf.getLong("hbase.regionserver.msginterval", 15 * 1000);
|
|
|
+ // Wait for 3 * hbase.regionserver.msginterval intervals or until all
|
|
|
+ // regionservers report in as closed.
|
|
|
+ long endTime = System.currentTimeMillis() + (regionServerMsgInterval * 3);
|
|
|
+ while (endTime > System.currentTimeMillis() &&
|
|
|
+ this.serversToServerInfo.size() > 0) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Waiting on regionservers: " + this.serversToServerInfo);
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ Thread.sleep(threadWakeFrequency);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ // Ignore interrupt.
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Turn off the HMaster. Sets a flag so that the main thread know to shut
|
|
|
* things down in an orderly fashion.
|
|
|
*/
|
|
|
- public void shutdown() throws IOException {
|
|
|
+ public void shutdown() {
|
|
|
TimerTask tt = new TimerTask() {
|
|
|
@Override
|
|
|
public void run() {
|
|
@@ -711,13 +749,10 @@ public class HMaster implements HConstants, HMasterInterface,
|
|
|
public HMsg[] regionServerReport(HServerInfo serverInfo, HMsg msgs[]) throws IOException {
|
|
|
String server = serverInfo.getServerAddress().toString().trim();
|
|
|
|
|
|
- if(closed) {
|
|
|
+ if (closed) {
|
|
|
// We're shutting down. Tell the server to go away.
|
|
|
serversToServerInfo.remove(server);
|
|
|
-
|
|
|
- HMsg returnMsgs[] = {
|
|
|
- new HMsg(HMsg.MSG_REGIONSERVER_STOP)
|
|
|
- };
|
|
|
+ HMsg returnMsgs[] = {new HMsg(HMsg.MSG_REGIONSERVER_STOP)};
|
|
|
return returnMsgs;
|
|
|
}
|
|
|
|