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-2541 Remove transactional contrib (Clint Morgan via Stack)
HBASE-2542 Fold stargate contrib into core HBASE-2542 Fold stargate contrib into core
HBASE-2565 Remove contrib module from hbase 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 BUG FIXES
HBASE-1791 Timeout in IndexRecordWriter (Bradford Stephens via Andrew 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 * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * 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.DataInput;
import java.io.DataOutput; import java.io.DataOutput;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap; import java.net.InetSocketAddress;
import java.util.Map; import java.util.Set;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.WritableComparable;
/** /**
* HServerInfo is meta info about an {@link HRegionServer}. * HServerInfo is meta info about an {@link HRegionServer}. It is the token
* Holds hostname, ports, regionserver startcode, and load. Each server has * 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 * 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> { public class HServerInfo implements WritableComparable<HServerInfo> {
/** /*
* This character is used as separator making up the <code>servername</code>. * This character is used as separator between server hostname and port and
* Servername is made of host, port, and startcode formatted as * its startcode. Servername is formatted as
* <code>&lt;hostname> '{@link #SERVERNAME_SEPARATOR}' &lt;port> '{@ink #SEPARATOR"}' &lt;startcode></code> * <code>&lt;hostname> '{@ink #SERVERNAME_SEPARATOR"}' &lt;port> '{@ink #SERVERNAME_SEPARATOR"}' &lt;startcode></code>.
* where {@link SEPARATOR is usually a ','.
*/ */
public static final String SERVERNAME_SEPARATOR = ","; private static final String SERVERNAME_SEPARATOR = ",";
private HServerAddress serverAddress; private HServerAddress serverAddress;
private long startCode; private long startCode;
@ -51,11 +58,23 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
private String serverName = null; private String serverName = null;
// Hostname of the regionserver. // Hostname of the regionserver.
private String hostname; private String hostname;
private static Map<String,String> dnsCache = new HashMap<String,String>(); private String cachedHostnamePort = null;
public HServerInfo() { public HServerInfo() {
this(new HServerAddress(), 0, this(new HServerAddress(), 0, HConstants.DEFAULT_REGIONSERVER_INFOPORT,
HConstants.DEFAULT_REGIONSERVER_INFOPORT, "default name"); "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, public HServerInfo(HServerAddress serverAddress, long startCode,
@ -104,13 +123,27 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
return this.infoPort; return this.infoPort;
} }
public void setInfoPort(int infoPort) { public String getHostname() {
this.infoPort = infoPort; return this.hostname;
} }
public synchronized void setStartCode(long startCode) { /**
this.startCode = startCode; * @return The hostname and port concatenated with a ':' as separator.
this.serverName = null; */
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() { public synchronized String getServerName() {
if (this.serverName == null) { if (this.serverName == null) {
// if we have the hostname of the RS, use it this.serverName = getServerName(this.hostname,
if(this.hostname != null) { this.serverAddress.getPort(), this.startCode);
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);
}
} }
return this.serverName; return this.serverName;
} }
/** public static synchronized String getServerName(final String hostAndPort,
* @param serverAddress In form <code>&lt;hostname> ':' &lt;port></code> final long startcode) {
* @param startCode Server startcode int index = hostAndPort.indexOf(":");
* @return Server name made of the concatenation of hostname, port and if (index <= 0) throw new IllegalArgumentException("Expected <hostname> ':' <port>");
* startcode formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code> return getServerName(hostAndPort.substring(0, index),
*/ Integer.parseInt(hostAndPort.substring(index + 1)), startcode);
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;
} }
/** /**
@ -175,7 +183,7 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
* @return Server name made of the concatenation of hostname, port and * @return Server name made of the concatenation of hostname, port and
* startcode formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code> * 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); StringBuilder name = new StringBuilder(hostName);
name.append(SERVERNAME_SEPARATOR); name.append(SERVERNAME_SEPARATOR);
name.append(port); name.append(port);
@ -233,4 +241,27 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
public int compareTo(HServerInfo o) { public int compareTo(HServerInfo o) {
return this.getServerName().compareTo(o.getServerName()); 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 regionServer
* @param meta * @param meta
* @param info * @param info
* @param serverAddress * @param hostnameAndPort hostname ':' port as it comes out of .META.
* @param startCode * @param startCode
* @throws IOException * @throws IOException
*/ */
protected void checkAssigned(final HRegionInterface regionServer, protected void checkAssigned(final HRegionInterface regionServer,
final MetaRegion meta, final HRegionInfo info, final MetaRegion meta, final HRegionInfo info,
final String serverAddress, final long startCode) final String hostnameAndPort, final long startCode)
throws IOException { throws IOException {
String serverName = null; String serverName = null;
String sa = serverAddress; String sa = hostnameAndPort;
long sc = startCode; long sc = startCode;
if (sa == null || sa.length() <= 0) { if (sa == null || sa.length() <= 0) {
// Scans are sloppy. They don't respect row locks and they get and // Scans are sloppy. They cache a row internally so may have data that
// cache a row internally so may have data that is a little stale. Make // is a little stale. Make sure that for sure this serverAddress is null.
// sure that for sure this serverAddress is null. We are trying to // We are trying to avoid double-assignments. See hbase-1784.
// 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.
Get g = new Get(info.getRegionName()); Get g = new Get(info.getRegionName());
g.addFamily(HConstants.CATALOG_FAMILY); g.addFamily(HConstants.CATALOG_FAMILY);
Result r = regionServer.get(meta.getRegionName(), g); Result r = regionServer.get(meta.getRegionName(), g);
if (r != null && !r.isEmpty()) { if (r != null && !r.isEmpty()) {
sa = getServerAddress(r); sa = getServerAddress(r);
if (sa != null && sa.length() > 0) {
// Reget startcode in case its changed in the meantime too.
sc = getStartCode(r); sc = getStartCode(r);
} }
} }
}
if (sa != null && sa.length() > 0) { if (sa != null && sa.length() > 0) {
serverName = HServerInfo.getServerName(sa, sc); 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) public MapWritable regionServerStartup(final HServerInfo serverInfo)
throws IOException { throws IOException {
// Set the address for now even tho it will not be persisted on HRS side // Set the ip into the passed in serverInfo. Its ip is more than likely
// If the address given is not the default one, use IP given by the user. // not the ip that the master sees here. See at end of this method where
if (serverInfo.getServerAddress().getBindAddress().equals(DEFAULT_HOST)) { // we pass it back to the regionserver by setting "hbase.regionserver.address"
String rsAddress = HBaseServer.getRemoteAddress(); String rsAddress = HBaseServer.getRemoteAddress();
serverInfo.setServerAddress(new HServerAddress(rsAddress, serverInfo.setServerAddress(new HServerAddress(rsAddress,
serverInfo.getServerAddress().getPort())); serverInfo.getServerAddress().getPort()));
}
// Register with server manager // Register with server manager
this.serverManager.regionServerStartup(serverInfo); this.serverManager.regionServerStartup(serverInfo);
// Send back some config info // 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() { protected MapWritable createConfigurationSubset() {
MapWritable mw = addConfig(new MapWritable(), HConstants.HBASE_DIR); 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"); 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. // Arguments are regionname and an optional server name.
byte [] regionname = ((ImmutableBytesWritable)args[0]).get(); byte [] regionname = ((ImmutableBytesWritable)args[0]).get();
LOG.debug("Attempting to close region: " + Bytes.toStringBinary(regionname)); LOG.debug("Attempting to close region: " + Bytes.toStringBinary(regionname));
String servername = null; String hostnameAndPort = null;
if (args.length == 2) { if (args.length == 2) {
servername = Bytes.toString(((ImmutableBytesWritable)args[1]).get()); hostnameAndPort = Bytes.toString(((ImmutableBytesWritable)args[1]).get());
} }
// Need hri // Need hri
Result rr = getFromMETA(regionname, HConstants.CATALOG_FAMILY); Result rr = getFromMETA(regionname, HConstants.CATALOG_FAMILY);
HRegionInfo hri = getHRegionInfo(rr.getRow(), rr); HRegionInfo hri = getHRegionInfo(rr.getRow(), rr);
if (servername == null) { if (hostnameAndPort == null) {
// Get server from the .META. if it wasn't passed as argument // Get server from the .META. if it wasn't passed as argument
servername = hostnameAndPort =
Bytes.toString(rr.getValue(CATALOG_FAMILY, SERVER_QUALIFIER)); Bytes.toString(rr.getValue(CATALOG_FAMILY, SERVER_QUALIFIER));
} }
// Take region out of the intransistions in case it got stuck there doing // Take region out of the intransistions in case it got stuck there doing
// an open or whatever. // an open or whatever.
this.regionManager.clearFromInTransition(regionname); this.regionManager.clearFromInTransition(regionname);
// If servername is still null, then none, exit. // If hostnameAndPort is still null, then none, exit.
if (servername == null) break; if (hostnameAndPort == null) break;
// Need to make up a HServerInfo 'servername' for that is how
// items are keyed in regionmanager Maps.
HServerAddress addr = new HServerAddress(servername);
long startCode = long startCode =
Bytes.toLong(rr.getValue(CATALOG_FAMILY, STARTCODE_QUALIFIER)); 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() + LOG.info("Marking " + hri.getRegionNameAsString() +
" as closing on " + name + "; cleaning SERVER + STARTCODE; " + " as closing on " + name + "; cleaning SERVER + STARTCODE; " +
"master will tell regionserver to close region on next heartbeat"); "master will tell regionserver to close region on next heartbeat");

View File

@ -69,18 +69,18 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
HRegionInterface server = HRegionInterface server =
master.getServerConnection().getHRegionConnection(getMetaRegion().getServer()); master.getServerConnection().getHRegionConnection(getMetaRegion().getServer());
LOG.info(regionInfo.getRegionNameAsString() + " open on " + LOG.info(regionInfo.getRegionNameAsString() + " open on " +
serverInfo.getServerAddress().toString()); serverInfo.getServerName());
// Register the newly-available Region's location. // Register the newly-available Region's location.
Put p = new Put(regionInfo.getRegionName()); Put p = new Put(regionInfo.getRegionName());
p.add(CATALOG_FAMILY, SERVER_QUALIFIER, p.add(CATALOG_FAMILY, SERVER_QUALIFIER,
Bytes.toBytes(serverInfo.getServerAddress().toString())); Bytes.toBytes(serverInfo.getHostnamePort()));
p.add(CATALOG_FAMILY, STARTCODE_QUALIFIER, p.add(CATALOG_FAMILY, STARTCODE_QUALIFIER,
Bytes.toBytes(serverInfo.getStartCode())); Bytes.toBytes(serverInfo.getStartCode()));
server.put(metaRegionName, p); server.put(metaRegionName, p);
LOG.info("Updated row " + regionInfo.getRegionNameAsString() + LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
" in region " + Bytes.toString(metaRegionName) + " with startcode=" + " in region " + Bytes.toString(metaRegionName) + " with startcode=" +
serverInfo.getStartCode() + ", server=" + serverInfo.getServerAddress()); serverInfo.getStartCode() + ", server=" + serverInfo.getHostnamePort());
synchronized (master.getRegionManager()) { synchronized (master.getRegionManager()) {
if (isMetaTable) { if (isMetaTable) {
// It's a meta region. // 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 // The map of known server names to server info
private final Map<String, HServerInfo> serversToServerInfo = private final Map<String, HServerInfo> serversToServerInfo =
new ConcurrentHashMap<String, HServerInfo>(); 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. * Set of known dead servers. On znode expiration, servers are added here.
@ -111,7 +109,7 @@ public class ServerManager implements HConstants {
@Override @Override
protected void chore() { protected void chore() {
int numServers = serverAddressToServerInfo.size(); int numServers = serversToServerInfo.size();
int numDeadServers = deadServers.size(); int numDeadServers = deadServers.size();
double averageLoad = getAverageLoad(); double averageLoad = getAverageLoad();
String deadServersList = null; String deadServersList = null;
@ -177,8 +175,7 @@ public class ServerManager implements HConstants {
// for processing by ProcessServerShutdown. // for processing by ProcessServerShutdown.
HServerInfo info = new HServerInfo(serverInfo); HServerInfo info = new HServerInfo(serverInfo);
String hostAndPort = info.getServerAddress().toString(); String hostAndPort = info.getServerAddress().toString();
HServerInfo existingServer = HServerInfo existingServer = haveServerWithSameHostAndPortAlready(info.getHostnamePort());
this.serverAddressToServerInfo.get(info.getServerAddress());
if (existingServer != null) { if (existingServer != null) {
LOG.info("Server start rejected; we already have " + hostAndPort + LOG.info("Server start rejected; we already have " + hostAndPort +
" registered; existingServer=" + existingServer + ", newServer=" + info); " registered; existingServer=" + existingServer + ", newServer=" + info);
@ -193,6 +190,17 @@ public class ServerManager implements HConstants {
recordNewServer(info); 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 * If this server is on the dead list, reject it with a LeaseStillHeldException
* @param serverName Server name formatted as host_port_startcode. * @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)); Watcher watcher = new ServerExpirer(new HServerInfo(info));
this.master.getZooKeeperWrapper().updateRSLocationGetWatch(info, watcher); this.master.getZooKeeperWrapper().updateRSLocationGetWatch(info, watcher);
this.serversToServerInfo.put(serverName, info); this.serversToServerInfo.put(serverName, info);
this.serverAddressToServerInfo.put(info.getServerAddress(), info);
this.serversToLoad.put(serverName, load); this.serversToLoad.put(serverName, load);
synchronized (this.loadToServers) { synchronized (this.loadToServers) {
Set<String> servers = this.loadToServers.get(load); Set<String> servers = this.loadToServers.get(load);
@ -317,7 +324,7 @@ public class ServerManager implements HConstants {
} }
synchronized (this.serversToServerInfo) { synchronized (this.serversToServerInfo) {
removeServerInfo(info.getServerName(), info.getServerAddress()); removeServerInfo(info.getServerName());
notifyServers(); notifyServers();
} }
@ -339,7 +346,7 @@ public class ServerManager implements HConstants {
synchronized (this.serversToServerInfo) { synchronized (this.serversToServerInfo) {
// This method removes ROOT/META from the list and marks them to be // This method removes ROOT/META from the list and marks them to be
// reassigned in addition to other housework. // 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. // Only process the exit message if the server still has registered info.
// Otherwise we could end up processing the server exit twice. // Otherwise we could end up processing the server exit twice.
LOG.info("Region server " + serverInfo.getServerName() + LOG.info("Region server " + serverInfo.getServerName() +
@ -391,7 +398,6 @@ public class ServerManager implements HConstants {
final HRegionInfo[] mostLoadedRegions, HMsg[] msgs) final HRegionInfo[] mostLoadedRegions, HMsg[] msgs)
throws IOException { throws IOException {
// Refresh the info object and the load information // Refresh the info object and the load information
this.serverAddressToServerInfo.put(serverInfo.getServerAddress(), serverInfo);
this.serversToServerInfo.put(serverInfo.getServerName(), serverInfo); this.serversToServerInfo.put(serverInfo.getServerName(), serverInfo);
HServerLoad load = this.serversToLoad.get(serverInfo.getServerName()); HServerLoad load = this.serversToLoad.get(serverInfo.getServerName());
if (load != null) { if (load != null) {
@ -659,10 +665,8 @@ public class ServerManager implements HConstants {
} }
/** Update a server load information because it's shutting down*/ /** Update a server load information because it's shutting down*/
private boolean removeServerInfo(final String serverName, private boolean removeServerInfo(final String serverName) {
final HServerAddress serverAddress) {
boolean infoUpdated = false; boolean infoUpdated = false;
this.serverAddressToServerInfo.remove(serverAddress);
HServerInfo info = this.serversToServerInfo.remove(serverName); HServerInfo info = this.serversToServerInfo.remove(serverName);
// Only update load information once. // Only update load information once.
// This method can be called a couple of times during shutdown. // 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
synchronized (this.serversToServerInfo) { * @return The HServerInfo whose HServerAddress is <code>hsa</code> or null
return Collections.unmodifiableMap(this.serverAddressToServerInfo); * if nothing found.
*/
public HServerInfo getHServerInfo(final HServerAddress hsa) {
synchronized(this.serversToServerInfo) {
// 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. * @return Read-only map of servers to load.
@ -841,7 +853,6 @@ public class ServerManager implements HConstants {
return; return;
} }
// Remove the server from the known servers lists and update load info // Remove the server from the known servers lists and update load info
this.serverAddressToServerInfo.remove(info.getServerAddress());
this.serversToServerInfo.remove(serverName); this.serversToServerInfo.remove(serverName);
HServerLoad load = this.serversToLoad.remove(serverName); HServerLoad load = this.serversToLoad.remove(serverName);
if (load != null) { if (load != null) {
@ -890,15 +901,7 @@ public class ServerManager implements HConstants {
static boolean isDead(final Set<String> deadServers, static boolean isDead(final Set<String> deadServers,
final String serverName, final boolean hostAndPortOnly) { final String serverName, final boolean hostAndPortOnly) {
if (!hostAndPortOnly) return deadServers.contains(serverName); return HServerInfo.isServer(deadServers, serverName, hostAndPortOnly);
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;
} }
Set<String> getDeadServers() { Set<String> getDeadServers() {

View File

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

View File

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

View File

@ -135,6 +135,10 @@ public class MiniHBaseCluster implements HConstants {
super(setDifferentUser(conf)); super(setDifferentUser(conf));
} }
public void setHServerInfo(final HServerInfo hsi) {
this.serverInfo = hsi;
}
/* /*
* @param c * @param c
* @param currentfs We return this if we did not make a new one. * @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; package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals; 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.HBaseTestingUtility;
import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import java.io.IOException;
public class TestMasterWrongRS { public class TestMasterWrongRS {
private static final Log LOG = LogFactory.getLog(TestMasterWrongRS.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@BeforeClass @BeforeClass
@ -58,9 +56,12 @@ public class TestMasterWrongRS {
@Test @Test
public void testRsReportsWrongServerName() throws Exception { public void testRsReportsWrongServerName() throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer firstServer = cluster.getRegionServer(0); MiniHBaseClusterRegionServer firstServer =
(MiniHBaseClusterRegionServer)cluster.getRegionServer(0);
HRegionServer secondServer = cluster.getRegionServer(1); 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 // Sleep while the region server pings back
Thread.sleep(2000); Thread.sleep(2000);
assertTrue(firstServer.isOnline()); assertTrue(firstServer.isOnline());