/**
@@ -151,7 +135,20 @@
* @param hri Region to which message <code>type</code> applies
*/
public HMsg(final HMsg.Type type, final HRegionInfo hri) {
- this(type, hri, null);
+ this(type, hri, null, false);
+ }
+
+ /**
+ * Constructor used by master to inform region servers if we are still in
+ * safe mode.
+ *
+ * @param type
+ * @param hri
+ * @param safeMode
+ */
+ public HMsg(final HMsg.Type type, final HRegionInfo hri,
+ final boolean safeMode) {
+ this(type, hri, null, safeMode);
}
/**
@@ -163,6 +160,19 @@
* @param msg Optional message (Stringified exception, etc.)
*/
public HMsg(final HMsg.Type type, final HRegionInfo hri, final byte[] msg) {
+ this(type, hri, msg, false);
+ }
+
+ /**
+ * Used by the master to inform region servers if we are still in safe mode
+ *
+ * @param type
+ * @param hri
+ * @param msg
+ * @param safemode
+ */
+ public HMsg(final HMsg.Type type, final HRegionInfo hri, final byte[] msg,
+ final boolean safemode) {
if (type == null) {
throw new NullPointerException("Message type cannot be null");
}
@@ -172,6 +182,7 @@
}
this.info = hri;
this.message = msg;
+ this.safeMode = safemode;
}
/**
@@ -199,6 +210,11 @@
return this.message;
}
+ /** @return safe mode */
+ public boolean isInSafeMode() {
+ return this.safeMode;
+ }
+
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
@@ -211,6 +227,7 @@
if (this.message != null && this.message.length > 0) {
sb.append(": " + Bytes.toString(this.message));
}
+ sb.append(": safeMode=" + safeMode);
return sb.toString();
}
@@ -244,6 +261,7 @@
out.writeBoolean(true);
Bytes.writeByteArray(out, this.message);
}
+ out.writeBoolean(this.safeMode);
}
public void readFields(DataInput in) throws IOException {
@@ -254,5 +272,6 @@
if (hasMessage) {
this.message = Bytes.readByteArray(in);
}
+ this.safeMode = in.readBoolean();
}
}
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
Fri Jan 23 11:56:14 2009
@@ -134,6 +134,7 @@
private final Map<String, HRegionInterface> servers =
new ConcurrentHashMap<String, HRegionInterface>();
+ // Used by master and region servers during safe mode only
private volatile HRegionLocation rootRegionLocation;
private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>>
@@ -177,10 +178,12 @@
return this.pause * HConstants.RETRY_BACKOFF[ntries];
}
+ // Used by master and region servers during safe mode only
public void unsetRootRegionLocation() {
this.rootRegionLocation = null;
}
+ // Used by master and region servers during safe mode only
public void setRootRegionLocation(HRegionLocation rootRegion) {
if (rootRegion == null) {
throw new IllegalArgumentException(
@@ -445,7 +448,7 @@
// second waits. The second thread will not do find.
if (!useCache || rootRegionLocation == null) {
- this.rootRegionLocation = locateRootRegion();
+ return locateRootRegion();
}
return rootRegionLocation;
}
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionClose.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionClose.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionClose.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionClose.java
Fri Jan 23 11:56:14 2009
@@ -61,7 +61,7 @@
Boolean result = null;
if (offlineRegion) {
result =
- new RetryableMetaOperation<Boolean>(this.metaRegion, this.master) {
+ new RetryableMetaOperation<Boolean>(getMetaRegion(), this.master) {
public Boolean call() throws IOException {
LOG.info("region closed: " + regionInfo.getRegionNameAsString());
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
Fri Jan 23 11:56:14 2009
@@ -64,7 +64,7 @@
@Override
protected boolean process() throws IOException {
Boolean result =
- new RetryableMetaOperation<Boolean>(this.metaRegion, this.master) {
+ new RetryableMetaOperation<Boolean>(getMetaRegion(), this.master) {
private final RegionHistorian historian =
RegionHistorian.getInstance();
public Boolean call() throws IOException {
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java
Fri Jan 23 11:56:14 2009
@@ -29,8 +29,8 @@
abstract class ProcessRegionStatusChange extends RegionServerOperation {
protected final boolean isMetaTable;
protected final HRegionInfo regionInfo;
- protected final MetaRegion metaRegion;
- protected final byte [] metaRegionName;
+ private volatile MetaRegion metaRegion = null;
+ protected volatile byte [] metaRegionName = null;
/**
* @param master
@@ -40,15 +40,6 @@
super(master);
this.regionInfo = regionInfo;
this.isMetaTable = regionInfo.isMetaTable();
- if (isMetaTable) {
- this.metaRegionName = HRegionInfo.ROOT_REGIONINFO.getRegionName();
- this.metaRegion = new MetaRegion(master.getRootRegionLocation(),
- this.metaRegionName, HConstants.EMPTY_START_ROW);
- } else {
- this.metaRegion =
- master.regionManager.getFirstMetaRegionForRegion(regionInfo);
- this.metaRegionName = this.metaRegion.getRegionName();
- }
}
protected boolean metaRegionAvailable() {
@@ -71,4 +62,17 @@
}
return available;
}
+
+ protected MetaRegion getMetaRegion() {
+ if (isMetaTable) {
+ this.metaRegionName = HRegionInfo.ROOT_REGIONINFO.getRegionName();
+ this.metaRegion = new MetaRegion(master.getRootRegionLocation(),
+ this.metaRegionName, HConstants.EMPTY_START_ROW);
+ } else {
+ this.metaRegion =
+ master.regionManager.getFirstMetaRegionForRegion(regionInfo);
+ this.metaRegionName = this.metaRegion.getRegionName();
+ }
+ return this.metaRegion;
+ }
}
\ No newline at end of file
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/RegionManager.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/RegionManager.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/RegionManager.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/RegionManager.java
Fri Jan 23 11:56:14 2009
@@ -274,7 +274,8 @@
" to server " + serverName);
s.setPendingOpen(serverName);
this.historian.addRegionAssignment(s.getRegionInfo(), serverName);
- returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, s.getRegionInfo()));
+ returnMsgs.add(
+ new HMsg(HMsg.Type.MSG_REGION_OPEN, s.getRegionInfo(),
inSafeMode()));
if (--nregions <= 0) {
break;
}
@@ -401,7 +402,8 @@
" to the only server " + serverName);
s.setPendingOpen(serverName);
this.historian.addRegionAssignment(s.getRegionInfo(), serverName);
- returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, s.getRegionInfo()));
+ returnMsgs.add(new HMsg(
+ HMsg.Type.MSG_REGION_OPEN, s.getRegionInfo(), inSafeMode()));
}
}
@@ -440,7 +442,7 @@
currentRegion.getRegionNameAsString());
// make a message to close the region
returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE, currentRegion,
- OVERLOADED));
+ OVERLOADED, inSafeMode()));
// mark the region as closing
setClosing(serverName, currentRegion, false);
setPendingClose(currentRegion.getRegionName());
@@ -901,6 +903,7 @@
public boolean inSafeMode() {
if (safeMode) {
if(isInitialMetaScanComplete() && regionsInTransition.size() == 0) {
+ master.connection.unsetRootRegionLocation();
safeMode = false;
LOG.info("exiting safe mode");
} else {
@@ -1065,7 +1068,7 @@
if (LOG.isDebugEnabled()) {
LOG.debug("Sending " + msg + " " + pair.getFirst() + " to " +
addr);
}
- returnMsgs.add(new HMsg(msg, pair.getFirst()));
+ returnMsgs.add(new HMsg(msg, pair.getFirst(), inSafeMode()));
i.remove();
}
}
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ServerManager.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ServerManager.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/master/ServerManager.java
Fri Jan 23 11:56:14 2009
@@ -44,6 +44,7 @@
import org.apache.hadoop.hbase.LeaseListener;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HMsg.Type;
/**
* The ServerManager class manages info about region servers - HServerInfo,
@@ -52,6 +53,13 @@
class ServerManager implements HConstants {
private static final Log LOG =
LogFactory.getLog(ServerManager.class.getName());
+ private static final HMsg REGIONSERVER_QUIESCE =
+ new HMsg(Type.MSG_REGIONSERVER_QUIESCE);
+ private static final HMsg REGIONSERVER_STOP =
+ new HMsg(Type.MSG_REGIONSERVER_STOP);
+ private static final HMsg CALL_SERVER_STARTUP =
+ new HMsg(Type.MSG_CALL_SERVER_STARTUP);
+ private static final HMsg [] EMPTY_HMSG_ARRAY = new HMsg[0];
private final AtomicInteger quiescedServers = new AtomicInteger(0);
@@ -225,7 +233,7 @@
if (msgs.length > 0) {
if (msgs[0].isType(HMsg.Type.MSG_REPORT_EXITING)) {
processRegionServerExit(serverName, msgs);
- return HMsg.EMPTY_HMSG_ARRAY;
+ return EMPTY_HMSG_ARRAY;
} else if (msgs[0].isType(HMsg.Type.MSG_REPORT_QUIESCED)) {
LOG.info("Region server " + serverName + " quiesced");
quiescedServers.incrementAndGet();
@@ -245,10 +253,10 @@
msgs[0].isType(HMsg.Type.MSG_REPORT_QUIESCED)) {
// Server is already quiesced, but we aren't ready to shut down
// return empty response
- return HMsg.EMPTY_HMSG_ARRAY;
+ return EMPTY_HMSG_ARRAY;
}
// Tell the server to stop serving any user regions
- return new HMsg [] {HMsg.REGIONSERVER_QUIESCE};
+ return new HMsg [] {REGIONSERVER_QUIESCE};
}
}
@@ -256,7 +264,7 @@
// Tell server to shut down if we are shutting down. This should
// happen after check of MSG_REPORT_EXITING above, since region server
// will send us one of these messages after it gets MSG_REGIONSERVER_STOP
- return new HMsg [] {HMsg.REGIONSERVER_STOP};
+ return new HMsg [] {REGIONSERVER_STOP};
}
HServerInfo storedInfo = serversToServerInfo.get(serverName);
@@ -267,7 +275,7 @@
// The HBaseMaster may have been restarted.
// Tell the RegionServer to start over and call regionServerStartup()
- return new HMsg[]{HMsg.CALL_SERVER_STARTUP};
+ return new HMsg[]{CALL_SERVER_STARTUP};
} else if (storedInfo.getStartCode() != serverInfo.getStartCode()) {
// This state is reachable if:
//
@@ -287,7 +295,7 @@
serversToServerInfo.notifyAll();
}
- return new HMsg[]{HMsg.REGIONSERVER_STOP};
+ return new HMsg[]{REGIONSERVER_STOP};
} else {
return processRegionServerAllsWell(serverName, serverInfo,
mostLoadedRegions, msgs);
@@ -435,7 +443,8 @@
synchronized (master.regionManager) {
// Tell the region server to close regions that we have marked for
closing.
for (HRegionInfo i: master.regionManager.getMarkedToClose(serverName)) {
- returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE, i));
+ returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE, i,
+ master.regionManager.inSafeMode()));
// Transition the region from toClose to closing state
master.regionManager.setPendingClose(i.getRegionName());
}
@@ -530,7 +539,8 @@
// 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()));
+ region, "Duplicate assignment".getBytes(),
+ master.regionManager.inSafeMode()));
} else {
if (region.isRootRegion()) {
// it was assigned, and it's not a duplicate assignment, so take it out
@@ -539,8 +549,10 @@
// Store the Root Region location (in memory)
HServerAddress rootServer = serverInfo.getServerAddress();
- master.connection.setRootRegionLocation(
- new HRegionLocation(region, rootServer));
+ if (master.regionManager.inSafeMode()) {
+ master.connection.setRootRegionLocation(
+ new HRegionLocation(region, rootServer));
+ }
master.regionManager.setRootRegionLocation(rootServer);
} else {
// Note that the table has been assigned and is waiting for the
@@ -563,7 +575,6 @@
synchronized (master.regionManager) {
if (region.isRootRegion()) {
// Root region
- master.connection.unsetRootRegionLocation();
master.regionManager.unsetRootRegion();
if (region.isOffline()) {
// Can't proceed without root region. Shutdown.
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HLog.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HLog.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HLog.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HLog.java
Fri Jan 23 11:56:14 2009
@@ -174,7 +174,7 @@
this.flushlogentries =
conf.getInt("hbase.regionserver.flushlogentries", 100);
this.blocksize =
- conf.getLong("hbase.regionserver.hlog.blocksize", 1024L * 1024L * 64L);
+ conf.getLong("hbase.regionserver.hlog.blocksize", 1024L * 1024L);
this.optionalFlushInterval =
conf.getLong("hbase.regionserver.optionallogflushinterval", 10 * 1000);
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
Modified:
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
(original)
+++
hadoop/hbase/branches/0.19/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
Fri Jan 23 11:56:14 2009
@@ -109,6 +109,8 @@
*/
public class HRegionServer implements HConstants, HRegionInterface, Runnable {
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);
// 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
@@ -200,8 +202,8 @@
final LogRoller logRoller;
final LogFlusher logFlusher;
- // safemode processing
- SafeModeThread safeModeThread;
+ // limit compactions while starting up
+ CompactionLimitThread compactionLimitThread;
// flag set after we're done setting up server threads (used for testing)
protected volatile boolean isOnline;
@@ -317,7 +319,7 @@
haveRootRegion.set(true);
}
}
- long now = System.currentTimeMillis();
+ long now = System.currentTimeMillis();
if (lastMsg != 0 && (now - lastMsg) >= serverLeaseTimeout) {
// It has been way too long since we last reported to the master.
LOG.warn("unable to report to master for " + (now - lastMsg) +
@@ -361,6 +363,15 @@
!restart && !stopRequested.get() && i < msgs.length;
i++) {
LOG.info(msgs[i].toString());
+ if (safeMode.get()) {
+ if (!msgs[i].isInSafeMode()) {
+ this.connection.unsetRootRegionLocation();
+ synchronized (safeMode) {
+ safeMode.set(false);
+ safeMode.notifyAll();
+ }
+ }
+ }
switch(msgs[i].getType()) {
case MSG_CALL_SERVER_STARTUP:
// We the MSG_CALL_SERVER_STARTUP on startup but we can also
@@ -503,7 +514,7 @@
}
try {
HMsg[] exitMsg = new HMsg[closedRegions.size() + 1];
- exitMsg[0] = HMsg.REPORT_EXITING;
+ exitMsg[0] = REPORT_EXITING;
// Tell the master what regions we are/were serving
int i = 1;
for (HRegion region: closedRegions) {
@@ -729,30 +740,24 @@
/**
* Thread for toggling safemode after some configurable interval.
*/
- private class SafeModeThread extends Thread {
+ private class CompactionLimitThread extends Thread {
@Override
public void run() {
- // first, wait the required interval before turning off safemode
- int safemodeInterval =
- conf.getInt("hbase.regionserver.safemode.period", 120 * 1000);
- try {
- Thread.sleep(safemodeInterval);
- } catch (InterruptedException ex) {
- // turn off safemode and limits on the way out due to some kind of
- // abnormal condition so we do not prevent such things as memcache
- // flushes and worsen the situation
- safeMode.set(false);
- compactSplitThread.setLimit(-1);
- if (LOG.isDebugEnabled()) {
- LOG.debug(this.getName() + " exiting on interrupt");
+ // First wait until we exit safe mode
+ synchronized (safeMode) {
+ while(safeMode.get()) {
+ LOG.debug("Waiting to exit safe mode");
+ try {
+ safeMode.wait();
+ } catch (InterruptedException e) {
+ // ignore
+ }
}
- return;
}
- LOG.info("leaving safe mode");
- safeMode.set(false);
// now that safemode is off, slowly increase the per-cycle compaction
// limit, finally setting it to unlimited (-1)
+
int compactionCheckInterval =
conf.getInt("hbase.regionserver.thread.splitcompactcheckfrequency",
20 * 1000);
@@ -1006,13 +1011,13 @@
}
// Set up the safe mode handler if safe mode has been configured.
- if (conf.getInt("hbase.regionserver.safemode.period", 0) < 1) {
+ if (!conf.getBoolean("hbase.regionserver.safemode", true)) {
safeMode.set(false);
compactSplitThread.setLimit(-1);
LOG.debug("skipping safe mode");
} else {
- this.safeModeThread = new SafeModeThread();
- Threads.setDaemonThreadRunning(this.safeModeThread, n + ".safeMode",
+ this.compactionLimitThread = new CompactionLimitThread();
+ Threads.setDaemonThreadRunning(this.compactionLimitThread, n +
".safeMode",
handler);
}
@@ -1482,9 +1487,9 @@
}
this.quiesced.set(true);
if (onlineRegions.size() == 0) {
- outboundMsgs.add(HMsg.REPORT_EXITING);
+ outboundMsgs.add(REPORT_EXITING);
} else {
- outboundMsgs.add(HMsg.REPORT_QUIESCED);
+ outboundMsgs.add(REPORT_QUIESCED);
}
}
Modified: hadoop/hbase/branches/0.19/src/test/hbase-site.xml
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/test/hbase-site.xml?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19/src/test/hbase-site.xml (original)
+++ hadoop/hbase/branches/0.19/src/test/hbase-site.xml Fri Jan 23 11:56:14 2009
@@ -98,6 +98,14 @@
</description>
</property>
<property>
+ <name>hbase.regionserver.safemode</name>
+ <value>false</value>
+ <description>
+ Turn on/off safe mode in region server. Always on for production, always
off
+ for tests.
+ </description>
+ </property>
+ <property>
<name>hbase.hregion.max.filesize</name>
<value>67108864</value>
<description>
@@ -111,11 +119,4 @@
<name>hadoop.log.dir</name>
<value>${user.dir}/../logs</value>
</property>
- <property>
- <name>hbase.regionserver.safemode.period</name>
- <value>0</value>
- <description>Time to wait on regionserver startup before beginning
- compactions and memcache flushes.
- </description>
- </property>
</configuration>
Modified:
hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/TestSerialization.java
URL:
http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/TestSerialization.java?rev=737166&r1=737165&r2=737166&view=diff
==============================================================================
---
hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/TestSerialization.java
(original)
+++
hadoop/hbase/branches/0.19/src/test/org/apache/hadoop/hbase/TestSerialization.java
Fri Jan 23 11:56:14 2009
@@ -33,10 +33,12 @@
*/
public class TestSerialization extends HBaseTestCase {
+ @Override
protected void setUp() throws Exception {
super.setUp();
}
+ @Override
protected void tearDown() throws Exception {
super.tearDown();
}
@@ -53,7 +55,7 @@
}
public void testHMsg() throws Exception {
- HMsg m = HMsg.REGIONSERVER_QUIESCE;
+ HMsg m = new HMsg(HMsg.Type.MSG_REGIONSERVER_QUIESCE);
byte [] mb = Writables.getBytes(m);
HMsg deserializedHMsg = (HMsg)Writables.getWritable(mb, new HMsg());
assertTrue(m.equals(deserializedHMsg));