HBASE-2599 BaseScanner says "Current assignment of X is not valid" over and over for same region

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@949327 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-05-29 00:40:54 +00:00
parent 25232bab12
commit 6cdd1822b6
10 changed files with 174 additions and 145 deletions

View File

@ -23,6 +23,8 @@ Release 0.21.0 - Unreleased
HBASE-2541 Remove transactional contrib (Clint Morgan via Stack)
HBASE-2542 Fold stargate contrib into core
HBASE-2565 Remove contrib module from hbase
HBASE-2599 BaseScanner says "Current assignment of X is not valid" over
and over for same region
BUG FIXES
HBASE-1791 Timeout in IndexRecordWriter (Bradford Stephens via Andrew

View File

@ -1,5 +1,5 @@
/**
* Copyright 2007 The Apache Software Foundation
* Copyright 2010 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -22,26 +22,33 @@ package org.apache.hadoop.hbase;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.net.InetSocketAddress;
import java.util.Set;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
/**
* HServerInfo is meta info about an {@link HRegionServer}.
* Holds hostname, ports, regionserver startcode, and load. Each server has
* HServerInfo is meta info about an {@link HRegionServer}. It is the token
* by which a master distingushes a particular regionserver from the rest.
* It holds hostname, ports, regionserver startcode, and load. Each server has
* a <code>servername</code> where servername is made up of a concatenation of
* hostname, port, and regionserver startcode.
* hostname, port, and regionserver startcode. This servername is used in
* various places identifying this regionserver. Its even used as part of
* a pathname in the filesystem. As part of the initialization,
* master will pass the regionserver the address that it knows this regionserver
* by. In subsequent communications, the regionserver will pass a HServerInfo
* with the master-supplied address.
*/
public class HServerInfo implements WritableComparable<HServerInfo> {
/**
* This character is used as separator making up the <code>servername</code>.
* Servername is made of host, port, and startcode formatted as
* <code>&lt;hostname> '{@link #SERVERNAME_SEPARATOR}' &lt;port> '{@ink #SEPARATOR"}' &lt;startcode></code>
* where {@link SEPARATOR is usually a ','.
/*
* This character is used as separator between server hostname and port and
* its startcode. Servername is formatted as
* <code>&lt;hostname> '{@ink #SERVERNAME_SEPARATOR"}' &lt;port> '{@ink #SERVERNAME_SEPARATOR"}' &lt;startcode></code>.
*/
public static final String SERVERNAME_SEPARATOR = ",";
private static final String SERVERNAME_SEPARATOR = ",";
private HServerAddress serverAddress;
private long startCode;
@ -51,11 +58,23 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
private String serverName = null;
// Hostname of the regionserver.
private String hostname;
private static Map<String,String> dnsCache = new HashMap<String,String>();
private String cachedHostnamePort = null;
public HServerInfo() {
this(new HServerAddress(), 0,
HConstants.DEFAULT_REGIONSERVER_INFOPORT, "default name");
this(new HServerAddress(), 0, HConstants.DEFAULT_REGIONSERVER_INFOPORT,
"default name");
}
/**
* Constructor that creates a HServerInfo with a generated startcode and an
* empty load.
* @param serverAddress An {@link InetSocketAddress} encased in a {@link Writable}
* @param infoPort Port the webui runs on.
* @param hostname Server hostname.
*/
public HServerInfo(HServerAddress serverAddress, final int infoPort,
final String hostname) {
this(serverAddress, System.currentTimeMillis(), infoPort, hostname);
}
public HServerInfo(HServerAddress serverAddress, long startCode,
@ -104,13 +123,27 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
return this.infoPort;
}
public void setInfoPort(int infoPort) {
this.infoPort = infoPort;
public String getHostname() {
return this.hostname;
}
public synchronized void setStartCode(long startCode) {
this.startCode = startCode;
this.serverName = null;
/**
* @return The hostname and port concatenated with a ':' as separator.
*/
public synchronized String getHostnamePort() {
if (this.cachedHostnamePort == null) {
this.cachedHostnamePort = getHostnamePort(this.hostname, this.serverAddress.getPort());
}
return this.cachedHostnamePort;
}
/**
* @param hostname
* @param port
* @return The hostname and port concatenated with a ':' as separator.
*/
public static String getHostnamePort(final String hostname, final int port) {
return hostname + ":" + port;
}
/**
@ -119,43 +152,18 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
*/
public synchronized String getServerName() {
if (this.serverName == null) {
// if we have the hostname of the RS, use it
if(this.hostname != null) {
this.serverName =
getServerName(this.hostname, this.serverAddress.getPort(), this.startCode);
}
// go to DNS name resolution only if we dont have the name of the RS
else {
this.serverName = getServerName(this.serverAddress, this.startCode);
}
this.serverName = getServerName(this.hostname,
this.serverAddress.getPort(), this.startCode);
}
return this.serverName;
}
/**
* @param serverAddress In form <code>&lt;hostname> ':' &lt;port></code>
* @param startCode Server startcode
* @return Server name made of the concatenation of hostname, port and
* startcode formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code>
*/
public static String getServerName(String serverAddress, long startCode) {
String name = null;
if (serverAddress != null) {
int colonIndex = serverAddress.lastIndexOf(':');
if(colonIndex < 0) {
throw new IllegalArgumentException("Not a host:port pair: " + serverAddress);
}
String host = serverAddress.substring(0, colonIndex);
int port =
Integer.valueOf(serverAddress.substring(colonIndex + 1)).intValue();
if(!dnsCache.containsKey(host)) {
HServerAddress address = new HServerAddress(serverAddress);
dnsCache.put(host, address.getHostname());
}
host = dnsCache.get(host);
name = getServerName(host, port, startCode);
}
return name;
public static synchronized String getServerName(final String hostAndPort,
final long startcode) {
int index = hostAndPort.indexOf(":");
if (index <= 0) throw new IllegalArgumentException("Expected <hostname> ':' <port>");
return getServerName(hostAndPort.substring(0, index),
Integer.parseInt(hostAndPort.substring(index + 1)), startcode);
}
/**
@ -175,7 +183,7 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
* @return Server name made of the concatenation of hostname, port and
* startcode formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code>
*/
private static String getServerName(String hostName, int port, long startCode) {
public static String getServerName(String hostName, int port, long startCode) {
StringBuilder name = new StringBuilder(hostName);
name.append(SERVERNAME_SEPARATOR);
name.append(port);
@ -233,4 +241,27 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
public int compareTo(HServerInfo o) {
return this.getServerName().compareTo(o.getServerName());
}
/**
* Utility method that does a find of a servername or a hostandport combination
* in the passed Set.
* @param servers Set of server names
* @param serverName Name to look for
* @param hostAndPortOnly If <code>serverName</code> is a
* <code>hostname ':' port</code>
* or <code>hostname , port , startcode</code>.
* @return True if <code>serverName</code> found in <code>servers</code>
*/
public static boolean isServer(final Set<String> servers,
final String serverName, final boolean hostAndPortOnly) {
if (!hostAndPortOnly) return servers.contains(serverName);
String serverNameColonReplaced =
serverName.replaceFirst(":", SERVERNAME_SEPARATOR);
for (String hostPortStartCode: servers) {
int index = hostPortStartCode.lastIndexOf(SERVERNAME_SEPARATOR);
String hostPortStrippedOfStartCode = hostPortStartCode.substring(0, index);
if (hostPortStrippedOfStartCode.equals(serverNameColonReplaced)) return true;
}
return false;
}
}

View File

@ -526,34 +526,29 @@ abstract class BaseScanner extends Chore implements HConstants {
* @param regionServer
* @param meta
* @param info
* @param serverAddress
* @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 serverAddress, final long startCode)
final String hostnameAndPort, final long startCode)
throws IOException {
String serverName = null;
String sa = serverAddress;
String sa = hostnameAndPort;
long sc = startCode;
if (sa == null || sa.length() <= 0) {
// Scans are sloppy. They don't respect row locks and they get and
// 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. Will have to wait till
// 0.21 hbase where we use zk to mediate state transitions to do better.
// 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);
if (sa != null && sa.length() > 0) {
// Reget startcode in case its changed in the meantime too.
sc = getStartCode(r);
}
}
}
if (sa != null && sa.length() > 0) {
serverName = HServerInfo.getServerName(sa, sc);
}

View File

@ -639,17 +639,18 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
public MapWritable regionServerStartup(final HServerInfo serverInfo)
throws IOException {
// Set the address for now even tho it will not be persisted on HRS side
// If the address given is not the default one, use IP given by the user.
if (serverInfo.getServerAddress().getBindAddress().equals(DEFAULT_HOST)) {
// Set the ip into the passed in serverInfo. Its ip is more than likely
// not the ip that the master sees here. See at end of this method where
// we pass it back to the regionserver by setting "hbase.regionserver.address"
String rsAddress = HBaseServer.getRemoteAddress();
serverInfo.setServerAddress(new HServerAddress(rsAddress,
serverInfo.getServerAddress().getPort()));
}
// Register with server manager
this.serverManager.regionServerStartup(serverInfo);
// Send back some config info
return createConfigurationSubset();
MapWritable mw = createConfigurationSubset();
mw.put(new Text("hbase.regionserver.address"), new Text(rsAddress));
return mw;
}
/**
@ -658,11 +659,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
*/
protected MapWritable createConfigurationSubset() {
MapWritable mw = addConfig(new MapWritable(), HConstants.HBASE_DIR);
// Get the real address of the HRS.
String rsAddress = HBaseServer.getRemoteAddress();
if (rsAddress != null) {
mw.put(new Text("hbase.regionserver.address"), new Text(rsAddress));
}
return addConfig(mw, "fs.default.name");
}
@ -993,29 +989,26 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
// Arguments are regionname and an optional server name.
byte [] regionname = ((ImmutableBytesWritable)args[0]).get();
LOG.debug("Attempting to close region: " + Bytes.toStringBinary(regionname));
String servername = null;
String hostnameAndPort = null;
if (args.length == 2) {
servername = Bytes.toString(((ImmutableBytesWritable)args[1]).get());
hostnameAndPort = Bytes.toString(((ImmutableBytesWritable)args[1]).get());
}
// Need hri
Result rr = getFromMETA(regionname, HConstants.CATALOG_FAMILY);
HRegionInfo hri = getHRegionInfo(rr.getRow(), rr);
if (servername == null) {
if (hostnameAndPort == null) {
// Get server from the .META. if it wasn't passed as argument
servername =
hostnameAndPort =
Bytes.toString(rr.getValue(CATALOG_FAMILY, SERVER_QUALIFIER));
}
// Take region out of the intransistions in case it got stuck there doing
// an open or whatever.
this.regionManager.clearFromInTransition(regionname);
// If servername is still null, then none, exit.
if (servername == null) break;
// Need to make up a HServerInfo 'servername' for that is how
// items are keyed in regionmanager Maps.
HServerAddress addr = new HServerAddress(servername);
// If hostnameAndPort is still null, then none, exit.
if (hostnameAndPort == null) break;
long startCode =
Bytes.toLong(rr.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER));
String name = HServerInfo.getServerName(addr, startCode);
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");

View File

@ -69,18 +69,18 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
HRegionInterface server =
master.getServerConnection().getHRegionConnection(getMetaRegion().getServer());
LOG.info(regionInfo.getRegionNameAsString() + " open on " +
serverInfo.getServerAddress().toString());
serverInfo.getServerName());
// Register the newly-available Region's location.
Put p = new Put(regionInfo.getRegionName());
p.add(CATALOG_FAMILY, SERVER_QUALIFIER,
Bytes.toBytes(serverInfo.getServerAddress().toString()));
Bytes.toBytes(serverInfo.getHostnamePort()));
p.add(CATALOG_FAMILY, 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.getServerAddress());
serverInfo.getStartCode() + ", server=" + serverInfo.getHostnamePort());
synchronized (master.getRegionManager()) {
if (isMetaTable) {
// It's a meta region.

View File

@ -67,8 +67,6 @@ public class ServerManager implements HConstants {
// The map of known server names to server info
private final Map<String, HServerInfo> serversToServerInfo =
new ConcurrentHashMap<String, HServerInfo>();
private final Map<HServerAddress, HServerInfo> serverAddressToServerInfo =
new ConcurrentHashMap<HServerAddress, HServerInfo>();
/*
* Set of known dead servers. On znode expiration, servers are added here.
@ -111,7 +109,7 @@ public class ServerManager implements HConstants {
@Override
protected void chore() {
int numServers = serverAddressToServerInfo.size();
int numServers = serversToServerInfo.size();
int numDeadServers = deadServers.size();
double averageLoad = getAverageLoad();
String deadServersList = null;
@ -177,8 +175,7 @@ public class ServerManager implements HConstants {
// for processing by ProcessServerShutdown.
HServerInfo info = new HServerInfo(serverInfo);
String hostAndPort = info.getServerAddress().toString();
HServerInfo existingServer =
this.serverAddressToServerInfo.get(info.getServerAddress());
HServerInfo existingServer = haveServerWithSameHostAndPortAlready(info.getHostnamePort());
if (existingServer != null) {
LOG.info("Server start rejected; we already have " + hostAndPort +
" registered; existingServer=" + existingServer + ", newServer=" + info);
@ -193,6 +190,17 @@ public class ServerManager implements HConstants {
recordNewServer(info);
}
private HServerInfo haveServerWithSameHostAndPortAlready(final String hostnamePort) {
synchronized (this.serversToServerInfo) {
for (Map.Entry<String, HServerInfo> e: this.serversToServerInfo.entrySet()) {
if (e.getValue().getHostnamePort().equals(hostnamePort)) {
return e.getValue();
}
}
}
return null;
}
/*
* If this server is on the dead list, reject it with a LeaseStillHeldException
* @param serverName Server name formatted as host_port_startcode.
@ -230,7 +238,6 @@ public class ServerManager implements HConstants {
Watcher watcher = new ServerExpirer(new HServerInfo(info));
this.master.getZooKeeperWrapper().updateRSLocationGetWatch(info, watcher);
this.serversToServerInfo.put(serverName, info);
this.serverAddressToServerInfo.put(info.getServerAddress(), info);
this.serversToLoad.put(serverName, load);
synchronized (this.loadToServers) {
Set<String> servers = this.loadToServers.get(load);
@ -317,7 +324,7 @@ public class ServerManager implements HConstants {
}
synchronized (this.serversToServerInfo) {
removeServerInfo(info.getServerName(), info.getServerAddress());
removeServerInfo(info.getServerName());
notifyServers();
}
@ -339,7 +346,7 @@ public class ServerManager implements HConstants {
synchronized (this.serversToServerInfo) {
// This method removes ROOT/META from the list and marks them to be
// reassigned in addition to other housework.
if (removeServerInfo(serverInfo.getServerName(), serverInfo.getServerAddress())) {
if (removeServerInfo(serverInfo.getServerName())) {
// Only process the exit message if the server still has registered info.
// Otherwise we could end up processing the server exit twice.
LOG.info("Region server " + serverInfo.getServerName() +
@ -391,7 +398,6 @@ public class ServerManager implements HConstants {
final HRegionInfo[] mostLoadedRegions, HMsg[] msgs)
throws IOException {
// Refresh the info object and the load information
this.serverAddressToServerInfo.put(serverInfo.getServerAddress(), serverInfo);
this.serversToServerInfo.put(serverInfo.getServerName(), serverInfo);
HServerLoad load = this.serversToLoad.get(serverInfo.getServerName());
if (load != null) {
@ -659,10 +665,8 @@ public class ServerManager implements HConstants {
}
/** Update a server load information because it's shutting down*/
private boolean removeServerInfo(final String serverName,
final HServerAddress serverAddress) {
private boolean removeServerInfo(final String serverName) {
boolean infoUpdated = false;
this.serverAddressToServerInfo.remove(serverAddress);
HServerInfo info = this.serversToServerInfo.remove(serverName);
// Only update load information once.
// This method can be called a couple of times during shutdown.
@ -746,12 +750,20 @@ public class ServerManager implements HConstants {
}
}
public Map<HServerAddress, HServerInfo> getServerAddressToServerInfo() {
// we use this one because all the puts to this map are parallel/synced with the other map.
/**
* @param hsa
* @return The HServerInfo whose HServerAddress is <code>hsa</code> or null
* if nothing found.
*/
public HServerInfo getHServerInfo(final HServerAddress hsa) {
synchronized(this.serversToServerInfo) {
return Collections.unmodifiableMap(this.serverAddressToServerInfo);
// TODO: This is primitive. Do a better search.
for (Map.Entry<String, HServerInfo> e: this.serversToServerInfo.entrySet()) {
if (e.getValue().getServerAddress().equals(hsa)) return e.getValue();
}
}
return null;
}
/**
* @return Read-only map of servers to load.
@ -841,7 +853,6 @@ public class ServerManager implements HConstants {
return;
}
// Remove the server from the known servers lists and update load info
this.serverAddressToServerInfo.remove(info.getServerAddress());
this.serversToServerInfo.remove(serverName);
HServerLoad load = this.serversToLoad.remove(serverName);
if (load != null) {
@ -890,15 +901,7 @@ public class ServerManager implements HConstants {
static boolean isDead(final Set<String> deadServers,
final String serverName, final boolean hostAndPortOnly) {
if (!hostAndPortOnly) return deadServers.contains(serverName);
String serverNameColonReplaced =
serverName.replaceFirst(":", HServerInfo.SERVERNAME_SEPARATOR);
for (String hostPortStartCode: deadServers) {
int index = hostPortStartCode.lastIndexOf(HServerInfo.SERVERNAME_SEPARATOR);
String hostPortStrippedOfStartCode = hostPortStartCode.substring(0, index);
if (hostPortStrippedOfStartCode.equals(serverNameColonReplaced)) return true;
}
return false;
return HServerInfo.isServer(deadServers, serverName, hostAndPortOnly);
}
Set<String> getDeadServers() {

View File

@ -595,8 +595,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
}
closeAllRegions(); // Don't leave any open file handles
}
LOG.info("aborting server at: " +
serverInfo.getServerAddress().toString());
LOG.info("aborting server at: " + this.serverInfo.getServerName());
} else {
ArrayList<HRegion> closedRegions = closeAllRegions();
try {
@ -618,14 +617,13 @@ public class HRegionServer implements HConstants, HRegionInterface,
}
LOG.info("telling master that region server is shutting down at: " +
serverInfo.getServerAddress().toString());
serverInfo.getServerName());
hbaseMaster.regionServerReport(serverInfo, exitMsg, (HRegionInfo[])null);
} catch (Throwable e) {
LOG.warn("Failed to send exiting message to master: ",
RemoteExceptionHandler.checkThrowable(e));
}
LOG.info("stopping server at: " +
serverInfo.getServerAddress().toString());
LOG.info("stopping server at: " + this.serverInfo.getServerName());
}
// Make sure the proxy is down.
@ -900,7 +898,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
}
if (fs.exists(logdir)) {
throw new RegionServerRunningException("region server already " +
"running at " + this.serverInfo.getServerAddress().toString() +
"running at " + this.serverInfo.getServerName() +
" because logdir " + logdir.toString() + " exists");
}
HLog newlog = instantiateHLog(logdir, oldLogDir);
@ -1035,8 +1033,10 @@ public class HRegionServer implements HConstants, HRegionInterface,
// auto bind enabled, try to use another port
LOG.info("Failed binding http info server to port: " + port);
port++;
// update HRS server info
this.serverInfo.setInfoPort(port);
// update HRS server info port.
this.serverInfo = new HServerInfo(this.serverInfo.getServerAddress(),
this.serverInfo.getStartCode(), port,
this.serverInfo.getHostname());
}
}
}
@ -1205,7 +1205,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
lastMsg = System.currentTimeMillis();
boolean startCodeOk = false;
while(!startCodeOk) {
serverInfo.setStartCode(System.currentTimeMillis());
this.serverInfo = createServerInfoWithNewStartCode(this.serverInfo);
startCodeOk = zooKeeperWrapper.writeRSLocation(this.serverInfo);
if(!startCodeOk) {
LOG.debug("Start code already taken, trying another one");
@ -1221,6 +1221,11 @@ public class HRegionServer implements HConstants, HRegionInterface,
return result;
}
private HServerInfo createServerInfoWithNewStartCode(final HServerInfo hsi) {
return new HServerInfo(hsi.getServerAddress(), hsi.getInfoPort(),
hsi.getHostname());
}
/* Add to the outbound message buffer */
private void reportOpen(HRegionInfo region) {
this.outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, region));

View File

@ -18,8 +18,6 @@
HBaseAdmin hbadmin = new HBaseAdmin(conf);
String tableName = request.getParameter("name");
HTable table = new HTable(conf, tableName);
Map<HServerAddress, HServerInfo> serverAddressToServerInfos =
master.getServerManager().getServerAddressToServerInfo();
String tableHeader = "<h2>Table Regions</h2><table><tr><th>Name</th><th>Region Server</th><th>Encoded Name</th><th>Start Key</th><th>End Key</th></tr>";
HServerAddress rootLocation = master.getRegionManager().getRootRegionLocation();
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
@ -86,7 +84,7 @@
%>
<%= tableHeader %>
<%
int infoPort = serverAddressToServerInfos.get(rootLocation).getInfoPort();
int infoPort = master.getServerManager().getHServerInfo(rootLocation).getInfoPort();
String url = "http://" + rootLocation.getHostname() + ":" + infoPort + "/";
%>
<tr>
@ -104,7 +102,7 @@
<%
Map<byte [], MetaRegion> onlineRegions = master.getRegionManager().getOnlineMetaRegions();
for (MetaRegion meta: onlineRegions.values()) {
int infoPort = serverAddressToServerInfos.get(meta.getServer()).getInfoPort();
int infoPort = master.getServerManager().getHServerInfo(meta.getServer()).getInfoPort();
String url = "http://" + meta.getServer().getHostname() + ":" + infoPort + "/";
%>
<tr>
@ -141,10 +139,7 @@
<%= tableHeader %>
<%
for(Map.Entry<HRegionInfo, HServerAddress> hriEntry : regions.entrySet()) {
int infoPort = serverAddressToServerInfos.get(
hriEntry.getValue()).getInfoPort();
int infoPort = master.getServerManager().getHServerInfo(hriEntry.getValue()).getInfoPort();
String urlRegionServer =
"http://" + hriEntry.getValue().getHostname().toString() + ":" + infoPort + "/";
%>

View File

@ -135,6 +135,10 @@ public class MiniHBaseCluster implements HConstants {
super(setDifferentUser(conf));
}
public void setHServerInfo(final HServerInfo hsi) {
this.serverInfo = hsi;
}
/*
* @param c
* @param currentfs We return this if we did not make a new one.

View File

@ -19,24 +19,22 @@
*/
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.IOException;
public class TestMasterWrongRS {
private static final Log LOG = LogFactory.getLog(TestMasterWrongRS.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@BeforeClass
@ -58,9 +56,12 @@ public class TestMasterWrongRS {
@Test
public void testRsReportsWrongServerName() throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer firstServer = cluster.getRegionServer(0);
MiniHBaseClusterRegionServer firstServer =
(MiniHBaseClusterRegionServer)cluster.getRegionServer(0);
HRegionServer secondServer = cluster.getRegionServer(1);
firstServer.getHServerInfo().setStartCode(12345);
HServerInfo hsi = firstServer.getServerInfo();
firstServer.setHServerInfo(new HServerInfo(hsi.getServerAddress(),
hsi.getInfoPort(), hsi.getHostname()));
// Sleep while the region server pings back
Thread.sleep(2000);
assertTrue(firstServer.isOnline());