HBASE-1502 Remove need for heartbeats in HBase

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1097275 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-04-27 23:12:42 +00:00
parent 4219e2accb
commit 529223130a
98 changed files with 3514 additions and 3622 deletions

View File

@ -9,6 +9,7 @@ Release 0.91.0 - Unreleased
HBASE-3762 HTableFactory.releaseHTableInterface() should throw IOException
instead of wrapping in RuntimeException (Ted Yu via garyh)
HBASE-3629 Update our thrift to 0.6 (Moaz Reyad)
HBASE-1502 Remove need for heartbeats in HBase
BUG FIXES
HBASE-3280 YouAreDeadException being swallowed in HRS getMaster

View File

@ -26,6 +26,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.TreeMap;
@ -55,13 +56,14 @@ public class ClusterStatus extends VersionedWritable {
* <dl>
* <dt>0</dt> <dd>initial version</dd>
* <dt>1</dt> <dd>added cluster ID</dd>
* <dt>2</dt> <dd>Added Map of ServerName to ServerLoad</dd>
* </dl>
*/
private static final byte VERSION = 1;
private static final byte VERSION = 2;
private String hbaseVersion;
private Collection<HServerInfo> liveServerInfo;
private Collection<String> deadServers;
private Map<ServerName, HServerLoad> liveServers;
private Collection<ServerName> deadServers;
private Map<String, RegionState> intransition;
private String clusterId;
@ -72,18 +74,28 @@ public class ClusterStatus extends VersionedWritable {
super();
}
public ClusterStatus(final String hbaseVersion, final String clusterid,
final Map<ServerName, HServerLoad> servers,
final Collection<ServerName> deadServers, final Map<String, RegionState> rit) {
this.hbaseVersion = hbaseVersion;
this.liveServers = servers;
this.deadServers = deadServers;
this.intransition = rit;
this.clusterId = clusterid;
}
/**
* @return the names of region servers on the dead list
*/
public Collection<String> getDeadServerNames() {
public Collection<ServerName> getDeadServerNames() {
return Collections.unmodifiableCollection(deadServers);
}
/**
* @return the number of region servers in the cluster
*/
public int getServers() {
return liveServerInfo.size();
public int getServersSize() {
return liveServers.size();
}
/**
@ -97,11 +109,8 @@ public class ClusterStatus extends VersionedWritable {
* @return the average cluster load
*/
public double getAverageLoad() {
int load = 0;
for (HServerInfo server: liveServerInfo) {
load += server.getLoad().getLoad();
}
return (double)load / (double)liveServerInfo.size();
int load = getRegionsCount();
return (double)load / (double)getServersSize();
}
/**
@ -109,8 +118,8 @@ public class ClusterStatus extends VersionedWritable {
*/
public int getRegionsCount() {
int count = 0;
for (HServerInfo server: liveServerInfo) {
count += server.getLoad().getNumberOfRegions();
for (Map.Entry<ServerName, HServerLoad> e: this.liveServers.entrySet()) {
count += e.getValue().getNumberOfRegions();
}
return count;
}
@ -120,8 +129,8 @@ public class ClusterStatus extends VersionedWritable {
*/
public int getRequestsCount() {
int count = 0;
for (HServerInfo server: liveServerInfo) {
count += server.getLoad().getNumberOfRequests();
for (Map.Entry<ServerName, HServerLoad> e: this.liveServers.entrySet()) {
count += e.getValue().getNumberOfRequests();
}
return count;
}
@ -133,13 +142,6 @@ public class ClusterStatus extends VersionedWritable {
return hbaseVersion;
}
/**
* @param version the HBase version string
*/
public void setHBaseVersion(String version) {
hbaseVersion = version;
}
/**
* @see java.lang.Object#equals(java.lang.Object)
*/
@ -152,7 +154,7 @@ public class ClusterStatus extends VersionedWritable {
}
return (getVersion() == ((ClusterStatus)o).getVersion()) &&
getHBaseVersion().equals(((ClusterStatus)o).getHBaseVersion()) &&
liveServerInfo.equals(((ClusterStatus)o).liveServerInfo) &&
this.liveServers.equals(((ClusterStatus)o).liveServers) &&
deadServers.equals(((ClusterStatus)o).deadServers);
}
@ -160,7 +162,7 @@ public class ClusterStatus extends VersionedWritable {
* @see java.lang.Object#hashCode()
*/
public int hashCode() {
return VERSION + hbaseVersion.hashCode() + liveServerInfo.hashCode() +
return VERSION + hbaseVersion.hashCode() + this.liveServers.hashCode() +
deadServers.hashCode();
}
@ -175,43 +177,34 @@ public class ClusterStatus extends VersionedWritable {
/**
* Returns detailed region server information: A list of
* {@link HServerInfo}, containing server load and resource usage
* statistics as {@link HServerLoad}, containing per-region
* statistics as {@link HServerLoad.RegionLoad}.
* {@link ServerName}.
* @return region server information
* @deprecated Use {@link #getServers()}
*/
public Collection<HServerInfo> getServerInfo() {
return Collections.unmodifiableCollection(liveServerInfo);
public Collection<ServerName> getServerInfo() {
return getServers();
}
//
// Setters
//
public void setServerInfo(Collection<HServerInfo> serverInfo) {
this.liveServerInfo = serverInfo;
public Collection<ServerName> getServers() {
return Collections.unmodifiableCollection(this.liveServers.keySet());
}
public void setDeadServers(Collection<String> deadServers) {
this.deadServers = deadServers;
/**
* @param sn
* @return Server's load or null if not found.
*/
public HServerLoad getLoad(final ServerName sn) {
return this.liveServers.get(sn);
}
public Map<String, RegionState> getRegionsInTransition() {
return this.intransition;
}
public void setRegionsInTransition(final Map<String, RegionState> m) {
this.intransition = m;
}
public String getClusterId() {
return clusterId;
}
public void setClusterId(String id) {
this.clusterId = id;
}
//
// Writable
//
@ -219,13 +212,14 @@ public class ClusterStatus extends VersionedWritable {
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeUTF(hbaseVersion);
out.writeInt(liveServerInfo.size());
for (HServerInfo server: liveServerInfo) {
server.write(out);
out.writeInt(getServersSize());
for (Map.Entry<ServerName, HServerLoad> e: this.liveServers.entrySet()) {
out.writeUTF(e.getKey().toString());
e.getValue().write(out);
}
out.writeInt(deadServers.size());
for (String server: deadServers) {
out.writeUTF(server);
for (ServerName server: deadServers) {
out.writeUTF(server.toString());
}
out.writeInt(this.intransition.size());
for (Map.Entry<String, RegionState> e: this.intransition.entrySet()) {
@ -239,16 +233,17 @@ public class ClusterStatus extends VersionedWritable {
super.readFields(in);
hbaseVersion = in.readUTF();
int count = in.readInt();
liveServerInfo = new ArrayList<HServerInfo>(count);
this.liveServers = new HashMap<ServerName, HServerLoad>(count);
for (int i = 0; i < count; i++) {
HServerInfo info = new HServerInfo();
info.readFields(in);
liveServerInfo.add(info);
String str = in.readUTF();
HServerLoad hsl = new HServerLoad();
hsl.readFields(in);
this.liveServers.put(new ServerName(str), hsl);
}
count = in.readInt();
deadServers = new ArrayList<String>(count);
deadServers = new ArrayList<ServerName>(count);
for (int i = 0; i < count; i++) {
deadServers.add(in.readUTF());
deadServers.add(new ServerName(in.readUTF()));
}
count = in.readInt();
this.intransition = new TreeMap<String, RegionState>();

View File

@ -373,6 +373,12 @@ public final class HConstants {
/** HBCK special code name used as server name when manipulating ZK nodes */
public static final String HBCK_CODE_NAME = "HBCKServerName";
public static final ServerName HBCK_CODE_SERVERNAME =
new ServerName(HBCK_CODE_NAME, -1, -1L);
public static final String KEY_FOR_HOSTNAME_SEEN_BY_MASTER =
"hbase.regionserver.hostname.seen.by.master";
public static final String HBASE_MASTER_LOGCLEANER_PLUGINS =
"hbase.master.logcleaner.plugins";

View File

@ -1,235 +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;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
/**
* HMsg is used to send messages between master and regionservers. Messages are
* sent as payload on the regionserver-to-master heartbeats. Region assignment
* does not use this mechanism. It goes via zookeeper.
*
* <p>Most of the time the messages are simple but some messages are accompanied
* by the region affected. HMsg may also carry an optional message.
*
* <p>TODO: Clean out all messages that go from master to regionserver; by
* design, these are to go via zk from here on out.
*/
public class HMsg implements Writable {
public static final HMsg [] EMPTY_HMSG_ARRAY = new HMsg[0];
public static enum Type {
/**
* When RegionServer receives this message, it goes into a sleep that only
* an exit will cure. This message is sent by unit tests simulating
* pathological states.
*/
TESTING_BLOCK_REGIONSERVER,
}
private Type type = null;
private HRegionInfo info = null;
private byte[] message = null;
private HRegionInfo daughterA = null;
private HRegionInfo daughterB = null;
/** Default constructor. Used during deserialization */
public HMsg() {
this(null);
}
/**
* Construct a message with the specified message and empty HRegionInfo
* @param type Message type
*/
public HMsg(final HMsg.Type type) {
this(type, new HRegionInfo(), null);
}
/**
* Construct a message with the specified message and HRegionInfo
* @param type Message type
* @param hri Region to which message <code>type</code> applies
*/
public HMsg(final HMsg.Type type, final HRegionInfo hri) {
this(type, hri, null);
}
/**
* Construct a message with the specified message and HRegionInfo
*
* @param type Message type
* @param hri Region to which message <code>type</code> applies. Cannot be
* null. If no info associated, used other Constructor.
* @param msg Optional message (Stringified exception, etc.)
*/
public HMsg(final HMsg.Type type, final HRegionInfo hri, final byte[] msg) {
this(type, hri, null, null, msg);
}
/**
* Construct a message with the specified message and HRegionInfo
*
* @param type Message type
* @param hri Region to which message <code>type</code> applies. Cannot be
* null. If no info associated, used other Constructor.
* @param daughterA
* @param daughterB
* @param msg Optional message (Stringified exception, etc.)
*/
public HMsg(final HMsg.Type type, final HRegionInfo hri,
final HRegionInfo daughterA, final HRegionInfo daughterB, final byte[] msg) {
this.type = type;
if (hri == null) {
throw new NullPointerException("Region cannot be null");
}
this.info = hri;
this.message = msg;
this.daughterA = daughterA;
this.daughterB = daughterB;
}
/**
* @return Region info or null if none associated with this message type.
*/
public HRegionInfo getRegionInfo() {
return this.info;
}
/** @return the type of message */
public Type getType() {
return this.type;
}
/**
* @param other Message type to compare to
* @return True if we are of same message type as <code>other</code>
*/
public boolean isType(final HMsg.Type other) {
return this.type.equals(other);
}
/** @return the message type */
public byte[] getMessage() {
return this.message;
}
/**
* @return First daughter if Type is MSG_REPORT_SPLIT_INCLUDES_DAUGHTERS else
* null
*/
public HRegionInfo getDaughterA() {
return this.daughterA;
}
/**
* @return Second daughter if Type is MSG_REPORT_SPLIT_INCLUDES_DAUGHTERS else
* null
*/
public HRegionInfo getDaughterB() {
return this.daughterB;
}
/**
* @see java.lang.Object#toString()
*/
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(this.type.toString());
// If null or empty region, don't bother printing it out.
if (this.info != null && this.info.getRegionName().length > 0) {
sb.append(": ");
sb.append(this.info.getRegionNameAsString());
}
if (this.message != null && this.message.length > 0) {
sb.append(": " + Bytes.toString(this.message));
}
return sb.toString();
}
/**
* @see java.lang.Object#equals(java.lang.Object)
*/
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
HMsg that = (HMsg)obj;
return this.type.equals(that.type) &&
(this.info != null)? this.info.equals(that.info):
that.info == null;
}
/**
* @see java.lang.Object#hashCode()
*/
@Override
public int hashCode() {
int result = this.type.hashCode();
if (this.info != null) {
result ^= this.info.hashCode();
}
return result;
}
// ////////////////////////////////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////////////////////////////////
/**
* @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
*/
public void write(DataOutput out) throws IOException {
out.writeInt(this.type.ordinal());
this.info.write(out);
if (this.message == null || this.message.length == 0) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
Bytes.writeByteArray(out, this.message);
}
}
/**
* @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
*/
public void readFields(DataInput in) throws IOException {
int ordinal = in.readInt();
this.type = HMsg.Type.values()[ordinal];
this.info.readFields(in);
boolean hasMessage = in.readBoolean();
if (hasMessage) {
this.message = Bytes.readByteArray(in);
}
}
}

View File

@ -19,24 +19,30 @@
*/
package org.apache.hadoop.hbase;
import java.net.InetSocketAddress;
import org.apache.hadoop.hbase.util.Addressing;
/**
* Contains the HRegionInfo for the region and the HServerAddress for the
* HRegionServer serving the region
* Data structure to hold HRegionInfo and the address for the hosting
* HRegionServer. Immutable.
*/
public class HRegionLocation implements Comparable<HRegionLocation> {
// TODO: Is this class necessary? Why not just have a Pair?
private HRegionInfo regionInfo;
private HServerAddress serverAddress;
private final HRegionInfo regionInfo;
private final String hostname;
private final int port;
/**
* Constructor
*
* @param regionInfo the HRegionInfo for the region
* @param serverAddress the HServerAddress for the region server
* @param hostname Hostname
* @param port port
*/
public HRegionLocation(HRegionInfo regionInfo, HServerAddress serverAddress) {
public HRegionLocation(HRegionInfo regionInfo, final String hostname,
final int port) {
this.regionInfo = regionInfo;
this.serverAddress = serverAddress;
this.hostname = hostname;
this.port = port;
}
/**
@ -44,8 +50,8 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
*/
@Override
public String toString() {
return "address: " + this.serverAddress.toString() + ", regioninfo: " +
this.regionInfo.getRegionNameAsString();
return "region=" + this.regionInfo.getRegionNameAsString() +
", hostname=" + this.hostname + ", port=" + this.port;
}
/**
@ -71,7 +77,8 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
@Override
public int hashCode() {
int result = this.regionInfo.hashCode();
result ^= this.serverAddress.hashCode();
result ^= this.hostname.hashCode();
result ^= this.port;
return result;
}
@ -80,9 +87,30 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
return regionInfo;
}
/** @return HServerAddress */
/** @return HServerAddress
* @deprecated Use {@link #getHostnamePort}
*/
public HServerAddress getServerAddress(){
return serverAddress;
return new HServerAddress(this.hostname, this.port);
}
public String getHostname() {
return this.hostname;
}
public int getPort() {
return this.port;
}
/**
* @return String made of hostname and port formatted as per {@link Addressing#createHostAndPortStr(String, int)}
*/
public String getHostnamePort() {
return Addressing.createHostAndPortStr(this.hostname, this.port);
}
public InetSocketAddress getInetSocketAddress() {
return new InetSocketAddress(this.hostname, this.port);
}
//
@ -91,9 +119,9 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
public int compareTo(HRegionLocation o) {
int result = this.regionInfo.compareTo(o.regionInfo);
if(result == 0) {
result = this.serverAddress.compareTo(o.serverAddress);
}
return result;
if (result != 0) return result;
result = this.hostname.compareTo(o.getHostname());
if (result != 0) return result;
return this.port - o.getPort();
}
}

View File

@ -19,25 +19,38 @@
*/
package org.apache.hadoop.hbase;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.WritableComparable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.WritableComparable;
/**
* HServerAddress is a "label" for a HBase server made of host and port number.
* HServerAddress hosts a {@link InetSocketAddress} and makes it
* {@link WritableComparable}. Resolves on construction AND on
* deserialization -- since we're internally creating an InetSocketAddress --
* so could end up with different results if the two ends of serialization have
* different resolvers. Be careful where you use it. Should only be used when
* you need to pass an InetSocketAddress across an RPC. Even then its a bad
* idea because of the above resolve issue.
* @deprecated Use {@link InetSocketAddress} or {@link ServerName} or
* a hostname String and port.
*/
public class HServerAddress implements WritableComparable<HServerAddress> {
private InetSocketAddress address;
String stringValue;
// Hard to deprecate this class. Its in the API as internal class,
// in particular as an inner class of HRegionLocation. Besides, sometimes
// we do want to serialize a InetSocketAddress; this class can be used then.
private InetSocketAddress address = null;
private String cachedToString = "";
/**
* Constructor for deserialization use only.
*/
public HServerAddress() {
this.address = null;
this.stringValue = null;
super();
}
/**
@ -46,34 +59,20 @@ public class HServerAddress implements WritableComparable<HServerAddress> {
*/
public HServerAddress(InetSocketAddress address) {
this.address = address;
this.stringValue = address.getAddress().getHostName() + ":" +
address.getPort();
checkBindAddressCanBeResolved();
this.cachedToString = createCachedToString();
}
private String createCachedToString() {
return this.address.toString();
}
/**
* @param hostAndPort Hostname and port formatted as <code>&lt;hostname> ':' &lt;port></code>
*/
public HServerAddress(String hostAndPort) {
int colonIndex = hostAndPort.lastIndexOf(':');
if (colonIndex < 0) {
throw new IllegalArgumentException("Not a host:port pair: " + hostAndPort);
}
String host = hostAndPort.substring(0, colonIndex);
int port = Integer.parseInt(hostAndPort.substring(colonIndex + 1));
this.address = new InetSocketAddress(host, port);
this.stringValue = address.getHostName() + ":" + port;
checkBindAddressCanBeResolved();
}
/**
* @param bindAddress Hostname
* @param hostname Hostname
* @param port Port number
*/
public HServerAddress(String bindAddress, int port) {
this.address = new InetSocketAddress(bindAddress, port);
this.stringValue = address.getHostName() + ":" + port;
checkBindAddressCanBeResolved();
public HServerAddress(final String hostname, final int port) {
this(new InetSocketAddress(hostname, port));
}
/**
@ -81,45 +80,48 @@ public class HServerAddress implements WritableComparable<HServerAddress> {
* @param other HServerAddress to copy from
*/
public HServerAddress(HServerAddress other) {
String bindAddress = other.getBindAddress();
int port = other.getPort();
this.address = new InetSocketAddress(bindAddress, port);
stringValue = other.stringValue;
checkBindAddressCanBeResolved();
this(new InetSocketAddress(other.getHostname(), other.getPort()));
}
/** @return Bind address */
/** @return Bind address -- the raw IP, the result of a call to
* {@link InetSocketAddress#getAddress()#getHostAddress()} --
* or null if cannot resolve */
public String getBindAddress() {
final InetAddress addr = address.getAddress();
if (addr != null) {
return addr.getHostAddress();
} else {
// This returns null if the address is not resolved.
final InetAddress addr = this.address.getAddress();
if (addr != null) return addr.getHostAddress();
LogFactory.getLog(HServerAddress.class).error("Could not resolve the"
+ " DNS name of " + stringValue);
+ " DNS name of " + this.address.toString());
return null;
}
}
private void checkBindAddressCanBeResolved() {
if (getBindAddress() == null) {
throw new IllegalArgumentException("Could not resolve the"
+ " DNS name of " + stringValue);
+ " DNS name of " + this.address.toString());
}
}
/** @return Port number */
public int getPort() {
return address.getPort();
return this.address.getPort();
}
/** @return Hostname */
public String getHostname() {
return address.getHostName();
return this.address.getHostName();
}
/**
* @return Returns <hostname> ':' <port>
*/
public String getHostnameAndPort() {
return getHostname() + ":" + getPort();
}
/** @return The InetSocketAddress */
public InetSocketAddress getInetSocketAddress() {
return address;
return this.address;
}
/**
@ -127,20 +129,14 @@ public class HServerAddress implements WritableComparable<HServerAddress> {
*/
@Override
public String toString() {
return stringValue == null ? "" : stringValue;
return this.cachedToString;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null) {
return false;
}
if (getClass() != o.getClass()) {
return false;
}
if (this == o) return true;
if (o == null) return false;
if (getClass() != o.getClass()) return false;
return compareTo((HServerAddress)o) == 0;
}
@ -158,24 +154,20 @@ public class HServerAddress implements WritableComparable<HServerAddress> {
public void readFields(DataInput in) throws IOException {
String hostname = in.readUTF();
int port = in.readInt();
if (hostname == null || hostname.length() == 0) {
address = null;
stringValue = null;
} else {
address = new InetSocketAddress(hostname, port);
stringValue = hostname + ":" + port;
if (hostname != null && hostname.length() > 0) {
this.address = new InetSocketAddress(hostname, port);
checkBindAddressCanBeResolved();
createCachedToString();
}
}
public void write(DataOutput out) throws IOException {
if (address == null) {
if (this.address == null) {
out.writeUTF("");
out.writeInt(0);
} else {
out.writeUTF(address.getAddress().getHostName());
out.writeInt(address.getPort());
out.writeUTF(this.address.getAddress().getHostName());
out.writeInt(this.address.getPort());
}
}
@ -187,7 +179,7 @@ public class HServerAddress implements WritableComparable<HServerAddress> {
// Addresses as Strings may not compare though address is for the one
// server with only difference being that one address has hostname
// resolved whereas other only has IP.
if (address.equals(o.address)) return 0;
if (this.address.equals(o.address)) return 0;
return toString().compareTo(o.toString());
}
}

View File

@ -23,77 +23,48 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Comparator;
import java.util.Set;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.io.VersionedWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
/**
* 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. 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.
* HServerInfo is meta info about an {@link HRegionServer}. It hosts the
* {@link HServerAddress}, its webui port, and its server startcode. It was
* used to pass meta info about a server across an RPC but we've since made
* it so regionserver info is up in ZooKeeper and so this class is on its
* way out. It used to carry {@link HServerLoad} but as off HBase 0.92.0, the
* HServerLoad is passed independent of this class. Also, we now no longer pass
* the webui from regionserver to master (TODO: Fix).
* @deprecated Use {@link InetSocketAddress} and or {@link ServerName} and or
* {@link HServerLoad}
*/
public class HServerInfo extends VersionedWritable
implements WritableComparable<HServerInfo> {
private static final byte VERSION = 0;
/*
* 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>.
*/
private static final String SERVERNAME_SEPARATOR = ",";
private HServerAddress serverAddress;
private static final byte VERSION = 1;
private HServerAddress serverAddress = new HServerAddress();
private long startCode;
private HServerLoad load;
private int infoPort;
// Servername is made of hostname, port and startcode.
private String serverName = null;
// Hostname of the regionserver.
private String hostname;
private String cachedHostnamePort = null;
/** @return the object version number */
public byte getVersion() {
return VERSION;
}
private int webuiport;
public HServerInfo() {
this(new HServerAddress(), 0, HConstants.DEFAULT_REGIONSERVER_INFOPORT,
"default name");
super();
}
/**
* 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.
* Constructor that creates a HServerInfo with a generated startcode
* @param serverAddress
* @param webuiport Port the webui runs on.
*/
public HServerInfo(HServerAddress serverAddress, final int infoPort,
final String hostname) {
this(serverAddress, System.currentTimeMillis(), infoPort, hostname);
public HServerInfo(final HServerAddress serverAddress, final int webuiport) {
this(serverAddress, System.currentTimeMillis(), webuiport);
}
public HServerInfo(HServerAddress serverAddress, long startCode,
final int infoPort, String hostname) {
final int webuiport) {
this.serverAddress = serverAddress;
this.startCode = startCode;
this.load = new HServerLoad();
this.infoPort = infoPort;
this.hostname = hostname;
this.webuiport = webuiport;
}
/**
@ -103,106 +74,32 @@ public class HServerInfo extends VersionedWritable
public HServerInfo(HServerInfo other) {
this.serverAddress = new HServerAddress(other.getServerAddress());
this.startCode = other.getStartCode();
this.load = other.getLoad();
this.infoPort = other.getInfoPort();
this.hostname = other.hostname;
this.webuiport = other.getInfoPort();
}
public HServerLoad getLoad() {
return load;
}
public void setLoad(HServerLoad load) {
this.load = load;
/** @return the object version number */
public byte getVersion() {
return VERSION;
}
public synchronized HServerAddress getServerAddress() {
return new HServerAddress(serverAddress);
}
public synchronized void setServerAddress(HServerAddress serverAddress) {
this.serverAddress = serverAddress;
this.hostname = serverAddress.getHostname();
this.serverName = null;
}
public synchronized long getStartCode() {
return startCode;
}
public int getInfoPort() {
return this.infoPort;
return getWebuiPort();
}
public int getWebuiPort() {
return this.webuiport;
}
public String getHostname() {
return this.hostname;
}
/**
* @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;
}
/**
* 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 <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code>
*/
public synchronized String getServerName() {
if (this.serverName == null) {
this.serverName = getServerName(this.hostname,
this.serverAddress.getPort(), this.startCode);
}
return this.serverName;
}
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);
}
/**
* @param address Server address
* @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(HServerAddress address, long startCode) {
return getServerName(address.getHostname(), address.getPort(), startCode);
}
/*
* @param hostName
* @param port
* @param 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 hostName, int port, long startCode) {
StringBuilder name = new StringBuilder(hostName);
name.append(SERVERNAME_SEPARATOR);
name.append(port);
name.append(SERVERNAME_SEPARATOR);
name.append(startCode);
return name.toString();
return this.serverAddress.getHostname();
}
/**
@ -211,97 +108,46 @@ public class HServerInfo extends VersionedWritable
* @see #getLoad()
*/
@Override
public String toString() {
return "serverName=" + getServerName() +
", load=(" + this.load.toString() + ")";
public synchronized String toString() {
return ServerName.getServerName(this.serverAddress.getHostnameAndPort(),
this.startCode);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null) {
return false;
}
if (getClass() != obj.getClass()) {
return false;
}
if (this == obj) return true;
if (obj == null) return false;
if (getClass() != obj.getClass()) return false;
return compareTo((HServerInfo)obj) == 0;
}
@Override
public int hashCode() {
return this.getServerName().hashCode();
int code = this.serverAddress.hashCode();
code ^= this.webuiport;
code ^= this.startCode;
return code;
}
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.serverAddress.readFields(in);
this.startCode = in.readLong();
this.load.readFields(in);
this.infoPort = in.readInt();
this.hostname = in.readUTF();
this.webuiport = in.readInt();
}
public void write(DataOutput out) throws IOException {
super.write(out);
this.serverAddress.write(out);
out.writeLong(this.startCode);
this.load.write(out);
out.writeInt(this.infoPort);
out.writeUTF(hostname);
out.writeInt(this.webuiport);
}
public int compareTo(HServerInfo o) {
return this.getServerName().compareTo(o.getServerName());
}
/**
* Orders HServerInfos by load then name. Natural/ascending order.
*/
public static class LoadComparator implements Comparator<HServerInfo> {
@Override
public int compare(HServerInfo left, HServerInfo right) {
int loadCompare = left.getLoad().compareTo(right.getLoad());
return loadCompare != 0 ? loadCompare : left.compareTo(right);
int compare = this.serverAddress.compareTo(o.getServerAddress());
if (compare != 0) return compare;
if (this.webuiport != o.getInfoPort()) return this.webuiport - o.getInfoPort();
if (this.startCode != o.getStartCode()) return (int)(this.startCode - o.getStartCode());
return 0;
}
}
/**
* 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;
}
/**
* Utility method to excise the start code from a server name
* @param inServerName full server name
* @return server name less its start code
*/
public static String getServerNameLessStartCode(String inServerName) {
if (inServerName != null && inServerName.length() > 0) {
int index = inServerName.lastIndexOf(SERVERNAME_SEPARATOR);
if (index > 0) {
return inServerName.substring(0, index);
}
}
return inServerName;
}
}

View File

@ -29,29 +29,31 @@ import java.util.TreeMap;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.io.VersionedWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
/**
* This class encapsulates metrics for determining the load on a HRegionServer
* This class is used exporting current state of load on a RegionServer.
*/
public class HServerLoad extends VersionedWritable
implements WritableComparable<HServerLoad> {
private static final byte VERSION = 0;
private static final byte VERSION = 1;
// Empty load instance.
public static final HServerLoad EMPTY_HSERVERLOAD = new HServerLoad();
/** Number of requests since last report
*/
// TODO: Instead build this up out of region counters.
private int numberOfRequests = 0;
/** number of regions */
// could just use regionLoad.size() but master.RegionManager likes to play
// around with this value while passing HServerLoad objects around during
// balancer calculations
private int numberOfRegions;
/** number of requests since last report */
private int numberOfRequests;
/** the amount of used heap, in MB */
private int usedHeapMB;
private int usedHeapMB = 0;
/** the maximum allowable size of the heap, in MB */
private int maxHeapMB;
private int maxHeapMB = 0;
/** per-region load metrics */
private Map<byte[], RegionLoad> regionLoad = new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR);
private Map<byte[], RegionLoad> regionLoad =
new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR);
/** @return the object version number */
public byte getVersion() {
@ -61,7 +63,14 @@ public class HServerLoad extends VersionedWritable
/**
* Encapsulates per-region loading metrics.
*/
public static class RegionLoad implements Writable {
public static class RegionLoad extends VersionedWritable {
private static final byte VERSION = 0;
/** @return the object version number */
public byte getVersion() {
return VERSION;
}
/** the region name */
private byte[] name;
/** the number of stores for the region */
@ -236,6 +245,9 @@ public class HServerLoad extends VersionedWritable
// Writable
public void readFields(DataInput in) throws IOException {
super.readFields(in);
int version = getVersion();
if (version != VERSION) throw new IOException("Version mismatch; " + version);
int namelen = in.readInt();
this.name = new byte[namelen];
in.readFully(this.name);
@ -249,6 +261,7 @@ public class HServerLoad extends VersionedWritable
}
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeInt(name.length);
out.write(name);
out.writeInt(stores);
@ -308,10 +321,11 @@ public class HServerLoad extends VersionedWritable
* @param maxHeapMB
*/
public HServerLoad(final int numberOfRequests, final int usedHeapMB,
final int maxHeapMB) {
final int maxHeapMB, final Map<byte[], RegionLoad> regionLoad) {
this.numberOfRequests = numberOfRequests;
this.usedHeapMB = usedHeapMB;
this.maxHeapMB = maxHeapMB;
this.regionLoad = regionLoad;
}
/**
@ -319,7 +333,7 @@ public class HServerLoad extends VersionedWritable
* @param hsl the template HServerLoad
*/
public HServerLoad(final HServerLoad hsl) {
this(hsl.numberOfRequests, hsl.usedHeapMB, hsl.maxHeapMB);
this(hsl.numberOfRequests, hsl.usedHeapMB, hsl.maxHeapMB, hsl.getRegionsLoad());
for (Map.Entry<byte[], RegionLoad> e : hsl.regionLoad.entrySet()) {
this.regionLoad.put(e.getKey(), e.getValue());
}
@ -338,7 +352,7 @@ public class HServerLoad extends VersionedWritable
// int load = numberOfRequests == 0 ? 1 : numberOfRequests;
// load *= numberOfRegions == 0 ? 1 : numberOfRegions;
// return load;
return numberOfRegions;
return this.regionLoad.size();
}
/**
@ -356,6 +370,7 @@ public class HServerLoad extends VersionedWritable
* @return The load as a String
*/
public String toString(int msgInterval) {
int numberOfRegions = this.regionLoad.size();
StringBuilder sb = new StringBuilder();
sb = Strings.appendKeyValue(sb, "requests",
Integer.valueOf(numberOfRequests/msgInterval));
@ -384,23 +399,13 @@ public class HServerLoad extends VersionedWritable
return compareTo((HServerLoad)o) == 0;
}
/**
* @see java.lang.Object#hashCode()
*/
@Override
public int hashCode() {
int result = Integer.valueOf(numberOfRequests).hashCode();
result ^= Integer.valueOf(numberOfRegions).hashCode();
return result;
}
// Getters
/**
* @return the numberOfRegions
*/
public int getNumberOfRegions() {
return numberOfRegions;
return this.regionLoad.size();
}
/**
@ -471,69 +476,16 @@ public class HServerLoad extends VersionedWritable
return count;
}
// Setters
/**
* @param numberOfRegions the number of regions
*/
public void setNumberOfRegions(int numberOfRegions) {
this.numberOfRegions = numberOfRegions;
}
/**
* @param numberOfRequests the number of requests to set
*/
public void setNumberOfRequests(int numberOfRequests) {
this.numberOfRequests = numberOfRequests;
}
/**
* @param usedHeapMB the amount of heap in use, in MB
*/
public void setUsedHeapMB(int usedHeapMB) {
this.usedHeapMB = usedHeapMB;
}
/**
* @param maxHeapMB the maximum allowable heap size, in MB
*/
public void setMaxHeapMB(int maxHeapMB) {
this.maxHeapMB = maxHeapMB;
}
/**
* @param load Instance of HServerLoad
*/
public void addRegionInfo(final HServerLoad.RegionLoad load) {
this.numberOfRegions++;
this.regionLoad.put(load.getName(), load);
}
/**
* @param name
* @param stores
* @param storefiles
* @param memstoreSizeMB
* @param storefileIndexSizeMB
* @param requestsCount
* @deprecated Use {@link #addRegionInfo(RegionLoad)}
*/
@Deprecated
public void addRegionInfo(final byte[] name, final int stores,
final int storefiles, final int storefileSizeMB,
final int memstoreSizeMB, final int storefileIndexSizeMB,
final int readRequestsCount, final int writeRequestsCount) {
this.regionLoad.put(name, new HServerLoad.RegionLoad(name, stores, storefiles,
storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, readRequestsCount, writeRequestsCount));
}
// Writable
public void readFields(DataInput in) throws IOException {
super.readFields(in);
int version = getVersion();
if (version != VERSION) throw new IOException("Version mismatch; " + version);
numberOfRequests = in.readInt();
usedHeapMB = in.readInt();
maxHeapMB = in.readInt();
numberOfRegions = in.readInt();
int numberOfRegions = in.readInt();
for (int i = 0; i < numberOfRegions; i++) {
RegionLoad rl = new RegionLoad();
rl.readFields(in);
@ -542,10 +494,11 @@ public class HServerLoad extends VersionedWritable
}
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeInt(numberOfRequests);
out.writeInt(usedHeapMB);
out.writeInt(maxHeapMB);
out.writeInt(numberOfRegions);
out.writeInt(this.regionLoad.size());
for (RegionLoad rl: regionLoad.values())
rl.write(out);
}

View File

@ -254,12 +254,10 @@ public class LocalHBaseCluster {
while (regionServerThread.isAlive()) {
try {
LOG.info("Waiting on " +
regionServerThread.getRegionServer().getHServerInfo().toString());
regionServerThread.getRegionServer().toString());
regionServerThread.join();
} catch (InterruptedException e) {
e.printStackTrace();
} catch (IOException e) {
e.printStackTrace();
}
}
return regionServerThread.getName();
@ -275,12 +273,10 @@ public class LocalHBaseCluster {
while (rst.isAlive()) {
try {
LOG.info("Waiting on " +
rst.getRegionServer().getHServerInfo().toString());
rst.getRegionServer().toString());
rst.join();
} catch (InterruptedException e) {
e.printStackTrace();
} catch (IOException e) {
e.printStackTrace();
}
}
for (int i=0;i<regionThreads.size();i++) {

View File

@ -55,12 +55,11 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
/**
* Get the address of the current master if one is available. Returns null
* if no current master.
*
* @return server address of current active master, or null if none available
* @return Server name or null if timed out.
*/
public HServerAddress getMasterAddress() {
public ServerName getMasterAddress() {
byte [] data = super.getData();
return data == null ? null : new HServerAddress(Bytes.toString(data));
return data == null ? null : new ServerName(Bytes.toString(data));
}
/**
@ -77,12 +76,12 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
* has passed.
*
* @param timeout maximum time to wait for master in millis, 0 for forever
* @return server address of current active master, null if timed out
* @return String of master host and port or null if timed out.
* @throws InterruptedException if the thread is interrupted while waiting
*/
public synchronized HServerAddress waitForMaster(long timeout)
public synchronized ServerName waitForMaster(long timeout)
throws InterruptedException {
byte [] data = super.blockUntilAvailable();
return data == null ? null : new HServerAddress(Bytes.toString(data));
return data == null ? null : new ServerName(Bytes.toString(data));
}
}

View File

@ -44,11 +44,7 @@ public interface Server extends Abortable, Stoppable {
public CatalogTracker getCatalogTracker();
/**
* Gets the unique server name for this server.
* If a RegionServer, it returns a concatenation of hostname, port and
* startcode formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code>.
* If the master, it returns <code>&lt;hostname> ':' &lt;port>'.
* @return unique server name
* @return The unique server name for this server.
*/
public String getServerName();
public ServerName getServerName();
}

View File

@ -0,0 +1,221 @@
/**
* Copyright 2011 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.util.Collection;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Instance of an HBase ServerName.
* A server name is used uniquely identifying a server instance and is made
* of the combination of hostname, port, and startcode. The startcode
* distingushes restarted servers on same hostname and port (startcode is
* usually timestamp of server startup). The {@link #toString()} format of
* ServerName is safe to use in the filesystem and as znode name up in
* ZooKeeper. Its format is:
* <code>&lt;hostname> '{@link #SERVERNAME_SEPARATOR"}' &lt;port> '{@ink #SERVERNAME_SEPARATOR"}' &lt;startcode></code>.
* For example, if hostname is <code>example.org</code>, port is <code>1234</code>,
* and the startcode for the regionserver is <code>1212121212</code>, then
* the {@link #toString()} would be <code>example.org,1234,1212121212</code>.
*
* <p>Immutable.
*/
public class ServerName implements Comparable<ServerName> {
/**
* This character is used as separator between server hostname, port and
* startcode.
*/
public static final String SERVERNAME_SEPARATOR = ",";
private final String servername;
private final String hostname;
private final int port;
private final long startcode;
private byte [] bytes;
public ServerName(final String hostname, final int port, final long startcode) {
this.hostname = hostname;
this.port = port;
this.startcode = startcode;
this.servername = getServerName(hostname, port, startcode);
}
public ServerName(final String serverName) {
this(parseHostname(serverName), parsePort(serverName),
parseStartcode(serverName));
}
public ServerName(final byte [] bytes) {
this(Bytes.toString(bytes));
}
public ServerName(final String hostAndPort, final long startCode) {
this(Addressing.parseHostname(hostAndPort),
Addressing.parsePort(hostAndPort), startCode);
}
public static String parseHostname(final String serverName) {
if (serverName == null || serverName.length() <= 0) {
throw new IllegalArgumentException("Passed hostname is null or empty");
}
int index = serverName.indexOf(SERVERNAME_SEPARATOR);
return serverName.substring(0, index);
}
public static int parsePort(final String serverName) {
String [] split = serverName.split(SERVERNAME_SEPARATOR);
return Integer.parseInt(split[1]);
}
public static long parseStartcode(final String serverName) {
int index = serverName.lastIndexOf(SERVERNAME_SEPARATOR);
return Long.parseLong(serverName.substring(index + 1));
}
@Override
public String toString() {
return getServerName();
}
/**
* @return {@link #getServerName()} as bytes
*/
public synchronized byte [] getBytes() {
if (this.bytes == null) this.bytes = Bytes.toBytes(getServerName());
return this.bytes;
}
public String getServerName() {
return servername;
}
public String getHostname() {
return hostname;
}
public int getPort() {
return port;
}
public long getStartcode() {
return startcode;
}
/**
* @param hostName
* @param port
* @param 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 hostName, int port, long startcode) {
StringBuilder name = new StringBuilder(hostName);
name.append(SERVERNAME_SEPARATOR);
name.append(port);
name.append(SERVERNAME_SEPARATOR);
name.append(startcode);
return name.toString();
}
/**
* @param hostAndPort String in form of &lt;hostname> ':' &lt;port>
* @param 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 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);
}
/**
* @return Hostname and port formatted as described at
* {@link Addressing#createHostAndPortStr(String, int)}
*/
public String getHostAndPort() {
return Addressing.createHostAndPortStr(this.hostname, this.port);
}
/**
* @param serverName ServerName in form specified by {@link #getServerName()}
* @return The server start code parsed from <code>servername</code>
*/
public static long getServerStartcodeFromServerName(final String serverName) {
int index = serverName.lastIndexOf(SERVERNAME_SEPARATOR);
return Long.parseLong(serverName.substring(index + 1));
}
/**
* Utility method to excise the start code from a server name
* @param inServerName full server name
* @return server name less its start code
*/
public static String getServerNameLessStartCode(String inServerName) {
if (inServerName != null && inServerName.length() > 0) {
int index = inServerName.lastIndexOf(SERVERNAME_SEPARATOR);
if (index > 0) {
return inServerName.substring(0, index);
}
}
return inServerName;
}
@Override
public int compareTo(ServerName other) {
int compare = this.getHostname().compareTo(other.getHostname());
if (compare != 0) return compare;
compare = this.getPort() - other.getPort();
if (compare != 0) return compare;
return (int)(this.getStartcode() - other.getStartcode());
}
@Override
public int hashCode() {
return getServerName().hashCode();
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null) return false;
if (!(o instanceof ServerName)) return false;
return this.compareTo((ServerName)o) == 0;
}
/**
* @return ServerName with matching hostname and port.
*/
public static ServerName findServerWithSameHostnamePort(final Collection<ServerName> names,
final ServerName serverName) {
for (ServerName sn: names) {
if (sn.getHostname().equals(serverName.getHostname()) &&
sn.getPort() == serverName.getPort()) {
return sn;
}
}
return null;
}
}

View File

@ -23,20 +23,17 @@ import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
import org.apache.avro.generic.GenericData;
import org.apache.avro.util.Utf8;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
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.io.hfile.Compression;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.avro.generated.AClusterStatus;
import org.apache.hadoop.hbase.avro.generated.AColumn;
import org.apache.hadoop.hbase.avro.generated.AColumnValue;
@ -54,11 +51,13 @@ import org.apache.hadoop.hbase.avro.generated.AServerAddress;
import org.apache.hadoop.hbase.avro.generated.AServerInfo;
import org.apache.hadoop.hbase.avro.generated.AServerLoad;
import org.apache.hadoop.hbase.avro.generated.ATableDescriptor;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
import org.apache.avro.generic.GenericData;
import org.apache.avro.util.Utf8;
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.io.hfile.Compression;
import org.apache.hadoop.hbase.util.Bytes;
public class AvroUtil {
@ -113,26 +112,26 @@ public class AvroUtil {
return asl;
}
static public AServerInfo hsiToASI(HServerInfo hsi) throws IOException {
static public AServerInfo hsiToASI(ServerName sn, HServerLoad hsl) throws IOException {
AServerInfo asi = new AServerInfo();
asi.infoPort = hsi.getInfoPort();
asi.load = hslToASL(hsi.getLoad());
asi.serverAddress = hsaToASA(hsi.getServerAddress());
asi.serverName = new Utf8(hsi.getServerName());
asi.startCode = hsi.getStartCode();
asi.infoPort = -1;
asi.load = hslToASL(hsl);
asi.serverAddress = hsaToASA(new HServerAddress(sn.getHostname(), sn.getPort()));
asi.serverName = new Utf8(sn.toString());
asi.startCode = sn.getStartcode();
return asi;
}
static public AClusterStatus csToACS(ClusterStatus cs) throws IOException {
AClusterStatus acs = new AClusterStatus();
acs.averageLoad = cs.getAverageLoad();
Collection<String> deadServerNames = cs.getDeadServerNames();
Collection<ServerName> deadServerNames = cs.getDeadServerNames();
Schema stringArraySchema = Schema.createArray(Schema.create(Schema.Type.STRING));
GenericData.Array<CharSequence> adeadServerNames = null;
if (deadServerNames != null) {
adeadServerNames = new GenericData.Array<CharSequence>(deadServerNames.size(), stringArraySchema);
for (String deadServerName : deadServerNames) {
adeadServerNames.add(new Utf8(deadServerName));
for (ServerName deadServerName : deadServerNames) {
adeadServerNames.add(new Utf8(deadServerName.toString()));
}
} else {
adeadServerNames = new GenericData.Array<CharSequence>(0, stringArraySchema);
@ -142,19 +141,19 @@ public class AvroUtil {
acs.hbaseVersion = new Utf8(cs.getHBaseVersion());
acs.regionsCount = cs.getRegionsCount();
acs.requestsCount = cs.getRequestsCount();
Collection<HServerInfo> hserverInfos = cs.getServerInfo();
Collection<ServerName> hserverInfos = cs.getServers();
Schema s = Schema.createArray(AServerInfo.SCHEMA$);
GenericData.Array<AServerInfo> aserverInfos = null;
if (hserverInfos != null) {
aserverInfos = new GenericData.Array<AServerInfo>(hserverInfos.size(), s);
for (HServerInfo hsi : hserverInfos) {
aserverInfos.add(hsiToASI(hsi));
for (ServerName hsi : hserverInfos) {
aserverInfos.add(hsiToASI(hsi, cs.getLoad(hsi)));
}
} else {
aserverInfos = new GenericData.Array<AServerInfo>(0, s);
}
acs.serverInfos = aserverInfos;
acs.servers = cs.getServers();
acs.servers = cs.getServers().size();
return acs;
}

View File

@ -30,9 +30,9 @@ 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.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
@ -63,12 +63,11 @@ public class CatalogTracker {
private final MetaNodeTracker metaNodeTracker;
private final AtomicBoolean metaAvailable = new AtomicBoolean(false);
/**
* Do not clear this address once set. Let it be cleared by
* {@link #setMetaLocation(HServerAddress)} only. Its needed when we do
* Do not clear this address once set. Its needed when we do
* server shutdown processing -- we need to know who had .META. last. If you
* want to know if the address is good, rely on {@link #metaAvailable} value.
*/
private HServerAddress metaLocation;
private ServerName metaLocation;
private final int defaultTimeout;
private boolean stopped = false;
@ -155,17 +154,18 @@ public class CatalogTracker {
/**
* Gets the current location for <code>-ROOT-</code> or null if location is
* not currently available.
* @return location of root, null if not available
* @return server name
* @throws InterruptedException
*/
public HServerAddress getRootLocation() throws InterruptedException {
public ServerName getRootLocation() throws InterruptedException {
return this.rootRegionTracker.getRootRegionLocation();
}
/**
* @return Location of meta or null if not yet available.
* @return Location of server hosting meta region formatted as per
* {@link ServerName}, or null if none available
*/
public HServerAddress getMetaLocation() {
public ServerName getMetaLocation() {
return this.metaLocation;
}
@ -184,18 +184,19 @@ public class CatalogTracker {
* 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
* @return Location of server hosting root region,
* or null if none available
* @throws InterruptedException if interrupted while waiting
* @throws NotAllMetaRegionsOnlineException if root not available before
* timeout
*/
HServerAddress waitForRoot(final long timeout)
ServerName waitForRoot(final long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException {
HServerAddress address = rootRegionTracker.waitRootRegionLocation(timeout);
if (address == null) {
ServerName sn = rootRegionTracker.waitRootRegionLocation(timeout);
if (sn == null) {
throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
}
return address;
return sn;
}
/**
@ -238,11 +239,11 @@ public class CatalogTracker {
*/
private HRegionInterface getRootServerConnection()
throws IOException, InterruptedException {
HServerAddress address = this.rootRegionTracker.getRootRegionLocation();
if (address == null) {
ServerName sn = this.rootRegionTracker.getRootRegionLocation();
if (sn == null) {
return null;
}
return getCachedConnection(address);
return getCachedConnection(sn);
}
/**
@ -278,7 +279,7 @@ public class CatalogTracker {
if (rootConnection == null) {
return null;
}
HServerAddress newLocation = MetaReader.readMetaLocation(rootConnection);
ServerName newLocation = MetaReader.readMetaLocation(rootConnection);
if (newLocation == null) {
return null;
}
@ -317,7 +318,7 @@ public class CatalogTracker {
* @throws NotAllMetaRegionsOnlineException if meta not available before
* timeout
*/
public HServerAddress waitForMeta(long timeout)
public ServerName waitForMeta(long timeout)
throws InterruptedException, IOException, NotAllMetaRegionsOnlineException {
long stop = System.currentTimeMillis() + timeout;
synchronized (metaAvailable) {
@ -372,18 +373,18 @@ public class CatalogTracker {
this.metaAvailable.set(false);
}
private void setMetaLocation(HServerAddress metaLocation) {
private void setMetaLocation(final ServerName metaLocation) {
metaAvailable.set(true);
this.metaLocation = metaLocation;
// no synchronization because these are private and already under lock
metaAvailable.notifyAll();
this.metaAvailable.notifyAll();
}
private HRegionInterface getCachedConnection(HServerAddress address)
private HRegionInterface getCachedConnection(ServerName sn)
throws IOException {
HRegionInterface protocol = null;
try {
protocol = connection.getHRegionConnection(address, false);
protocol = connection.getHRegionConnection(sn.getHostname(), sn.getPort());
} catch (RetriesExhaustedException e) {
if (e.getCause() != null && e.getCause() instanceof ConnectException) {
// Catch this; presume it means the cached connection has gone bad.
@ -392,10 +393,10 @@ public class CatalogTracker {
}
} catch (SocketTimeoutException e) {
// Return 'protocol' == null.
LOG.debug("Timed out connecting to " + address);
LOG.debug("Timed out connecting to " + sn);
} catch (SocketException e) {
// Return 'protocol' == null.
LOG.debug("Exception connecting to " + address);
LOG.debug("Exception connecting to " + sn);
} catch (IOException ioe) {
Throwable cause = ioe.getCause();
if (cause != null && cause instanceof EOFException) {
@ -412,7 +413,7 @@ public class CatalogTracker {
}
private boolean verifyRegionLocation(HRegionInterface metaServer,
final HServerAddress address,
final ServerName address,
byte [] regionName)
throws IOException {
if (metaServer == null) {
@ -469,7 +470,8 @@ public class CatalogTracker {
throw e;
}
return (connection == null)? false:
verifyRegionLocation(connection,this.rootRegionTracker.getRootRegionLocation(),
verifyRegionLocation(connection,
this.rootRegionTracker.getRootRegionLocation(),
HRegionInfo.ROOT_REGIONINFO.getRegionName());
}

View File

@ -26,8 +26,8 @@ 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.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
@ -87,18 +87,17 @@ public class MetaEditor {
}
public static void addDaughter(final CatalogTracker catalogTracker,
final HRegionInfo regionInfo, final HServerInfo serverInfo)
final HRegionInfo regionInfo, final ServerName sn)
throws NotAllMetaRegionsOnlineException, IOException {
HRegionInterface server = catalogTracker.waitForMetaServerConnectionDefault();
byte [] catalogRegionName = CatalogTracker.META_REGION;
Put put = new Put(regionInfo.getRegionName());
addRegionInfo(put, regionInfo);
if (serverInfo != null) addLocation(put, serverInfo);
if (sn != null) addLocation(put, sn);
server.put(catalogRegionName, put);
LOG.info("Added daughter " + regionInfo.getRegionNameAsString() +
" in region " + Bytes.toString(catalogRegionName) +
(serverInfo == null?
", serverInfo=null": ", serverInfo=" + serverInfo.getServerName()));
(sn == null? ", serverName=null": ", serverName=" + sn.toString()));
}
/**
@ -110,18 +109,18 @@ public class MetaEditor {
*
* @param catalogTracker catalog tracker
* @param regionInfo region to update location of
* @param serverInfo server the region is located on
* @param sn Server name
* @throws IOException
* @throws ConnectException Usually because the regionserver carrying .META.
* is down.
* @throws NullPointerException Because no -ROOT- server connection
*/
public static void updateMetaLocation(CatalogTracker catalogTracker,
HRegionInfo regionInfo, HServerInfo serverInfo)
HRegionInfo regionInfo, ServerName sn)
throws IOException, ConnectException {
HRegionInterface server = catalogTracker.waitForRootServerConnectionDefault();
if (server == null) throw new IOException("No server for -ROOT-");
updateLocation(server, CatalogTracker.ROOT_REGION, regionInfo, serverInfo);
updateLocation(server, CatalogTracker.ROOT_REGION, regionInfo, sn);
}
/**
@ -133,14 +132,14 @@ public class MetaEditor {
*
* @param catalogTracker catalog tracker
* @param regionInfo region to update location of
* @param serverInfo server the region is located on
* @param sn Server name
* @throws IOException
*/
public static void updateRegionLocation(CatalogTracker catalogTracker,
HRegionInfo regionInfo, HServerInfo serverInfo)
HRegionInfo regionInfo, ServerName sn)
throws IOException {
updateLocation(catalogTracker.waitForMetaServerConnectionDefault(),
CatalogTracker.META_REGION, regionInfo, serverInfo);
CatalogTracker.META_REGION, regionInfo, sn);
}
/**
@ -152,20 +151,19 @@ public class MetaEditor {
* @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
* @param sn Server name
* @throws IOException In particular could throw {@link java.net.ConnectException}
* if the server is down on other end.
*/
private static void updateLocation(HRegionInterface server,
byte [] catalogRegionName, HRegionInfo regionInfo, HServerInfo serverInfo)
byte [] catalogRegionName, HRegionInfo regionInfo, ServerName sn)
throws IOException {
Put put = new Put(regionInfo.getRegionName());
addLocation(put, serverInfo);
addLocation(put, sn);
server.put(catalogRegionName, put);
LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
" in region " + Bytes.toString(catalogRegionName) + " with " +
"server=" + serverInfo.getHostnamePort() + ", " +
"startcode=" + serverInfo.getStartCode());
"serverName=" + sn.toString());
}
/**
@ -228,11 +226,11 @@ public class MetaEditor {
return p;
}
private static Put addLocation(final Put p, final HServerInfo hsi) {
private static Put addLocation(final Put p, final ServerName sn) {
p.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(hsi.getHostnamePort()));
Bytes.toBytes(sn.getHostAndPort()));
p.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(hsi.getStartCode()));
Bytes.toBytes(sn.getStartcode()));
return p;
}
}

View File

@ -30,11 +30,10 @@ import java.util.TreeSet;
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.HTableDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@ -125,10 +124,11 @@ public class MetaReader {
* 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
* @return map of regions to their currently assigned server where server is
* a String of &lt;host> ':' &lt;port>
* @throws IOException
*/
public static Map<HRegionInfo,HServerAddress> fullScan(
public static Map<HRegionInfo, ServerName> fullScan(
CatalogTracker catalogTracker)
throws IOException {
return fullScan(catalogTracker, new TreeSet<String>());
@ -147,7 +147,7 @@ public class MetaReader {
* @return map of regions to their currently assigned server
* @throws IOException
*/
public static Map<HRegionInfo,HServerAddress> fullScan(
public static Map<HRegionInfo, ServerName> fullScan(
CatalogTracker catalogTracker, final Set<String> disabledTables)
throws IOException {
return fullScan(catalogTracker, disabledTables, false);
@ -168,17 +168,17 @@ public class MetaReader {
* @return map of regions to their currently assigned server
* @throws IOException
*/
public static Map<HRegionInfo,HServerAddress> fullScan(
public static Map<HRegionInfo, ServerName> fullScan(
CatalogTracker catalogTracker, final Set<String> disabledTables,
final boolean excludeOfflinedSplitParents)
throws IOException {
final Map<HRegionInfo,HServerAddress> regions =
new TreeMap<HRegionInfo,HServerAddress>();
final Map<HRegionInfo, ServerName> regions =
new TreeMap<HRegionInfo, ServerName>();
Visitor v = new Visitor() {
@Override
public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true;
Pair<HRegionInfo,HServerAddress> region = metaRowToRegionPair(r);
Pair<HRegionInfo, ServerName> region = metaRowToRegionPair(r);
if (region == null) return true;
HRegionInfo hri = region.getFirst();
if (disabledTables.contains(
@ -199,8 +199,6 @@ public class MetaReader {
* 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.
* <p>
* Returns HServerInfo which includes server startcode.
*
* @return map of regions to their currently assigned server
* @throws IOException
@ -273,10 +271,10 @@ public class MetaReader {
/**
* Reads the location of META from ROOT.
* @param metaServer connection to server hosting ROOT
* @return location of META in ROOT, null if not available
* @return location of META in ROOT where location, or null if not available
* @throws IOException
*/
public static HServerAddress readMetaLocation(HRegionInterface metaServer)
public static ServerName readMetaLocation(HRegionInterface metaServer)
throws IOException {
return readLocation(metaServer, CatalogTracker.ROOT_REGION,
CatalogTracker.META_REGION);
@ -286,10 +284,10 @@ public class MetaReader {
* 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
* @return location of META in ROOT where location is, or null if not available
* @throws IOException
*/
public static HServerAddress readRegionLocation(CatalogTracker catalogTracker,
public static ServerName readRegionLocation(CatalogTracker catalogTracker,
byte [] regionName)
throws IOException {
if (isMetaRegion(regionName)) throw new IllegalArgumentException("See readMetaLocation");
@ -297,14 +295,17 @@ public class MetaReader {
CatalogTracker.META_REGION, regionName);
}
private static HServerAddress readLocation(HRegionInterface metaServer,
private static ServerName readLocation(HRegionInterface metaServer,
byte [] catalogRegionName, byte [] regionName)
throws IOException {
Result r = null;
try {
r = metaServer.get(catalogRegionName,
new Get(regionName).addColumn(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER));
new Get(regionName).
addColumn(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER).
addColumn(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER));
} catch (java.net.SocketTimeoutException e) {
// Treat this exception + message as unavailable catalog table. Catch it
// and fall through to return a null
@ -334,78 +335,57 @@ public class MetaReader {
if (r == null || r.isEmpty()) {
return null;
}
byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
return new HServerAddress(Bytes.toString(value));
return getServerNameFromResult(r);
}
/**
* 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
* @return location of META in ROOT where location is
* a String of &lt;host> ':' &lt;port>, or null if not available
* @throws IOException
*/
public static Pair<HRegionInfo, HServerAddress> getRegion(
public static Pair<HRegionInfo, ServerName> getRegion(
CatalogTracker catalogTracker, byte [] regionName)
throws IOException {
Get get = new Get(regionName);
get.addFamily(HConstants.CATALOG_FAMILY);
byte [] meta = getCatalogRegionNameForRegion(regionName);
Result r = catalogTracker.waitForMetaServerConnectionDefault().get(meta, get);
if(r == null || r.isEmpty()) {
return null;
}
return metaRowToRegionPair(r);
return (r == null || r.isEmpty())? null: metaRowToRegionPair(r);
}
/**
* @param data A .META. table row.
* @return A pair of the regioninfo and the server address from <code>data</code>
* or null for server address if no address set in .META. or null for a result
* if no HRegionInfo found.
* @throws IOException
*/
public static Pair<HRegionInfo, HServerAddress> metaRowToRegionPair(
Result data) throws IOException {
byte [] bytes =
data.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) return null;
HRegionInfo info = Writables.getHRegionInfo(bytes);
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<HRegionInfo,HServerAddress>(info, server);
} else {
return new Pair<HRegionInfo, HServerAddress>(info, null);
}
}
/**
* @param data A .META. table row.
* @return A pair of the regioninfo and the server info from <code>data</code>
* @return A pair of the regioninfo and the ServerName
* (or null for server address if no address set in .META.).
* @throws IOException
*/
public static Pair<HRegionInfo, HServerInfo> metaRowToRegionPairWithInfo(
Result data) throws IOException {
public static Pair<HRegionInfo, ServerName> metaRowToRegionPair(Result data)
throws IOException {
byte [] bytes = data.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (bytes == null) return null;
HRegionInfo info = Writables.getHRegionInfo(bytes);
final byte[] value = data.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value != null && value.length > 0) {
final long startCode = Bytes.toLong(data.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER));
HServerAddress server = new HServerAddress(Bytes.toString(value));
HServerInfo hsi = new HServerInfo(server, startCode, 0,
server.getHostname());
return new Pair<HRegionInfo,HServerInfo>(info, hsi);
} else {
return new Pair<HRegionInfo, HServerInfo>(info, null);
ServerName sn = getServerNameFromResult(data);
// sn can be null in case where no server inof.
return new Pair<HRegionInfo, ServerName>(info, sn);
}
/**
* @param data Result to interrogate.
* @return A ServerName instance or null if necessary fields not found or empty.
*/
private static ServerName getServerNameFromResult(final Result data) {
byte[] value = data.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value == null || value.length == 0) return null;
String hostAndPort = Bytes.toString(value);
value = data.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
if (value == null || value.length == 0) return null;
return new ServerName(hostAndPort, Bytes.toLong(value));
}
/**
@ -528,26 +508,27 @@ public class MetaReader {
/**
* @param catalogTracker
* @param tableName
* @return Return list of regioninfos and server addresses.
* @return Return list of regioninfos and server.
* @throws IOException
* @throws InterruptedException
*/
public static List<Pair<HRegionInfo, HServerAddress>>
public static List<Pair<HRegionInfo, ServerName>>
getTableRegionsAndLocations(CatalogTracker catalogTracker, String tableName)
throws IOException, InterruptedException {
byte [] tableNameBytes = Bytes.toBytes(tableName);
if (Bytes.equals(tableNameBytes, HConstants.ROOT_TABLE_NAME)) {
// If root, do a bit of special handling.
HServerAddress hsa = catalogTracker.getRootLocation();
List<Pair<HRegionInfo, HServerAddress>> list =
new ArrayList<Pair<HRegionInfo, HServerAddress>>();
list.add(new Pair<HRegionInfo, HServerAddress>(HRegionInfo.ROOT_REGIONINFO, hsa));
ServerName serverName = catalogTracker.getRootLocation();
List<Pair<HRegionInfo, ServerName>> list =
new ArrayList<Pair<HRegionInfo, ServerName>>();
list.add(new Pair<HRegionInfo, ServerName>(HRegionInfo.ROOT_REGIONINFO,
serverName));
return list;
}
HRegionInterface metaServer =
getCatalogRegionInterface(catalogTracker, tableNameBytes);
List<Pair<HRegionInfo, HServerAddress>> regions =
new ArrayList<Pair<HRegionInfo, HServerAddress>>();
List<Pair<HRegionInfo, ServerName>> regions =
new ArrayList<Pair<HRegionInfo, ServerName>>();
Scan scan = getScanForTableName(tableNameBytes);
scan.addFamily(HConstants.CATALOG_FAMILY);
long scannerid =
@ -556,7 +537,7 @@ public class MetaReader {
Result data;
while((data = metaServer.next(scannerid)) != null) {
if (data != null && data.size() > 0) {
Pair<HRegionInfo, HServerAddress> region = metaRowToRegionPair(data);
Pair<HRegionInfo, ServerName> region = metaRowToRegionPair(data);
if (region == null) continue;
regions.add(region);
}
@ -575,7 +556,7 @@ public class MetaReader {
* @throws IOException
*/
public static NavigableMap<HRegionInfo, Result>
getServerUserRegions(CatalogTracker catalogTracker, final HServerInfo hsi)
getServerUserRegions(CatalogTracker catalogTracker, final ServerName serverName)
throws IOException {
HRegionInterface metaServer =
catalogTracker.waitForMetaServerConnectionDefault();
@ -588,10 +569,9 @@ public class MetaReader {
Result result;
while((result = metaServer.next(scannerid)) != null) {
if (result != null && result.size() > 0) {
Pair<HRegionInfo, HServerInfo> pair =
metaRowToRegionPairWithInfo(result);
Pair<HRegionInfo, ServerName> pair = metaRowToRegionPair(result);
if (pair == null) continue;
if (pair.getSecond() == null || !pair.getSecond().equals(hsi)) {
if (pair.getSecond() == null || !serverName.equals(pair.getSecond())) {
continue;
}
hris.put(pair.getFirst(), result);

View File

@ -21,7 +21,7 @@ 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.ServerName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -53,11 +53,11 @@ public class RootLocationEditor {
* Sets the location of <code>-ROOT-</code> in ZooKeeper to the
* specified server address.
* @param zookeeper zookeeper reference
* @param location server address hosting root
* @param location The server hosting <code>-ROOT-</code>
* @throws KeeperException unexpected zookeeper exception
*/
public static void setRootLocation(ZooKeeperWatcher zookeeper,
HServerAddress location)
final ServerName location)
throws KeeperException {
LOG.info("Setting ROOT region location in ZooKeeper as " + location);
try {

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException;
@ -371,7 +373,7 @@ public class HBaseAdmin implements Abortable {
}
// Wait until all regions deleted
HRegionInterface server =
connection.getHRegionConnection(firstMetaServer.getServerAddress());
connection.getHRegionConnection(firstMetaServer.getHostname(), firstMetaServer.getPort());
for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
long scannerId = -1L;
try {
@ -762,18 +764,15 @@ public class HBaseAdmin implements Abortable {
CatalogTracker ct = getCatalogTracker();
try {
if (hostAndPort != null) {
HServerAddress hsa = new HServerAddress(hostAndPort);
Pair<HRegionInfo, HServerAddress> pair =
MetaReader.getRegion(ct, regionname);
Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
if (pair == null || pair.getSecond() == null) {
LOG.info("No server in .META. for " +
Bytes.toString(regionname) + "; pair=" + pair);
} else {
closeRegion(hsa, pair.getFirst());
closeRegion(pair.getSecond(), pair.getFirst());
}
} else {
Pair<HRegionInfo, HServerAddress> pair =
MetaReader.getRegion(ct, regionname);
Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
if (pair == null || pair.getSecond() == null) {
LOG.info("No server in .META. for " +
Bytes.toString(regionname) + "; pair=" + pair);
@ -786,9 +785,10 @@ public class HBaseAdmin implements Abortable {
}
}
private void closeRegion(final HServerAddress hsa, final HRegionInfo hri)
private void closeRegion(final ServerName sn, final HRegionInfo hri)
throws IOException {
HRegionInterface rs = this.connection.getHRegionConnection(hsa);
HRegionInterface rs =
this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
// Close the region without updating zk state.
rs.closeRegion(hri, false);
}
@ -820,7 +820,7 @@ public class HBaseAdmin implements Abortable {
CatalogTracker ct = getCatalogTracker();
try {
if (isRegionName) {
Pair<HRegionInfo, HServerAddress> pair =
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(ct, tableNameOrRegionName);
if (pair == null || pair.getSecond() == null) {
LOG.info("No server in .META. for " +
@ -829,10 +829,10 @@ public class HBaseAdmin implements Abortable {
flush(pair.getSecond(), pair.getFirst());
}
} else {
List<Pair<HRegionInfo, HServerAddress>> pairs =
List<Pair<HRegionInfo, ServerName>> pairs =
MetaReader.getTableRegionsAndLocations(ct,
Bytes.toString(tableNameOrRegionName));
for (Pair<HRegionInfo, HServerAddress> pair: pairs) {
for (Pair<HRegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
@ -850,9 +850,10 @@ public class HBaseAdmin implements Abortable {
}
}
private void flush(final HServerAddress hsa, final HRegionInfo hri)
private void flush(final ServerName sn, final HRegionInfo hri)
throws IOException {
HRegionInterface rs = this.connection.getHRegionConnection(hsa);
HRegionInterface rs =
this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
rs.flushRegion(hri);
}
@ -922,7 +923,7 @@ public class HBaseAdmin implements Abortable {
CatalogTracker ct = getCatalogTracker();
try {
if (isRegionName(tableNameOrRegionName)) {
Pair<HRegionInfo, HServerAddress> pair =
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(ct, tableNameOrRegionName);
if (pair == null || pair.getSecond() == null) {
LOG.info("No server in .META. for " +
@ -931,10 +932,10 @@ public class HBaseAdmin implements Abortable {
compact(pair.getSecond(), pair.getFirst(), major);
}
} else {
List<Pair<HRegionInfo, HServerAddress>> pairs =
List<Pair<HRegionInfo, ServerName>> pairs =
MetaReader.getTableRegionsAndLocations(ct,
Bytes.toString(tableNameOrRegionName));
for (Pair<HRegionInfo, HServerAddress> pair: pairs) {
for (Pair<HRegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
@ -953,10 +954,11 @@ public class HBaseAdmin implements Abortable {
}
}
private void compact(final HServerAddress hsa, final HRegionInfo hri,
private void compact(final ServerName sn, final HRegionInfo hri,
final boolean major)
throws IOException {
HRegionInterface rs = this.connection.getHRegionConnection(hsa);
HRegionInterface rs =
this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
rs.compactRegion(hri, major);
}
@ -969,7 +971,7 @@ public class HBaseAdmin implements Abortable {
* @param destServerName The servername of the destination regionserver. If
* passed the empty byte array we'll assign to a random server. A server name
* is made of host, port and startcode. Here is an example:
* <code> host187.example.com,60020,1289493121758</code>.
* <code> host187.example.com,60020,1289493121758</code>
* @throws UnknownRegionException Thrown if we can't find a region named
* <code>encodedRegionName</code>
* @throws ZooKeeperConnectionException
@ -1077,7 +1079,7 @@ public class HBaseAdmin implements Abortable {
try {
if (isRegionName(tableNameOrRegionName)) {
// Its a possible region name.
Pair<HRegionInfo, HServerAddress> pair =
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(ct, tableNameOrRegionName);
if (pair == null || pair.getSecond() == null) {
LOG.info("No server in .META. for " +
@ -1086,10 +1088,10 @@ public class HBaseAdmin implements Abortable {
split(pair.getSecond(), pair.getFirst(), splitPoint);
}
} else {
List<Pair<HRegionInfo, HServerAddress>> pairs =
List<Pair<HRegionInfo, ServerName>> pairs =
MetaReader.getTableRegionsAndLocations(ct,
Bytes.toString(tableNameOrRegionName));
for (Pair<HRegionInfo, HServerAddress> pair: pairs) {
for (Pair<HRegionInfo, ServerName> pair: pairs) {
// May not be a server for a particular row
if (pair.getSecond() == null) continue;
HRegionInfo r = pair.getFirst();
@ -1106,9 +1108,10 @@ public class HBaseAdmin implements Abortable {
}
}
private void split(final HServerAddress hsa, final HRegionInfo hri,
private void split(final ServerName sn, final HRegionInfo hri,
byte[] splitPoint) throws IOException {
HRegionInterface rs = this.connection.getHRegionConnection(hsa);
HRegionInterface rs =
this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
rs.splitRegion(hri, splitPoint);
}
@ -1179,10 +1182,27 @@ public class HBaseAdmin implements Abortable {
/**
* Stop the designated regionserver.
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link #stopRegionServer(String)}
*/
public synchronized void stopRegionServer(final HServerAddress hsa)
throws IOException {
HRegionInterface rs = this.connection.getHRegionConnection(hsa);
HRegionInterface rs =
this.connection.getHRegionConnection(hsa);
rs.stop("Called by admin client " + this.connection.toString());
}
/**
* Stop the designated regionserver
* @param hostnamePort Hostname and port delimited by a <code>:</code> as in
* <code>example.org:1234</code>
* @throws IOException if a remote or network exception occurs
*/
public synchronized void stopRegionServer(final String hostnamePort)
throws IOException {
String hostname = Addressing.parseHostname(hostnamePort);
int port = Addressing.parsePort(hostnamePort);
HRegionInterface rs =
this.connection.getHRegionConnection(hostname, port);
rs.stop("Called by admin client " + this.connection.toString());
}

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
@ -188,19 +189,44 @@ public interface HConnection extends Abortable {
* @param regionServer - the server to connect to
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link #getHRegionConnection(InetSocketAddress)}
*/
public HRegionInterface getHRegionConnection(HServerAddress regionServer)
throws IOException;
/**
* Establishes a connection to the region server at the specified address.
* @param hostname RegionServer hostname
* @param port RegionServer port
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
*
*/
public HRegionInterface getHRegionConnection(final String hostname, final int port)
throws IOException;
/**
* Establishes a connection to the region server at the specified address.
* @param regionServer - the server to connect to
* @param getMaster - do we check if master is alive
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link #getHRegionConnection(HServerAddress, boolean)}
*/
public HRegionInterface getHRegionConnection(
HServerAddress regionServer, boolean getMaster)
public HRegionInterface getHRegionConnection(HServerAddress regionServer,
boolean getMaster)
throws IOException;
/**
* Establishes a connection to the region server at the specified address.
* @param hostname RegionServer hostname
* @param port RegionServer port
* @param getMaster - do we check if master is alive
* @return proxy for HRegionServer
* @throws IOException if a remote or network exception occurs
*/
public HRegionInterface getHRegionConnection(final String hostname,
final int port, boolean getMaster)
throws IOException;
/**
@ -343,6 +369,7 @@ public interface HConnection extends Abortable {
* Scan zookeeper to get the number of region servers
* @return the number of region servers that are currently running
* @throws IOException if a remote or network exception occurs
* @deprecated This method will be changed from public to package protected.
*/
public int getCurrentNrHRS() throws IOException;
}

View File

@ -22,8 +22,16 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.lang.reflect.Proxy;
import java.lang.reflect.UndeclaredThrowableException;
import java.util.*;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArraySet;
@ -46,11 +54,17 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterAddressTracker;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.*;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import org.apache.hadoop.hbase.ipc.ExecRPCInvoker;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.SoftValueSortedMap;
@ -243,7 +257,8 @@ public class HConnectionManager {
private final Map<String, HRegionInterface> servers =
new ConcurrentHashMap<String, HRegionInterface>();
private final ConcurrentHashMap<String, String> connectionLock = new ConcurrentHashMap<String, String>();
private final ConcurrentHashMap<String, String> connectionLock =
new ConcurrentHashMap<String, String>();
/**
* Map of table to table {@link HRegionLocation}s. The table key is made
@ -340,7 +355,7 @@ public class HConnectionManager {
}
}
HServerAddress masterLocation = null;
ServerName sn = null;
synchronized (this.masterLock) {
for (int tries = 0;
!this.closed &&
@ -349,8 +364,8 @@ public class HConnectionManager {
tries++) {
try {
masterLocation = masterAddressTracker.getMasterAddress();
if(masterLocation == null) {
sn = masterAddressTracker.getMasterAddress();
if (sn == null) {
LOG.info("ZooKeeper available but no active master location found");
throw new MasterNotRunningException();
}
@ -358,9 +373,11 @@ public class HConnectionManager {
if (clusterId.hasId()) {
conf.set(HConstants.CLUSTER_ID, clusterId.getId());
}
InetSocketAddress isa =
new InetSocketAddress(sn.getHostname(), sn.getPort());
HMasterInterface tryMaster = (HMasterInterface)HBaseRPC.getProxy(
HMasterInterface.class, HMasterInterface.VERSION,
masterLocation.getInetSocketAddress(), this.conf, this.rpcTimeout);
HMasterInterface.class, HMasterInterface.VERSION, isa, this.conf,
this.rpcTimeout);
if (tryMaster.isMasterRunning()) {
this.master = tryMaster;
@ -391,10 +408,10 @@ public class HConnectionManager {
this.masterChecked = true;
}
if (this.master == null) {
if (masterLocation == null) {
if (sn == null) {
throw new MasterNotRunningException();
}
throw new MasterNotRunningException(masterLocation.toString());
throw new MasterNotRunningException(sn.toString());
}
return this.master;
}
@ -577,12 +594,13 @@ public class HConnectionManager {
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
try {
HServerAddress hsa =
ServerName servername =
this.rootRegionTracker.waitRootRegionLocation(this.rpcTimeout);
LOG.debug("Lookedup root region location, connection=" + this +
"; hsa=" + hsa);
if (hsa == null) return null;
return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa);
"; serverName=" + ((servername == null)? "": servername.toString()));
if (servername == null) return null;
return new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
servername.getHostname(), servername.getPort());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;
@ -631,11 +649,14 @@ public class HConnectionManager {
if (value == null) {
return true; // don't cache it
}
final String serverAddress = Bytes.toString(value);
final String hostAndPort = Bytes.toString(value);
String hostname = Addressing.parseHostname(hostAndPort);
int port = Addressing.parsePort(hostAndPort);
value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
// instantiate the location
HRegionLocation loc = new HRegionLocation(regionInfo,
new HServerAddress(serverAddress));
HRegionLocation loc =
new HRegionLocation(regionInfo, hostname, port);
// cache this meta entry
cacheLocation(tableName, loc);
}
@ -690,7 +711,7 @@ public class HConnectionManager {
// If null still, go around again.
if (metaLocation == null) continue;
HRegionInterface server =
getHRegionConnection(metaLocation.getServerAddress());
getHRegionConnection(metaLocation.getHostname(), metaLocation.getPort());
Result regionInfoRow = null;
// This block guards against two threads trying to load the meta
@ -746,19 +767,22 @@ public class HConnectionManager {
value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
String serverAddress = "";
String hostAndPort = "";
if (value != null) {
serverAddress = Bytes.toString(value);
hostAndPort = Bytes.toString(value);
}
if (serverAddress.equals("")) {
if (hostAndPort.equals("")) {
throw new NoServerForRegionException("No server address listed " +
"in " + Bytes.toString(parentTable) + " for region " +
regionInfo.getRegionNameAsString());
}
// instantiate the location
location = new HRegionLocation(regionInfo,
new HServerAddress(serverAddress));
// Instantiate the location
String hostname = Addressing.parseHostname(hostAndPort);
int port = Addressing.parsePort(hostAndPort);
value = regionInfoRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
location = new HRegionLocation(regionInfo, hostname, port);
cacheLocation(tableName, location);
return location;
} catch (TableNotFoundException e) {
@ -936,14 +960,48 @@ public class HConnectionManager {
}
}
public HRegionInterface getHRegionConnection(
HServerAddress regionServer, boolean getMaster)
public HRegionInterface getHRegionConnection(HServerAddress hsa)
throws IOException {
if (getMaster) {
getMaster();
return getHRegionConnection(hsa, false);
}
@Override
public HRegionInterface getHRegionConnection(final String hostname,
final int port)
throws IOException {
return getHRegionConnection(hostname, port, false);
}
public HRegionInterface getHRegionConnection(HServerAddress hsa,
boolean master)
throws IOException {
return getHRegionConnection(null, -1, hsa.getInetSocketAddress(), master);
}
@Override
public HRegionInterface getHRegionConnection(final String hostname,
final int port, final boolean master)
throws IOException {
return getHRegionConnection(hostname, port, null, master);
}
/**
* Either the passed <code>isa</code> is null or <code>hostname</code>
* can be but not both.
* @param hostname
* @param port
* @param isa
* @param master
* @return Proxy.
* @throws IOException
*/
HRegionInterface getHRegionConnection(final String hostname, final int port,
final InetSocketAddress isa, final boolean master)
throws IOException {
if (master) getMaster();
HRegionInterface server;
String rsName = regionServer.toString();
String rsName = isa != null?
isa.toString(): Addressing.createHostAndPortStr(hostname, port);
// See if we already have a connection (common case)
server = this.servers.get(rsName);
if (server == null) {
@ -958,12 +1016,15 @@ public class HConnectionManager {
if (clusterId.hasId()) {
conf.set(HConstants.CLUSTER_ID, clusterId.getId());
}
// Only create isa when we need to.
InetSocketAddress address = isa != null? isa:
new InetSocketAddress(hostname, port);
// definitely a cache miss. establish an RPC for this RS
server = (HRegionInterface) HBaseRPC.waitForProxy(
serverInterfaceClass, HRegionInterface.VERSION,
regionServer.getInetSocketAddress(), this.conf,
address, this.conf,
this.maxRPCAttempts, this.rpcTimeout, this.rpcTimeout);
this.servers.put(rsName, server);
this.servers.put(address.toString(), server);
} catch (RemoteException e) {
LOG.warn("RemoteException connecting to RS", e);
// Throw what the RemoteException was carrying.
@ -975,12 +1036,6 @@ public class HConnectionManager {
return server;
}
public HRegionInterface getHRegionConnection(
HServerAddress regionServer)
throws IOException {
return getHRegionConnection(regionServer, false);
}
/**
* Get the ZooKeeper instance for this TableServers instance.
*
@ -1065,10 +1120,8 @@ public class HConnectionManager {
this.closed = true;
}
private <R> Callable<MultiResponse> createCallable(
final HServerAddress address,
final MultiAction<R> multi,
final byte [] tableName) {
private <R> Callable<MultiResponse> createCallable(final HRegionLocation loc,
final MultiAction<R> multi, final byte [] tableName) {
final HConnection connection = this;
return new Callable<MultiResponse>() {
public MultiResponse call() throws IOException {
@ -1079,7 +1132,8 @@ public class HConnectionManager {
}
@Override
public void instantiateServer(boolean reload) throws IOException {
server = connection.getHRegionConnection(address);
server =
connection.getHRegionConnection(loc.getHostname(), loc.getPort());
}
}
);
@ -1191,8 +1245,10 @@ public class HConnectionManager {
}
// Keep track of the most recent servers for any given item for better
// exceptional reporting.
HServerAddress [] lastServers = new HServerAddress[results.length];
// exceptional reporting. We keep HRegionLocation to save on parsing.
// Later below when we use lastServers, we'll pull what we need from
// lastServers.
HRegionLocation [] lastServers = new HRegionLocation[results.length];
List<Row> workingList = new ArrayList<Row>(list);
boolean retry = true;
// count that helps presize actions array
@ -1208,43 +1264,41 @@ public class HConnectionManager {
Thread.sleep(sleepTime);
}
// step 1: break up into regionserver-sized chunks and build the data structs
Map<HServerAddress, MultiAction<R>> actionsByServer =
new HashMap<HServerAddress, MultiAction<R>>();
Map<HRegionLocation, MultiAction<R>> actionsByServer =
new HashMap<HRegionLocation, MultiAction<R>>();
for (int i = 0; i < workingList.size(); i++) {
Row row = workingList.get(i);
if (row != null) {
HRegionLocation loc = locateRegion(tableName, row.getRow(), true);
HServerAddress address = loc.getServerAddress();
byte[] regionName = loc.getRegionInfo().getRegionName();
MultiAction<R> actions = actionsByServer.get(address);
MultiAction<R> actions = actionsByServer.get(loc);
if (actions == null) {
actions = new MultiAction<R>();
actionsByServer.put(address, actions);
actionsByServer.put(loc, actions);
}
Action<R> action = new Action<R>(regionName, row, i);
lastServers[i] = address;
lastServers[i] = loc;
actions.add(regionName, action);
}
}
// step 2: make the requests
Map<HServerAddress,Future<MultiResponse>> futures =
new HashMap<HServerAddress, Future<MultiResponse>>(
Map<HRegionLocation, Future<MultiResponse>> futures =
new HashMap<HRegionLocation, Future<MultiResponse>>(
actionsByServer.size());
for (Entry<HServerAddress, MultiAction<R>> e
: actionsByServer.entrySet()) {
for (Entry<HRegionLocation, MultiAction<R>> e: actionsByServer.entrySet()) {
futures.put(e.getKey(), pool.submit(createCallable(e.getKey(), e.getValue(), tableName)));
}
// step 3: collect the failures and successes and prepare for retry
for (Entry<HServerAddress, Future<MultiResponse>> responsePerServer
for (Entry<HRegionLocation, Future<MultiResponse>> responsePerServer
: futures.entrySet()) {
HServerAddress address = responsePerServer.getKey();
HRegionLocation loc = responsePerServer.getKey();
try {
Future<MultiResponse> future = responsePerServer.getValue();
@ -1252,7 +1306,8 @@ public class HConnectionManager {
if (resp == null) {
// Entire server failed
LOG.debug("Failed all for server: " + address + ", removing from cache");
LOG.debug("Failed all for server: " + loc.getHostnamePort() +
", removing from cache");
continue;
}
@ -1277,7 +1332,7 @@ public class HConnectionManager {
}
}
} catch (ExecutionException e) {
LOG.debug("Failed all from " + address, e);
LOG.debug("Failed all from " + loc, e);
}
}
@ -1320,13 +1375,13 @@ public class HConnectionManager {
List<Throwable> exceptions = new ArrayList<Throwable>(actionCount);
List<Row> actions = new ArrayList<Row>(actionCount);
List<HServerAddress> addresses = new ArrayList<HServerAddress>(actionCount);
List<String> addresses = new ArrayList<String>(actionCount);
for (int i = 0 ; i < results.length; i++) {
if (results[i] == null || results[i] instanceof Throwable) {
exceptions.add((Throwable)results[i]);
actions.add(list.get(i));
addresses.add(lastServers[i]);
addresses.add(lastServers[i].getHostnamePort());
}
}
@ -1418,11 +1473,14 @@ public class HConnectionManager {
return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName));
}
public void prewarmRegionCache(final byte[] tableName,
final Map<HRegionInfo, HServerAddress> regions) {
@Override
public void prewarmRegionCache(byte[] tableName,
Map<HRegionInfo, HServerAddress> regions) {
for (Map.Entry<HRegionInfo, HServerAddress> e : regions.entrySet()) {
HServerAddress hsa = e.getValue();
if (hsa == null || hsa.getInetSocketAddress() == null) continue;
cacheLocation(tableName,
new HRegionLocation(e.getKey(), e.getValue()));
new HRegionLocation(e.getKey(), hsa.getHostname(), hsa.getPort()));
}
}

View File

@ -29,6 +29,7 @@ import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.SynchronousQueue;
@ -49,12 +50,14 @@ import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import org.apache.hadoop.hbase.ipc.ExecRPCInvoker;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
@ -283,6 +286,7 @@ public class HTable implements HTableInterface {
* <em>INTERNAL</em> Used by unit tests and tools to do low-level
* manipulations.
* @return An HConnection instance.
* @deprecated This method will be changed from public to package protected.
*/
// TODO(tsuna): Remove this. Unit tests shouldn't require public helpers.
public HConnection getConnection() {
@ -378,10 +382,9 @@ public class HTable implements HTableInterface {
/**
* Gets all the regions and their address for this table.
* <p>
* This is mainly useful for the MapReduce integration.
* @return A map of HRegionInfo with it's server address
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link #getRegionLocations()} or {@link #getStartEndKeys()}
*/
public Map<HRegionInfo, HServerAddress> getRegionsInfo() throws IOException {
final Map<HRegionInfo, HServerAddress> regionMap =
@ -401,8 +404,8 @@ public class HTable implements HTableInterface {
byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value != null && value.length > 0) {
String address = Bytes.toString(value);
server = new HServerAddress(address);
String hostAndPort = Bytes.toString(value);
server = new HServerAddress(Addressing.createInetSocketAddressFromHostAndPortStr(hostAndPort));
}
if (!(info.isOffline() || info.isSplit())) {
@ -416,6 +419,17 @@ public class HTable implements HTableInterface {
return regionMap;
}
/**
* Gets all the regions and their address for this table.
* <p>
* This is mainly useful for the MapReduce integration.
* @return A map of HRegionInfo with it's server address
* @throws IOException if a remote or network exception occurs
*/
public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
return MetaScanner.allTableRegions(getConfiguration(), getTableName(), false);
}
/**
* Save the passed region information and the table's regions
* cache.

View File

@ -23,13 +23,20 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.TreeSet;
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.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
@ -225,8 +232,7 @@ public class MetaScanner {
public static List<HRegionInfo> listAllRegions(Configuration conf, final boolean offlined)
throws IOException {
final List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
MetaScannerVisitor visitor =
new MetaScannerVisitor() {
MetaScannerVisitor visitor = new MetaScannerVisitor() {
@Override
public boolean processRow(Result result) throws IOException {
if (result == null || result.isEmpty()) {
@ -249,6 +255,51 @@ public class MetaScanner {
return regions;
}
/**
* Lists all of the table regions currently in META.
* @param conf
* @param offlined True if we are to include offlined regions, false and we'll
* leave out offlined regions from returned list.
* @return Map of all user-space regions to servers
* @throws IOException
*/
public static NavigableMap<HRegionInfo, ServerName> allTableRegions(Configuration conf, final byte [] tablename, final boolean offlined)
throws IOException {
final NavigableMap<HRegionInfo, ServerName> regions =
new TreeMap<HRegionInfo, ServerName>();
MetaScannerVisitor visitor = new MetaScannerVisitor() {
@Override
public boolean processRow(Result rowResult) throws IOException {
HRegionInfo info = Writables.getHRegionInfo(
rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
if (!(Bytes.equals(info.getTableDesc().getName(), tablename))) {
return false;
}
byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
String hostAndPort = null;
if (value != null && value.length > 0) {
hostAndPort = Bytes.toString(value);
}
value = rowResult.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
long startcode = -1L;
if (value != null && value.length > 0) startcode = Bytes.toLong(value);
if (!(info.isOffline() || info.isSplit())) {
ServerName sn = null;
if (hostAndPort != null && hostAndPort.length() > 0) {
sn = new ServerName(hostAndPort, startcode);
}
regions.put(new UnmodifyableHRegionInfo(info), sn);
}
return true;
}
};
metaScan(conf, visitor);
return regions;
}
/**
* Visitor class called to process each row of the .META. table
*/

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.util.Addressing;
import java.util.Collection;
import java.util.HashMap;
@ -39,22 +40,23 @@ import java.util.Set;
* known server addresses via {@link #getNumExceptions()} and
* {@link #getCause(int)}, {@link #getRow(int)} and {@link #getAddress(int)}.
*/
public class RetriesExhaustedWithDetailsException extends RetriesExhaustedException {
@SuppressWarnings("serial")
public class RetriesExhaustedWithDetailsException
extends RetriesExhaustedException {
List<Throwable> exceptions;
List<Row> actions;
List<HServerAddress> addresses;
List<String> hostnameAndPort;
public RetriesExhaustedWithDetailsException(List<Throwable> exceptions,
List<Row> actions,
List<HServerAddress> addresses) {
List<String> hostnameAndPort) {
super("Failed " + exceptions.size() + " action" +
pluralize(exceptions) + ": " +
getDesc(exceptions,actions,addresses));
getDesc(exceptions, actions, hostnameAndPort));
this.exceptions = exceptions;
this.actions = actions;
this.addresses = addresses;
this.hostnameAndPort = hostnameAndPort;
}
public List<Throwable> getCauses() {
@ -73,8 +75,17 @@ public class RetriesExhaustedWithDetailsException extends RetriesExhaustedExcept
return actions.get(i);
}
/**
* @param i
* @return
* @deprecated
*/
public HServerAddress getAddress(int i) {
return addresses.get(i);
return new HServerAddress(Addressing.createInetSocketAddressFromHostAndPortStr(getHostnamePort(i)));
}
public String getHostnamePort(final int i) {
return this.hostnameAndPort.get(i);
}
public boolean mayHaveClusterIssues() {
@ -100,12 +111,12 @@ public class RetriesExhaustedWithDetailsException extends RetriesExhaustedExcept
public static String getDesc(List<Throwable> exceptions,
List<Row> actions,
List<HServerAddress> addresses) {
List<String> hostnamePort) {
String s = getDesc(classifyExs(exceptions));
s += "servers with issues: ";
Set<HServerAddress> uniqAddr = new HashSet<HServerAddress>();
uniqAddr.addAll(addresses);
for(HServerAddress addr : uniqAddr) {
Set<String> uniqAddr = new HashSet<String>();
uniqAddr.addAll(hostnamePort);
for(String addr : uniqAddr) {
s += addr + ", ";
}
return s;

View File

@ -22,8 +22,8 @@ package org.apache.hadoop.hbase.coprocessor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.UnknownRegionException;
import java.io.IOException;

View File

@ -138,23 +138,22 @@ public interface MasterObserver extends Coprocessor {
* Called prior to moving a given region from one region server to another.
*/
void preMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final HRegionInfo region, final HServerInfo srcServer,
final HServerInfo destServer)
final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
throws UnknownRegionException;
/**
* Called after the region move has been requested.
*/
void postMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final HRegionInfo region, final HServerInfo srcServer,
final HServerInfo destServer)
final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
throws UnknownRegionException;
/**
* Called prior to assigning a specific region.
*/
void preAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final byte [] regionName, final boolean force) throws IOException;
final byte [] regionName, final boolean force)
throws IOException;
/**
* Called after the region assignment has been requested.

View File

@ -23,6 +23,7 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
@ -42,7 +43,7 @@ public class RegionTransitionData implements Writable {
private byte [] regionName;
/** Server event originated from. Optional. */
private String serverName;
private ServerName origin;
/** Time the event was created. Required but automatically set. */
private long stamp;
@ -89,11 +90,11 @@ public class RegionTransitionData implements Writable {
*
* @param eventType type of event
* @param regionName name of region as per <code>HRegionInfo#getRegionName()</code>
* @param serverName name of server setting data
* @param origin Originating {@link ServerName}
*/
public RegionTransitionData(EventType eventType, byte [] regionName,
String serverName) {
this(eventType, regionName, serverName, null);
final ServerName origin) {
this(eventType, regionName, origin, null);
}
/**
@ -107,16 +108,16 @@ public class RegionTransitionData implements Writable {
*
* @param eventType type of event
* @param regionName name of region as per <code>HRegionInfo#getRegionName()</code>
* @param serverName name of server setting data
* @param origin Originating {@link ServerName}
* @param payload Payload examples include the daughters involved in a
* {@link EventType#RS_ZK_REGION_SPLIT}. Can be null
*/
public RegionTransitionData(EventType eventType, byte [] regionName,
String serverName, final byte [] payload) {
final ServerName serverName, final byte [] payload) {
this.eventType = eventType;
this.stamp = System.currentTimeMillis();
this.regionName = regionName;
this.serverName = serverName;
this.origin = serverName;
this.payload = payload;
}
@ -155,8 +156,8 @@ public class RegionTransitionData implements Writable {
*
* @return server name of originating regionserver, or null if from master
*/
public String getServerName() {
return serverName;
public ServerName getOrigin() {
return origin;
}
/**
@ -186,9 +187,7 @@ public class RegionTransitionData implements Writable {
// 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;
this.origin = new ServerName(in.readUTF());
}
if (in.readBoolean()) {
this.payload = Bytes.readByteArray(in);
@ -201,9 +200,9 @@ public class RegionTransitionData implements Writable {
out.writeLong(System.currentTimeMillis());
Bytes.writeByteArray(out, regionName);
// remaining fields are optional so prefixed with boolean
out.writeBoolean(serverName != null);
if(serverName != null) {
out.writeUTF(serverName);
out.writeBoolean(this.origin != null);
if(this.origin != null) {
out.writeUTF(this.origin.toString());
}
out.writeBoolean(this.payload != null);
if (this.payload != null) {
@ -244,7 +243,7 @@ public class RegionTransitionData implements Writable {
@Override
public String toString() {
return "region=" + Bytes.toString(regionName) + ", server=" + serverName +
return "region=" + Bytes.toString(regionName) + ", origin=" + this.origin +
", state=" + eventType;
}
}

View File

@ -41,10 +41,10 @@ import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HColumnDescriptor;
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.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete;
@ -148,8 +148,14 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
// Hbase types
addToMap(HColumnDescriptor.class, code++);
addToMap(HConstants.Modify.class, code++);
addToMap(HMsg.class, code++);
addToMap(HMsg[].class, code++);
// We used to have a class named HMsg but its been removed. Rather than
// just axe it, use following random Integer class -- we just chose any
// class from java.lang -- instead just so codes that follow stay
// in same relative place.
addToMap(Integer.class, code++);
addToMap(Integer[].class, code++);
addToMap(HRegion.class, code++);
addToMap(HRegion[].class, code++);
addToMap(HRegionInfo.class, code++);
@ -225,6 +231,8 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
addToMap(CompareOp.class, code++);
addToMap(ColumnRangeFilter.class, code++);
addToMap(HServerLoad.class, code++);
}
private Class<?> declaredClass;

View File

@ -21,7 +21,6 @@
package org.apache.hadoop.hbase.ipc;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
@ -41,7 +40,13 @@ import java.nio.channels.Selector;
import java.nio.channels.ServerSocketChannel;
import java.nio.channels.SocketChannel;
import java.nio.channels.WritableByteChannel;
import java.util.*;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
@ -52,15 +57,14 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.WritableWithSize;
import org.apache.hadoop.hbase.util.ByteBufferOutputStream;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.ObjectWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.hbase.util.ByteBufferOutputStream;
import com.google.common.base.Function;
import com.google.common.util.concurrent.ThreadFactoryBuilder;

View File

@ -19,22 +19,16 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.hbase.HMsg;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import java.io.IOException;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.io.MapWritable;
import org.apache.hadoop.ipc.VersionedProtocol;
import java.io.IOException;
/**
* HRegionServers interact with the HMasterRegionInterface to report on local
* goings-on and to obtain data-handling instructions from the HMaster.
* <p>Changes here need to be reflected in HbaseObjectWritable HbaseRPC#Invoker.
*
* <p>NOTE: if you change the interface, you must change the RPC version
* number in HBaseRPCProtocolVersion
*
* The Master publishes this Interface for RegionServers to register themselves
* on.
*/
public interface HMasterRegionInterface extends VersionedProtocol {
/**
@ -44,32 +38,27 @@ public interface HMasterRegionInterface extends VersionedProtocol {
// maintained a single global version number on all HBase Interfaces. This
// meant all HBase RPC was broke though only one of the three RPC Interfaces
// had changed. This has since been undone.
public static final long VERSION = 28L;
public static final long VERSION = 29L;
/**
* Called when a region server first starts
* @param info server info
* Called when a region server first starts.
* @param port Port number this regionserver is up on.
* @param serverStartcode This servers' startcode.
* @param serverCurrentTime The current time of the region server in ms
* @throws IOException e
* @return Configuration for the regionserver to use: e.g. filesystem,
* hbase rootdir, etc.
* hbase rootdir, the hostname to use creating the RegionServer ServerName,
* etc.
*/
public MapWritable regionServerStartup(HServerInfo info,
long serverCurrentTime) throws IOException;
public MapWritable regionServerStartup(final int port,
final long serverStartcode, final long serverCurrentTime)
throws IOException;
/**
* Called to renew lease, tell master what the region server is doing and to
* receive new instructions from the master
*
* @param info server's address and start code
* @param msgs things the region server wants to tell the master
* @param mostLoadedRegions Array of HRegionInfos that should contain the
* reporting server's most loaded regions. These are candidates for being
* rebalanced.
* @return instructions from the master to the region server
* @throws IOException e
* @param sn {@link ServerName#getBytes()}
* @param hsl Server load.
* @throws IOException
*/
public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[],
HRegionInfo mostLoadedRegions[])
public void regionServerReport(byte [] sn, HServerLoad hsl)
throws IOException;
}

View File

@ -292,8 +292,10 @@ public interface HRegionInterface extends VersionedProtocol, Stoppable, Abortabl
/**
* Method used when a master is taking the place of another failed one.
* @return The HSI
* @return This servers {@link HServerInfo}; it has RegionServer POV on the
* hostname which may not agree w/ how the Master sees this server.
* @throws IOException e
* @deprecated
*/
public HServerInfo getHServerInfo() throws IOException;

View File

@ -140,8 +140,9 @@ class WritableRpcEngine implements RpcEngine {
client.call(new Invocation(method, args), address,
protocol, ticket, rpcTimeout);
if (logDebug) {
long callTime = System.currentTimeMillis() - startTime;
LOG.debug("Call: " + method.getName() + " " + callTime);
// FIGURE HOW TO TURN THIS OFF!
// long callTime = System.currentTimeMillis() - startTime;
// LOG.debug("Call: " + method.getName() + " " + callTime);
}
return value.get();
}

View File

@ -23,8 +23,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
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.ZooKeeperWatcher;
@ -48,13 +49,17 @@ class ActiveMasterManager extends ZooKeeperListener {
final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
private final HServerAddress address;
private final ServerName sn;
private final Server master;
ActiveMasterManager(ZooKeeperWatcher watcher, HServerAddress address,
Server master) {
/**
* @param watcher
* @param sn ServerName
* @param master In an instance of a Master.
*/
ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
super(watcher);
this.address = address;
this.sn = sn;
this.master = master;
}
@ -122,11 +127,11 @@ class ActiveMasterManager extends ZooKeeperListener {
boolean cleanSetOfActiveMaster = true;
// Try to become the active master, watch if there is another master
try {
if (ZKUtil.setAddressAndWatch(this.watcher,
this.watcher.masterAddressZNode, this.address)) {
if (ZKUtil.createEphemeralNodeAndWatch(this.watcher,
this.watcher.masterAddressZNode, Bytes.toBytes(this.sn.toString()))) {
// We are the master, return
this.clusterHasActiveMaster.set(true);
LOG.info("Master=" + this.address);
LOG.info("Master=" + this.sn);
return cleanSetOfActiveMaster;
}
cleanSetOfActiveMaster = false;
@ -134,9 +139,10 @@ class ActiveMasterManager extends ZooKeeperListener {
// There is another active master running elsewhere or this is a restart
// and the master ephemeral node has not expired yet.
this.clusterHasActiveMaster.set(true);
HServerAddress currentMaster =
ZKUtil.getDataAsAddress(this.watcher, this.watcher.masterAddressZNode);
if (currentMaster != null && currentMaster.equals(this.address)) {
byte [] bytes =
ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
ServerName currentMaster = new ServerName(Bytes.toString(bytes));
if (currentMaster != null && currentMaster.equals(this.sn)) {
LOG.info("Current master has this master's address, " + currentMaster +
"; master was restarted? Waiting on znode to expire...");
// Hurry along the expiration of the znode.
@ -177,11 +183,11 @@ class ActiveMasterManager extends ZooKeeperListener {
public void stop() {
try {
// If our address is in ZK, delete it on our way out
HServerAddress zkAddress =
ZKUtil.getDataAsAddress(watcher, watcher.masterAddressZNode);
byte [] bytes =
ZKUtil.getDataAndWatch(watcher, watcher.masterAddressZNode);
// TODO: redo this to make it atomic (only added for tests)
if(zkAddress != null &&
zkAddress.equals(address)) {
ServerName master = new ServerName(Bytes.toString(bytes));
if(master != null && master.equals(this.sn)) {
ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
}
} catch (KeeperException e) {

View File

@ -24,6 +24,7 @@ import java.io.DataOutput;
import java.io.IOException;
import java.lang.Thread.UncaughtExceptionHandler;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@ -43,11 +44,10 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Chore;
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.HTableDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
@ -124,8 +124,8 @@ public class AssignmentManager extends ZooKeeperListener {
* with the other under a lock on {@link #regions}
* @see #regions
*/
private final NavigableMap<HServerInfo, List<HRegionInfo>> servers =
new TreeMap<HServerInfo, List<HRegionInfo>>();
private final NavigableMap<ServerName, List<HRegionInfo>> servers =
new TreeMap<ServerName, List<HRegionInfo>>();
/**
* Region to server assignment map.
@ -134,8 +134,8 @@ public class AssignmentManager extends ZooKeeperListener {
* with the other under a lock on {@link #regions}
* @see #servers
*/
private final SortedMap<HRegionInfo,HServerInfo> regions =
new TreeMap<HRegionInfo,HServerInfo>();
private final SortedMap<HRegionInfo, ServerName> regions =
new TreeMap<HRegionInfo, ServerName>();
private final ExecutorService executorService;
@ -168,6 +168,26 @@ public class AssignmentManager extends ZooKeeperListener {
this.master.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10);
}
/**
* Compute the average load across all region servers.
* Currently, this uses a very naive computation - just uses the number of
* regions being served, ignoring stats about number of requests.
* @return the average load
*/
double getAverageLoad() {
int totalLoad = 0;
int numServers = 0;
// Sync on this.regions because access to this.servers always synchronizes
// in this order.
synchronized (this.regions) {
for (Map.Entry<ServerName, List<HRegionInfo>> e: servers.entrySet()) {
numServers++;
totalLoad += e.getValue().size();
}
}
return (double)totalLoad / (double)numServers;
}
/**
* @return Instance of ZKTable.
*/
@ -191,33 +211,31 @@ public class AssignmentManager extends ZooKeeperListener {
}
/**
* Handle failover. Restore state from META and ZK. Handle any regions in
* transition. Presumes <code>.META.</code> and <code>-ROOT-</code> deployed.
* @throws KeeperException
* Called on startup.
* Figures whether a fresh cluster start of we are joining extant running cluster.
* @throws IOException
* @throws KeeperException
* @throws InterruptedException
*/
void processFailover() throws KeeperException, IOException, InterruptedException {
void joinCluster() throws IOException, KeeperException, InterruptedException {
// Concurrency note: In the below the accesses on regionsInTransition are
// outside of a synchronization block where usually all accesses to RIT are
// synchronized. The presumption is that in this case it is safe since this
// method is being played by a single thread on startup.
// TODO: Check list of user regions and their assignments against regionservers.
// TODO: Regions that have a null location and are not in regionsInTransitions
// need to be handled.
// Add -ROOT- and .META. on regions map. They must be deployed if we got
// this far. Caller takes care of it.
HServerInfo hsi =
this.serverManager.getHServerInfo(this.catalogTracker.getMetaLocation());
regionOnline(HRegionInfo.FIRST_META_REGIONINFO, hsi);
hsi = this.serverManager.getHServerInfo(this.catalogTracker.getRootLocation());
regionOnline(HRegionInfo.ROOT_REGIONINFO, hsi);
// this far.
ServerName sn = this.catalogTracker.getMetaLocation();
regionOnline(HRegionInfo.FIRST_META_REGIONINFO, sn);
sn = this.catalogTracker.getRootLocation();
regionOnline(HRegionInfo.ROOT_REGIONINFO, sn);
// Scan META to build list of existing regions, servers, and assignment
// Returns servers who have not checked in (assumed dead) and their regions
Map<HServerInfo,List<Pair<HRegionInfo,Result>>> deadServers =
Map<ServerName,List<Pair<HRegionInfo,Result>>> deadServers =
rebuildUserRegions();
// Process list of dead servers
processDeadServers(deadServers);
@ -228,16 +246,37 @@ public class AssignmentManager extends ZooKeeperListener {
public void processRegionsInTransition() throws KeeperException, IOException {
List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher,
watcher.assignmentZNode);
if (nodes.isEmpty()) {
LOG.info("No regions in transition in ZK to process on failover");
return;
// Run through all regions. If they are not assigned and not in RIT, then
// its a clean cluster startup, else its a failover.
boolean userRegionsOutOnCluster = false;
for (Map.Entry<HRegionInfo, ServerName> e: this.regions.entrySet()) {
if (!e.getKey().isMetaRegion() && e.getValue() != null) {
LOG.debug("Found " + e + " out on cluster");
userRegionsOutOnCluster = true;
break;
}
LOG.info("Failed-over master needs to process " + nodes.size() +
" regions in transition");
if (nodes.contains(e.getKey().getEncodedName())) {
LOG.debug("Found " + e + " in RITs");
userRegionsOutOnCluster = true;
break;
}
}
// If we found user regions out on cluster, its a failover.
if (userRegionsOutOnCluster) {
LOG.info("Found regions out on cluster or in RIT; failover");
processDeadServers(deadServers);
if (!nodes.isEmpty()) {
for (String encodedRegionName: nodes) {
processRegionInTransition(encodedRegionName, null);
}
}
} else {
// Fresh cluster startup.
cleanoutUnassigned();
assignAllUserRegions();
}
}
/**
* If region is up in zk in transition, then do fixup and block and wait until
@ -264,10 +303,10 @@ public class AssignmentManager extends ZooKeeperListener {
}
/**
* Process failover of <code>encodedName</code>. Look in
* Process failover of <code>servername</code>. Look in RIT.
* @param encodedRegionName Region to process failover for.
* @param encodedRegionName RegionInfo. If null we'll go get it from meta table.
* @return
* @param regionInfo If null we'll go get it from meta table.
* @return True if we processed <code>regionInfo</code> as a RIT.
* @throws KeeperException
* @throws IOException
*/
@ -278,7 +317,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (data == null) return false;
HRegionInfo hri = regionInfo;
if (hri == null) {
Pair<HRegionInfo, HServerAddress> p =
Pair<HRegionInfo, ServerName> p =
MetaReader.getRegion(catalogTracker, data.getRegionName());
if (p == null) return false;
hri = p.getFirst();
@ -327,17 +366,18 @@ public class AssignmentManager extends ZooKeeperListener {
// Region is opened, insert into RIT and handle it
regionsInTransition.put(encodedRegionName, new RegionState(
regionInfo, RegionState.State.OPENING, data.getStamp()));
HServerInfo hsi = serverManager.getServerInfo(data.getServerName());
ServerName sn =
data.getOrigin() == null? null: data.getOrigin();
// hsi could be null if this server is no longer online. If
// that the case, just let this RIT timeout; it'll be assigned
// to new server then.
if (hsi == null) {
if (sn == null) {
LOG.warn("Region in transition " + regionInfo.getEncodedName() +
" references a server no longer up " + data.getServerName() +
"; letting RIT timeout so will be assigned elsewhere");
" references a null server; letting RIT timeout so will be " +
"assigned elsewhere");
break;
}
new OpenedRegionHandler(master, this, regionInfo, hsi).process();
new OpenedRegionHandler(master, this, regionInfo, sn).process();
break;
}
}
@ -354,18 +394,19 @@ public class AssignmentManager extends ZooKeeperListener {
*/
private void handleRegion(final RegionTransitionData data) {
synchronized(regionsInTransition) {
if (data == null || data.getServerName() == null) {
if (data == null || data.getOrigin() == null) {
LOG.warn("Unexpected NULL input " + data);
return;
}
ServerName sn = data.getOrigin();
// Check if this is a special HBCK transition
if (data.getServerName().equals(HConstants.HBCK_CODE_NAME)) {
if (sn.equals(HConstants.HBCK_CODE_SERVERNAME)) {
handleHBCK(data);
return;
}
// Verify this is a known server
if (!serverManager.isServerOnline(data.getServerName()) &&
!this.master.getServerName().equals(data.getServerName())) {
if (!serverManager.isServerOnline(sn) &&
!this.master.getServerName().equals(sn)) {
LOG.warn("Attempted to handle region transition for server but " +
"server is not online: " + data.getRegionName());
return;
@ -387,7 +428,7 @@ public class AssignmentManager extends ZooKeeperListener {
case RS_ZK_REGION_SPLITTING:
if (!isInStateForSplitting(regionState)) break;
addSplittingToRIT(data.getServerName(), encodedName);
addSplittingToRIT(sn.toString(), encodedName);
break;
case RS_ZK_REGION_SPLIT:
@ -396,9 +437,9 @@ public class AssignmentManager extends ZooKeeperListener {
// If null, add SPLITTING state before going to SPLIT
if (regionState == null) {
LOG.info("Received SPLIT for region " + prettyPrintedRegionName +
" from server " + data.getServerName() +
" from server " + sn +
" but region was not first in SPLITTING state; continuing");
addSplittingToRIT(data.getServerName(), encodedName);
addSplittingToRIT(sn.toString(), encodedName);
}
// Check it has daughters.
byte [] payload = data.getPayload();
@ -412,14 +453,13 @@ public class AssignmentManager extends ZooKeeperListener {
}
assert daughters.size() == 2;
// Assert that we can get a serverinfo for this server.
HServerInfo hsi = getAndCheckHServerInfo(data.getServerName());
if (hsi == null) {
LOG.error("Dropped split! No HServerInfo for " + data.getServerName());
if (!this.serverManager.isServerOnline(sn)) {
LOG.error("Dropped split! ServerName=" + sn + " unknown.");
break;
}
// Run handler to do the rest of the SPLIT handling.
this.executorService.submit(new SplitRegionHandler(master, this,
regionState.getRegion(), hsi, daughters));
regionState.getRegion(), sn, daughters));
break;
case RS_ZK_REGION_CLOSING:
@ -428,7 +468,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (regionState == null ||
(!regionState.isPendingClose() && !regionState.isClosing())) {
LOG.warn("Received CLOSING for region " + prettyPrintedRegionName +
" from server " + data.getServerName() + " but region was in " +
" from server " + data.getOrigin() + " but region was in " +
" the state " + regionState + " and not " +
"in expected PENDING_CLOSE or CLOSING states");
return;
@ -442,7 +482,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (regionState == null ||
(!regionState.isPendingClose() && !regionState.isClosing())) {
LOG.warn("Received CLOSED for region " + prettyPrintedRegionName +
" from server " + data.getServerName() + " but region was in " +
" from server " + data.getOrigin() + " but region was in " +
" the state " + regionState + " and not " +
"in expected PENDING_CLOSE or CLOSING states");
return;
@ -462,7 +502,7 @@ public class AssignmentManager extends ZooKeeperListener {
(!regionState.isPendingOpen() && !regionState.isOpening())) {
LOG.warn("Received OPENING for region " +
prettyPrintedRegionName +
" from server " + data.getServerName() + " but region was in " +
" from server " + data.getOrigin() + " but region was in " +
" the state " + regionState + " and not " +
"in expected PENDING_OPEN or OPENING states");
return;
@ -477,7 +517,7 @@ public class AssignmentManager extends ZooKeeperListener {
(!regionState.isPendingOpen() && !regionState.isOpening())) {
LOG.warn("Received OPENED for region " +
prettyPrintedRegionName +
" from server " + data.getServerName() + " but region was in " +
" from server " + data.getOrigin() + " but region was in " +
" the state " + regionState + " and not " +
"in expected PENDING_OPEN or OPENING states");
return;
@ -486,7 +526,7 @@ public class AssignmentManager extends ZooKeeperListener {
regionState.update(RegionState.State.OPEN, data.getStamp());
this.executorService.submit(
new OpenedRegionHandler(master, this, regionState.getRegion(),
this.serverManager.getServerInfo(data.getServerName())));
data.getOrigin()));
break;
}
}
@ -524,12 +564,6 @@ public class AssignmentManager extends ZooKeeperListener {
return true;
}
private HServerInfo getAndCheckHServerInfo(final String serverName) {
HServerInfo hsi = this.serverManager.getServerInfo(serverName);
if (hsi == null) LOG.debug("No serverinfo for " + serverName);
return hsi;
}
/**
* @param serverName
* @param encodedName
@ -572,9 +606,9 @@ public class AssignmentManager extends ZooKeeperListener {
*/
private HRegionInfo findHRegionInfo(final String serverName,
final String encodedName) {
HServerInfo hsi = getAndCheckHServerInfo(serverName);
if (hsi == null) return null;
List<HRegionInfo> hris = this.servers.get(hsi);
ServerName sn = new ServerName(serverName);
if (!this.serverManager.isServerOnline(sn)) return null;
List<HRegionInfo> hris = this.servers.get(sn);
HRegionInfo foundHri = null;
for (HRegionInfo hri: hris) {
if (hri.getEncodedName().equals(encodedName)) {
@ -594,7 +628,7 @@ public class AssignmentManager extends ZooKeeperListener {
private void handleHBCK(RegionTransitionData data) {
String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
LOG.info("Handling HBCK triggered transition=" + data.getEventType() +
", server=" + data.getServerName() + ", region=" +
", server=" + data.getOrigin() + ", region=" +
HRegionInfo.prettyPrint(encodedName));
RegionState regionState = regionsInTransition.get(encodedName);
switch (data.getEventType()) {
@ -741,9 +775,9 @@ public class AssignmentManager extends ZooKeeperListener {
* <p>
* Used when a region has been successfully opened on a region server.
* @param regionInfo
* @param serverInfo
* @param sn
*/
public void regionOnline(HRegionInfo regionInfo, HServerInfo serverInfo) {
public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
synchronized (this.regionsInTransition) {
RegionState rs =
this.regionsInTransition.remove(regionInfo.getEncodedName());
@ -753,22 +787,22 @@ public class AssignmentManager extends ZooKeeperListener {
}
synchronized (this.regions) {
// Add check
HServerInfo hsi = this.regions.get(regionInfo);
if (hsi != null) LOG.warn("Overwriting " + regionInfo.getEncodedName() +
" on " + hsi);
this.regions.put(regionInfo, serverInfo);
addToServers(serverInfo, regionInfo);
ServerName oldSn = this.regions.get(regionInfo);
if (oldSn != null) LOG.warn("Overwriting " + regionInfo.getEncodedName() +
" on " + oldSn + " with " + sn);
this.regions.put(regionInfo, sn);
addToServers(sn, regionInfo);
this.regions.notifyAll();
}
// Remove plan if one.
clearRegionPlan(regionInfo);
// Update timers for all regions in transition going against this server.
updateTimers(serverInfo);
updateTimers(sn);
}
/**
* Touch timers for all regions in transition that have the passed
* <code>hsi</code> in common.
* <code>sn</code> in common.
* Call this method whenever a server checks in. Doing so helps the case where
* a new regionserver has joined the cluster and its been given 1k regions to
* open. If this method is tickled every time the region reports in a
@ -777,9 +811,9 @@ public class AssignmentManager extends ZooKeeperListener {
* as part of bulk assign -- there we have a different mechanism for extending
* the regions in transition timer (we turn it off temporarily -- because
* there is no regionplan involved when bulk assigning.
* @param hsi
* @param sn
*/
private void updateTimers(final HServerInfo hsi) {
private void updateTimers(final ServerName sn) {
// This loop could be expensive.
// First make a copy of current regionPlan rather than hold sync while
// looping because holding sync can cause deadlock. Its ok in this loop
@ -789,7 +823,7 @@ public class AssignmentManager extends ZooKeeperListener {
copy.putAll(this.regionPlans);
}
for (Map.Entry<String, RegionPlan> e: copy.entrySet()) {
if (!e.getValue().getDestination().equals(hsi)) continue;
if (!e.getValue().getDestination().equals(sn)) continue;
RegionState rs = null;
synchronized (this.regionsInTransition) {
rs = this.regionsInTransition.get(e.getKey());
@ -828,11 +862,11 @@ public class AssignmentManager extends ZooKeeperListener {
*/
public void setOffline(HRegionInfo regionInfo) {
synchronized (this.regions) {
HServerInfo serverInfo = this.regions.remove(regionInfo);
if (serverInfo == null) return;
List<HRegionInfo> serverRegions = this.servers.get(serverInfo);
ServerName sn = this.regions.remove(regionInfo);
if (sn == null) return;
List<HRegionInfo> serverRegions = this.servers.get(sn);
if (!serverRegions.remove(regionInfo)) {
LOG.warn("No " + regionInfo + " on " + serverInfo);
LOG.warn("No " + regionInfo + " on " + sn);
}
}
}
@ -906,10 +940,10 @@ public class AssignmentManager extends ZooKeeperListener {
* @param destination
* @param regions Regions to assign.
*/
void assign(final HServerInfo destination,
void assign(final ServerName destination,
final List<HRegionInfo> regions) {
LOG.debug("Bulk assigning " + regions.size() + " region(s) to " +
destination.getServerName());
destination.toString());
List<RegionState> states = new ArrayList<RegionState>(regions.size());
synchronized (this.regionsInTransition) {
@ -932,14 +966,19 @@ public class AssignmentManager extends ZooKeeperListener {
for (int oldCounter = 0; true;) {
int count = counter.get();
if (oldCounter != count) {
LOG.info(destination.getServerName() + " unassigned znodes=" + count +
LOG.info(destination.toString() + " unassigned znodes=" + count +
" of total=" + total);
oldCounter = count;
}
if (count == total) break;
Threads.sleep(1);
}
// Move on to open regions.
try {
// Send OPEN RPC. This can fail if the server on other end is is not up.
// If we fail, fail the startup by aborting the server. There is one
// exception we will tolerate: ServerNotRunningException. This is thrown
// between report of regionserver being up and
long maxWaitTime = System.currentTimeMillis() +
this.master.getConfiguration().
getLong("hbase.regionserver.rpc.startup.waittime", 60000);
@ -962,7 +1001,7 @@ public class AssignmentManager extends ZooKeeperListener {
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
LOG.debug("Bulk assigning done for " + destination.getServerName());
LOG.debug("Bulk assigning done for " + destination.toString());
}
/**
@ -971,11 +1010,11 @@ public class AssignmentManager extends ZooKeeperListener {
static class CreateUnassignedAsyncCallback implements AsyncCallback.StringCallback {
private final Log LOG = LogFactory.getLog(CreateUnassignedAsyncCallback.class);
private final ZooKeeperWatcher zkw;
private final HServerInfo destination;
private final ServerName destination;
private final AtomicInteger counter;
CreateUnassignedAsyncCallback(final ZooKeeperWatcher zkw,
final HServerInfo destination, final AtomicInteger counter) {
final ServerName destination, final AtomicInteger counter) {
this.zkw = zkw;
this.destination = destination;
this.counter = counter;
@ -991,7 +1030,7 @@ public class AssignmentManager extends ZooKeeperListener {
", rc=" + rc, null);
return;
}
LOG.debug("rs=" + (RegionState)ctx + ", server=" + this.destination.getServerName());
LOG.debug("rs=" + (RegionState)ctx + ", server=" + this.destination.toString());
// Async exists to set a watcher so we'll get triggered when
// unassigned node changes.
this.zkw.getZooKeeper().exists(path, this.zkw,
@ -1078,7 +1117,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (plan == null) return; // Should get reassigned later when RIT times out.
try {
LOG.debug("Assigning region " + state.getRegion().getRegionNameAsString() +
" to " + plan.getDestination().getServerName());
" to " + plan.getDestination().toString());
// Transition RegionState to PENDING_OPEN
state.update(RegionState.State.PENDING_OPEN);
// Send OPEN RPC. This can fail if the server on other end is is not up.
@ -1118,7 +1157,7 @@ public class AssignmentManager extends ZooKeeperListener {
state.update(RegionState.State.OFFLINE);
try {
if(!ZKAssign.createOrForceNodeOffline(master.getZooKeeper(),
state.getRegion(), master.getServerName())) {
state.getRegion(), this.master.getServerName())) {
LOG.warn("Attempted to create/force node into OFFLINE state before " +
"completing assignment but failed to do so for " + state);
return false;
@ -1147,7 +1186,7 @@ public class AssignmentManager extends ZooKeeperListener {
state.update(RegionState.State.OFFLINE);
try {
ZKAssign.asyncCreateNodeOffline(master.getZooKeeper(), state.getRegion(),
master.getServerName(), cb, ctx);
this.master.getServerName(), cb, ctx);
} catch (KeeperException e) {
master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
return false;
@ -1175,10 +1214,10 @@ public class AssignmentManager extends ZooKeeperListener {
* if no servers to assign, it returns null).
*/
RegionPlan getRegionPlan(final RegionState state,
final HServerInfo serverToExclude, final boolean forceNewPlan) {
final ServerName serverToExclude, final boolean forceNewPlan) {
// Pickup existing plan or make a new one
String encodedName = state.getRegion().getEncodedName();
List<HServerInfo> servers = this.serverManager.getOnlineServersList();
List<ServerName> servers = this.serverManager.getOnlineServersList();
// The remove below hinges on the fact that the call to
// serverManager.getOnlineServersList() returns a copy
if (serverToExclude != null) servers.remove(serverToExclude);
@ -1266,7 +1305,7 @@ public class AssignmentManager extends ZooKeeperListener {
}
}
// Send CLOSE RPC
HServerInfo server = null;
ServerName server = null;
synchronized (this.regions) {
server = regions.get(region);
}
@ -1347,6 +1386,29 @@ public class AssignmentManager extends ZooKeeperListener {
* Assigns all user regions, if any. Used during cluster startup.
* <p>
* This is a synchronous call and will return once every region has been
* assigned. If anything fails, an exception is thrown
* @throws InterruptedException
* @throws IOException
*/
public void assignUserRegions(List<HRegionInfo> regions, List<ServerName> servers)
throws IOException, InterruptedException {
if (regions == null)
return;
Map<ServerName, List<HRegionInfo>> bulkPlan = null;
// Generate a round-robin bulk assignment plan
bulkPlan = LoadBalancer.roundRobinAssignment(regions, servers);
LOG.info("Bulk assigning " + regions.size() + " region(s) round-robin across " +
servers.size() + " server(s)");
// Use fixed count thread pool assigning.
BulkAssigner ba = new StartupBulkAssigner(this.master, bulkPlan, this);
ba.bulkAssign();
LOG.info("Bulk assigning done");
}
/**
* Assigns all user regions, if any exist. Used during cluster startup.
* <p>
* 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.
* @throws InterruptedException
@ -1354,10 +1416,10 @@ public class AssignmentManager extends ZooKeeperListener {
*/
public void assignAllUserRegions() throws IOException, InterruptedException {
// Get all available servers
List<HServerInfo> servers = serverManager.getOnlineServersList();
List<ServerName> servers = serverManager.getOnlineServersList();
// Scan META for all user regions, skipping any disabled tables
Map<HRegionInfo,HServerAddress> allRegions =
Map<HRegionInfo, ServerName> allRegions =
MetaReader.fullScan(catalogTracker, this.zkTable.getDisabledTables(), true);
if (allRegions == null || allRegions.isEmpty()) return;
@ -1365,15 +1427,14 @@ public class AssignmentManager extends ZooKeeperListener {
boolean retainAssignment = master.getConfiguration().
getBoolean("hbase.master.startup.retainassign", true);
Map<HServerInfo, List<HRegionInfo>> bulkPlan = null;
Map<ServerName, List<HRegionInfo>> bulkPlan = null;
if (retainAssignment) {
// Reuse existing assignment info
bulkPlan = LoadBalancer.retainAssignment(allRegions, servers);
} else {
// assign regions in round-robin fashion
HRegionInfo [] regions =
allRegions.keySet().toArray(new HRegionInfo[allRegions.size()]);
bulkPlan = LoadBalancer.roundRobinAssignment(regions, servers);
assignUserRegions(new ArrayList<HRegionInfo>(allRegions.keySet()), servers);
return;
}
LOG.info("Bulk assigning " + allRegions.size() + " region(s) across " +
servers.size() + " server(s), retainAssignment=" + retainAssignment);
@ -1391,11 +1452,11 @@ public class AssignmentManager extends ZooKeeperListener {
* which will abort the Server if exception.
*/
static class StartupBulkAssigner extends BulkAssigner {
final Map<HServerInfo, List<HRegionInfo>> bulkPlan;
final Map<ServerName, List<HRegionInfo>> bulkPlan;
final AssignmentManager assignmentManager;
StartupBulkAssigner(final Server server,
final Map<HServerInfo, List<HRegionInfo>> bulkPlan,
final Map<ServerName, List<HRegionInfo>> bulkPlan,
final AssignmentManager am) {
super(server);
this.bulkPlan = bulkPlan;
@ -1421,9 +1482,9 @@ public class AssignmentManager extends ZooKeeperListener {
@Override
protected void populatePool(java.util.concurrent.ExecutorService pool) {
for (Map.Entry<HServerInfo, List<HRegionInfo>> e: this.bulkPlan.entrySet()) {
for (Map.Entry<ServerName, List<HRegionInfo>> e: this.bulkPlan.entrySet()) {
pool.execute(new SingleServerBulkAssigner(e.getKey(), e.getValue(),
this.assignmentManager, true));
this.assignmentManager));
}
}
@ -1456,7 +1517,7 @@ public class AssignmentManager extends ZooKeeperListener {
*/
static class GeneralBulkAssigner extends StartupBulkAssigner {
GeneralBulkAssigner(final Server server,
final Map<HServerInfo, List<HRegionInfo>> bulkPlan,
final Map<ServerName, List<HRegionInfo>> bulkPlan,
final AssignmentManager am) {
super(server, bulkPlan, am);
}
@ -1476,13 +1537,12 @@ public class AssignmentManager extends ZooKeeperListener {
* Manage bulk assigning to a server.
*/
static class SingleServerBulkAssigner implements Runnable {
private final HServerInfo regionserver;
private final ServerName regionserver;
private final List<HRegionInfo> regions;
private final AssignmentManager assignmentManager;
SingleServerBulkAssigner(final HServerInfo regionserver,
final List<HRegionInfo> regions, final AssignmentManager am,
final boolean startUp) {
SingleServerBulkAssigner(final ServerName regionserver,
final List<HRegionInfo> regions, final AssignmentManager am) {
this.regionserver = regionserver;
this.regions = regions;
this.assignmentManager = am;
@ -1562,26 +1622,24 @@ public class AssignmentManager extends ZooKeeperListener {
* in META
* @throws IOException
*/
private Map<HServerInfo,List<Pair<HRegionInfo,Result>>> rebuildUserRegions()
Map<ServerName, List<Pair<HRegionInfo, Result>>> rebuildUserRegions()
throws IOException {
// Region assignment from META
List<Result> results = MetaReader.fullScanOfResults(catalogTracker);
List<Result> results = MetaReader.fullScanOfResults(this.catalogTracker);
// Map of offline servers and their regions to be returned
Map<HServerInfo,List<Pair<HRegionInfo,Result>>> offlineServers =
new TreeMap<HServerInfo,List<Pair<HRegionInfo,Result>>>();
Map<ServerName, List<Pair<HRegionInfo,Result>>> offlineServers =
new TreeMap<ServerName, List<Pair<HRegionInfo, Result>>>();
// Iterate regions in META
for (Result result : results) {
Pair<HRegionInfo,HServerInfo> region =
MetaReader.metaRowToRegionPairWithInfo(result);
Pair<HRegionInfo, ServerName> region = MetaReader.metaRowToRegionPair(result);
if (region == null) continue;
HServerInfo regionLocation = region.getSecond();
HRegionInfo regionInfo = region.getFirst();
ServerName regionLocation = region.getSecond();
if (regionLocation == null) {
// Region not being served, add to region map with no assignment
// If this needs to be assigned out, it will also be in ZK as RIT
this.regions.put(regionInfo, null);
} else if (!serverManager.isServerOnline(
regionLocation.getServerName())) {
} else if (!this.serverManager.isServerOnline(regionLocation)) {
// Region is located on a server that isn't online
List<Pair<HRegionInfo, Result>> offlineRegions =
offlineServers.get(regionLocation);
@ -1592,7 +1650,7 @@ public class AssignmentManager extends ZooKeeperListener {
offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
} else {
// Region is being served and on an active server
regions.put(regionInfo, regionLocation);
this.regions.put(regionInfo, regionLocation);
addToServers(regionLocation, regionInfo);
}
}
@ -1613,9 +1671,9 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws KeeperException
*/
private void processDeadServers(
Map<HServerInfo, List<Pair<HRegionInfo, Result>>> deadServers)
Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers)
throws IOException, KeeperException {
for (Map.Entry<HServerInfo, List<Pair<HRegionInfo,Result>>> deadServer :
for (Map.Entry<ServerName, List<Pair<HRegionInfo,Result>>> deadServer:
deadServers.entrySet()) {
List<Pair<HRegionInfo,Result>> regions = deadServer.getValue();
for (Pair<HRegionInfo,Result> region : regions) {
@ -1624,7 +1682,7 @@ public class AssignmentManager extends ZooKeeperListener {
// If region was in transition (was in zk) force it offline for reassign
try {
ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
master.getServerName());
this.master.getServerName());
} catch (KeeperException.NoNodeException nne) {
// This is fine
}
@ -1640,11 +1698,11 @@ public class AssignmentManager extends ZooKeeperListener {
* @param hsi
* @param hri
*/
private void addToServers(final HServerInfo hsi, final HRegionInfo hri) {
List<HRegionInfo> hris = servers.get(hsi);
private void addToServers(final ServerName sn, final HRegionInfo hri) {
List<HRegionInfo> hris = servers.get(sn);
if (hris == null) {
hris = new ArrayList<HRegionInfo>();
servers.put(hsi, hris);
servers.put(sn, hris);
}
hris.add(hri);
}
@ -1922,16 +1980,16 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Process shutdown server removing any assignments.
* @param hsi Server that went down.
* @param sn Server that went down.
* @return list of regions in transition on this server
*/
public List<RegionState> processServerShutdown(final HServerInfo hsi) {
public List<RegionState> processServerShutdown(final ServerName sn) {
// Clean out any existing assignment plans for this server
synchronized (this.regionPlans) {
for (Iterator <Map.Entry<String, RegionPlan>> i =
this.regionPlans.entrySet().iterator(); i.hasNext();) {
Map.Entry<String, RegionPlan> e = i.next();
if (e.getValue().getDestination().equals(hsi)) {
if (e.getValue().getDestination().equals(sn)) {
// Use iterator's remove else we'll get CME
i.remove();
}
@ -1943,7 +2001,7 @@ public class AssignmentManager extends ZooKeeperListener {
Set<HRegionInfo> deadRegions = null;
List<RegionState> rits = new ArrayList<RegionState>();
synchronized (this.regions) {
List<HRegionInfo> assignedRegions = this.servers.remove(hsi);
List<HRegionInfo> assignedRegions = this.servers.remove(sn);
if (assignedRegions == null || assignedRegions.isEmpty()) {
// No regions on this server, we are done, return empty list of RITs
return rits;
@ -1968,16 +2026,16 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* Update inmemory structures.
* @param hsi Server that reported the split
* @param sn Server that reported the split
* @param parent Parent region that was split
* @param a Daughter region A
* @param b Daughter region B
*/
public void handleSplitReport(final HServerInfo hsi, final HRegionInfo parent,
public void handleSplitReport(final ServerName sn, final HRegionInfo parent,
final HRegionInfo a, final HRegionInfo b) {
regionOffline(parent);
regionOnline(a, hsi);
regionOnline(b, hsi);
regionOnline(a, sn);
regionOnline(b, sn);
// There's a possibility that the region was splitting while a user asked
// the master to disable, we need to make sure we close those regions in
@ -1995,21 +2053,16 @@ public class AssignmentManager extends ZooKeeperListener {
* If a new server has come in and it has no regions, it will not be included
* in the returned Map.
*/
Map<HServerInfo, List<HRegionInfo>> getAssignments() {
Map<ServerName, List<HRegionInfo>> getAssignments() {
// This is an EXPENSIVE clone. Cloning though is the safest thing to do.
// Can't let out original since it can change and at least the loadbalancer
// wants to iterate this exported list. We need to synchronize on regions
// since all access to this.servers is under a lock on this.regions.
Map<HServerInfo, List<HRegionInfo>> result = null;
Map<ServerName, List<HRegionInfo>> result = null;
synchronized (this.regions) {
result = new HashMap<HServerInfo, List<HRegionInfo>>(this.servers.size());
for (Map.Entry<HServerInfo, List<HRegionInfo>> e: this.servers.entrySet()) {
List<HRegionInfo> shallowCopy = new ArrayList<HRegionInfo>(e.getValue());
HServerInfo clone = new HServerInfo(e.getKey());
// Set into server load the number of regions this server is carrying
// The load balancer calculation needs it at least and its handy.
clone.getLoad().setNumberOfRegions(e.getValue().size());
result.put(clone, shallowCopy);
result = new HashMap<ServerName, List<HRegionInfo>>(this.servers.size());
for (Map.Entry<ServerName, List<HRegionInfo>> e: this.servers.entrySet()) {
result.put(e.getKey(), new ArrayList<HRegionInfo>(e.getValue()));
}
}
return result;
@ -2018,14 +2071,14 @@ public class AssignmentManager extends ZooKeeperListener {
/**
* @param encodedRegionName Region encoded name.
* @return Null or a {@link Pair} instance that holds the full {@link HRegionInfo}
* and the hosting servers {@link HServerInfo}.
* and the hosting servers {@link ServerName}.
*/
Pair<HRegionInfo, HServerInfo> getAssignment(final byte [] encodedRegionName) {
Pair<HRegionInfo, ServerName> getAssignment(final byte [] encodedRegionName) {
String name = Bytes.toString(encodedRegionName);
synchronized(this.regions) {
for (Map.Entry<HRegionInfo, HServerInfo> e: this.regions.entrySet()) {
for (Map.Entry<HRegionInfo, ServerName> e: this.regions.entrySet()) {
if (e.getKey().getEncodedName().equals(name)) {
return new Pair<HRegionInfo, HServerInfo>(e.getKey(), e.getValue());
return new Pair<HRegionInfo, ServerName>(e.getKey(), e.getValue());
}
}
}
@ -2042,29 +2095,13 @@ public class AssignmentManager extends ZooKeeperListener {
unassign(plan.getRegionInfo());
}
/**
* @param hsi
* @return True if this server is carrying a catalog region, a region from
* -ROOT- or .META. table.
*/
boolean isMetaRegionServer(final HServerInfo hsi) {
synchronized (this.regions) {
List<HRegionInfo> regions = this.servers.get(hsi);
if (regions == null || regions.isEmpty()) return false;
for (HRegionInfo hri: regions) {
if (hri.isMetaRegion()) return true;
}
}
return false;
}
/**
* Run through remaining regionservers and unassign all catalog regions.
*/
void unassignCatalogRegions() {
this.servers.entrySet();
synchronized (this.regions) {
for (Map.Entry<HServerInfo, List<HRegionInfo>> e: this.servers.entrySet()) {
for (Map.Entry<ServerName, List<HRegionInfo>> e: this.servers.entrySet()) {
List<HRegionInfo> regions = e.getValue();
if (regions == null || regions.isEmpty()) continue;
for (HRegionInfo hri: regions) {
@ -2084,10 +2121,10 @@ public class AssignmentManager extends ZooKeeperListener {
* @throws IOException
*/
void bulkAssignUserRegions(final HRegionInfo [] regions,
final List<HServerInfo> servers, final boolean sync)
final List<ServerName> servers, final boolean sync)
throws IOException {
Map<HServerInfo, List<HRegionInfo>> bulkPlan =
LoadBalancer.roundRobinAssignment(regions, servers);
Map<ServerName, List<HRegionInfo>> bulkPlan =
LoadBalancer.roundRobinAssignment(Arrays.asList(regions), servers);
LOG.info("Bulk assigning " + regions.length + " region(s) " +
"round-robin across " + servers.size() + " server(s)");
// Use fixed count thread pool assigning.

View File

@ -22,17 +22,15 @@ package org.apache.hadoop.hbase.master;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.ServerName;
/**
* Class to hold dead servers list and utility querying dead server list.
*/
public class DeadServer implements Set<String> {
public class DeadServer implements Set<ServerName> {
/**
* 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
@ -40,26 +38,22 @@ public class DeadServer implements Set<String> {
* and it's server logs are recovered, it will be told to call server startup
* because by then, its regions have probably been reassigned.
*/
private final Set<String> deadServers = new HashSet<String>();
/** Maximum number of dead servers to keep track of */
private final int maxDeadServers;
private final Set<ServerName> deadServers = new HashSet<ServerName>();
/** Number of dead servers currently being processed */
private int numProcessing;
public DeadServer(int maxDeadServers) {
public DeadServer() {
super();
this.maxDeadServers = maxDeadServers;
this.numProcessing = 0;
}
/**
* @param serverName
* @param serverName Server name
* @return true if server is dead
*/
public boolean isDeadServer(final String serverName) {
return isDeadServer(serverName, false);
return isDeadServer(new ServerName(serverName));
}
/**
@ -68,31 +62,27 @@ public class DeadServer implements Set<String> {
* <code>host,port,startcode</code>.
* @return true if this server was dead before and coming back alive again
*/
public boolean cleanPreviousInstance(final String newServerName) {
String serverAddress =
HServerInfo.getServerNameLessStartCode(newServerName);
for (String serverName: deadServers) {
String deadServerAddress =
HServerInfo.getServerNameLessStartCode(serverName);
if (deadServerAddress.equals(serverAddress)) {
remove(serverName);
return true;
}
}
return false;
public boolean cleanPreviousInstance(final ServerName newServerName) {
ServerName sn =
ServerName.findServerWithSameHostnamePort(this.deadServers, newServerName);
if (sn == null) return false;
return this.deadServers.remove(sn);
}
/**
* @param serverName Servername as either <code>host:port</code> or
* <code>host,port,startcode</code>.
* @param hostAndPortOnly True if <code>serverName</code> is host and
* port only (<code>host:port</code>) and if so, then we do a prefix compare
* (ignoring start codes) looking for dead server.
* @return true if server is dead
* @param serverName
* @return true if this server is on the dead servers list.
*/
boolean isDeadServer(final String serverName, final boolean hostAndPortOnly) {
return HServerInfo.isServer(this, serverName, hostAndPortOnly);
boolean isDeadServer(final ServerName serverName) {
return this.deadServers.contains(serverName);
}
/**
* @return True if we have a server with matching hostname and port.
*/
boolean isDeadServerWithSameHostnamePort(final ServerName serverName) {
return ServerName.findServerWithSameHostnamePort(this.deadServers,
serverName) != null;
}
/**
@ -105,18 +95,18 @@ public class DeadServer implements Set<String> {
return numProcessing != 0;
}
public synchronized Set<String> clone() {
Set<String> clone = new HashSet<String>(this.deadServers.size());
public synchronized Set<ServerName> clone() {
Set<ServerName> clone = new HashSet<ServerName>(this.deadServers.size());
clone.addAll(this.deadServers);
return clone;
}
public synchronized boolean add(String e) {
public synchronized boolean add(ServerName e) {
this.numProcessing++;
return deadServers.add(e);
}
public synchronized void finish(String e) {
public synchronized void finish(ServerName e) {
this.numProcessing--;
}
@ -132,7 +122,7 @@ public class DeadServer implements Set<String> {
return deadServers.contains(o);
}
public Iterator<String> iterator() {
public Iterator<ServerName> iterator() {
return this.deadServers.iterator();
}
@ -152,7 +142,7 @@ public class DeadServer implements Set<String> {
return deadServers.containsAll(c);
}
public synchronized boolean addAll(Collection<? extends String> c) {
public synchronized boolean addAll(Collection<? extends ServerName> c) {
return deadServers.addAll(c);
}

View File

@ -23,8 +23,8 @@ import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
@ -36,14 +36,13 @@ import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HColumnDescriptor;
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.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
@ -54,8 +53,8 @@ import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
@ -131,8 +130,12 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
// RPC server for the HMaster
private final RpcServer rpcServer;
// Address of the HMaster
private final HServerAddress address;
/**
* This servers address.
*/
private final InetSocketAddress isa;
// Metrics for the HMaster
private final MasterMetrics metrics;
// file system manager for the master FS operations
@ -172,6 +175,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
private LogCleaner logCleaner;
private MasterCoprocessorHost cpHost;
private final ServerName serverName;
/**
* Initializes the HMaster. The steps are as follows:
@ -189,43 +193,48 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
throws IOException, KeeperException, InterruptedException {
this.conf = conf;
/*
* Determine address and initialize RPC server (but do not start).
* The RPC server ports can be ephemeral. Create a ZKW instance.
*/
HServerAddress a = new HServerAddress(getMyAddress(this.conf));
int numHandlers = conf.getInt("hbase.regionserver.handler.count", 10);
// Server to handle client requests.
String hostname = DNS.getDefaultHost(
conf.get("hbase.master.dns.interface", "default"),
conf.get("hbase.master.dns.nameserver", "default"));
int port = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT);
// Creation of a HSA will force a resolve.
InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
if (initialIsa.getAddress() == null) {
throw new IllegalArgumentException("Failed resolve of " + this.isa);
}
int numHandlers = conf.getInt("hbase.master.handler.count",
conf.getInt("hbase.regionserver.handler.count", 25));
this.rpcServer = HBaseRPC.getServer(this,
new Class<?>[]{HMasterInterface.class, HMasterRegionInterface.class},
a.getBindAddress(), a.getPort(),
initialIsa.getHostName(), // BindAddress is IP we got for this server.
initialIsa.getPort(),
numHandlers,
0, // we dont use high priority handlers in master
false, conf,
conf.getBoolean("hbase.rpc.verbose", false), conf,
0); // this is a DNC w/o high priority handlers
this.address = new HServerAddress(rpcServer.getListenerAddress());
// Set our address.
this.isa = this.rpcServer.getListenerAddress();
this.serverName = new ServerName(this.isa.getHostName(),
this.isa.getPort(), System.currentTimeMillis());
// initialize server principal (if using secure Hadoop)
User.login(conf, "hbase.master.keytab.file",
"hbase.master.kerberos.principal", this.address.getHostname());
"hbase.master.kerberos.principal", this.isa.getHostName());
// set the thread name now we have an address
setName(MASTER + "-" + this.address);
setName(MASTER + "-" + this.serverName.toString());
Replication.decorateMasterConfiguration(this.conf);
this.rpcServer.startThreads();
// Hack! Maps DFSClient => Master for logs. HDFS made this
// config param for task trackers, but we can piggyback off of it.
if (this.conf.get("mapred.task.id") == null) {
this.conf.set("mapred.task.id", "hb_m_" + this.address.toString() +
"_" + System.currentTimeMillis());
this.conf.set("mapred.task.id", "hb_m_" + this.serverName.toString());
}
this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" +
address.getPort(), this);
this.metrics = new MasterMetrics(getServerName());
this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" + isa.getPort(), this);
this.metrics = new MasterMetrics(getServerName().toString());
}
/**
@ -397,9 +406,9 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
fileSystemManager.getClusterId());
this.connection = HConnectionManager.getConnection(conf);
this.executorService = new ExecutorService(getServerName());
this.executorService = new ExecutorService(getServerName().toString());
this.serverManager = new ServerManager(this, this, metrics);
this.serverManager = new ServerManager(this, this);
initializeZKBasedSystemTrackers();
@ -409,29 +418,25 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
// start up all service threads.
startServiceThreads();
// Wait for region servers to report in. Returns count of regions.
int regionCount = this.serverManager.waitForRegionServers();
// Wait for region servers to report in.
this.serverManager.waitForRegionServers();
// Check zk for regionservers that are up but didn't register
for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
if (!this.serverManager.isServerOnline(sn)) {
// Not registered; add it.
LOG.info("Registering server found up in zk: " + sn);
this.serverManager.recordNewServer(sn, HServerLoad.EMPTY_HSERVERLOAD);
}
}
// TODO: Should do this in background rather than block master startup
this.fileSystemManager.
splitLogAfterStartup(this.serverManager.getOnlineServers());
splitLogAfterStartup(this.serverManager.getOnlineServers().keySet());
// Make sure root and meta assigned before proceeding.
assignRootAndMeta();
// Is this fresh start with no regions assigned or are we a master joining
// an already-running cluster? If regionsCount == 0, then for sure a
// fresh start. TOOD: Be fancier. If regionsCount == 2, perhaps the
// 2 are .META. and -ROOT- and we should fall into the fresh startup
// branch below. For now, do processFailover.
if (regionCount == 0) {
LOG.info("Master startup proceeding: cluster startup");
this.assignmentManager.cleanoutUnassigned();
this.assignmentManager.assignAllUserRegions();
} else {
LOG.info("Master startup proceeding: master failover");
this.assignmentManager.processFailover();
}
// Fixup assignment manager status
this.assignmentManager.joinCluster();
// Start balancer and meta catalog janitor after meta and regions have
// been assigned.
@ -466,7 +471,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
} else {
// Region already assigned. We didnt' assign it. Add to in-memory state.
this.assignmentManager.regionOnline(HRegionInfo.ROOT_REGIONINFO,
this.serverManager.getHServerInfo(this.catalogTracker.getRootLocation()));
this.catalogTracker.getRootLocation());
}
LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getRootLocation());
@ -484,32 +489,13 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
} else {
// Region already assigned. We didnt' assign it. Add to in-memory state.
this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
this.serverManager.getHServerInfo(this.catalogTracker.getMetaLocation()));
this.catalogTracker.getMetaLocation());
}
LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
return assigned;
}
/*
* @return This masters' address.
* @throws UnknownHostException
*/
private static String getMyAddress(final Configuration c)
throws UnknownHostException {
// Find out our address up in DNS.
String s = DNS.getDefaultHost(c.get("hbase.master.dns.interface","default"),
c.get("hbase.master.dns.nameserver","default"));
s += ":" + c.get(HConstants.MASTER_PORT,
Integer.toString(HConstants.DEFAULT_MASTER_PORT));
return s;
}
/** @return HServerAddress of the master server */
public HServerAddress getMasterAddress() {
return this.address;
}
public long getProtocolVersion(String protocol, long clientVersion) {
if (HMasterInterface.class.getName().equals(protocol)) {
return HMasterInterface.VERSION;
@ -645,25 +631,16 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
@Override
public MapWritable regionServerStartup(final HServerInfo serverInfo,
final long serverCurrentTime)
public MapWritable regionServerStartup(final int port,
final long serverStartCode, final long serverCurrentTime)
throws IOException {
// 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"
// Everafter, the HSI combination 'server name' is what uniquely identifies
// the incoming RegionServer.
InetSocketAddress address = new InetSocketAddress(
HBaseServer.getRemoteIp().getHostName(),
serverInfo.getServerAddress().getPort());
serverInfo.setServerAddress(new HServerAddress(address));
// Register with server manager
this.serverManager.regionServerStartup(serverInfo, serverCurrentTime);
this.serverManager.regionServerStartup(HBaseServer.getRemoteIp(), port,
serverStartCode, serverCurrentTime);
// Send back some config info
MapWritable mw = createConfigurationSubset();
mw.put(new Text("hbase.regionserver.address"),
serverInfo.getServerAddress());
mw.put(new Text(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER),
new Text(this.serverName.getHostname()));
return mw;
}
@ -682,23 +659,13 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
@Override
public HMsg [] regionServerReport(HServerInfo serverInfo, HMsg msgs[],
HRegionInfo[] mostLoadedRegions)
public void regionServerReport(byte[] sn, HServerLoad hsl)
throws IOException {
return adornRegionServerAnswer(serverInfo,
this.serverManager.regionServerReport(serverInfo, msgs, mostLoadedRegions));
this.serverManager.regionServerReport(new ServerName(sn), hsl);
if (hsl != null && this.metrics != null) {
// Up our metrics.
this.metrics.incrementRequests(hsl.getNumberOfRequests());
}
/**
* Override if you'd add messages to return to regionserver <code>hsi</code>
* or to send an exception.
* @param msgs Messages to add to
* @return Messages to return to
* @throws IOException exceptions that were injected for the region servers
*/
protected HMsg [] adornRegionServerAnswer(final HServerInfo hsi,
final HMsg [] msgs) throws IOException {
return msgs;
}
public boolean isMasterRunning() {
@ -758,14 +725,13 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
}
Map<HServerInfo, List<HRegionInfo>> assignments =
Map<ServerName, List<HRegionInfo>> assignments =
this.assignmentManager.getAssignments();
// Returned Map from AM does not include mention of servers w/o assignments.
for (Map.Entry<String, HServerInfo> e:
for (Map.Entry<ServerName, HServerLoad> e:
this.serverManager.getOnlineServers().entrySet()) {
HServerInfo hsi = e.getValue();
if (!assignments.containsKey(hsi)) {
assignments.put(hsi, new ArrayList<HRegionInfo>());
if (!assignments.containsKey(e.getKey())) {
assignments.put(e.getKey(), new ArrayList<HRegionInfo>());
}
}
List<RegionPlan> plans = this.balancer.balanceCluster(assignments);
@ -832,12 +798,12 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
@Override
public void move(final byte[] encodedRegionName, final byte[] destServerName)
throws UnknownRegionException {
Pair<HRegionInfo, HServerInfo> p =
Pair<HRegionInfo, ServerName> p =
this.assignmentManager.getAssignment(encodedRegionName);
if (p == null)
throw new UnknownRegionException(Bytes.toString(encodedRegionName));
HRegionInfo hri = p.getFirst();
HServerInfo dest = null;
ServerName dest = null;
if (destServerName == null || destServerName.length == 0) {
LOG.info("Passed destination servername is null/empty so " +
"choosing a server at random");
@ -845,12 +811,12 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
// Unassign will reassign it elsewhere choosing random server.
this.assignmentManager.unassign(hri);
} else {
dest = this.serverManager.getServerInfo(new String(destServerName));
dest = new ServerName(Bytes.toString(destServerName));
if (this.cpHost != null) {
this.cpHost.preMove(p.getFirst(), p.getSecond(), dest);
}
RegionPlan rp = new RegionPlan(p.getFirst(), p.getSecond(), dest);
LOG.info("Added move plan " + rp + ", running balancer");
this.assignmentManager.balance(rp);
if (this.cpHost != null) {
this.cpHost.postMove(p.getFirst(), p.getSecond(), dest);
@ -928,8 +894,13 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
// 5. Trigger immediate assignment of the regions in round-robin fashion
List<HServerInfo> servers = serverManager.getOnlineServersList();
this.assignmentManager.bulkAssignUserRegions(newRegions, servers, sync);
List<ServerName> servers = serverManager.getOnlineServersList();
try {
this.assignmentManager.assignUserRegions(Arrays.asList(newRegions), servers);
} catch (InterruptedException ie) {
LOG.error("Caught " + ie + " during round-robin assignment");
throw new IOException(ie);
}
// 6. If sync, wait for assignment of regions
if (sync) {
@ -1027,11 +998,11 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
* is found, but not currently deployed, the second element of the pair
* may be null.
*/
Pair<HRegionInfo,HServerAddress> getTableRegionForRow(
Pair<HRegionInfo, ServerName> getTableRegionForRow(
final byte [] tableName, final byte [] rowKey)
throws IOException {
final AtomicReference<Pair<HRegionInfo, HServerAddress>> result =
new AtomicReference<Pair<HRegionInfo, HServerAddress>>(null);
final AtomicReference<Pair<HRegionInfo, ServerName>> result =
new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
MetaScannerVisitor visitor =
new MetaScannerVisitor() {
@ -1040,13 +1011,11 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
if (data == null || data.size() <= 0) {
return true;
}
Pair<HRegionInfo, HServerAddress> pair =
MetaReader.metaRowToRegionPair(data);
Pair<HRegionInfo, ServerName> pair = MetaReader.metaRowToRegionPair(data);
if (pair == null) {
return false;
}
if (!Bytes.equals(pair.getFirst().getTableDesc().getName(),
tableName)) {
if (!Bytes.equals(pair.getFirst().getTableDesc().getName(), tableName)) {
return false;
}
result.set(pair);
@ -1095,13 +1064,11 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
* @return cluster status
*/
public ClusterStatus getClusterStatus() {
ClusterStatus status = new ClusterStatus();
status.setHBaseVersion(VersionInfo.getVersion());
status.setServerInfo(serverManager.getOnlineServers().values());
status.setDeadServers(serverManager.getDeadServers());
status.setRegionsInTransition(assignmentManager.getRegionsInTransition());
status.setClusterId(fileSystemManager.getClusterId());
return status;
return new ClusterStatus(VersionInfo.getVersion(),
this.fileSystemManager.getClusterId(),
this.serverManager.getOnlineServers(),
this.serverManager.getDeadServers(),
this.assignmentManager.getRegionsInTransition());
}
public String getClusterId() {
@ -1183,8 +1150,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
@Override
public String getServerName() {
return address.toString();
public ServerName getServerName() {
return this.serverName;
}
@Override
@ -1274,7 +1241,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
return;
}
}
Pair<HRegionInfo, HServerAddress> pair =
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(this.catalogTracker, regionName);
if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
assignRegion(pair.getFirst());
@ -1295,7 +1262,7 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
return;
}
}
Pair<HRegionInfo, HServerAddress> pair =
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(this.catalogTracker, regionName);
if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
HRegionInfo hri = pair.getFirst();
@ -1306,6 +1273,16 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
}
/**
* Compute the average load across all region servers.
* Currently, this uses a very naive computation - just uses the number of
* regions being served, ignoring stats about number of requests.
* @return the average load
*/
public double getAverageLoad() {
return this.assignmentManager.getAverageLoad();
}
/**
* Utility for constructing an instance of the passed HMaster class.
* @param masterClass
@ -1331,7 +1308,6 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
}
}
/**
* @see org.apache.hadoop.hbase.master.HMasterCommandLine
*/

View File

@ -27,6 +27,7 @@ import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.Random;
import java.util.TreeMap;
@ -40,8 +41,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.ServerName;
import com.google.common.collect.MinMaxPriorityQueue;
@ -103,6 +103,27 @@ public class LoadBalancer {
}
static RegionPlanComparator rpComparator = new RegionPlanComparator();
/**
* Data structure that holds servername and 'load'.
*/
static class ServerAndLoad implements Comparable<ServerAndLoad> {
private final ServerName sn;
private final int load;
ServerAndLoad(final ServerName sn, final int load) {
this.sn = sn;
this.load = load;
}
ServerName getServerName() {return this.sn;}
int getLoad() {return this.load;}
@Override
public int compareTo(ServerAndLoad other) {
int diff = this.load - other.load;
return diff != 0? diff: this.sn.compareTo(other.getServerName());
}
}
/**
* Generate a global load balancing plan according to the specified map of
* server information to the most loaded regions of each server.
@ -189,28 +210,25 @@ public class LoadBalancer {
* or null if cluster is already balanced
*/
public List<RegionPlan> balanceCluster(
Map<HServerInfo,List<HRegionInfo>> clusterState) {
Map<ServerName, List<HRegionInfo>> clusterState) {
boolean emptyRegionServerPresent = false;
long startTime = System.currentTimeMillis();
// Make a map sorted by load and count regions
TreeMap<HServerInfo,List<HRegionInfo>> serversByLoad =
new TreeMap<HServerInfo,List<HRegionInfo>>(
new HServerInfo.LoadComparator());
int numServers = clusterState.size();
if (numServers == 0) {
LOG.debug("numServers=0 so skipping load balancing");
return null;
}
NavigableMap<ServerAndLoad, List<HRegionInfo>> serversByLoad =
new TreeMap<ServerAndLoad, List<HRegionInfo>>();
int numRegions = 0;
// Iterate so we can count regions as we build the map
for(Map.Entry<HServerInfo, List<HRegionInfo>> server:
clusterState.entrySet()) {
int sz = server.getValue().size();
for (Map.Entry<ServerName, List<HRegionInfo>> server: clusterState.entrySet()) {
List<HRegionInfo> regions = server.getValue();
int sz = regions.size();
if (sz == 0) emptyRegionServerPresent = true;
server.getKey().getLoad().setNumberOfRegions(sz);
numRegions += server.getKey().getLoad().getNumberOfRegions();
serversByLoad.put(server.getKey(), server.getValue());
numRegions += sz;
serversByLoad.put(new ServerAndLoad(server.getKey(), sz), regions);
}
// Check if we even need to do any load balancing
@ -218,13 +236,14 @@ public class LoadBalancer {
// HBASE-3681 check sloppiness first
int floor = (int) Math.floor(average * (1 - slop));
int ceiling = (int) Math.ceil(average * (1 + slop));
if(serversByLoad.lastKey().getLoad().getNumberOfRegions() <= ceiling &&
serversByLoad.firstKey().getLoad().getNumberOfRegions() >= floor) {
if (serversByLoad.lastKey().getLoad() <= ceiling &&
serversByLoad.firstKey().getLoad() >= floor) {
// Skipped because no server outside (min,max) range
LOG.info("Skipping load balancing. servers=" + numServers + " " +
LOG.info("Skipping load balancing because balanced cluster; " +
"servers=" + numServers + " " +
"regions=" + numRegions + " average=" + average + " " +
"mostloaded=" + serversByLoad.lastKey().getLoad().getNumberOfRegions() +
" leastloaded=" + serversByLoad.firstKey().getLoad().getNumberOfRegions());
"mostloaded=" + serversByLoad.lastKey().getLoad() +
" leastloaded=" + serversByLoad.lastKey().getLoad());
return null;
}
int min = numRegions / numServers;
@ -232,21 +251,22 @@ public class LoadBalancer {
// Balance the cluster
// TODO: Look at data block locality or a more complex load to do this
MinMaxPriorityQueue<RegionPlan> regionsToMove = MinMaxPriorityQueue.orderedBy(rpComparator).create();
MinMaxPriorityQueue<RegionPlan> regionsToMove =
MinMaxPriorityQueue.orderedBy(rpComparator).create();
List<RegionPlan> regionsToReturn = new ArrayList<RegionPlan>();
// Walk down most loaded, pruning each to the max
int serversOverloaded = 0;
// flag used to fetch regions from head and tail of list, alternately
boolean fetchFromTail = false;
Map<HServerInfo,BalanceInfo> serverBalanceInfo =
new TreeMap<HServerInfo,BalanceInfo>();
for(Map.Entry<HServerInfo, List<HRegionInfo>> server :
Map<ServerName, BalanceInfo> serverBalanceInfo =
new TreeMap<ServerName, BalanceInfo>();
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server:
serversByLoad.descendingMap().entrySet()) {
HServerInfo serverInfo = server.getKey();
int regionCount = serverInfo.getLoad().getNumberOfRegions();
ServerAndLoad sal = server.getKey();
int regionCount = sal.getLoad();
if (regionCount <= max) {
serverBalanceInfo.put(serverInfo, new BalanceInfo(0, 0));
serverBalanceInfo.put(sal.getServerName(), new BalanceInfo(0, 0));
break;
}
serversOverloaded++;
@ -264,7 +284,7 @@ public class LoadBalancer {
i++;
// Don't rebalance meta regions.
if (hri.isMetaRegion()) continue;
regionsToMove.add(new RegionPlan(hri, serverInfo, null));
regionsToMove.add(new RegionPlan(hri, sal.getServerName(), null));
numTaken++;
if (numTaken >= numToOffload) break;
// fetch in alternate order if there is new region server
@ -272,7 +292,7 @@ public class LoadBalancer {
fetchFromTail = !fetchFromTail;
}
}
serverBalanceInfo.put(serverInfo,
serverBalanceInfo.put(sal.getServerName(),
new BalanceInfo(numToOffload, (-1)*numTaken));
}
int totalNumMoved = regionsToMove.size();
@ -280,36 +300,32 @@ public class LoadBalancer {
// Walk down least loaded, filling each to the min
int neededRegions = 0; // number of regions needed to bring all up to min
fetchFromTail = false;
RegionPlan rp = null;
Map<HServerInfo, Integer> underloadedServers = new HashMap<HServerInfo, Integer>();
for(Map.Entry<HServerInfo, List<HRegionInfo>> server :
Map<ServerName, Integer> underloadedServers = new HashMap<ServerName, Integer>();
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server:
serversByLoad.entrySet()) {
int regionCount = server.getKey().getLoad().getNumberOfRegions();
int regionCount = server.getKey().getLoad();
if (regionCount >= min) {
break;
}
underloadedServers.put(server.getKey(), min - regionCount);
underloadedServers.put(server.getKey().getServerName(), min - regionCount);
}
// number of servers that get new regions
int serversUnderloaded = underloadedServers.size();
int incr = 1;
List<HServerInfo> serverInfos = Arrays.asList(underloadedServers.keySet().
toArray(new HServerInfo[serversUnderloaded]));
Collections.shuffle(serverInfos, RANDOM);
List<ServerName> sns =
Arrays.asList(underloadedServers.keySet().toArray(new ServerName[serversUnderloaded]));
Collections.shuffle(sns, RANDOM);
while (regionsToMove.size() > 0) {
int cnt = 0;
int i = incr > 0 ? 0 : underloadedServers.size()-1;
for (; i >= 0 && i < underloadedServers.size(); i += incr) {
if (0 == regionsToMove.size()) break;
HServerInfo si = serverInfos.get(i);
if (regionsToMove.isEmpty()) break;
ServerName si = sns.get(i);
int numToTake = underloadedServers.get(si);
if (numToTake == 0) continue;
if (!fetchFromTail) rp = regionsToMove.remove();
else rp = regionsToMove.removeLast();
rp.setDestination(si);
regionsToReturn.add(rp);
addRegionPlan(regionsToMove, fetchFromTail, si, regionsToReturn);
if (emptyRegionServerPresent) {
fetchFromTail = !fetchFromTail;
}
@ -325,7 +341,6 @@ public class LoadBalancer {
}
if (cnt == 0) break;
// iterates underloadedServers in the other direction
LOG.info("First pass inner loop assigned " + cnt + " regions");
incr = -incr;
}
for (Integer i : underloadedServers.values()) {
@ -335,7 +350,7 @@ public class LoadBalancer {
// If none needed to fill all to min and none left to drain all to max,
// we are done
if(neededRegions == 0 && 0 == regionsToMove.size()) {
if (neededRegions == 0 && regionsToMove.isEmpty()) {
long endTime = System.currentTimeMillis();
LOG.info("Calculated a load balance in " + (endTime-startTime) + "ms. " +
"Moving " + totalNumMoved + " regions off of " +
@ -350,15 +365,16 @@ public class LoadBalancer {
// If we need more to fill min, grab one from each most loaded until enough
if (neededRegions != 0) {
// Walk down most loaded, grabbing one from each until we get enough
for(Map.Entry<HServerInfo, List<HRegionInfo>> server :
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server :
serversByLoad.descendingMap().entrySet()) {
BalanceInfo balanceInfo = serverBalanceInfo.get(server.getKey());
BalanceInfo balanceInfo =
serverBalanceInfo.get(server.getKey().getServerName());
int idx =
balanceInfo == null ? 0 : balanceInfo.getNextRegionForUnload();
if (idx >= server.getValue().size()) break;
HRegionInfo region = server.getValue().get(idx);
if (region.isMetaRegion()) continue; // Don't move meta regions.
regionsToMove.add(new RegionPlan(region, server.getKey(), null));
regionsToMove.add(new RegionPlan(region, server.getKey().getServerName(), null));
totalNumMoved++;
if (--neededRegions == 0) {
// No more regions needed, done shedding
@ -371,11 +387,11 @@ public class LoadBalancer {
// Assign each underloaded up to the min, then if leftovers, assign to max
// Walk down least loaded, assigning to each to fill up to min
for(Map.Entry<HServerInfo, List<HRegionInfo>> server :
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server :
serversByLoad.entrySet()) {
int regionCount = server.getKey().getLoad().getNumberOfRegions();
int regionCount = server.getKey().getLoad();
if (regionCount >= min) break;
BalanceInfo balanceInfo = serverBalanceInfo.get(server.getKey());
BalanceInfo balanceInfo = serverBalanceInfo.get(server.getKey().getServerName());
if(balanceInfo != null) {
regionCount += balanceInfo.getNumRegionsAdded();
}
@ -385,11 +401,8 @@ public class LoadBalancer {
int numToTake = min - regionCount;
int numTaken = 0;
while(numTaken < numToTake && 0 < regionsToMove.size()) {
if (!fetchFromTail) rp = regionsToMove.remove();
else rp = regionsToMove.removeLast();
rp.setDestination(server.getKey());
regionsToReturn.add(rp);
addRegionPlan(regionsToMove, fetchFromTail,
server.getKey().getServerName(), regionsToReturn);
numTaken++;
if (emptyRegionServerPresent) {
fetchFromTail = !fetchFromTail;
@ -399,20 +412,18 @@ public class LoadBalancer {
// If we still have regions to dish out, assign underloaded to max
if (0 < regionsToMove.size()) {
for(Map.Entry<HServerInfo, List<HRegionInfo>> server :
for (Map.Entry<ServerAndLoad, List<HRegionInfo>> server :
serversByLoad.entrySet()) {
int regionCount = server.getKey().getLoad().getNumberOfRegions();
int regionCount = server.getKey().getLoad();
if(regionCount >= max) {
break;
}
if (!fetchFromTail) rp = regionsToMove.remove();
else rp = regionsToMove.removeLast();
rp.setDestination(server.getKey());
regionsToReturn.add(rp);
addRegionPlan(regionsToMove, fetchFromTail,
server.getKey().getServerName(), regionsToReturn);
if (emptyRegionServerPresent) {
fetchFromTail = !fetchFromTail;
}
if(0 == regionsToMove.size()) {
if (regionsToMove.isEmpty()) {
break;
}
}
@ -420,15 +431,15 @@ public class LoadBalancer {
long endTime = System.currentTimeMillis();
if (0 != regionsToMove.size() || neededRegions != 0) {
if (!regionsToMove.isEmpty() || neededRegions != 0) {
// Emit data so can diagnose how balancer went astray.
LOG.warn("regionsToMove=" + totalNumMoved +
", numServers=" + numServers + ", serversOverloaded=" + serversOverloaded +
", serversUnderloaded=" + serversUnderloaded);
StringBuilder sb = new StringBuilder();
for (Map.Entry<HServerInfo, List<HRegionInfo>> e: clusterState.entrySet()) {
for (Map.Entry<ServerName, List<HRegionInfo>> e: clusterState.entrySet()) {
if (sb.length() > 0) sb.append(", ");
sb.append(e.getKey().getServerName());
sb.append(e.getKey().toString());
sb.append(" ");
sb.append(e.getValue().size());
}
@ -444,6 +455,18 @@ public class LoadBalancer {
return regionsToReturn;
}
/**
* Add a region from the head or tail to the List of regions to return.
*/
void addRegionPlan(final MinMaxPriorityQueue<RegionPlan> regionsToMove,
final boolean fetchFromTail, final ServerName sn, List<RegionPlan> regionsToReturn) {
RegionPlan rp = null;
if (!fetchFromTail) rp = regionsToMove.remove();
else rp = regionsToMove.removeLast();
rp.setDestination(sn);
regionsToReturn.add(rp);
}
/**
* @param regions
* @return Randomization of passed <code>regions</code>
@ -457,11 +480,6 @@ public class LoadBalancer {
* Stores additional per-server information about the regions added/removed
* during the run of the balancing algorithm.
*
* For servers that receive additional regions, we are not updating the number
* of regions in HServerInfo once we decide to reassign regions to a server,
* but we need this information later in the algorithm. This is stored in
* <b>numRegionsAdded</b>.
*
* For servers that shed regions, we need to track which regions we have
* already shed. <b>nextRegionForUnload</b> contains the index in the list
* of regions on the server that is the next to be shed.
@ -506,14 +524,14 @@ public class LoadBalancer {
* @return map of server to the regions it should take, or null if no
* assignment is possible (ie. no regions or no servers)
*/
public static Map<HServerInfo, List<HRegionInfo>> roundRobinAssignment(
HRegionInfo [] regions, List<HServerInfo> servers) {
if(regions.length == 0 || servers.size() == 0) {
public static Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
List<HRegionInfo> regions, List<ServerName> servers) {
if (regions.isEmpty() || servers.isEmpty()) {
return null;
}
Map<HServerInfo,List<HRegionInfo>> assignments =
new TreeMap<HServerInfo,List<HRegionInfo>>();
int numRegions = regions.length;
Map<ServerName, List<HRegionInfo>> assignments =
new TreeMap<ServerName,List<HRegionInfo>>();
int numRegions = regions.size();
int numServers = servers.size();
int max = (int)Math.ceil((float)numRegions/numServers);
int serverIdx = 0;
@ -522,10 +540,10 @@ public class LoadBalancer {
}
int regionIdx = 0;
for (int j = 0; j < numServers; j++) {
HServerInfo server = servers.get((j+serverIdx) % numServers);
ServerName server = servers.get((j + serverIdx) % numServers);
List<HRegionInfo> serverRegions = new ArrayList<HRegionInfo>(max);
for (int i=regionIdx; i<numRegions; i += numServers) {
serverRegions.add(regions[i % numRegions]);
serverRegions.add(regions.get(i % numRegions));
}
assignments.put(server, serverRegions);
regionIdx++;
@ -549,25 +567,20 @@ public class LoadBalancer {
* @param servers available servers
* @return map of servers and regions to be assigned to them
*/
public static Map<HServerInfo, List<HRegionInfo>> retainAssignment(
Map<HRegionInfo, HServerAddress> regions, List<HServerInfo> servers) {
Map<HServerInfo, List<HRegionInfo>> assignments =
new TreeMap<HServerInfo, List<HRegionInfo>>();
// Build a map of server addresses to server info so we can match things up
Map<HServerAddress, HServerInfo> serverMap =
new TreeMap<HServerAddress, HServerInfo>();
for (HServerInfo server : servers) {
serverMap.put(server.getServerAddress(), server);
public static Map<ServerName, List<HRegionInfo>> retainAssignment(
Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
Map<ServerName, List<HRegionInfo>> assignments =
new TreeMap<ServerName, List<HRegionInfo>>();
for (ServerName server : servers) {
assignments.put(server, new ArrayList<HRegionInfo>());
}
for (Map.Entry<HRegionInfo, HServerAddress> region : regions.entrySet()) {
HServerAddress hsa = region.getValue();
HServerInfo server = hsa == null? null: serverMap.get(hsa);
if (server != null) {
assignments.get(server).add(region.getKey());
for (Map.Entry<HRegionInfo, ServerName> region : regions.entrySet()) {
ServerName sn = region.getValue();
if (sn != null && servers.contains(sn)) {
assignments.get(sn).add(region.getKey());
} else {
assignments.get(servers.get(RANDOM.nextInt(assignments.size()))).add(
region.getKey());
int size = assignments.size();
assignments.get(servers.get(RANDOM.nextInt(size))).add(region.getKey());
}
}
return assignments;
@ -692,17 +705,17 @@ public class LoadBalancer {
* @param servers
* @return map of regions to the server it should be assigned to
*/
public static Map<HRegionInfo,HServerInfo> immediateAssignment(
List<HRegionInfo> regions, List<HServerInfo> servers) {
Map<HRegionInfo,HServerInfo> assignments =
new TreeMap<HRegionInfo,HServerInfo>();
public static Map<HRegionInfo, ServerName> immediateAssignment(
List<HRegionInfo> regions, List<ServerName> servers) {
Map<HRegionInfo,ServerName> assignments =
new TreeMap<HRegionInfo,ServerName>();
for(HRegionInfo region : regions) {
assignments.put(region, servers.get(RANDOM.nextInt(servers.size())));
}
return assignments;
}
public static HServerInfo randomAssignment(List<HServerInfo> servers) {
public static ServerName randomAssignment(List<ServerName> servers) {
if (servers == null || servers.isEmpty()) {
LOG.warn("Wanted to do random assignment but no servers to assign to");
return null;
@ -722,21 +735,21 @@ public class LoadBalancer {
*/
public static class RegionPlan implements Comparable<RegionPlan> {
private final HRegionInfo hri;
private final HServerInfo source;
private HServerInfo dest;
private final ServerName source;
private ServerName dest;
/**
* Instantiate a plan for a region move, moving the specified region from
* the specified source server to the specified destination server.
*
* Destination server can be instantiated as null and later set
* with {@link #setDestination(HServerInfo)}.
* with {@link #setDestination(ServerName)}.
*
* @param hri region to be moved
* @param source regionserver region should be moved from
* @param dest regionserver region should be moved to
*/
public RegionPlan(final HRegionInfo hri, HServerInfo source, HServerInfo dest) {
public RegionPlan(final HRegionInfo hri, ServerName source, ServerName dest) {
this.hri = hri;
this.source = source;
this.dest = dest;
@ -745,7 +758,7 @@ public class LoadBalancer {
/**
* Set the destination server for the plan for this region.
*/
public void setDestination(HServerInfo dest) {
public void setDestination(ServerName dest) {
this.dest = dest;
}
@ -753,7 +766,7 @@ public class LoadBalancer {
* Get the source server for the plan for this region.
* @return server info for source
*/
public HServerInfo getSource() {
public ServerName getSource() {
return source;
}
@ -761,7 +774,7 @@ public class LoadBalancer {
* Get the destination server for the plan for this region.
* @return server info for destination
*/
public HServerInfo getDestination() {
public ServerName getDestination() {
return dest;
}
@ -789,8 +802,8 @@ public class LoadBalancer {
@Override
public String toString() {
return "hri=" + this.hri.getRegionNameAsString() + ", src=" +
(this.source == null? "": this.source.getServerName()) +
", dest=" + (this.dest == null? "": this.dest.getServerName());
(this.source == null? "": this.source.toString()) +
", dest=" + (this.dest == null? "": this.dest.toString());
}
}
}

View File

@ -287,7 +287,7 @@ public class MasterCoprocessorHost
}
}
void preMove(final HRegionInfo region, final HServerInfo srcServer, final HServerInfo destServer)
void preMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
throws UnknownRegionException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {
@ -302,7 +302,7 @@ public class MasterCoprocessorHost
}
}
void postMove(final HRegionInfo region, final HServerInfo srcServer, final HServerInfo destServer)
void postMove(final HRegionInfo region, final ServerName srcServer, final ServerName destServer)
throws UnknownRegionException {
ObserverContext<MasterCoprocessorEnvironment> ctx = null;
for (MasterEnvironment env: coprocessors) {

View File

@ -20,7 +20,7 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@ -34,9 +34,9 @@ import org.apache.hadoop.fs.Path;
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.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
@ -95,7 +95,7 @@ public class MasterFileSystem {
conf.getBoolean("hbase.master.distributed.log.splitting", true);
if (this.distributedLogSplitting) {
this.splitLogManager = new SplitLogManager(master.getZooKeeper(),
master.getConfiguration(), master, master.getServerName());
master.getConfiguration(), master, master.getServerName().toString());
this.splitLogManager.finishInitialization();
} else {
this.splitLogManager = null;
@ -175,9 +175,9 @@ public class MasterFileSystem {
* Inspect the log directory to recover any log file without
* an active region server.
* @param onlineServers Map of online servers keyed by
* {@link HServerInfo#getServerName()}
* {@link ServerName}
*/
void splitLogAfterStartup(final Map<String, HServerInfo> onlineServers) {
void splitLogAfterStartup(final Set<ServerName> onlineServers) {
Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
try {
if (!this.fs.exists(logsDirPath)) {
@ -197,8 +197,8 @@ public class MasterFileSystem {
return;
}
for (FileStatus status : logFolders) {
String serverName = status.getPath().getName();
if (onlineServers.get(serverName) == null) {
ServerName serverName = new ServerName(status.getPath().getName());
if (!onlineServers.contains(serverName)) {
LOG.info("Log folder " + status.getPath() + " doesn't belong " +
"to a known region server, splitting");
splitLog(serverName);
@ -209,9 +209,9 @@ public class MasterFileSystem {
}
}
public void splitLog(final String serverName) {
public void splitLog(final ServerName serverName) {
long splitTime = 0, splitLogSize = 0;
Path logDir = new Path(this.rootdir, HLog.getHLogDirectoryName(serverName));
Path logDir = new Path(this.rootdir, HLog.getHLogDirectoryName(serverName.toString()));
if (distributedLogSplitting) {
splitTime = EnvironmentEdgeManager.currentTimeMillis();
try {

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@ -32,13 +33,12 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
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.PleaseHoldException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
@ -47,12 +47,9 @@ import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler;
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
/**
* The ServerManager class manages info about region servers - HServerInfo,
* load numbers, dying servers, etc.
* The ServerManager class manages info about region servers.
* <p>
* Maintains lists of online and dead servers. Processes the startups,
* shutdowns, and deaths of region servers.
@ -70,23 +67,20 @@ public class ServerManager {
// Set if we are to shutdown the cluster.
private volatile boolean clusterShutdown = false;
/** The map of known server names to server info */
private final Map<String, HServerInfo> onlineServers =
new ConcurrentHashMap<String, HServerInfo>();
/** Map of registered servers to their current load */
private final Map<ServerName, HServerLoad> onlineServers =
new ConcurrentHashMap<ServerName, HServerLoad>();
// 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<String, HRegionInterface> serverConnections =
new HashMap<String, HRegionInterface>();
private final Map<ServerName, HRegionInterface> serverConnections =
new HashMap<ServerName, HRegionInterface>();
private final Server master;
private final MasterServices services;
// Reporting to track master metrics.
private final MasterMetrics metrics;
private final DeadServer deadservers;
private final long maxSkew;
@ -95,26 +89,25 @@ public class ServerManager {
* Constructor.
* @param master
* @param services
* @param metrics
*/
public ServerManager(final Server master, final MasterServices services,
MasterMetrics metrics) {
public ServerManager(final Server master, final MasterServices services) {
this.master = master;
this.services = services;
this.metrics = metrics;
Configuration c = master.getConfiguration();
maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
this.deadservers =
new DeadServer(c.getInt("hbase.master.maxdeadservers", 100));
this.deadservers = new DeadServer();
}
/**
* Let the server manager know a new regionserver has come online
* @param serverInfo
* @param ia The remote address
* @param port The remote port
* @param serverStartcode
* @param serverCurrentTime The current time of the region server in ms
* @throws IOException
*/
void regionServerStartup(final HServerInfo serverInfo, long serverCurrentTime)
void regionServerStartup(final InetAddress ia, final int port,
final long serverStartcode, long serverCurrentTime)
throws IOException {
// Test for case where we get a region startup message from a regionserver
// that has been quickly restarted but whose znode expiration handler has
@ -123,58 +116,66 @@ public class ServerManager {
// is, reject the server and trigger its expiration. The next time it comes
// in, it should have been removed from serverAddressToServerInfo and queued
// for processing by ProcessServerShutdown.
HServerInfo info = new HServerInfo(serverInfo);
checkIsDead(info.getServerName(), "STARTUP");
checkAlreadySameHostPort(info);
checkClockSkew(info, serverCurrentTime);
recordNewServer(info, false, null);
ServerName sn = new ServerName(ia.getHostName(), port, serverStartcode);
checkClockSkew(sn, serverCurrentTime);
checkIsDead(sn, "STARTUP");
checkAlreadySameHostPort(sn);
recordNewServer(sn, HServerLoad.EMPTY_HSERVERLOAD);
}
void regionServerReport(ServerName sn, HServerLoad hsl)
throws YouAreDeadException, PleaseHoldException {
checkIsDead(sn, "REPORT");
if (!this.onlineServers.containsKey(sn)) {
// Already have this host+port combo and its just different start code?
checkAlreadySameHostPort(sn);
// Just let the server in. Presume master joining a running cluster.
// recordNewServer is what happens at the end of reportServerStartup.
// The only thing we are skipping is passing back to the regionserver
// the ServerName to use. Here we presume a master has already done
// that so we'll press on with whatever it gave us for ServerName.
recordNewServer(sn, hsl);
} else {
this.onlineServers.put(sn, hsl);
}
}
/**
* Test to see if we have a server of same host and port already.
* @param serverInfo
* @param serverName
* @throws PleaseHoldException
*/
void checkAlreadySameHostPort(final HServerInfo serverInfo)
void checkAlreadySameHostPort(final ServerName serverName)
throws PleaseHoldException {
String hostAndPort = serverInfo.getServerAddress().toString();
HServerInfo existingServer =
haveServerWithSameHostAndPortAlready(serverInfo.getHostnamePort());
ServerName existingServer =
ServerName.findServerWithSameHostnamePort(getOnlineServersList(), serverName);
if (existingServer != null) {
String message = "Server start rejected; we already have " + hostAndPort +
" registered; existingServer=" + existingServer + ", newServer=" + serverInfo;
String message = "Server serverName=" + serverName +
" rejected; we already have " + existingServer.toString() +
" registered with same hostname and port";
LOG.info(message);
if (existingServer.getStartCode() < serverInfo.getStartCode()) {
if (existingServer.getStartcode() < serverName.getStartcode()) {
LOG.info("Triggering server recovery; existingServer " +
existingServer.getServerName() + " looks stale");
existingServer + " looks stale");
expireServer(existingServer);
}
throw new PleaseHoldException(message);
}
}
private HServerInfo haveServerWithSameHostAndPortAlready(final String hostnamePort) {
synchronized (this.onlineServers) {
for (Map.Entry<String, HServerInfo> e: this.onlineServers.entrySet()) {
if (e.getValue().getHostnamePort().equals(hostnamePort)) {
return e.getValue();
}
}
}
return null;
}
/**
* Checks if the clock skew between the server and the master. If the clock
* skew is too much it will throw an Exception.
* @param serverName Incoming servers's name
* @param serverCurrentTime
* @throws ClockOutOfSyncException
*/
private void checkClockSkew(final HServerInfo serverInfo,
private void checkClockSkew(final ServerName serverName,
final long serverCurrentTime)
throws ClockOutOfSyncException {
long skew = System.currentTimeMillis() - serverCurrentTime;
if (skew > maxSkew) {
String message = "Server " + serverInfo.getServerName() + " has been " +
String message = "Server " + serverName + " has been " +
"rejected; Reported time is too far out of sync with master. " +
"Time difference of " + skew + "ms > max allowed of " + maxSkew + "ms";
LOG.warn(message);
@ -186,11 +187,11 @@ public class ServerManager {
* If this server is on the dead list, reject it with a YouAreDeadException.
* If it was dead but came back with a new start code, remove the old entry
* from the dead list.
* @param serverName Server name formatted as host_port_startcode.
* @param serverName
* @param what START or REPORT
* @throws YouAreDeadException
*/
private void checkIsDead(final String serverName, final String what)
private void checkIsDead(final ServerName serverName, final String what)
throws YouAreDeadException {
if (this.deadservers.isDeadServer(serverName)) {
// host name, port and start code all match with existing one of the
@ -210,157 +211,34 @@ public class ServerManager {
}
/**
* Adds the HSI to the RS list
* @param info The region server informations
* @param useInfoLoad True if the load from the info should be used; e.g.
* under a master failover
* @param hri Region interface. Can be null.
* Adds the onlineServers list.
* @param hsl
* @param serverName The remote servers name.
*/
void recordNewServer(HServerInfo info, boolean useInfoLoad,
HRegionInterface hri) {
HServerLoad load = useInfoLoad? info.getLoad(): new HServerLoad();
String serverName = info.getServerName();
LOG.info("Registering server=" + serverName + ", regionCount=" +
load.getLoad() + ", userLoad=" + useInfoLoad);
info.setLoad(load);
// TODO: Why did we update the RS location ourself? Shouldn't RS do this?
// masterStatus.getZooKeeper().updateRSLocationGetWatch(info, watcher);
// -- If I understand the question, the RS does not update the location
// because could be disagreement over locations because of DNS issues; only
// master does DNS now -- St.Ack 20100929.
this.onlineServers.put(serverName, info);
if (hri == null) {
serverConnections.remove(serverName);
} else {
serverConnections.put(serverName, hri);
}
}
/**
* Called to process the messages sent from the region server to the master
* along with the heart beat.
*
* @param serverInfo
* @param msgs
* @param mostLoadedRegions Array of regions the region server is submitting
* as candidates to be rebalanced, should it be overloaded
* @return messages from master to region server indicating what region
* server should do.
*
* @throws IOException
*/
HMsg [] regionServerReport(final HServerInfo serverInfo,
final HMsg [] msgs, final HRegionInfo[] mostLoadedRegions)
throws IOException {
// Be careful. This method does returns in the middle.
HServerInfo info = new HServerInfo(serverInfo);
// Check if dead. If it is, it'll get a 'You Are Dead!' exception.
checkIsDead(info.getServerName(), "REPORT");
// If we don't know this server, tell it shutdown.
HServerInfo storedInfo = this.onlineServers.get(info.getServerName());
if (storedInfo == null) {
// Maybe we already have this host+port combo and its just different
// start code?
checkAlreadySameHostPort(info);
// Just let the server in. Presume master joining a running cluster.
// recordNewServer is what happens at the end of reportServerStartup.
// The only thing we are skipping is passing back to the regionserver
// the HServerInfo to use. Here we presume a master has already done
// that so we'll press on with whatever it gave us for HSI.
recordNewServer(info, true, null);
// If msgs, put off their processing but this is not enough because
// its possible that the next time the server reports in, we'll still
// not be up and serving. For example, if a split, we'll need the
// regions and servers setup in the master before the below
// handleSplitReport will work. TODO: FIx!!
if (msgs.length > 0)
throw new PleaseHoldException("FIX! Putting off " +
"message processing because not yet rwady but possible we won't be " +
"ready next on next report");
}
for (HMsg msg: msgs) {
LOG.info("Received " + msg + " from " + serverInfo.getServerName());
switch (msg.getType()) {
default:
LOG.error("Unhandled msg type " + msg);
}
}
HMsg [] reply = null;
if (this.clusterShutdown) {
if (isOnlyMetaRegionServersOnline()) {
LOG.info("Only catalog regions remaining; running unassign");
// The only remaining regions are catalog regions.
// Shutdown needs to be staggered; the meta regions need to close last
// in case they need to be updated during the close melee. If only
// catalog reigons remaining, tell them they can go down now too. On
// close of region, the regionservers should then shut themselves down.
this.services.getAssignmentManager().unassignCatalogRegions();
}
}
return processRegionServerAllsWell(info, mostLoadedRegions, reply);
}
/**
* @return True if all online servers are carrying one or more catalog
* regions, there are no servers online carrying user regions only
*/
private boolean isOnlyMetaRegionServersOnline() {
List<HServerInfo> onlineServers = getOnlineServersList();
for (HServerInfo hsi: onlineServers) {
if (!this.services.getAssignmentManager().isMetaRegionServer(hsi)) {
return false;
}
}
return true;
}
/**
* RegionServer is checking in, no exceptional circumstances
* @param serverInfo
* @param mostLoadedRegions
* @param msgs
* @return
* @throws IOException
*/
private HMsg[] processRegionServerAllsWell(HServerInfo serverInfo,
final HRegionInfo[] mostLoadedRegions, HMsg[] msgs)
throws IOException {
// Refresh the info object and the load information
this.onlineServers.put(serverInfo.getServerName(), serverInfo);
HServerLoad load = serverInfo.getLoad();
if (load != null && this.metrics != null) {
this.metrics.incrementRequests(load.getNumberOfRequests());
}
// No more piggyback messages on heartbeats for other stuff
return msgs;
}
/**
* Make server load accessible to AssignmentManager
* @param serverName
* @return
* @throws HServerLoad if serverName is known
*/
HServerLoad getLoad(String serverName) {
HServerInfo hsi = this.onlineServers.get(serverName);
if (hsi == null) return null;
return hsi.getLoad();
void recordNewServer(final ServerName serverName, final HServerLoad hsl) {
LOG.info("Registering server=" + serverName);
this.onlineServers.put(serverName, hsl);
this.serverConnections.remove(serverName);
}
/**
* @param serverName
* @return True if we removed server from the list.
* @return HServerLoad if serverName is known else null
*/
private boolean removeServerInfo(final String serverName) {
HServerInfo info = this.onlineServers.remove(serverName);
if (info != null) {
return true;
public HServerLoad getLoad(final ServerName serverName) {
return this.onlineServers.get(serverName.toString());
}
return false;
/**
* @param serverName
* @return HServerLoad if serverName is known else null
* @deprecated Use {@link #getLoad(HServerAddress)}
*/
public HServerLoad getLoad(final HServerAddress address) {
ServerName sn = new ServerName(address.toString(), -1);
ServerName actual =
ServerName.findServerWithSameHostnamePort(this.getOnlineServersList(), sn);
return actual == null? null: getLoad(actual);
}
/**
@ -373,9 +251,9 @@ public class ServerManager {
int totalLoad = 0;
int numServers = 0;
double averageLoad = 0.0;
for (HServerInfo hsi : onlineServers.values()) {
for (HServerLoad hsl: this.onlineServers.values()) {
numServers++;
totalLoad += hsi.getLoad().getNumberOfRegions();
totalLoad += hsl.getNumberOfRegions();
}
averageLoad = (double)totalLoad / (double)numServers;
return averageLoad;
@ -387,25 +265,17 @@ public class ServerManager {
return this.onlineServers.size();
}
/**
* @param name server name
* @return HServerInfo for the given server address
*/
public HServerInfo getServerInfo(String name) {
return this.onlineServers.get(name);
}
/**
* @return Read-only map of servers to serverinfo
*/
public Map<String, HServerInfo> getOnlineServers() {
public Map<ServerName, HServerLoad> getOnlineServers() {
// Presumption is that iterating the returned Map is OK.
synchronized (this.onlineServers) {
return Collections.unmodifiableMap(this.onlineServers);
}
}
public Set<String> getDeadServers() {
public Set<ServerName> getDeadServers() {
return this.deadservers.clone();
}
@ -417,40 +287,11 @@ public class ServerManager {
return this.deadservers.areDeadServersInProgress();
}
/**
* @param hsa
* @return The HServerInfo whose HServerAddress is <code>hsa</code> or null
* if nothing found.
*/
public HServerInfo getHServerInfo(final HServerAddress hsa) {
synchronized(this.onlineServers) {
// TODO: This is primitive. Do a better search.
for (Map.Entry<String, HServerInfo> e: this.onlineServers.entrySet()) {
if (e.getValue().getServerAddress().equals(hsa)) {
return e.getValue();
}
}
}
return null;
}
private void notifyOnlineServers() {
synchronized (this.onlineServers) {
this.onlineServers.notifyAll();
}
}
/*
* Wait on regionservers to report in
* with {@link #regionServerReport(HServerInfo, HMsg[])} so they get notice
* the master is going down. Waits until all region servers come back with
* a MSG_REGIONSERVER_STOP.
*/
void letRegionServersShutdown() {
synchronized (onlineServers) {
while (onlineServers.size() > 0) {
while (!onlineServers.isEmpty()) {
StringBuilder sb = new StringBuilder();
for (String key: this.onlineServers.keySet()) {
for (ServerName key: this.onlineServers.keySet()) {
if (sb.length() > 0) {
sb.append(", ");
}
@ -470,19 +311,15 @@ public class ServerManager {
* Expire the passed server. Add it to list of deadservers and queue a
* shutdown processing.
*/
public synchronized void expireServer(final HServerInfo hsi) {
// First check a server to expire. ServerName is of the form:
// <hostname> , <port> , <startcode>
String serverName = hsi.getServerName();
HServerInfo info = this.onlineServers.get(serverName);
if (info == null) {
LOG.warn("Received expiration of " + hsi.getServerName() +
public synchronized void expireServer(final ServerName serverName) {
if (!this.onlineServers.containsKey(serverName)) {
LOG.warn("Received expiration of " + serverName +
" but server is not currently online");
return;
}
if (this.deadservers.contains(serverName)) {
// TODO: Can this happen? It shouldn't be online in this case?
LOG.warn("Received expiration of " + hsi.getServerName() +
LOG.warn("Received expiration of " + serverName +
" but server shutdown is already in progress");
return;
}
@ -495,7 +332,7 @@ public class ServerManager {
// If cluster is going down, yes, servers are going to be expiring; don't
// process as a dead server
if (this.clusterShutdown) {
LOG.info("Cluster shutdown set; " + hsi.getServerName() +
LOG.info("Cluster shutdown set; " + serverName +
" expired; onlineServers=" + this.onlineServers.size());
if (this.onlineServers.isEmpty()) {
master.stop("Cluster shutdown set; onlineServer=0");
@ -506,9 +343,8 @@ public class ServerManager {
// Was this server carrying root?
boolean carryingRoot;
try {
HServerAddress address = ct.getRootLocation();
carryingRoot = address != null &&
hsi.getServerAddress().equals(address);
ServerName address = ct.getRootLocation();
carryingRoot = address.equals(serverName);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.info("Interrupted");
@ -519,15 +355,14 @@ public class ServerManager {
// run into fact that meta is dead). I can ask assignment manager. It
// has an inmemory list of who has what. This list will be cleared as we
// process the dead server but should be find asking it now.
HServerAddress address = ct.getMetaLocation();
boolean carryingMeta =
address != null && hsi.getServerAddress().equals(address);
ServerName address = ct.getMetaLocation();
boolean carryingMeta = address.equals(serverName);
if (carryingRoot || carryingMeta) {
this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
this.services, this.deadservers, info, carryingRoot, carryingMeta));
this.services, this.deadservers, serverName, carryingRoot, carryingMeta));
} else {
this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
this.services, this.deadservers, info));
this.services, this.deadservers, serverName));
}
LOG.debug("Added=" + serverName +
" to dead servers, submitted shutdown handler to be executed, root=" +
@ -544,12 +379,12 @@ public class ServerManager {
* @param server server to open a region
* @param region region to open
*/
public void sendRegionOpen(HServerInfo server, HRegionInfo region)
public void sendRegionOpen(final ServerName server, HRegionInfo region)
throws IOException {
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");
LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
" failed because no RPC connection found to this server");
return;
}
hri.openRegion(region);
@ -563,12 +398,12 @@ public class ServerManager {
* @param server server to open a region
* @param regions regions to open
*/
public void sendRegionOpen(HServerInfo server, List<HRegionInfo> regions)
public void sendRegionOpen(ServerName server, List<HRegionInfo> regions)
throws IOException {
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");
LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
" failed because no RPC connection found to this server");
return;
}
hri.openRegions(regions);
@ -584,13 +419,13 @@ public class ServerManager {
* @return true if server acknowledged close, false if not
* @throws IOException
*/
public boolean sendRegionClose(HServerInfo server, HRegionInfo region)
public boolean sendRegionClose(ServerName server, HRegionInfo region)
throws IOException {
if (server == null) throw new NullPointerException("Passed server is null");
HRegionInterface hri = getServerConnection(server);
if (hri == null) {
throw new IOException("Attempting to send CLOSE RPC to server " +
server.getServerName() + " for region " +
server.toString() + " for region " +
region.getRegionNameAsString() +
" failed because no RPC connection found to this server");
}
@ -598,31 +433,30 @@ public class ServerManager {
}
/**
* @param info
* @param sn
* @return
* @throws IOException
* @throws RetriesExhaustedException wrapping a ConnectException if failed
* putting up proxy.
*/
private HRegionInterface getServerConnection(HServerInfo info)
private HRegionInterface getServerConnection(final ServerName sn)
throws IOException {
HConnection connection =
HConnectionManager.getConnection(this.master.getConfiguration());
HRegionInterface hri = serverConnections.get(info.getServerName());
HRegionInterface hri = this.serverConnections.get(sn.toString());
if (hri == null) {
LOG.debug("New connection to " + info.getServerName());
hri = connection.getHRegionConnection(info.getServerAddress(), false);
this.serverConnections.put(info.getServerName(), hri);
LOG.debug("New connection to " + sn.toString());
hri = connection.getHRegionConnection(sn.getHostname(), sn.getPort());
this.serverConnections.put(sn, hri);
}
return hri;
}
/**
* Waits for the regionservers to report in.
* @return Count of regions out on cluster
* @throws InterruptedException
*/
public int waitForRegionServers()
public void waitForRegionServers()
throws InterruptedException {
long interval = this.master.getConfiguration().
getLong("hbase.master.wait.on.regionservers.interval", 3000);
@ -640,31 +474,18 @@ public class ServerManager {
}
oldcount = count;
}
// Count how many regions deployed out on cluster. If fresh start, it'll
// be none but if not a fresh start, we'll have registered servers when
// they came in on the {@link #regionServerReport(HServerInfo)} as opposed to
// {@link #regionServerStartup(HServerInfo)} and it'll be carrying an
// actual server load.
int regionCount = 0;
for (Map.Entry<String, HServerInfo> e: this.onlineServers.entrySet()) {
HServerLoad load = e.getValue().getLoad();
if (load != null) regionCount += load.getLoad();
}
LOG.info("Exiting wait on regionserver(s) to checkin; count=" + count +
", stopped=" + this.master.isStopped() +
", count of regions out on cluster=" + regionCount);
return regionCount;
}
/**
* @return A copy of the internal list of online servers.
*/
public List<HServerInfo> getOnlineServersList() {
public List<ServerName> getOnlineServersList() {
// TODO: optimize the load balancer call so we don't need to make a new list
return new ArrayList<HServerInfo>(onlineServers.values());
// TODO: FIX. THIS IS POPULAR CALL.
return new ArrayList<ServerName>(this.onlineServers.keySet());
}
public boolean isServerOnline(String serverName) {
public boolean isServerOnline(ServerName serverName) {
return onlineServers.containsKey(serverName);
}
@ -681,6 +502,5 @@ public class ServerManager {
* Stop the ServerManager. Currently does nothing.
*/
public void stop() {
}
}

View File

@ -156,11 +156,14 @@ public class SplitLogManager extends ZooKeeperListener {
}
public void finishInitialization() {
Threads.setDaemonThreadRunning(timeoutMonitor, serverName
+ ".splitLogManagerTimeoutMonitor");
Threads.setDaemonThreadRunning(timeoutMonitor, serverName +
".splitLogManagerTimeoutMonitor");
// Watcher can be null during tests with Mock'd servers.
if (this.watcher != null) {
this.watcher.registerListener(this);
lookForOrphans();
}
}
/**
* The caller will block until all the log files of the given region server

View File

@ -19,8 +19,8 @@
*/
package org.apache.hadoop.hbase.master.handler;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.DeadServer;
import org.apache.hadoop.hbase.master.MasterServices;
@ -34,9 +34,9 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
public MetaServerShutdownHandler(final Server server,
final MasterServices services,
final DeadServer deadServers, final HServerInfo hsi,
final DeadServer deadServers, final ServerName serverName,
final boolean carryingRoot, final boolean carryingMeta) {
super(server, services, deadServers, hsi, EventType.M_META_SERVER_SHUTDOWN);
super(server, services, deadServers, serverName, EventType.M_META_SERVER_SHUTDOWN);
this.carryingRoot = carryingRoot;
this.carryingMeta = carryingMeta;
}

View File

@ -22,8 +22,8 @@ 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.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@ -36,7 +36,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
private static final Log LOG = LogFactory.getLog(OpenedRegionHandler.class);
private final AssignmentManager assignmentManager;
private final HRegionInfo regionInfo;
private final HServerInfo serverInfo;
private final ServerName sn;
private final OpenedPriority priority;
private enum OpenedPriority {
@ -55,11 +55,11 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
public OpenedRegionHandler(Server server,
AssignmentManager assignmentManager, HRegionInfo regionInfo,
HServerInfo serverInfo) {
ServerName sn) {
super(server, EventType.RS_ZK_REGION_OPENED);
this.assignmentManager = assignmentManager;
this.regionInfo = regionInfo;
this.serverInfo = serverInfo;
this.sn = sn;
if(regionInfo.isRootRegion()) {
priority = OpenedPriority.ROOT;
} else if(regionInfo.isMetaRegion()) {
@ -94,7 +94,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
// Code to defend against case where we get SPLIT before region open
// processing completes; temporary till we make SPLITs go via zk -- 0.92.
if (this.assignmentManager.isRegionInTransition(regionInfo) != null) {
this.assignmentManager.regionOnline(regionInfo, serverInfo);
this.assignmentManager.regionOnline(regionInfo, this.sn);
} else {
LOG.warn("Skipping the onlining of " + regionInfo.getRegionNameAsString() +
" because regions is NOT in RIT -- presuming this is because it SPLIT");
@ -106,7 +106,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
assignmentManager.unassign(regionInfo);
} else {
LOG.debug("Opened region " + regionInfo.getRegionNameAsString() +
" on " + serverInfo.getServerName());
" on " + this.sn.toString());
}
}
}

View File

@ -28,8 +28,8 @@ 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.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
@ -47,29 +47,29 @@ import org.apache.zookeeper.KeeperException;
/**
* Process server shutdown.
* Server-to-handle must be already in the deadservers lists. See
* {@link ServerManager#expireServer(HServerInfo)}.
* {@link ServerManager#expireServer(ServerName)}
*/
public class ServerShutdownHandler extends EventHandler {
private static final Log LOG = LogFactory.getLog(ServerShutdownHandler.class);
private final HServerInfo hsi;
private final ServerName serverName;
private final Server server;
private final MasterServices services;
private final DeadServer deadServers;
public ServerShutdownHandler(final Server server, final MasterServices services,
final DeadServer deadServers, final HServerInfo hsi) {
this(server, services, deadServers, hsi, EventType.M_SERVER_SHUTDOWN);
final DeadServer deadServers, final ServerName serverName) {
this(server, services, deadServers, serverName, EventType.M_SERVER_SHUTDOWN);
}
ServerShutdownHandler(final Server server, final MasterServices services,
final DeadServer deadServers, final HServerInfo hsi, EventType type) {
final DeadServer deadServers, final ServerName serverName, EventType type) {
super(server, type);
this.hsi = hsi;
this.serverName = serverName;
this.server = server;
this.services = services;
this.deadServers = deadServers;
if (!this.deadServers.contains(hsi.getServerName())) {
LOG.warn(hsi.getServerName() + " is NOT in deadservers; it should be!");
if (!this.deadServers.contains(this.serverName)) {
LOG.warn(this.serverName + " is NOT in deadservers; it should be!");
}
}
@ -89,7 +89,7 @@ public class ServerShutdownHandler extends EventHandler {
@Override
public void process() throws IOException {
final String serverName = this.hsi.getServerName();
final ServerName serverName = this.serverName;
LOG.info("Splitting logs for " + serverName);
this.services.getMasterFileSystem().splitLog(serverName);
@ -99,7 +99,7 @@ public class ServerShutdownHandler extends EventHandler {
// OFFLINE? -- and then others after like CLOSING that depend on log
// splitting.
List<RegionState> regionsInTransition =
this.services.getAssignmentManager().processServerShutdown(this.hsi);
this.services.getAssignmentManager().processServerShutdown(this.serverName);
// Assign root and meta if we were carrying them.
if (isCarryingRoot()) { // -ROOT-
@ -134,7 +134,7 @@ public class ServerShutdownHandler extends EventHandler {
try {
this.server.getCatalogTracker().waitForMeta();
hris = MetaReader.getServerUserRegions(this.server.getCatalogTracker(),
this.hsi);
this.serverName);
break;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();

View File

@ -24,8 +24,8 @@ 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.HServerInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@ -38,7 +38,7 @@ public class SplitRegionHandler extends EventHandler implements TotesHRegionInfo
private static final Log LOG = LogFactory.getLog(SplitRegionHandler.class);
private final AssignmentManager assignmentManager;
private final HRegionInfo parent;
private final HServerInfo serverInfo;
private final ServerName sn;
private final List<HRegionInfo> daughters;
/**
* For testing only! Set to true to skip handling of split.
@ -47,11 +47,11 @@ public class SplitRegionHandler extends EventHandler implements TotesHRegionInfo
public SplitRegionHandler(Server server,
AssignmentManager assignmentManager, HRegionInfo regionInfo,
HServerInfo serverInfo, final List<HRegionInfo> daughters) {
ServerName sn, final List<HRegionInfo> daughters) {
super(server, EventType.RS_ZK_REGION_SPLIT);
this.assignmentManager = assignmentManager;
this.parent = regionInfo;
this.serverInfo = serverInfo;
this.sn = sn;
this.daughters = daughters;
}
@ -70,7 +70,7 @@ public class SplitRegionHandler extends EventHandler implements TotesHRegionInfo
LOG.warn("Skipping split message, TEST_SKIP is set");
return;
}
this.assignmentManager.handleSplitReport(this.serverInfo, this.parent,
this.assignmentManager.handleSplitReport(this.sn, this.parent,
this.daughters.get(0), this.daughters.get(1));
// Remove region from ZK
try {

View File

@ -58,12 +58,12 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
@ -51,12 +50,6 @@ public interface RegionServerServices extends OnlineRegions {
*/
public FlushRequester getFlushRequester();
/**
* Return data structure that has Server address and startcode.
* @return The HServerInfo for this RegionServer.
*/
public HServerInfo getServerInfo();
/**
* @return the RegionServerAccounting for this Region Server
*/

View File

@ -198,7 +198,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
try {
taskReadyLock.wait();
} catch (InterruptedException e) {
LOG.warn("SplitLogWorker inteurrpted while waiting for task," +
LOG.warn("SplitLogWorker interurrpted while waiting for task," +
" exiting", e);
assert exitWorker == true;
return;

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
@ -476,7 +477,7 @@ public class SplitTransaction {
}
// Look for any exception
for (Future future : futures) {
for (Future<Void> future: futures) {
try {
future.get();
} catch (InterruptedException e) {
@ -690,7 +691,7 @@ public class SplitTransaction {
* @throws IOException
*/
private static int createNodeSplitting(final ZooKeeperWatcher zkw,
final HRegionInfo region, final String serverName)
final HRegionInfo region, final ServerName serverName)
throws KeeperException, IOException {
LOG.debug(zkw.prefix("Creating ephemeral node for " +
region.getEncodedName() + " in SPLITTING state"));
@ -744,7 +745,7 @@ public class SplitTransaction {
* @throws IOException
*/
private static int transitionNodeSplit(ZooKeeperWatcher zkw,
HRegionInfo parent, HRegionInfo a, HRegionInfo b, String serverName,
HRegionInfo parent, HRegionInfo a, HRegionInfo b, ServerName serverName,
final int znodeVersion)
throws KeeperException, IOException {
byte [] payload = Writables.getBytes(a, b);
@ -755,7 +756,7 @@ public class SplitTransaction {
private static int transitionNodeSplitting(final ZooKeeperWatcher zkw,
final HRegionInfo parent,
final String serverName, final int version)
final ServerName serverName, final int version)
throws KeeperException, IOException {
return ZKAssign.transitionNode(zkw, parent, serverName,
EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);

View File

@ -40,7 +40,6 @@ import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.regex.Matcher;
@ -57,8 +56,8 @@ import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.FSUtils;
@ -146,9 +145,6 @@ public class HLog implements Syncable {
private Method getNumCurrentReplicas; // refers to DFSOutputStream.getNumCurrentReplicas
final static Object [] NO_ARGS = new Object []{};
// used to indirectly tell syncFs to force the sync
private boolean forceSync = false;
public interface Reader {
void init(FileSystem fs, Path path, Configuration c) throws IOException;
void close() throws IOException;
@ -1279,36 +1275,10 @@ public class HLog implements Syncable {
/**
* Construct the HLog directory name
*
* @param info HServerInfo for server
* @param serverName Server name formatted as described in {@link ServerName}
* @return the HLog directory name
*/
public static String getHLogDirectoryName(HServerInfo info) {
return getHLogDirectoryName(info.getServerName());
}
/**
* Construct the HLog directory name
*
* @param serverAddress
* @param startCode
* @return the HLog directory name
*/
public static String getHLogDirectoryName(String serverAddress,
long startCode) {
if (serverAddress == null || serverAddress.length() == 0) {
return null;
}
return getHLogDirectoryName(
HServerInfo.getServerName(serverAddress, startCode));
}
/**
* Construct the HLog directory name
*
* @param serverName
* @return the HLog directory name
*/
public static String getHLogDirectoryName(String serverName) {
public static String getHLogDirectoryName(final String serverName) {
StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
dirName.append("/");
dirName.append(serverName);

View File

@ -24,7 +24,7 @@ import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
@ -37,8 +37,7 @@ public class ReplicationPeer {
private final String clusterKey;
private final String id;
private List<HServerAddress> regionServers =
new ArrayList<HServerAddress>(0);
private List<ServerName> regionServers = new ArrayList<ServerName>(0);
private final AtomicBoolean peerEnabled = new AtomicBoolean();
// Cannot be final since a new object needs to be recreated when session fails
private ZooKeeperWatcher zkw;
@ -82,7 +81,7 @@ public class ReplicationPeer {
* for this peer cluster
* @return list of addresses
*/
public List<HServerAddress> getRegionServers() {
public List<ServerName> getRegionServers() {
return regionServers;
}
@ -90,7 +89,7 @@ public class ReplicationPeer {
* Set the list of region servers for that peer
* @param regionServers list of addresses for the region servers
*/
public void setRegionServers(List<HServerAddress> regionServers) {
public void setRegionServers(List<ServerName> regionServers) {
this.regionServers = regionServers;
}

View File

@ -35,14 +35,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
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.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
/**
@ -137,7 +136,7 @@ public class ReplicationZookeeper {
this.peerClusters = new HashMap<String, ReplicationPeer>();
ZKUtil.createWithParents(this.zookeeper,
ZKUtil.joinZNode(this.replicationZNode, this.replicationStateNodeName));
this.rsServerNameZnode = ZKUtil.joinZNode(rsZNode, server.getServerName());
this.rsServerNameZnode = ZKUtil.joinZNode(rsZNode, server.getServerName().toString());
ZKUtil.createWithParents(this.zookeeper, this.rsServerNameZnode);
connectExistingPeers();
}
@ -204,14 +203,14 @@ public class ReplicationZookeeper {
* @param peerClusterId (byte) the cluster to interrogate
* @return addresses of all region servers
*/
public List<HServerAddress> getSlavesAddresses(String peerClusterId)
public List<ServerName> getSlavesAddresses(String peerClusterId)
throws KeeperException {
if (this.peerClusters.size() == 0) {
return new ArrayList<HServerAddress>(0);
return new ArrayList<ServerName>(0);
}
ReplicationPeer peer = this.peerClusters.get(peerClusterId);
if (peer == null) {
return new ArrayList<HServerAddress>(0);
return new ArrayList<ServerName>(0);
}
peer.setRegionServers(fetchSlavesAddresses(peer.getZkw()));
return peer.getRegionServers();
@ -222,16 +221,47 @@ public class ReplicationZookeeper {
* @param zkw zk connection to use
* @return list of region server addresses
*/
private List<HServerAddress> fetchSlavesAddresses(ZooKeeperWatcher zkw) {
List<HServerAddress> rss = null;
private List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw) {
List<ServerName> rss = null;
try {
rss = ZKUtil.listChildrenAndGetAsAddresses(zkw, zkw.rsZNode);
rss = listChildrenAndGetAsServerNames(zkw, zkw.rsZNode);
} catch (KeeperException e) {
LOG.warn("Cannot get peer's region server addresses", e);
}
return rss;
}
/**
* Lists the children of the specified znode, retrieving the data of each
* child as a server address.
*
* Used to list the currently online regionservers and their addresses.
*
* Sets no watches at all, this method is best effort.
*
* Returns an empty list if the node has no children. Returns null if the
* parent node itself does not exist.
*
* @param zkw zookeeper reference
* @param znode node to get children of as addresses
* @return list of data of children of specified znode, empty if no children,
* null if parent does not exist
* @throws KeeperException if unexpected zookeeper exception
*/
public static List<ServerName> listChildrenAndGetAsServerNames(
ZooKeeperWatcher zkw, String znode)
throws KeeperException {
List<String> children = ZKUtil.listChildrenNoWatch(zkw, znode);
if(children == null) {
return null;
}
List<ServerName> addresses = new ArrayList<ServerName>(children.size());
for (String child : children) {
addresses.add(new ServerName(child));
}
return addresses;
}
/**
* This method connects this cluster to another one and registers it
* in this region server's replication znode

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -202,7 +203,7 @@ public class ReplicationSource extends Thread
*/
private void chooseSinks() throws KeeperException {
this.currentPeers.clear();
List<HServerAddress> addresses =
List<ServerName> addresses =
this.zkHelper.getSlavesAddresses(peerClusterId);
Set<HServerAddress> setOfAddr = new HashSet<HServerAddress>();
int nbPeers = (int) (Math.ceil(addresses.size() * ratio));
@ -212,7 +213,8 @@ public class ReplicationSource extends Thread
HServerAddress address;
// Make sure we get one address that we don't already have
do {
address = addresses.get(this.random.nextInt(addresses.size()));
ServerName sn = addresses.get(this.random.nextInt(addresses.size()));
address = new HServerAddress(sn.getHostname(), sn.getPort());
} while (setOfAddr.contains(address));
LOG.info("Choosing peer " + address);
setOfAddr.add(address);

View File

@ -35,8 +35,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
@ -73,13 +73,13 @@ public class StorageClusterStatusResource extends ResourceBase {
model.setRegions(status.getRegionsCount());
model.setRequests(status.getRequestsCount());
model.setAverageLoad(status.getAverageLoad());
for (HServerInfo info: status.getServerInfo()) {
HServerLoad load = info.getLoad();
for (ServerName info: status.getServers()) {
HServerLoad load = status.getLoad(info);
StorageClusterStatusModel.Node node =
model.addLiveNode(
info.getServerAddress().getHostname() + ":" +
Integer.toString(info.getServerAddress().getPort()),
info.getStartCode(), load.getUsedHeapMB(),
info.getHostname() + ":" +
Integer.toString(info.getPort()),
info.getStartcode(), load.getUsedHeapMB(),
load.getMaxHeapMB());
node.setRequests(load.getNumberOfRequests());
for (HServerLoad.RegionLoad region: load.getRegionsLoad().values()) {
@ -88,8 +88,8 @@ public class StorageClusterStatusResource extends ResourceBase {
region.getMemStoreSizeMB(), region.getStorefileIndexSizeMB());
}
}
for (String name: status.getDeadServerNames()) {
model.addDeadNode(name);
for (ServerName name: status.getDeadServerNames()) {
model.addDeadNode(name.toString());
}
ResponseBuilder response = Response.ok(model);
response.cacheControl(cacheControl);

View File

@ -0,0 +1,75 @@
/**
* Copyright 2011 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.util;
import java.net.InetSocketAddress;
/**
* Utility for network addresses, resolving and naming.
*/
public class Addressing {
public static final String HOSTNAME_PORT_SEPARATOR = ":";
/**
* @param hostAndPort Formatted as <code>&lt;hostname> ':' &lt;port></code>
* @return An InetSocketInstance
*/
public static InetSocketAddress createInetSocketAddressFromHostAndPortStr(
final String hostAndPort) {
return new InetSocketAddress(parseHostname(hostAndPort), parsePort(hostAndPort));
}
/**
* @param hostname Server hostname
* @param port Server port
* @return Returns a concatenation of <code>hostname</code> and
* <code>port</code> in following
* form: <code>&lt;hostname> ':' &lt;port></code>. For example, if hostname
* is <code>example.org</code> and port is 1234, this method will return
* <code>example.org:1234</code>
*/
public static String createHostAndPortStr(final String hostname, final int port) {
return hostname + HOSTNAME_PORT_SEPARATOR + port;
}
/**
* @param hostAndPort Formatted as <code>&lt;hostname> ':' &lt;port></code>
* @return The hostname portion of <code>hostAndPort</code>
*/
public static String parseHostname(final String hostAndPort) {
int colonIndex = hostAndPort.lastIndexOf(HOSTNAME_PORT_SEPARATOR);
if (colonIndex < 0) {
throw new IllegalArgumentException("Not a host:port pair: " + hostAndPort);
}
return hostAndPort.substring(0, colonIndex);
}
/**
* @param hostAndPort Formatted as <code>&lt;hostname> ':' &lt;port></code>
* @return The port portion of <code>hostAndPort</code>
*/
public static int parsePort(final String hostAndPort) {
int colonIndex = hostAndPort.lastIndexOf(HOSTNAME_PORT_SEPARATOR);
if (colonIndex < 0) {
throw new IllegalArgumentException("Not a host:port pair: " + hostAndPort);
}
return Integer.parseInt(hostAndPort.substring(colonIndex + 1));
}
}

View File

@ -39,25 +39,28 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
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.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -172,21 +175,21 @@ public class HBaseFsck {
}
// From the master, get a list of all known live region servers
Collection<HServerInfo> regionServers = status.getServerInfo();
Collection<ServerName> regionServers = status.getServers();
errors.print("Number of live region servers: " +
regionServers.size());
if (details) {
for (HServerInfo rsinfo: regionServers) {
errors.print(" " + rsinfo.getServerName());
for (ServerName rsinfo: regionServers) {
errors.print(" " + rsinfo);
}
}
// From the master, get a list of all dead region servers
Collection<String> deadRegionServers = status.getDeadServerNames();
Collection<ServerName> deadRegionServers = status.getDeadServerNames();
errors.print("Number of dead region servers: " +
deadRegionServers.size());
if (details) {
for (String name: deadRegionServers) {
for (ServerName name: deadRegionServers) {
errors.print(" " + name);
}
}
@ -302,31 +305,55 @@ public class HBaseFsck {
// Check if Root region is valid and existing
if (rootLocation == null || rootLocation.getRegionInfo() == null ||
rootLocation.getServerAddress() == null) {
rootLocation.getHostname() == null) {
errors.reportError("Root Region or some of its attributes is null.");
return false;
}
MetaEntry m = new MetaEntry(rootLocation.getRegionInfo(),
rootLocation.getServerAddress(), null, System.currentTimeMillis());
ServerName sn;
try {
sn = getRootRegionServerName();
} catch (InterruptedException e) {
throw new IOException("Interrupted", e);
}
MetaEntry m =
new MetaEntry(rootLocation.getRegionInfo(), sn, System.currentTimeMillis());
HbckInfo hbInfo = new HbckInfo(m);
regionInfo.put(rootLocation.getRegionInfo().getEncodedName(), hbInfo);
return true;
}
private ServerName getRootRegionServerName()
throws IOException, InterruptedException {
RootRegionTracker rootRegionTracker =
new RootRegionTracker(this.connection.getZooKeeperWatcher(), new Abortable() {
@Override
public void abort(String why, Throwable e) {
LOG.error(why, e);
System.exit(1);
}
});
rootRegionTracker.start();
ServerName sn = null;
try {
sn = rootRegionTracker.getRootRegionLocation();
} finally {
rootRegionTracker.stop();
}
return sn;
}
/**
* Contacts each regionserver and fetches metadata about regions.
* @param regionServerList - the list of region servers to connect to
* @throws IOException if a remote or network exception occurs
*/
void processRegionServers(Collection<HServerInfo> regionServerList)
void processRegionServers(Collection<ServerName> regionServerList)
throws IOException, InterruptedException {
WorkItemRegion[] work = new WorkItemRegion[regionServerList.size()];
int num = 0;
// loop to contact each region server in parallel
for (HServerInfo rsinfo:regionServerList) {
for (ServerName rsinfo: regionServerList) {
work[num] = new WorkItemRegion(this, rsinfo, errors, connection);
executor.execute(work[num]);
num++;
@ -478,7 +505,7 @@ public class HBaseFsck {
if (modTInfo == null) {
modTInfo = new TInfo(tableName);
}
for (HServerAddress server : hbi.deployedOn) {
for (ServerName server : hbi.deployedOn) {
modTInfo.addServer(server);
}
modTInfo.addEdge(hbi.metaEntry.getStartKey(), hbi.metaEntry.getEndKey());
@ -498,19 +525,19 @@ public class HBaseFsck {
private class TInfo {
String tableName;
TreeMap <byte[], byte[]> edges;
TreeSet <HServerAddress> deployedOn;
TreeSet <ServerName> deployedOn;
TInfo(String name) {
this.tableName = name;
edges = new TreeMap <byte[], byte[]> (Bytes.BYTES_COMPARATOR);
deployedOn = new TreeSet <HServerAddress>();
deployedOn = new TreeSet <ServerName>();
}
public void addEdge(byte[] fromNode, byte[] toNode) {
this.edges.put(fromNode, toNode);
}
public void addServer(HServerAddress server) {
public void addServer(ServerName server) {
this.deployedOn.add(server);
}
@ -647,7 +674,7 @@ public class HBaseFsck {
errors.print("Trying to fix a problem with .META...");
setShouldRerun();
// try fix it (treat is a dupe assignment)
List <HServerAddress> deployedOn = Lists.newArrayList();
List <ServerName> deployedOn = Lists.newArrayList();
for (HbckInfo mRegion : metaRegions) {
deployedOn.add(mRegion.metaEntry.regionServer);
}
@ -681,35 +708,19 @@ public class HBaseFsck {
// record the latest modification of this META record
long ts = Collections.max(result.list(), comp).getTimestamp();
// record region details
byte [] value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (value == null || value.length == 0) {
Pair<HRegionInfo, ServerName> pair =
MetaReader.metaRowToRegionPair(result);
if (pair == null || pair.getFirst() == null) {
emptyRegionInfoQualifiers.add(result);
return true;
}
HRegionInfo info = Writables.getHRegionInfo(value);
HServerAddress server = null;
byte[] startCode = null;
// record assigned region server
value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER);
if (value != null && value.length > 0) {
String address = Bytes.toString(value);
server = new HServerAddress(address);
ServerName sn = null;
if (pair.getSecond() != null) {
sn = pair.getSecond();
}
// record region's start key
value = result.getValue(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER);
if (value != null) {
startCode = value;
}
MetaEntry m = new MetaEntry(info, server, startCode, ts);
MetaEntry m = new MetaEntry(pair.getFirst(), sn, ts);
HbckInfo hbInfo = new HbckInfo(m);
HbckInfo previous = regionInfo.put(info.getEncodedName(), hbInfo);
HbckInfo previous = regionInfo.put(pair.getFirst().getEncodedName(), hbInfo);
if (previous != null) {
throw new IOException("Two entries in META are same " + previous);
}
@ -740,11 +751,10 @@ public class HBaseFsck {
* Stores the entries scanned from META
*/
private static class MetaEntry extends HRegionInfo {
HServerAddress regionServer; // server hosting this region
ServerName regionServer; // server hosting this region
long modTime; // timestamp of most recent modification metadata
public MetaEntry(HRegionInfo rinfo, HServerAddress regionServer,
byte[] startCode, long modTime) {
public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime) {
super(rinfo);
this.regionServer = regionServer;
this.modTime = modTime;
@ -758,13 +768,13 @@ public class HBaseFsck {
boolean onlyEdits = false;
MetaEntry metaEntry = null;
FileStatus foundRegionDir = null;
List<HServerAddress> deployedOn = Lists.newArrayList();
List<ServerName> deployedOn = Lists.newArrayList();
HbckInfo(MetaEntry metaEntry) {
this.metaEntry = metaEntry;
}
public synchronized void addServer(HServerAddress server) {
public synchronized void addServer(ServerName server) {
this.deployedOn.add(server);
}
@ -792,7 +802,7 @@ public class HBaseFsck {
}
System.out.println(" Number of regions: " + tInfo.getNumRegions());
System.out.print(" Deployed on: ");
for (HServerAddress server : tInfo.deployedOn) {
for (ServerName server : tInfo.deployedOn) {
System.out.print(" " + server.toString());
}
System.out.println();
@ -865,12 +875,12 @@ public class HBaseFsck {
*/
static class WorkItemRegion implements Runnable {
private HBaseFsck hbck;
private HServerInfo rsinfo;
private ServerName rsinfo;
private ErrorReporter errors;
private HConnection connection;
private boolean done;
WorkItemRegion(HBaseFsck hbck, HServerInfo info,
WorkItemRegion(HBaseFsck hbck, ServerName info,
ErrorReporter errors, HConnection connection) {
this.hbck = hbck;
this.rsinfo = info;
@ -888,8 +898,7 @@ public class HBaseFsck {
public synchronized void run() {
errors.progress();
try {
HRegionInterface server = connection.getHRegionConnection(
rsinfo.getServerAddress());
HRegionInterface server = connection.getHRegionConnection(new HServerAddress(rsinfo.getHostname(), rsinfo.getPort()));
// list all online regions from this region server
List<HRegionInfo> regions = server.getOnlineRegions();
@ -908,7 +917,7 @@ public class HBaseFsck {
// check to see if the existance of this region matches the region in META
for (HRegionInfo r:regions) {
HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
hbi.addServer(rsinfo.getServerAddress());
hbi.addServer(rsinfo);
}
} catch (IOException e) { // unable to connect to the region server.
errors.reportError("RegionServer: " + rsinfo.getServerName() +

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
@ -47,13 +48,13 @@ public class HBaseFsckRepair {
* @throws InterruptedException
*/
public static void fixDupeAssignment(Configuration conf, HRegionInfo region,
List<HServerAddress> servers)
List<ServerName> servers)
throws IOException, KeeperException, InterruptedException {
HRegionInfo actualRegion = new HRegionInfo(region);
// Close region on the servers silently
for(HServerAddress server : servers) {
for(ServerName server : servers) {
closeRegionSilentlyAndWait(conf, server, actualRegion);
}
@ -82,14 +83,14 @@ public class HBaseFsckRepair {
throws ZooKeeperConnectionException, KeeperException, IOException {
ZKAssign.createOrForceNodeOffline(
HConnectionManager.getConnection(conf).getZooKeeperWatcher(),
region, HConstants.HBCK_CODE_NAME);
region, HConstants.HBCK_CODE_SERVERNAME);
}
private static void closeRegionSilentlyAndWait(Configuration conf,
HServerAddress server, HRegionInfo region)
ServerName server, HRegionInfo region)
throws IOException, InterruptedException {
HRegionInterface rs =
HConnectionManager.getConnection(conf).getHRegionConnection(server);
HConnectionManager.getConnection(conf).getHRegionConnection(new HServerAddress(server.getHostname(), server.getPort()));
rs.closeRegion(region, false);
long timeout = conf.getLong("hbase.hbck.close.timeout", 120000);
long expiration = timeout + System.currentTimeMillis();

View File

@ -189,7 +189,7 @@ public class JVMClusterUtil {
while (true) {
for (JVMClusterUtil.MasterThread t : masters) {
if (t.master.isActiveMaster()) {
return t.master.getMasterAddress().toString();
return t.master.getServerName().toString();
}
}
try {

View File

@ -19,14 +19,18 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import java.util.TreeSet;
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.HServerInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData;
import org.apache.zookeeper.KeeperException;
/**
@ -41,7 +45,7 @@ import org.apache.zookeeper.KeeperException;
*/
public class RegionServerTracker extends ZooKeeperListener {
private static final Log LOG = LogFactory.getLog(RegionServerTracker.class);
private NavigableSet<ServerName> regionServers = new TreeSet<ServerName>();
private ServerManager serverManager;
private Abortable abortable;
@ -58,10 +62,29 @@ public class RegionServerTracker extends ZooKeeperListener {
* <p>All RSs will be tracked after this method is called.
*
* @throws KeeperException
* @throws IOException
*/
public void start() throws KeeperException {
public void start() throws KeeperException, IOException {
watcher.registerListener(this);
List<NodeAndData> servers =
ZKUtil.watchAndGetNewChildren(watcher, watcher.rsZNode);
add(servers);
}
private void add(final List<NodeAndData> servers) throws IOException {
synchronized(this.regionServers) {
this.regionServers.clear();
for (NodeAndData n: servers) {
ServerName sn = new ServerName(ZKUtil.getNodeName(n.getNode()));
this.regionServers.add(sn);
}
}
}
private void remove(final ServerName sn) {
synchronized(this.regionServers) {
this.regionServers.remove(sn);
}
}
@Override
@ -70,12 +93,13 @@ public class RegionServerTracker extends ZooKeeperListener {
String serverName = ZKUtil.getNodeName(path);
LOG.info("RegionServer ephemeral node deleted, processing expiration [" +
serverName + "]");
HServerInfo hsi = serverManager.getServerInfo(serverName);
if(hsi == null) {
LOG.info("No HServerInfo found for " + serverName);
ServerName sn = new ServerName(serverName);
if (!serverManager.isServerOnline(sn)) {
LOG.info(serverName.toString() + " is not online");
return;
}
serverManager.expireServer(hsi);
remove(sn);
this.serverManager.expireServer(sn);
}
}
@ -83,7 +107,11 @@ public class RegionServerTracker extends ZooKeeperListener {
public void nodeChildrenChanged(String path) {
if (path.equals(watcher.rsZNode)) {
try {
List<NodeAndData> servers =
ZKUtil.watchAndGetNewChildren(watcher, watcher.rsZNode);
add(servers);
} catch (IOException e) {
abortable.abort("Unexpected zk exception getting RS nodes", e);
} catch (KeeperException e) {
abortable.abort("Unexpected zk exception getting RS nodes", e);
}
@ -92,10 +120,12 @@ public class RegionServerTracker extends ZooKeeperListener {
/**
* Gets the online servers.
* @return list of online servers from zk
* @return list of online servers
* @throws KeeperException
*/
public List<HServerAddress> getOnlineServers() throws KeeperException {
return ZKUtil.listChildrenAndGetAsAddresses(watcher, watcher.rsZNode);
public List<ServerName> getOnlineServers() {
synchronized (this.regionServers) {
return new ArrayList<ServerName>(this.regionServers);
}
}
}

View File

@ -20,9 +20,8 @@
package org.apache.hadoop.hbase.zookeeper;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.RootLocationEditor;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -54,31 +53,34 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
/**
* Gets the root region location, if available. Null if not. Does not block.
* @return server address for server hosting root region, null if none available
* @return server name
* @throws InterruptedException
*/
public HServerAddress getRootRegionLocation() throws InterruptedException {
return dataToHServerAddress(super.getData());
public ServerName getRootRegionLocation() throws InterruptedException {
byte [] data = super.getData();
return data == null? null: new ServerName(dataToString(data));
}
/**
* 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
* @return server address for server hosting root region, null if timed out
* @return server name for server hosting root region formatted as per
* {@link ServerName}, or null if none available
* @throws InterruptedException if interrupted while waiting
*/
public HServerAddress waitRootRegionLocation(long timeout)
public ServerName waitRootRegionLocation(long timeout)
throws InterruptedException {
return dataToHServerAddress(super.blockUntilAvailable(timeout));
String str = dataToString(super.blockUntilAvailable(timeout));
return str == null? null: new ServerName(str);
}
/*
* @param data
* @return Returns null if <code>data</code> is null else converts passed data
* to an HServerAddress instance.
* to a String instance.
*/
private static HServerAddress dataToHServerAddress(final byte [] data) {
return data == null ? null: new HServerAddress(Bytes.toString(data));
private static String dataToString(final byte [] data) {
return data == null ? null: Bytes.toString(data);
}
}

View File

@ -24,6 +24,7 @@ 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.ServerName;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.zookeeper.AsyncCallback;
@ -130,13 +131,13 @@ public class ZKAssign {
* @throws KeeperException.NodeExistsException if node already exists
*/
public static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
String serverName)
ServerName serverName)
throws KeeperException, KeeperException.NodeExistsException {
createNodeOffline(zkw, region, serverName, EventType.M_ZK_REGION_OFFLINE);
}
public static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
String serverName, final EventType event)
ServerName serverName, final EventType event)
throws KeeperException, KeeperException.NodeExistsException {
LOG.debug(zkw.prefix("Creating unassigned node for " +
region.getEncodedName() + " in OFFLINE state"));
@ -165,7 +166,7 @@ public class ZKAssign {
* @throws KeeperException.NodeExistsException if node already exists
*/
public static void asyncCreateNodeOffline(ZooKeeperWatcher zkw,
HRegionInfo region, String serverName,
HRegionInfo region, ServerName serverName,
final AsyncCallback.StringCallback cb, final Object ctx)
throws KeeperException {
LOG.debug(zkw.prefix("Async create of unassigned node for " +
@ -198,7 +199,7 @@ public class ZKAssign {
* @throws KeeperException.NoNodeException if node does not exist
*/
public static void forceNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
String serverName)
ServerName serverName)
throws KeeperException, KeeperException.NoNodeException {
LOG.debug(zkw.prefix("Forcing existing unassigned node for " +
region.getEncodedName() + " to OFFLINE state"));
@ -231,7 +232,7 @@ public class ZKAssign {
* @throws KeeperException.NodeExistsException if node already exists
*/
public static boolean createOrForceNodeOffline(ZooKeeperWatcher zkw,
HRegionInfo region, String serverName)
HRegionInfo region, ServerName serverName)
throws KeeperException {
LOG.debug(zkw.prefix("Creating (or updating) unassigned node for " +
region.getEncodedName() + " with OFFLINE state"));
@ -464,7 +465,7 @@ public class ZKAssign {
* @throws KeeperException.NodeExistsException if node already exists
*/
public static int createNodeClosing(ZooKeeperWatcher zkw, HRegionInfo region,
String serverName)
ServerName serverName)
throws KeeperException, KeeperException.NodeExistsException {
LOG.debug(zkw.prefix("Creating unassigned node for " +
region.getEncodedName() + " in a CLOSING state"));
@ -506,7 +507,7 @@ public class ZKAssign {
* @throws KeeperException if unexpected zookeeper exception
*/
public static int transitionNodeClosed(ZooKeeperWatcher zkw,
HRegionInfo region, String serverName, int expectedVersion)
HRegionInfo region, ServerName serverName, int expectedVersion)
throws KeeperException {
return transitionNode(zkw, region, serverName,
EventType.RS_ZK_REGION_CLOSING,
@ -540,14 +541,14 @@ public class ZKAssign {
* @throws KeeperException if unexpected zookeeper exception
*/
public static int transitionNodeOpening(ZooKeeperWatcher zkw,
HRegionInfo region, String serverName)
HRegionInfo region, ServerName serverName)
throws KeeperException {
return transitionNodeOpening(zkw, region, serverName,
EventType.M_ZK_REGION_OFFLINE);
}
public static int transitionNodeOpening(ZooKeeperWatcher zkw,
HRegionInfo region, String serverName, final EventType beginState)
HRegionInfo region, ServerName serverName, final EventType beginState)
throws KeeperException {
return transitionNode(zkw, region, serverName, beginState,
EventType.RS_ZK_REGION_OPENING, -1);
@ -580,7 +581,7 @@ public class ZKAssign {
* @throws KeeperException if unexpected zookeeper exception
*/
public static int retransitionNodeOpening(ZooKeeperWatcher zkw,
HRegionInfo region, String serverName, int expectedVersion)
HRegionInfo region, ServerName serverName, int expectedVersion)
throws KeeperException {
return transitionNode(zkw, region, serverName,
EventType.RS_ZK_REGION_OPENING,
@ -616,7 +617,7 @@ public class ZKAssign {
* @throws KeeperException if unexpected zookeeper exception
*/
public static int transitionNodeOpened(ZooKeeperWatcher zkw,
HRegionInfo region, String serverName, int expectedVersion)
HRegionInfo region, ServerName serverName, int expectedVersion)
throws KeeperException {
return transitionNode(zkw, region, serverName,
EventType.RS_ZK_REGION_OPENING,
@ -652,7 +653,7 @@ public class ZKAssign {
* @throws KeeperException if unexpected zookeeper exception
*/
public static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region,
String serverName, EventType beginState, EventType endState,
ServerName serverName, EventType beginState, EventType endState,
int expectedVersion)
throws KeeperException {
return transitionNode(zkw, region, serverName, beginState, endState,
@ -660,7 +661,7 @@ public class ZKAssign {
}
public static int transitionNode(ZooKeeperWatcher zkw, HRegionInfo region,
String serverName, EventType beginState, EventType endState,
ServerName serverName, EventType beginState, EventType endState,
int expectedVersion, final byte [] payload)
throws KeeperException {
String encoded = region.getEncodedName();
@ -699,7 +700,7 @@ public class ZKAssign {
"unassigned node for " + encoded +
" from " + beginState + " to " + endState + " failed, " +
"the node existed but was in the state " + existingData.getEventType() +
" set by the server " + existingData.getServerName()));
" set by the server " + serverName));
return -1;
}

View File

@ -33,8 +33,6 @@ 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.RegionTransitionData;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.zookeeper.AsyncCallback;
@ -125,19 +123,6 @@ public class ZKUtil {
return idx <= 0 ? null : node.substring(0, idx);
}
/**
* Get the unique node-name for the specified regionserver.
*
* Used when a server puts up an ephemeral node for itself and needs to use
* a unique name.
*
* @param serverInfo server information
* @return unique, zookeeper-safe znode path for the server instance
*/
public static String getNodeName(HServerInfo serverInfo) {
return serverInfo.getServerName();
}
/**
* Get the name of the current node from the specified fully-qualified path.
* @param path fully-qualified path
@ -332,38 +317,6 @@ public class ZKUtil {
return children;
}
/**
* Lists the children of the specified znode, retrieving the data of each
* child as a server address.
*
* Used to list the currently online regionservers and their addresses.
*
* Sets no watches at all, this method is best effort.
*
* Returns an empty list if the node has no children. Returns null if the
* parent node itself does not exist.
*
* @param zkw zookeeper reference
* @param znode node to get children of as addresses
* @return list of data of children of specified znode, empty if no children,
* null if parent does not exist
* @throws KeeperException if unexpected zookeeper exception
*/
public static List<HServerAddress> listChildrenAndGetAsAddresses(
ZooKeeperWatcher zkw, String znode)
throws KeeperException {
List<String> children = listChildrenNoWatch(zkw, znode);
if(children == null) {
return null;
}
List<HServerAddress> addresses =
new ArrayList<HServerAddress>(children.size());
for(String child : children) {
addresses.add(getDataAsAddress(zkw, joinZNode(znode, child)));
}
return addresses;
}
/**
* Lists the children of the specified znode without setting any watches.
*
@ -601,32 +554,6 @@ public class ZKUtil {
}
}
/**
* Get the data at the specified znode, deserialize it as an HServerAddress,
* and set a watch.
*
* Returns the data as a server address and sets a watch if the node exists.
* Returns null and no watch is set if the node does not exist or there is an
* exception.
*
* @param zkw zk reference
* @param znode path of node
* @return data of the specified node as a server address, or null
* @throws KeeperException if unexpected zookeeper exception
*/
public static HServerAddress getDataAsAddress(ZooKeeperWatcher zkw,
String znode)
throws KeeperException {
byte [] data = getDataAndWatch(zkw, znode);
if(data == null) {
return null;
}
String addrString = Bytes.toString(data);
LOG.debug(zkw.prefix("Read server address from znode " + znode + ": " +
addrString));
return new HServerAddress(addrString);
}
/**
* Update the data of an existing node with the expected version to have the
* specified data.
@ -656,31 +583,6 @@ public class ZKUtil {
// Data setting
//
/**
* Set the specified znode to be an ephemeral node carrying the specified
* server address. Used by masters for their ephemeral node and regionservers
* for their ephemeral node.
*
* If the node is created successfully, a watcher is also set on the node.
*
* If the node is not created successfully because it already exists, this
* method will also set a watcher on the node.
*
* If there is another problem, a KeeperException will be thrown.
*
* @param zkw zk reference
* @param znode path of node
* @param address server address
* @return true if address set, false if not, watch set in both cases
* @throws KeeperException if unexpected zookeeper exception
*/
public static boolean setAddressAndWatch(ZooKeeperWatcher zkw,
String znode, HServerAddress address)
throws KeeperException {
return createEphemeralNodeAndWatch(zkw, znode,
Bytes.toBytes(address.toString()));
}
/**
* Sets the data of the existing znode to be the specified data. Ensures that
* the current data has the specified expected version.
@ -745,8 +647,7 @@ public class ZKUtil {
* @param data data to set for node
* @throws KeeperException if unexpected zookeeper exception
*/
public static void setData(ZooKeeperWatcher zkw, String znode,
byte [] data)
public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data)
throws KeeperException, KeeperException.NoNodeException {
setData(zkw, znode, data, -1);
}
@ -1024,12 +925,11 @@ public class ZKUtil {
public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node)
throws KeeperException {
List<String> children = ZKUtil.listChildrenNoWatch(zkw, node);
if(children != null || !children.isEmpty()) {
if (children == null || children.isEmpty()) return;
for(String child : children) {
deleteNodeRecursively(zkw, joinZNode(node, child));
}
}
}
//
// ZooKeeper cluster information
@ -1041,13 +941,12 @@ public class ZKUtil {
try {
sb.append("HBase is rooted at ").append(zkw.baseZNode);
sb.append("\nMaster address: ").append(
getDataAsAddress(zkw, zkw.masterAddressZNode));
Bytes.toStringBinary(getData(zkw, zkw.masterAddressZNode)));
sb.append("\nRegion server holding ROOT: ").append(
getDataAsAddress(zkw, zkw.rootServerZNode));
Bytes.toStringBinary(getData(zkw, zkw.rootServerZNode)));
sb.append("\nRegion servers:");
for (HServerAddress address : listChildrenAndGetAsAddresses(zkw,
zkw.rsZNode)) {
sb.append("\n ").append(address);
for (String child: listChildrenNoWatch(zkw, zkw.rsZNode)) {
sb.append("\n ").append(child);
}
sb.append("\nQuorum Server Statistics:");
String[] servers = zkw.getQuorum().split(",");

View File

@ -7,17 +7,15 @@
import="org.apache.hadoop.hbase.util.FSUtils"
import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.HConstants"
import="org.apache.hadoop.hbase.ServerName"
import="org.apache.hadoop.hbase.client.HBaseAdmin"
import="org.apache.hadoop.hbase.client.HConnectionManager"
import="org.apache.hadoop.hbase.HServerInfo"
import="org.apache.hadoop.hbase.HServerAddress"
import="org.apache.hadoop.hbase.HTableDescriptor" %><%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
Configuration conf = master.getConfiguration();
HServerAddress rootLocation = master.getCatalogTracker().getRootLocation();
ServerName rootLocation = master.getCatalogTracker().getRootLocation();
boolean metaOnline = master.getCatalogTracker().getMetaLocation() != null;
Map<String, HServerInfo> serverToServerInfos =
master.getServerManager().getOnlineServers();
List<ServerName> servers = master.getServerManager().getOnlineServersList();
int interval = conf.getInt("hbase.regionserver.msginterval", 1000)/1000;
if (interval == 0) {
interval = 1;
@ -32,12 +30,12 @@
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
<html xmlns="http://www.w3.org/1999/xhtml">
<head><meta http-equiv="Content-Type" content="text/html;charset=UTF-8"/>
<title>HBase Master: <%= master.getMasterAddress().getHostname()%>:<%= master.getMasterAddress().getPort() %></title>
<title>HBase Master: <%= master.getServerName().getHostAndPort() %></title>
<link rel="stylesheet" type="text/css" href="/static/hbase.css" />
</head>
<body>
<a id="logo" href="http://wiki.apache.org/lucene-hadoop/Hbase"><img src="/static/hbase_logo_med.gif" alt="HBase Logo" title="HBase Logo" /></a>
<h1 id="page_title">Master: <%=master.getMasterAddress().getHostname()%>:<%=master.getMasterAddress().getPort()%></h1>
<h1 id="page_title">Master: <%=master.getServerName().getHostname()%>:<%=master.getServerName().getPort()%></h1>
<p id="links_menu"><a href="/logs/">Local logs</a>, <a href="/stacks">Thread Dump</a>, <a href="/logLevel">Log Level</a></p>
<!-- Various warnings that cluster admins should be aware of -->
@ -137,26 +135,27 @@
<% } %>
<h2>Region Servers</h2>
<% if (serverToServerInfos != null && serverToServerInfos.size() > 0) { %>
<% if (servers != null && servers.size() > 0) { %>
<% int totalRegions = 0;
int totalRequests = 0;
%>
<table>
<tr><th rowspan="<%= serverToServerInfos.size() + 1%>"></th><th>Address</th><th>Start Code</th><th>Load</th></tr>
<% String[] serverNames = serverToServerInfos.keySet().toArray(new String[serverToServerInfos.size()]);
<tr><th rowspan="<%= servers.size() + 1%>"></th><th>Address</th><th>Start Code</th><th>Load</th></tr>
<% ServerName [] serverNames = servers.toArray(new ServerName[servers.size()]);
Arrays.sort(serverNames);
for (String serverName: serverNames) {
HServerInfo hsi = serverToServerInfos.get(serverName);
String hostname = hsi.getServerAddress().getHostname() + ":" + hsi.getInfoPort();
for (ServerName serverName: serverNames) {
// HARDCODED FOR NOW; FIX -- READ FROM ZK
String hostname = serverName.getHostname() + ":60020";
String url = "http://" + hostname + "/";
totalRegions += hsi.getLoad().getNumberOfRegions();
totalRequests += hsi.getLoad().getNumberOfRequests() / interval;
long startCode = hsi.getStartCode();
// TODO: FIX
totalRegions += 0;
totalRequests += 0;
long startCode = serverName.getStartcode();
%>
<tr><td><a href="<%= url %>"><%= hostname %></a></td><td><%= startCode %></td><td><%= hsi.getLoad().toString(interval) %></td></tr>
<tr><td><a href="<%= url %>"><%= hostname %></a></td><td><%= startCode %></td><td><%= 0 %></td></tr>
<% } %>
<tr><th>Total: </th><td>servers: <%= serverToServerInfos.size() %></td><td>&nbsp;</td><td>requests=<%= totalRequests %>, regions=<%= totalRegions %></td></tr>
<tr><th>Total: </th><td>servers: <%= servers.size() %></td><td>&nbsp;</td><td>requests=<%= totalRequests %>, regions=<%= totalRegions %></td></tr>
</table>
<p>Load is requests per second and count of regions loaded</p>

View File

@ -6,6 +6,7 @@
import="org.apache.hadoop.hbase.client.HBaseAdmin"
import="org.apache.hadoop.hbase.client.HConnectionManager"
import="org.apache.hadoop.hbase.HRegionInfo"
import="org.apache.hadoop.hbase.ServerName"
import="org.apache.hadoop.hbase.HServerAddress"
import="org.apache.hadoop.hbase.HServerInfo"
import="org.apache.hadoop.hbase.HServerLoad"
@ -22,7 +23,7 @@
String tableName = request.getParameter("name");
HTable table = new HTable(conf, tableName);
String tableHeader = "<h2>Table Regions</h2><table><tr><th>Name</th><th>Region Server</th><th>Start Key</th><th>End Key</th><th>Requests</th></tr>";
HServerAddress rl = master.getCatalogTracker().getRootLocation();
ServerName rl = master.getCatalogTracker().getRootLocation();
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
Map<String, Integer> frags = null;
if (showFragmentation) {
@ -83,8 +84,9 @@
if(tableName.equals(Bytes.toString(HConstants.ROOT_TABLE_NAME))) {
%>
<%= tableHeader %>
// HARDCODED FOR NOW TODO: FIX GET FROM ZK
<%
int infoPort = master.getServerManager().getHServerInfo(rl).getInfoPort();
int infoPort = 60020; // HARDCODED FOR NOW -- TODO FIX
String url = "http://" + rl.getHostname() + ":" + infoPort + "/";
%>
<tr>
@ -102,9 +104,9 @@
<%
// NOTE: Presumes one meta region only.
HRegionInfo meta = HRegionInfo.FIRST_META_REGIONINFO;
HServerAddress metaLocation = master.getCatalogTracker().getMetaLocation();
ServerName metaLocation = master.getCatalogTracker().getMetaLocation();
for (int i = 0; i < 1; i++) {
int infoPort = master.getServerManager().getHServerInfo(metaLocation).getInfoPort();
int infoPort = 60020; // HARDCODED FOR NOW -- TODO FIX
String url = "http://" + metaLocation.getHostname() + ":" + infoPort + "/";
%>
<tr>
@ -150,16 +152,16 @@
String urlRegionServer = null;
if (addr != null) {
HServerInfo info = master.getServerManager().getHServerInfo(addr);
if (info != null) {
HServerLoad sl = info.getLoad();
HServerLoad sl = master.getServerManager().getLoad(addr);
if (sl != null) {
Map<byte[], RegionLoad> map = sl.getRegionsLoad();
if (map.containsKey(regionInfo.getRegionName())) {
req = map.get(regionInfo.getRegionName()).getRequestsCount();
}
infoPort = info.getInfoPort();
// This port might be wrong if RS actually ended up using something else.
int port = conf.getInt("hbase.regionserver.info.port", 60030);
urlRegionServer =
"http://" + addr.getHostname().toString() + ":" + infoPort + "/";
"http://" + addr.getHostname().toString() + ":" + port + "/";
Integer i = regDistribution.get(urlRegionServer);
if (null == i) i = new Integer(0);
regDistribution.put(urlRegionServer, i+1);

View File

@ -23,8 +23,6 @@ import java.io.IOException;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -39,7 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.MapWritable;
import org.apache.zookeeper.KeeperException;
/**
* This class creates a single process HBase cluster.
@ -83,75 +80,6 @@ public class MiniHBaseCluster {
return this.conf;
}
/**
* Override Master so can add inject behaviors testing.
*/
public static class MiniHBaseClusterMaster extends HMaster {
private final Map<HServerInfo, List<HMsg>> messages =
new ConcurrentHashMap<HServerInfo, List<HMsg>>();
private final Map<HServerInfo, IOException> exceptions =
new ConcurrentHashMap<HServerInfo, IOException>();
public MiniHBaseClusterMaster(final Configuration conf)
throws IOException, KeeperException, InterruptedException {
super(conf);
}
/**
* Add a message to send to a regionserver next time it checks in.
* @param hsi RegionServer's HServerInfo.
* @param msg Message to add.
*/
void addMessage(final HServerInfo hsi, HMsg msg) {
synchronized(this.messages) {
List<HMsg> hmsgs = this.messages.get(hsi);
if (hmsgs == null) {
hmsgs = new ArrayList<HMsg>();
this.messages.put(hsi, hmsgs);
}
hmsgs.add(msg);
}
}
void addException(final HServerInfo hsi, final IOException ex) {
this.exceptions.put(hsi, ex);
}
/**
* This implementation is special, exceptions will be treated first and
* message won't be sent back to the region servers even if some are
* specified.
* @param hsi the rs
* @param msgs Messages to add to
* @return
* @throws IOException will be throw if any added for this region server
*/
@Override
protected HMsg[] adornRegionServerAnswer(final HServerInfo hsi,
final HMsg[] msgs) throws IOException {
IOException ex = this.exceptions.remove(hsi);
if (ex != null) {
throw ex;
}
HMsg [] answerMsgs = msgs;
synchronized (this.messages) {
List<HMsg> hmsgs = this.messages.get(hsi);
if (hmsgs != null && !hmsgs.isEmpty()) {
int size = answerMsgs.length;
HMsg [] newAnswerMsgs = new HMsg[size + hmsgs.size()];
System.arraycopy(answerMsgs, 0, newAnswerMsgs, 0, answerMsgs.length);
for (int i = 0; i < hmsgs.size(); i++) {
newAnswerMsgs[answerMsgs.length + i] = hmsgs.get(i);
}
answerMsgs = newAnswerMsgs;
hmsgs.clear();
}
}
return super.adornRegionServerAnswer(hsi, answerMsgs);
}
}
/**
* Subclass so can get at protected methods (none at moment). Also, creates
* a FileSystem instance per instantiation. Adds a shutdown own FileSystem
@ -176,10 +104,6 @@ public class MiniHBaseCluster {
return super.closeRegion(region);
}
public void setHServerInfo(final HServerInfo hsi) {
this.serverInfo = hsi;
}
/*
* @param c
* @param currentfs We return this if we did not make a new one.
@ -266,8 +190,7 @@ public class MiniHBaseCluster {
try {
// start up a LocalHBaseCluster
hbaseCluster = new LocalHBaseCluster(conf, nMasterNodes, 0,
MiniHBaseCluster.MiniHBaseClusterMaster.class,
MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
HMaster.class, MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
// manually add the regionservers as other users
for (int i=0; i<nRegionNodes; i++) {
@ -384,15 +307,6 @@ public class MiniHBaseCluster {
return t;
}
/**
* @return Returns the rpc address actually used by the currently active
* master server, because the supplied port is not necessarily the actual port
* used.
*/
public HServerAddress getHMasterAddress() {
return this.hbaseCluster.getActiveMaster().getMasterAddress();
}
/**
* Returns the current active master, if available.
* @return the active HMaster, null if none is active.
@ -606,59 +520,6 @@ public class MiniHBaseCluster {
return index;
}
/**
* Add an exception to send when a region server checks back in
* @param serverNumber Which server to send it to
* @param ex The exception that will be sent
* @throws IOException
*/
public void addExceptionToSendRegionServer(final int serverNumber,
IOException ex) throws IOException {
MiniHBaseClusterRegionServer hrs =
(MiniHBaseClusterRegionServer)getRegionServer(serverNumber);
addExceptionToSendRegionServer(hrs, ex);
}
/**
* Add an exception to send when a region server checks back in
* @param hrs Which server to send it to
* @param ex The exception that will be sent
* @throws IOException
*/
public void addExceptionToSendRegionServer(
final MiniHBaseClusterRegionServer hrs, IOException ex)
throws IOException {
((MiniHBaseClusterMaster)getMaster()).addException(hrs.getHServerInfo(),ex);
}
/**
* Add a message to include in the responses send a regionserver when it
* checks back in.
* @param serverNumber Which server to send it to.
* @param msg The MESSAGE
* @throws IOException
*/
public void addMessageToSendRegionServer(final int serverNumber,
final HMsg msg)
throws IOException {
MiniHBaseClusterRegionServer hrs =
(MiniHBaseClusterRegionServer)getRegionServer(serverNumber);
addMessageToSendRegionServer(hrs, msg);
}
/**
* Add a message to include in the responses send a regionserver when it
* checks back in.
* @param hrs Which region server.
* @param msg The MESSAGE
* @throws IOException
*/
public void addMessageToSendRegionServer(final MiniHBaseClusterRegionServer hrs,
final HMsg msg)
throws IOException {
((MiniHBaseClusterMaster)getMaster()).addMessage(hrs.getHServerInfo(), msg);
}
/**
* Counts the total numbers of regions being served by the currently online
* region servers by asking each how many regions they have. Does not look

View File

@ -0,0 +1,67 @@
/**
* Copyright 2011 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertTrue;
import org.junit.Test;
public class TestHRegionLocation {
@Test
public void testHashAndEqualsCode() {
ServerName hsa1 = new ServerName("localhost", 1234, -1L);
HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
hsa1.getHostname(), hsa1.getPort());
HRegionLocation hrl2 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
hsa1.getHostname(), hsa1.getPort());
assertEquals(hrl1.hashCode(), hrl2.hashCode());
assertTrue(hrl1.equals(hrl2));
HRegionLocation hrl3 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
hsa1.getHostname(), hsa1.getPort());
assertNotSame(hrl1, hrl3);
assertFalse(hrl1.equals(hrl3));
}
@Test
public void testToString() {
ServerName hsa1 = new ServerName("localhost", 1234, -1L);
HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
hsa1.getHostname(), hsa1.getPort());
System.out.println(hrl1.toString());
}
@Test
public void testCompareTo() {
ServerName hsa1 = new ServerName("localhost", 1234, -1L);
HRegionLocation hsl1 =
new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa1.getHostname(), hsa1.getPort());
ServerName hsa2 = new ServerName("localhost", 1235, -1L);
HRegionLocation hsl2 =
new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa2.getHostname(), hsa2.getPort());
assertTrue(hsl1.compareTo(hsl1) == 0);
assertTrue(hsl2.compareTo(hsl2) == 0);
int compare1 = hsl1.compareTo(hsl2);
int compare2 = hsl2.compareTo(hsl1);
assertTrue((compare1 > 0)? compare2 < 0: compare2 > 0);
}
}

View File

@ -0,0 +1,83 @@
/**
* Copyright 2011 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 static org.junit.Assert.*;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.hbase.util.Writables;
import org.junit.Test;
/**
* Tests for {@link HServerAddress}
*/
public class TestHServerAddress {
@Test
public void testHashCode() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerAddress hsa2 = new HServerAddress("localhost", 1234);
assertEquals(hsa1.hashCode(), hsa2.hashCode());
HServerAddress hsa3 = new HServerAddress("localhost", 1235);
assertNotSame(hsa1.hashCode(), hsa3.hashCode());
}
@Test
public void testHServerAddress() {
new HServerAddress();
}
@Test
public void testHServerAddressInetSocketAddress() {
HServerAddress hsa1 =
new HServerAddress(new InetSocketAddress("localhost", 1234));
System.out.println(hsa1.toString());
}
@Test
public void testHServerAddressString() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerAddress hsa2 =
new HServerAddress(new InetSocketAddress("localhost", 1234));
assertTrue(hsa1.equals(hsa2));
}
@Test
public void testHServerAddressHServerAddress() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerAddress hsa2 = new HServerAddress(hsa1);
assertEquals(hsa1, hsa2);
}
@Test
public void testReadFields() throws IOException {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerAddress hsa2 = new HServerAddress("localhost", 1235);
byte [] bytes = Writables.getBytes(hsa1);
HServerAddress deserialized =
(HServerAddress)Writables.getWritable(bytes, new HServerAddress());
assertEquals(hsa1, deserialized);
bytes = Writables.getBytes(hsa2);
deserialized =
(HServerAddress)Writables.getWritable(bytes, new HServerAddress());
assertNotSame(hsa1, deserialized);
}
}

View File

@ -0,0 +1,80 @@
package org.apache.hadoop.hbase;
import static org.junit.Assert.*;
import java.io.IOException;
import org.apache.hadoop.hbase.util.Writables;
import org.junit.Test;
public class TestHServerInfo {
@Test
public void testHashCodeAndEquals() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerInfo hsi1 = new HServerInfo(hsa1, 1L, 5678);
HServerInfo hsi2 = new HServerInfo(hsa1, 1L, 5678);
HServerInfo hsi3 = new HServerInfo(hsa1, 2L, 5678);
HServerInfo hsi4 = new HServerInfo(hsa1, 1L, 5677);
HServerAddress hsa2 = new HServerAddress("localhost", 1235);
HServerInfo hsi5 = new HServerInfo(hsa2, 1L, 5678);
assertEquals(hsi1.hashCode(), hsi2.hashCode());
assertTrue(hsi1.equals(hsi2));
assertNotSame(hsi1.hashCode(), hsi3.hashCode());
assertFalse(hsi1.equals(hsi3));
assertNotSame(hsi1.hashCode(), hsi4.hashCode());
assertFalse(hsi1.equals(hsi4));
assertNotSame(hsi1.hashCode(), hsi5.hashCode());
assertFalse(hsi1.equals(hsi5));
}
@Test
public void testHServerInfoHServerInfo() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerInfo hsi1 = new HServerInfo(hsa1, 1L, 5678);
HServerInfo hsi2 = new HServerInfo(hsi1);
assertEquals(hsi1, hsi2);
}
@Test
public void testGetServerAddress() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerInfo hsi1 = new HServerInfo(hsa1, 1L, 5678);
assertEquals(hsi1.getServerAddress(), hsa1);
}
@Test
public void testToString() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerInfo hsi1 = new HServerInfo(hsa1, 1L, 5678);
System.out.println(hsi1.toString());
}
@Test
public void testReadFields() throws IOException {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerInfo hsi1 = new HServerInfo(hsa1, 1L, 5678);
HServerAddress hsa2 = new HServerAddress("localhost", 1235);
HServerInfo hsi2 = new HServerInfo(hsa2, 1L, 5678);
byte [] bytes = Writables.getBytes(hsi1);
HServerInfo deserialized =
(HServerInfo)Writables.getWritable(bytes, new HServerInfo());
assertEquals(hsi1, deserialized);
bytes = Writables.getBytes(hsi2);
deserialized = (HServerInfo)Writables.getWritable(bytes, new HServerInfo());
assertNotSame(hsa1, deserialized);
}
@Test
public void testCompareTo() {
HServerAddress hsa1 = new HServerAddress("localhost", 1234);
HServerInfo hsi1 = new HServerInfo(hsa1, 1L, 5678);
HServerAddress hsa2 = new HServerAddress("localhost", 1235);
HServerInfo hsi2 = new HServerInfo(hsa2, 1L, 5678);
assertTrue(hsi1.compareTo(hsi1) == 0);
assertTrue(hsi2.compareTo(hsi2) == 0);
int compare1 = hsi1.compareTo(hsi2);
int compare2 = hsi2.compareTo(hsi1);
assertTrue((compare1 > 0)? compare2 < 0: compare2 > 0);
}
}

View File

@ -22,10 +22,15 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -33,7 +38,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
/**
* Test whether region rebalancing works. (HBASE-71)
@ -93,8 +97,16 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
* For HBASE-71. Try a few different configurations of starting and stopping
* region servers to see if the assignment or regions is pretty balanced.
* @throws IOException
* @throws InterruptedException
*/
public void testRebalancing() throws IOException {
public void testRebalancing() throws IOException, InterruptedException {
HConnection connection = HConnectionManager.getConnection(conf);
CatalogTracker ct = new CatalogTracker(connection);
ct.start();
Map<HRegionInfo, ServerName> regions = MetaReader.fullScan(ct);
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
LOG.info(e);
}
table = new HTable(conf, "test");
assertEquals("Test table should have 20 regions",
20, table.getStartKeys().length);
@ -102,39 +114,34 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
// verify that the region assignments are balanced to start out
assertRegionsAreBalanced();
LOG.debug("Adding 2nd region server.");
// add a region server - total of 2
LOG.info("Started=" +
LOG.info("Started second server=" +
cluster.startRegionServer().getRegionServer().getServerName());
cluster.getMaster().balance();
assertRegionsAreBalanced();
// add a region server - total of 3
LOG.debug("Adding 3rd region server.");
LOG.info("Started=" +
LOG.info("Started third server=" +
cluster.startRegionServer().getRegionServer().getServerName());
cluster.getMaster().balance();
assertRegionsAreBalanced();
// kill a region server - total of 2
LOG.debug("Killing the 3rd region server.");
LOG.info("Stopped=" + cluster.stopRegionServer(2, false));
LOG.info("Stopped third server=" + cluster.stopRegionServer(2, false));
cluster.waitOnRegionServer(2);
cluster.getMaster().balance();
assertRegionsAreBalanced();
// start two more region servers - total of 4
LOG.debug("Adding 3rd region server");
LOG.info("Started=" +
LOG.info("Readding third server=" +
cluster.startRegionServer().getRegionServer().getServerName());
LOG.debug("Adding 4th region server");
LOG.info("Started=" +
LOG.info("Added fourth server=" +
cluster.startRegionServer().getRegionServer().getServerName());
cluster.getMaster().balance();
assertRegionsAreBalanced();
for (int i = 0; i < 6; i++){
LOG.debug("Adding " + (i + 5) + "th region server");
LOG.info("Adding " + (i + 5) + "th region server");
cluster.startRegionServer();
}
cluster.getMaster().balance();
@ -169,7 +176,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
int regionCount = getRegionCount();
List<HRegionServer> servers = getOnlineRegionServers();
double avg = cluster.getMaster().getServerManager().getAverageLoad();
double avg = cluster.getMaster().getAverageLoad();
int avgLoadPlusSlop = (int)Math.ceil(avg * (1 + slop));
int avgLoadMinusSlop = (int)Math.floor(avg * (1 - slop)) - 1;
LOG.debug("There are " + servers.size() + " servers and " + regionCount

View File

@ -130,19 +130,6 @@ public class TestSerialization {
HConstants.EMPTY_END_ROW);
}
/**
* Test ServerInfo serialization
* @throws Exception
*/
@Test public void testServerInfo() throws Exception {
HServerInfo hsi = new HServerInfo(new HServerAddress("0.0.0.0:123"), -1,
1245, "default name");
byte [] b = Writables.getBytes(hsi);
HServerInfo deserializedHsi =
(HServerInfo)Writables.getWritable(b, new HServerInfo());
assertTrue(hsi.equals(deserializedHsi));
}
@Test public void testPut() throws Exception{
byte[] row = "row".getBytes();
byte[] fam = "fam".getBytes();

View File

@ -0,0 +1,56 @@
/**
* Copyright 2011 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 static org.junit.Assert.*;
import java.util.HashSet;
import java.util.Set;
import org.junit.Test;
public class TestServerName {
@Test
public void testServerName() {
ServerName sn = new ServerName("www.example.org", 1234, 5678);
ServerName sn2 = new ServerName("www.example.org", 1234, 5678);
ServerName sn3 = new ServerName("www.example.org", 1234, 56789);
assertTrue(sn.equals(sn2));
assertFalse(sn.equals(sn3));
assertEquals(sn.hashCode(), sn2.hashCode());
assertNotSame(sn.hashCode(), sn3.hashCode());
assertEquals(sn.toString(),
ServerName.getServerName("www.example.org", 1234, 5678));
assertEquals(sn.toString(),
ServerName.getServerName("www.example.org:1234", 5678));
assertEquals(sn.toString(),
"www.example.org" + ServerName.SERVERNAME_SEPARATOR +
"1234" + ServerName.SERVERNAME_SEPARATOR + "5678");
}
@Test
public void getServerStartcodeFromServerName() {
ServerName sn = new ServerName("www.example.org", 1234, 5678);
assertEquals(5678,
ServerName.getServerStartcodeFromServerName(sn.toString()));
assertNotSame(5677,
ServerName.getServerStartcodeFromServerName(sn.toString()));
}
}

View File

@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
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.KeyValue;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Result;
@ -63,8 +63,8 @@ import org.mockito.Mockito;
public class TestCatalogTracker {
private static final Log LOG = LogFactory.getLog(TestCatalogTracker.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final HServerAddress HSA =
new HServerAddress("example.org:1234");
private static final ServerName HSA =
new ServerName("example.org", 1234, System.currentTimeMillis());
private ZooKeeperWatcher watcher;
private Abortable abortable;
@ -115,7 +115,7 @@ public class TestCatalogTracker {
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
RootLocationEditor.setRootLocation(this.watcher,
new HServerAddress("example.com:1234"));
new ServerName("example.com", 1234, System.currentTimeMillis()));
} finally {
// Clean out root location or later tests will be confused... they presume
// start fresh in zk.
@ -131,9 +131,9 @@ public class TestCatalogTracker {
@Test public void testInterruptWaitOnMetaAndRoot()
throws IOException, InterruptedException {
final CatalogTracker ct = constructAndStartCatalogTracker();
HServerAddress hsa = ct.getRootLocation();
ServerName hsa = ct.getRootLocation();
Assert.assertNull(hsa);
HServerAddress meta = ct.getMetaLocation();
ServerName meta = ct.getMetaLocation();
Assert.assertNull(meta);
Thread t = new Thread() {
@Override
@ -169,7 +169,7 @@ public class TestCatalogTracker {
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
RootLocationEditor.setRootLocation(this.watcher,
new HServerAddress("example.com:1234"));
new ServerName("example.com", 1234, System.currentTimeMillis()));
Assert.assertFalse(ct.verifyMetaRegionLocation(100));
} finally {
// Clean out root location or later tests will be confused... they presume
@ -200,7 +200,7 @@ public class TestCatalogTracker {
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
RootLocationEditor.setRootLocation(this.watcher,
new HServerAddress("example.com:1234"));
new ServerName("example.com", 1234, System.currentTimeMillis()));
Assert.assertFalse(ct.verifyRootRegionLocation(100));
} finally {
// Clean out root location or later tests will be confused... they presume
@ -232,7 +232,7 @@ public class TestCatalogTracker {
@Test public void testNoTimeoutWaitForRoot()
throws IOException, InterruptedException, KeeperException {
final CatalogTracker ct = constructAndStartCatalogTracker();
HServerAddress hsa = ct.getRootLocation();
ServerName hsa = ct.getRootLocation();
Assert.assertNull(hsa);
// Now test waiting on root location getting set.
@ -246,7 +246,7 @@ public class TestCatalogTracker {
Assert.assertTrue(ct.getRootLocation().equals(hsa));
}
private HServerAddress setRootLocation() throws KeeperException {
private ServerName setRootLocation() throws KeeperException {
RootLocationEditor.setRootLocation(this.watcher, HSA);
return HSA;
}
@ -270,7 +270,7 @@ public class TestCatalogTracker {
thenReturn(mockHRI);
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
HServerAddress hsa = ct.getMetaLocation();
ServerName hsa = ct.getMetaLocation();
Assert.assertNull(hsa);
// Now test waiting on meta location getting set.
@ -300,8 +300,7 @@ public class TestCatalogTracker {
// been assigned.
String node = ct.getMetaNodeTracker().getNode();
ZKUtil.createAndFailSilent(this.watcher, node);
MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO,
new HServerInfo(HSA, -1, "example.com"));
MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, HSA);
ZKUtil.deleteNode(this.watcher, node);
// Join the thread... should exit shortly.
t.join();

View File

@ -23,7 +23,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
@ -53,7 +53,8 @@ public class TestCatalogTrackerOnCluster {
LOG.error("Abort was called on 'bad root location writer'", e);
}
});
HServerAddress nonsense = new HServerAddress("example.org:1234");
ServerName nonsense =
new ServerName("example.org", 1234, System.currentTimeMillis());
RootLocationEditor.setRootLocation(zookeeper, nonsense);
// Bring back up the hbase cluster. See if it can deal with nonsense root
// location.

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -116,7 +116,7 @@ public class TestMetaReaderEditor {
// Test it works getting a region from user table.
List<HRegionInfo> regions = MetaReader.getTableRegions(ct, nameBytes);
assertEquals(regionCount, regions.size());
Pair<HRegionInfo, HServerAddress> pair =
Pair<HRegionInfo, ServerName> pair =
MetaReader.getRegion(ct, regions.get(0).getRegionName());
assertEquals(regions.get(0).getEncodedName(),
pair.getFirst().getEncodedName());

View File

@ -120,6 +120,154 @@ public class TestFromClientSide {
// Nothing to do.
}
/**
* HBASE-2468 use case 1 and 2: region info de/serialization
*/
@Test
public void testRegionCacheDeSerialization() throws Exception {
// 1. test serialization.
LOG.info("Starting testRegionCacheDeSerialization");
final byte[] TABLENAME = Bytes.toBytes("testCachePrewarm2");
final byte[] FAMILY = Bytes.toBytes("family");
Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.createTable(TABLENAME, FAMILY);
// Set up test table:
// Create table:
HTable table = new HTable(conf, TABLENAME);
// Create multiple regions for this table
TEST_UTIL.createMultiRegions(table, FAMILY);
Scan s = new Scan();
ResultScanner scanner = table.getScanner(s);
while (scanner.next() != null) continue;
Path tempPath = new Path(HBaseTestingUtility.getTestDir(), "regions.dat");
final String tempFileName = tempPath.toString();
FileOutputStream fos = new FileOutputStream(tempFileName);
DataOutputStream dos = new DataOutputStream(fos);
// serialize the region info and output to a local file.
table.serializeRegionInfo(dos);
dos.flush();
dos.close();
// read a local file and deserialize the region info from it.
FileInputStream fis = new FileInputStream(tempFileName);
DataInputStream dis = new DataInputStream(fis);
Map<HRegionInfo, HServerAddress> deserRegions =
table.deserializeRegionInfo(dis);
dis.close();
// regions obtained from meta scanner.
Map<HRegionInfo, HServerAddress> loadedRegions =
table.getRegionsInfo();
// set the deserialized regions to the global cache.
table.getConnection().clearRegionCache();
table.getConnection().prewarmRegionCache(table.getTableName(),
deserRegions);
// verify whether the 2 maps are identical or not.
assertEquals("Number of cached region is incorrect",
HConnectionManager.getCachedRegionCount(conf, TABLENAME),
loadedRegions.size());
// verify each region is prefetched or not.
for (Map.Entry<HRegionInfo, HServerAddress> e: loadedRegions.entrySet()) {
HRegionInfo hri = e.getKey();
assertTrue(HConnectionManager.isRegionCached(conf,
hri.getTableDesc().getName(), hri.getStartKey()));
}
// delete the temp file
File f = new java.io.File(tempFileName);
f.delete();
LOG.info("Finishing testRegionCacheDeSerialization");
}
/**
* HBASE-2468 use case 3:
*/
@Test
public void testRegionCachePreWarm() throws Exception {
LOG.info("Starting testRegionCachePreWarm");
final byte [] TABLENAME = Bytes.toBytes("testCachePrewarm");
Configuration conf = TEST_UTIL.getConfiguration();
// Set up test table:
// Create table:
TEST_UTIL.createTable(TABLENAME, FAMILY);
// disable region cache for the table.
HTable.setRegionCachePrefetch(conf, TABLENAME, false);
assertFalse("The table is disabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
HTable table = new HTable(conf, TABLENAME);
// create many regions for the table.
TEST_UTIL.createMultiRegions(table, FAMILY);
// This count effectively waits until the regions have been
// fully assigned
TEST_UTIL.countRows(table);
table.getConnection().clearRegionCache();
assertEquals("Clearing cache should have 0 cached ", 0,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
// A Get is suppose to do a region lookup request
Get g = new Get(Bytes.toBytes("aaa"));
table.get(g);
// only one region should be cached if the cache prefetch is disabled.
assertEquals("Number of cached region is incorrect ", 1,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
// now we enable cached prefetch.
HTable.setRegionCachePrefetch(conf, TABLENAME, true);
assertTrue("The table is enabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
HTable.setRegionCachePrefetch(conf, TABLENAME, false);
assertFalse("The table is disabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
HTable.setRegionCachePrefetch(conf, TABLENAME, true);
assertTrue("The table is enabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
table.getConnection().clearRegionCache();
assertEquals("Number of cached region is incorrect ", 0,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
// if there is a cache miss, some additional regions should be prefetched.
Get g2 = new Get(Bytes.toBytes("bbb"));
table.get(g2);
// Get the configured number of cache read-ahead regions.
int prefetchRegionNumber = conf.getInt("hbase.client.prefetch.limit", 10);
// the total number of cached regions == region('aaa") + prefeched regions.
LOG.info("Testing how many regions cached");
assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
table.getConnection().clearRegionCache();
Get g3 = new Get(Bytes.toBytes("abc"));
table.get(g3);
assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
LOG.info("Finishing testRegionCachePreWarm");
}
/**
* Verifies that getConfiguration returns the same Configuration object used
* to create the HTable instance.
@ -3762,149 +3910,6 @@ public class TestFromClientSide {
assertTrue(scan.getFamilyMap().containsKey(FAMILY));
}
/**
* HBASE-2468 use case 1 and 2: region info de/serialization
*/
@Test
public void testRegionCacheDeSerialization() throws Exception {
// 1. test serialization.
LOG.info("Starting testRegionCacheDeSerialization");
final byte[] TABLENAME = Bytes.toBytes("testCachePrewarm2");
final byte[] FAMILY = Bytes.toBytes("family");
Configuration conf = TEST_UTIL.getConfiguration();
TEST_UTIL.createTable(TABLENAME, FAMILY);
// Set up test table:
// Create table:
HTable table = new HTable(conf, TABLENAME);
// Create multiple regions for this table
TEST_UTIL.createMultiRegions(table, FAMILY);
Path tempPath = new Path(HBaseTestingUtility.getTestDir(), "regions.dat");
final String tempFileName = tempPath.toString();
FileOutputStream fos = new FileOutputStream(tempFileName);
DataOutputStream dos = new DataOutputStream(fos);
// serialize the region info and output to a local file.
table.serializeRegionInfo(dos);
dos.flush();
dos.close();
// read a local file and deserialize the region info from it.
FileInputStream fis = new FileInputStream(tempFileName);
DataInputStream dis = new DataInputStream(fis);
Map<HRegionInfo, HServerAddress> deserRegions =
table.deserializeRegionInfo(dis);
dis.close();
// regions obtained from meta scanner.
Map<HRegionInfo, HServerAddress> loadedRegions =
table.getRegionsInfo();
// set the deserialized regions to the global cache.
table.getConnection().clearRegionCache();
table.getConnection().prewarmRegionCache(table.getTableName(),
deserRegions);
// verify whether the 2 maps are identical or not.
assertEquals("Number of cached region is incorrect",
HConnectionManager.getCachedRegionCount(conf, TABLENAME),
loadedRegions.size());
// verify each region is prefetched or not.
for (Map.Entry<HRegionInfo, HServerAddress> e: loadedRegions.entrySet()) {
HRegionInfo hri = e.getKey();
assertTrue(HConnectionManager.isRegionCached(conf,
hri.getTableDesc().getName(), hri.getStartKey()));
}
// delete the temp file
File f = new java.io.File(tempFileName);
f.delete();
LOG.info("Finishing testRegionCacheDeSerialization");
}
/**
* HBASE-2468 use case 3:
*/
@Test
public void testRegionCachePreWarm() throws Exception {
LOG.info("Starting testRegionCachePreWarm");
final byte [] TABLENAME = Bytes.toBytes("testCachePrewarm");
Configuration conf = TEST_UTIL.getConfiguration();
// Set up test table:
// Create table:
TEST_UTIL.createTable(TABLENAME, FAMILY);
// disable region cache for the table.
HTable.setRegionCachePrefetch(conf, TABLENAME, false);
assertFalse("The table is disabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
HTable table = new HTable(conf, TABLENAME);
// create many regions for the table.
TEST_UTIL.createMultiRegions(table, FAMILY);
// This count effectively waits until the regions have been
// fully assigned
TEST_UTIL.countRows(table);
table.getConnection().clearRegionCache();
assertEquals("Clearing cache should have 0 cached ", 0,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
// A Get is suppose to do a region lookup request
Get g = new Get(Bytes.toBytes("aaa"));
table.get(g);
// only one region should be cached if the cache prefetch is disabled.
assertEquals("Number of cached region is incorrect ", 1,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
// now we enable cached prefetch.
HTable.setRegionCachePrefetch(conf, TABLENAME, true);
assertTrue("The table is enabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
HTable.setRegionCachePrefetch(conf, TABLENAME, false);
assertFalse("The table is disabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
HTable.setRegionCachePrefetch(conf, TABLENAME, true);
assertTrue("The table is enabled for region cache prefetch",
HTable.getRegionCachePrefetch(conf, TABLENAME));
table.getConnection().clearRegionCache();
assertEquals("Number of cached region is incorrect ", 0,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
// if there is a cache miss, some additional regions should be prefetched.
Get g2 = new Get(Bytes.toBytes("bbb"));
table.get(g2);
// Get the configured number of cache read-ahead regions.
int prefetchRegionNumber = conf.getInt("hbase.client.prefetch.limit", 10);
// the total number of cached regions == region('aaa") + prefeched regions.
LOG.info("Testing how many regions cached");
assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
table.getConnection().clearRegionCache();
Get g3 = new Get(Bytes.toBytes("abc"));
table.get(g3);
assertEquals("Number of cached region is incorrect ", prefetchRegionNumber,
HConnectionManager.getCachedRegionCount(conf, TABLENAME));
LOG.info("Finishing testRegionCachePreWarm");
}
@Test
public void testIncrement() throws Exception {

View File

@ -20,15 +20,24 @@
package org.apache.hadoop.hbase.coprocessor;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
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.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -41,13 +50,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.*;
/**
* Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.MasterObserver}
* interface hooks at all appropriate times during normal HMaster operations.
@ -217,14 +219,14 @@ public class TestMasterObserver {
@Override
public void preMove(ObserverContext<MasterCoprocessorEnvironment> env,
HRegionInfo region, HServerInfo srcServer, HServerInfo destServer)
HRegionInfo region, ServerName srcServer, ServerName destServer)
throws UnknownRegionException {
preMoveCalled = true;
}
@Override
public void postMove(ObserverContext<MasterCoprocessorEnvironment> env, HRegionInfo region,
HServerInfo srcServer, HServerInfo destServer)
ServerName srcServer, ServerName destServer)
throws UnknownRegionException {
postMoveCalled = true;
}
@ -449,11 +451,13 @@ public class TestMasterObserver {
regions.entrySet().iterator().next();
// try to force a move
Collection<HServerInfo> servers = master.getClusterStatus().getServerInfo();
Collection<ServerName> servers = master.getClusterStatus().getServers();
String destName = null;
for (HServerInfo info : servers) {
if (!info.getServerAddress().equals(firstRegion.getValue())) {
destName = info.getServerName();
for (ServerName info : servers) {
HServerAddress hsa =
new HServerAddress(info.getHostname(), info.getPort());
if (!hsa.equals(firstRegion.getValue())) {
destName = info.toString();
break;
}
}
@ -471,7 +475,7 @@ public class TestMasterObserver {
master.balanceSwitch(false);
// move half the open regions from RS 0 to RS 1
HRegionServer rs = cluster.getRegionServer(0);
byte[] destRS = Bytes.toBytes(cluster.getRegionServer(1).getServerName());
byte[] destRS = Bytes.toBytes(cluster.getRegionServer(1).getServerName().toString());
List<HRegionInfo> openRegions = rs.getOnlineRegions();
int moveCnt = openRegions.size()/2;
for (int i=0; i<moveCnt; i++) {

View File

@ -1,103 +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 static org.junit.Assert.assertEquals;
import java.io.IOException;
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.YouAreDeadException;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
public class BROKE_FIX_TestKillingServersFromMaster {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static MiniHBaseCluster cluster;
@BeforeClass
public static void beforeAllTests() throws Exception {
TEST_UTIL.startMiniCluster(2);
cluster = TEST_UTIL.getHBaseCluster();
}
@AfterClass
public static void afterAllTests() throws IOException {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setup() throws IOException {
TEST_UTIL.ensureSomeRegionServersAvailable(2);
}
/**
* Test that a region server that reports with the wrong start code
* gets shut down
* See HBASE-2613
* @throws Exception
*/
@Ignore @Test (timeout=180000)
public void testRsReportsWrongStartCode() throws Exception {
MiniHBaseClusterRegionServer firstServer =
(MiniHBaseClusterRegionServer)cluster.getRegionServer(0);
HServerInfo hsi = firstServer.getServerInfo();
// This constructor creates a new startcode
firstServer.setHServerInfo(new HServerInfo(hsi.getServerAddress(),
hsi.getInfoPort(), hsi.getHostname()));
cluster.waitOnRegionServer(0);
assertEquals(1, cluster.getLiveRegionServerThreads().size());
}
/**
* Test that a region server that reports with the wrong address
* gets shut down
* See HBASE-2613
* @throws Exception
*/
@Ignore @Test (timeout=180000)
public void testRsReportsWrongAddress() throws Exception {
MiniHBaseClusterRegionServer firstServer =
(MiniHBaseClusterRegionServer)cluster.getRegionServer(0);
firstServer.getHServerInfo().setServerAddress(
new HServerAddress("0.0.0.0", 60010));
cluster.waitOnRegionServer(0);
assertEquals(1, cluster.getLiveRegionServerThreads().size());
}
/**
* Send a YouAreDeadException to the region server and expect it to shut down
* See HBASE-2691
* @throws Exception
*/
@Ignore @Test (timeout=180000)
public void testSendYouAreDead() throws Exception {
cluster.addExceptionToSendRegionServer(0, new YouAreDeadException("bam!"));
cluster.waitOnRegionServer(0);
assertEquals(1, cluster.getLiveRegionServerThreads().size());
}
}

View File

@ -1,58 +0,0 @@
/**
* Copyright 2007 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.List;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HMsg;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.zookeeper.KeeperException;
/**
* An HMaster that runs out of memory.
* Everytime a region server reports in, add to the retained heap of memory.
* Needs to be started manually as in
* <code>${HBASE_HOME}/bin/hbase ./bin/hbase org.apache.hadoop.hbase.OOMEHMaster start/code>.
*/
public class OOMEHMaster extends HMaster {
private List<byte []> retainer = new ArrayList<byte[]>();
public OOMEHMaster(HBaseConfiguration conf)
throws IOException, KeeperException, InterruptedException {
super(conf);
}
@Override
public HMsg[] regionServerReport(HServerInfo serverInfo, HMsg[] msgs,
HRegionInfo[] mostLoadedRegions)
throws IOException {
// Retain 1M.
this.retainer.add(new byte [1024 * 1024]);
return super.regionServerReport(serverInfo, msgs, mostLoadedRegions);
}
public static void main(String[] args) throws Exception {
new HMasterCommandLine(OOMEHMaster.class).doMain(args);
}
}

View File

@ -30,9 +30,10 @@ 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.HServerAddress;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
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.ZooKeeperWatcher;
@ -67,7 +68,7 @@ public class TestActiveMasterManager {
} catch(KeeperException.NoNodeException nne) {}
// Create the master node with a dummy address
HServerAddress master = new HServerAddress("localhost", 1);
ServerName master = new ServerName("localhost", 1, System.currentTimeMillis());
// Should not have a master yet
DummyMaster dummyMaster = new DummyMaster();
ActiveMasterManager activeMasterManager = new ActiveMasterManager(zk,
@ -106,8 +107,10 @@ public class TestActiveMasterManager {
} catch(KeeperException.NoNodeException nne) {}
// Create the master node with a dummy address
HServerAddress firstMasterAddress = new HServerAddress("localhost", 1);
HServerAddress secondMasterAddress = new HServerAddress("localhost", 2);
ServerName firstMasterAddress =
new ServerName("localhost", 1, System.currentTimeMillis());
ServerName secondMasterAddress =
new ServerName("localhost", 2, System.currentTimeMillis());
// Should not have a master yet
DummyMaster ms1 = new DummyMaster();
@ -177,8 +180,10 @@ public class TestActiveMasterManager {
* @throws KeeperException
*/
private void assertMaster(ZooKeeperWatcher zk,
HServerAddress expectedAddress) throws KeeperException {
HServerAddress readAddress = ZKUtil.getDataAsAddress(zk, zk.masterAddressZNode);
ServerName expectedAddress)
throws KeeperException {
ServerName readAddress =
new ServerName(Bytes.toString(ZKUtil.getData(zk, zk.masterAddressZNode)));
assertNotNull(readAddress);
assertTrue(expectedAddress.equals(readAddress));
}
@ -188,8 +193,7 @@ public class TestActiveMasterManager {
ActiveMasterManager manager;
boolean isActiveMaster;
public WaitToBeMasterThread(ZooKeeperWatcher zk,
HServerAddress address) {
public WaitToBeMasterThread(ZooKeeperWatcher zk, ServerName address) {
this.manager = new ActiveMasterManager(zk, address,
new DummyMaster());
isActiveMaster = false;
@ -248,7 +252,7 @@ public class TestActiveMasterManager {
}
@Override
public String getServerName() {
public ServerName getServerName() {
return null;
}

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.executor.ExecutorService;
@ -84,8 +85,8 @@ public class TestCatalogJanitor {
}
@Override
public String getServerName() {
return "mockserver.example.org,1234,-1L";
public ServerName getServerName() {
return new ServerName("mockserver.example.org", 1234, -1L);
}
@Override
@ -161,7 +162,7 @@ public class TestCatalogJanitor {
}
@Override
public String getServerName() {
public ServerName getServerName() {
return null;
}

View File

@ -19,6 +19,8 @@
*/
package org.apache.hadoop.hbase.master;
import java.net.InetAddress;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
@ -26,9 +28,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
@ -53,7 +54,7 @@ public class TestClockSkewDetection {
}
@Override
public String getServerName() {
public ServerName getServerName() {
return null;
}
@ -72,20 +73,18 @@ public class TestClockSkewDetection {
@Override
public void stop(String why) {
}}, null, null);
}}, null);
LOG.debug("regionServerStartup 1");
HServerInfo hsi1 = new HServerInfo(new HServerAddress("example.org:1234"),
System.currentTimeMillis(), -1, "example.com");
sm.regionServerStartup(hsi1, System.currentTimeMillis());
InetAddress ia1 = InetAddress.getLocalHost();
sm.regionServerStartup(ia1, 1234, -1, System.currentTimeMillis());
long maxSkew = 30000;
try {
LOG.debug("regionServerStartup 2");
HServerInfo hsi2 = new HServerInfo(new HServerAddress("example.org:1235"),
System.currentTimeMillis(), -1, "example.com");
sm.regionServerStartup(hsi2, System.currentTimeMillis() - maxSkew * 2);
InetAddress ia2 = InetAddress.getLocalHost();
sm.regionServerStartup(ia2, 1235, -1, System.currentTimeMillis() - maxSkew * 2);
Assert.assertTrue("HMaster should have thrown an ClockOutOfSyncException "
+ "but didn't.", false);
} catch(ClockOutOfSyncException e) {

View File

@ -20,34 +20,25 @@ package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.ServerName;
import org.junit.Test;
public class TestDeadServer {
@Test public void testIsDead() {
DeadServer ds = new DeadServer(2);
final String hostname123 = "127.0.0.1,123,3";
assertFalse(ds.isDeadServer(hostname123, false));
assertFalse(ds.isDeadServer(hostname123, true));
DeadServer ds = new DeadServer();
final ServerName hostname123 = new ServerName("127.0.0.1", 123, 3L);
ds.add(hostname123);
assertTrue(ds.isDeadServer(hostname123, false));
assertFalse(ds.isDeadServer("127.0.0.1:1", true));
assertFalse(ds.isDeadServer("127.0.0.1:1234", true));
assertTrue(ds.isDeadServer("127.0.0.1:123", true));
assertTrue(ds.areDeadServersInProgress());
ds.finish(hostname123);
assertFalse(ds.areDeadServersInProgress());
final String hostname1234 = "127.0.0.2,1234,4";
final ServerName hostname1234 = new ServerName("127.0.0.2", 1234, 4L);
ds.add(hostname1234);
assertTrue(ds.isDeadServer(hostname123, false));
assertTrue(ds.isDeadServer(hostname1234, false));
assertTrue(ds.areDeadServersInProgress());
ds.finish(hostname1234);
assertFalse(ds.areDeadServersInProgress());
final String hostname12345 = "127.0.0.2,12345,4";
final ServerName hostname12345 = new ServerName("127.0.0.2", 12345, 4L);
ds.add(hostname12345);
assertTrue(ds.isDeadServer(hostname1234, false));
assertTrue(ds.isDeadServer(hostname12345, false));
assertTrue(ds.areDeadServersInProgress());
ds.finish(hostname12345);
assertFalse(ds.areDeadServersInProgress());
@ -55,14 +46,14 @@ public class TestDeadServer {
// Already dead = 127.0.0.1,9090,112321
// Coming back alive = 127.0.0.1,9090,223341
final String deadServer = "127.0.0.1,9090,112321";
final ServerName deadServer = new ServerName("127.0.0.1", 9090, 112321L);
assertFalse(ds.cleanPreviousInstance(deadServer));
ds.add(deadServer);
assertTrue(ds.isDeadServer(deadServer));
final String deadServerHostComingAlive = "127.0.0.1,9090,112321";
final ServerName deadServerHostComingAlive =
new ServerName("127.0.0.1", 9090, 112321L);
assertTrue(ds.cleanPreviousInstance(deadServerHostComingAlive));
assertFalse(ds.isDeadServer(deadServer));
assertFalse(ds.cleanPreviousInstance(deadServerHostComingAlive));
}
}

View File

@ -19,11 +19,12 @@
*/
package org.apache.hadoop.hbase.master;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_final_transistion_failed;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_err;
import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_resigned;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Iterator;
@ -62,8 +63,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
import org.junit.Before;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class TestDistributedLogSplitting {
@ -156,7 +157,7 @@ public class TestDistributedLogSplitting {
HRegionServer hrs = rsts.get(0).getRegionServer();
Path rootdir = FSUtils.getRootDir(conf);
final Path logDir = new Path(rootdir,
HLog.getHLogDirectoryName(hrs.getServerName()));
HLog.getHLogDirectoryName(hrs.getServerName().toString()));
installTable(new ZooKeeperWatcher(conf, "table-creation", null),
"table", "family", 40);
@ -205,7 +206,7 @@ public class TestDistributedLogSplitting {
HRegionServer hrs = rsts.get(0).getRegionServer();
Path rootdir = FSUtils.getRootDir(conf);
final Path logDir = new Path(rootdir,
HLog.getHLogDirectoryName(hrs.getServerName()));
HLog.getHLogDirectoryName(hrs.getServerName().toString()));
installTable(new ZooKeeperWatcher(conf, "table-creation", null),
"table", "family", 40);
@ -253,11 +254,10 @@ public class TestDistributedLogSplitting {
HRegionServer hrs = rsts.get(0).getRegionServer();
Path rootdir = FSUtils.getRootDir(conf);
final Path logDir = new Path(rootdir,
HLog.getHLogDirectoryName(hrs.getServerName()));
HLog.getHLogDirectoryName(hrs.getServerName().toString()));
installTable(new ZooKeeperWatcher(conf, "table-creation", null),
"table", "family", 40);
byte[] table = Bytes.toBytes("table");
makeHLog(hrs.getWAL(), hrs.getOnlineRegions(), "table",
NUM_LOG_LINES, 100);
@ -400,11 +400,6 @@ public class TestDistributedLogSplitting {
master.assignmentManager.waitUntilNoRegionsInTransition(60000);
}
private void blockUntilRIT(ZooKeeperWatcher zkw)
throws KeeperException, InterruptedException {
ZKAssign.blockUntilRIT(zkw);
}
private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,
byte [] ...families)
throws IOException {

View File

@ -23,10 +23,12 @@ package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.ipc.RemoteException;
@ -43,12 +45,11 @@ public class TestHMasterRPCException {
HMaster hm = new HMaster(conf);
HServerAddress hma = hm.getMasterAddress();
ServerName sm = hm.getServerName();
InetSocketAddress isa = new InetSocketAddress(sm.getHostname(), sm.getPort());
try {
HMasterInterface inf =
(HMasterInterface) HBaseRPC.getProxy(
HMasterInterface.class, HMasterInterface.VERSION,
hma.getInetSocketAddress(), conf, 100);
HMasterInterface inf = (HMasterInterface) HBaseRPC.getProxy(
HMasterInterface.class, HMasterInterface.VERSION, isa, conf, 100);
inf.isMasterRunning();
fail();
} catch (RemoteException ex) {

View File

@ -20,10 +20,11 @@
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
@ -39,9 +40,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
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.ServerName;
import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.BeforeClass;
@ -138,6 +138,38 @@ public class TestLoadBalancer {
new int [] { 12, 100 },
};
@Test
public void testRandomizer() {
for(int [] mockCluster : clusterStateMocks) {
if (mockCluster.length < 5) continue;
Map<ServerName, List<HRegionInfo>> servers =
mockClusterServers(mockCluster);
for (Map.Entry<ServerName, List<HRegionInfo>> e: servers.entrySet()) {
List<HRegionInfo> original = e.getValue();
if (original.size() < 5) continue;
// Try ten times in case random chances upon original order more than
// one or two times in a row.
boolean same = true;
for (int i = 0; i < 10 && same; i++) {
List<HRegionInfo> copy = new ArrayList<HRegionInfo>(original);
System.out.println("Randomizing before " + copy.size());
for (HRegionInfo hri: copy) {
System.out.println(hri.getEncodedName());
}
List<HRegionInfo> randomized = LoadBalancer.randomize(copy);
System.out.println("Randomizing after " + randomized.size());
for (HRegionInfo hri: randomized) {
System.out.println(hri.getEncodedName());
}
if (original.equals(randomized)) continue;
same = false;
break;
}
assertFalse(same);
}
}
}
/**
* Test the load balancing algorithm.
*
@ -150,13 +182,14 @@ public class TestLoadBalancer {
public void testBalanceCluster() throws Exception {
for(int [] mockCluster : clusterStateMocks) {
Map<HServerInfo,List<HRegionInfo>> servers = mockClusterServers(mockCluster);
LOG.info("Mock Cluster : " + printMock(servers) + " " + printStats(servers));
Map<ServerName, List<HRegionInfo>> servers = mockClusterServers(mockCluster);
List <LoadBalancer.ServerAndLoad> list = convertToList(servers);
LOG.info("Mock Cluster : " + printMock(list) + " " + printStats(list));
List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
List<HServerInfo> balancedCluster = reconcile(servers, plans);
List<LoadBalancer.ServerAndLoad> balancedCluster = reconcile(list, plans);
LOG.info("Mock Balance : " + printMock(balancedCluster));
assertClusterAsBalanced(balancedCluster);
for(Map.Entry<HServerInfo, List<HRegionInfo>> entry : servers.entrySet()) {
for(Map.Entry<ServerName, List<HRegionInfo>> entry : servers.entrySet()) {
returnRegions(entry.getValue());
returnServer(entry.getKey());
}
@ -168,13 +201,13 @@ public class TestLoadBalancer {
* Invariant is that all servers have between floor(avg) and ceiling(avg)
* number of regions.
*/
public void assertClusterAsBalanced(List<HServerInfo> servers) {
public void assertClusterAsBalanced(List<LoadBalancer.ServerAndLoad> servers) {
int numServers = servers.size();
int numRegions = 0;
int maxRegions = 0;
int minRegions = Integer.MAX_VALUE;
for(HServerInfo server : servers) {
int nr = server.getLoad().getNumberOfRegions();
for(LoadBalancer.ServerAndLoad server : servers) {
int nr = server.getLoad();
if(nr > maxRegions) {
maxRegions = nr;
}
@ -190,9 +223,9 @@ public class TestLoadBalancer {
int min = numRegions / numServers;
int max = numRegions % numServers == 0 ? min : min + 1;
for(HServerInfo server : servers) {
assertTrue(server.getLoad().getNumberOfRegions() <= max);
assertTrue(server.getLoad().getNumberOfRegions() >= min);
for(LoadBalancer.ServerAndLoad server : servers) {
assertTrue(server.getLoad() <= max);
assertTrue(server.getLoad() >= min);
}
}
@ -208,12 +241,13 @@ public class TestLoadBalancer {
for(int [] mock : regionsAndServersMocks) {
LOG.debug("testImmediateAssignment with " + mock[0] + " regions and " + mock[1] + " servers");
List<HRegionInfo> regions = randomRegions(mock[0]);
List<HServerInfo> servers = randomServers(mock[1], 0);
Map<HRegionInfo,HServerInfo> assignments =
LoadBalancer.immediateAssignment(regions, servers);
assertImmediateAssignment(regions, servers, assignments);
List<LoadBalancer.ServerAndLoad> servers = randomServers(mock[1], 0);
List<ServerName> list = getListOfServerNames(servers);
Map<HRegionInfo,ServerName> assignments =
LoadBalancer.immediateAssignment(regions, list);
assertImmediateAssignment(regions, list, assignments);
returnRegions(regions);
returnServers(servers);
returnServers(list);
}
}
@ -224,7 +258,7 @@ public class TestLoadBalancer {
* @param assignments
*/
private void assertImmediateAssignment(List<HRegionInfo> regions,
List<HServerInfo> servers, Map<HRegionInfo,HServerInfo> assignments) {
List<ServerName> servers, Map<HRegionInfo, ServerName> assignments) {
for(HRegionInfo region : regions) {
assertTrue(assignments.containsKey(region));
}
@ -243,9 +277,10 @@ public class TestLoadBalancer {
for(int [] mock : regionsAndServersMocks) {
LOG.debug("testBulkAssignment with " + mock[0] + " regions and " + mock[1] + " servers");
List<HRegionInfo> regions = randomRegions(mock[0]);
List<HServerInfo> servers = randomServers(mock[1], 0);
Map<HServerInfo,List<HRegionInfo>> assignments =
LoadBalancer.roundRobinAssignment(regions.toArray(new HRegionInfo[regions.size()]), servers);
List<LoadBalancer.ServerAndLoad> servers = randomServers(mock[1], 0);
List<ServerName> list = getListOfServerNames(servers);
Map<ServerName, List<HRegionInfo>> assignments =
LoadBalancer.roundRobinAssignment(regions, list);
float average = (float)regions.size()/servers.size();
int min = (int)Math.floor(average);
int max = (int)Math.ceil(average);
@ -255,7 +290,7 @@ public class TestLoadBalancer {
}
}
returnRegions(regions);
returnServers(servers);
returnServers(list);
}
}
@ -267,31 +302,43 @@ public class TestLoadBalancer {
@Test
public void testRetainAssignment() throws Exception {
// Test simple case where all same servers are there
List<HServerInfo> servers = randomServers(10, 10);
List<LoadBalancer.ServerAndLoad> servers = randomServers(10, 10);
List<HRegionInfo> regions = randomRegions(100);
Map<HRegionInfo, HServerAddress> existing =
new TreeMap<HRegionInfo, HServerAddress>();
Map<HRegionInfo, ServerName> existing =
new TreeMap<HRegionInfo, ServerName>();
for (int i = 0; i < regions.size(); i++) {
existing.put(regions.get(i),
servers.get(i % servers.size()).getServerAddress());
existing.put(regions.get(i), servers.get(i % servers.size()).getServerName());
}
Map<HServerInfo, List<HRegionInfo>> assignment =
LoadBalancer.retainAssignment(existing, servers);
assertRetainedAssignment(existing, servers, assignment);
List<ServerName> listOfServerNames = getListOfServerNames(servers);
Map<ServerName, List<HRegionInfo>> assignment =
LoadBalancer.retainAssignment(existing, listOfServerNames);
assertRetainedAssignment(existing, listOfServerNames, assignment);
// Include two new servers that were not there before
List<HServerInfo> servers2 = new ArrayList<HServerInfo>(servers);
List<LoadBalancer.ServerAndLoad> servers2 =
new ArrayList<LoadBalancer.ServerAndLoad>(servers);
servers2.add(randomServer(10));
servers2.add(randomServer(10));
assignment = LoadBalancer.retainAssignment(existing, servers2);
assertRetainedAssignment(existing, servers2, assignment);
listOfServerNames = getListOfServerNames(servers2);
assignment = LoadBalancer.retainAssignment(existing, listOfServerNames);
assertRetainedAssignment(existing, listOfServerNames, assignment);
// Remove two of the servers that were previously there
List<HServerInfo> servers3 = new ArrayList<HServerInfo>(servers);
List<LoadBalancer.ServerAndLoad> servers3 =
new ArrayList<LoadBalancer.ServerAndLoad>(servers);
servers3.remove(servers3.size()-1);
servers3.remove(servers3.size()-2);
assignment = LoadBalancer.retainAssignment(existing, servers3);
assertRetainedAssignment(existing, servers3, assignment);
listOfServerNames = getListOfServerNames(servers2);
assignment = LoadBalancer.retainAssignment(existing, listOfServerNames);
assertRetainedAssignment(existing, listOfServerNames, assignment);
}
private List<ServerName> getListOfServerNames(final List<LoadBalancer.ServerAndLoad> sals) {
List<ServerName> list = new ArrayList<ServerName>();
for (LoadBalancer.ServerAndLoad e: sals) {
list.add(e.getServerName());
}
return list;
}
/**
@ -308,12 +355,12 @@ public class TestLoadBalancer {
* @param assignment
*/
private void assertRetainedAssignment(
Map<HRegionInfo, HServerAddress> existing, List<HServerInfo> servers,
Map<HServerInfo, List<HRegionInfo>> assignment) {
Map<HRegionInfo, ServerName> existing, List<ServerName> servers,
Map<ServerName, List<HRegionInfo>> assignment) {
// Verify condition 1, every region assigned, and to online server
Set<HServerInfo> onlineServerSet = new TreeSet<HServerInfo>(servers);
Set<ServerName> onlineServerSet = new TreeSet<ServerName>(servers);
Set<HRegionInfo> assignedRegions = new TreeSet<HRegionInfo>();
for (Map.Entry<HServerInfo, List<HRegionInfo>> a : assignment.entrySet()) {
for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
assertTrue("Region assigned to server that was not listed as online",
onlineServerSet.contains(a.getKey()));
for (HRegionInfo r : a.getValue()) assignedRegions.add(r);
@ -321,23 +368,23 @@ public class TestLoadBalancer {
assertEquals(existing.size(), assignedRegions.size());
// Verify condition 2, if server had existing assignment, must have same
Set<HServerAddress> onlineAddresses = new TreeSet<HServerAddress>();
for (HServerInfo s : servers) onlineAddresses.add(s.getServerAddress());
for (Map.Entry<HServerInfo, List<HRegionInfo>> a : assignment.entrySet()) {
Set<ServerName> onlineAddresses = new TreeSet<ServerName>();
for (ServerName s : servers) onlineAddresses.add(s);
for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
for (HRegionInfo r : a.getValue()) {
HServerAddress address = existing.get(r);
ServerName address = existing.get(r);
if (address != null && onlineAddresses.contains(address)) {
assertTrue(a.getKey().getServerAddress().equals(address));
assertTrue(a.getKey().equals(address));
}
}
}
}
private String printStats(Map<HServerInfo, List<HRegionInfo>> servers) {
private String printStats(List<LoadBalancer.ServerAndLoad> servers) {
int numServers = servers.size();
int totalRegions = 0;
for(HServerInfo server : servers.keySet()) {
totalRegions += server.getLoad().getNumberOfRegions();
for(LoadBalancer.ServerAndLoad server : servers) {
totalRegions += server.getLoad();
}
float average = (float)totalRegions / numServers;
int max = (int)Math.ceil(average);
@ -345,20 +392,31 @@ public class TestLoadBalancer {
return "[srvr=" + numServers + " rgns=" + totalRegions + " avg=" + average + " max=" + max + " min=" + min + "]";
}
private String printMock(Map<HServerInfo, List<HRegionInfo>> servers) {
return printMock(Arrays.asList(servers.keySet().toArray(new HServerInfo[servers.size()])));
private List<LoadBalancer.ServerAndLoad> convertToList(final Map<ServerName, List<HRegionInfo>> servers) {
List<LoadBalancer.ServerAndLoad> list =
new ArrayList<LoadBalancer.ServerAndLoad>(servers.size());
for (Map.Entry<ServerName, List<HRegionInfo>> e: servers.entrySet()) {
list.add(new LoadBalancer.ServerAndLoad(e.getKey(), e.getValue().size()));
}
return list;
}
private String printMock(List<HServerInfo> balancedCluster) {
SortedSet<HServerInfo> sorted = new TreeSet<HServerInfo>(balancedCluster);
HServerInfo [] arr = sorted.toArray(new HServerInfo[sorted.size()]);
private String printMock(Map<ServerName, List<HRegionInfo>> servers) {
return printMock(convertToList(servers));
}
private String printMock(List<LoadBalancer.ServerAndLoad> balancedCluster) {
SortedSet<LoadBalancer.ServerAndLoad> sorted =
new TreeSet<LoadBalancer.ServerAndLoad>(balancedCluster);
LoadBalancer.ServerAndLoad [] arr =
sorted.toArray(new LoadBalancer.ServerAndLoad[sorted.size()]);
StringBuilder sb = new StringBuilder(sorted.size() * 4 + 4);
sb.append("{ ");
for(int i = 0; i < arr.length; i++) {
if (i != 0) {
sb.append(" , ");
}
sb.append(arr[i].getLoad().getNumberOfRegions());
sb.append(arr[i].getLoad());
}
sb.append(" }");
return sb.toString();
@ -371,29 +429,42 @@ public class TestLoadBalancer {
* @param plans
* @return
*/
private List<HServerInfo> reconcile(
Map<HServerInfo, List<HRegionInfo>> servers, List<RegionPlan> plans) {
if(plans != null) {
private List<LoadBalancer.ServerAndLoad> reconcile(List<LoadBalancer.ServerAndLoad> list,
List<RegionPlan> plans) {
List<LoadBalancer.ServerAndLoad> result =
new ArrayList<LoadBalancer.ServerAndLoad>(list.size());
if (plans == null) return result;
Map<ServerName, LoadBalancer.ServerAndLoad> map =
new HashMap<ServerName, LoadBalancer.ServerAndLoad>(list.size());
for (RegionPlan plan : plans) {
plan.getSource().getLoad().setNumberOfRegions(
plan.getSource().getLoad().getNumberOfRegions() - 1);
plan.getDestination().getLoad().setNumberOfRegions(
plan.getDestination().getLoad().getNumberOfRegions() + 1);
ServerName source = plan.getSource();
updateLoad(map, source, -1);
ServerName destination = plan.getDestination();
updateLoad(map, destination, +1);
}
}
return Arrays.asList(servers.keySet().toArray(new HServerInfo[servers.size()]));
result.clear();
result.addAll(map.values());
return result;
}
private Map<HServerInfo, List<HRegionInfo>> mockClusterServers(
private void updateLoad(Map<ServerName, LoadBalancer.ServerAndLoad> map,
final ServerName sn, final int diff) {
LoadBalancer.ServerAndLoad sal = map.get(sn);
if (sal == null) return;
sal = new LoadBalancer.ServerAndLoad(sn, sal.getLoad() + diff);
map.put(sn, sal);
}
private Map<ServerName, List<HRegionInfo>> mockClusterServers(
int [] mockCluster) {
int numServers = mockCluster.length;
Map<HServerInfo,List<HRegionInfo>> servers =
new TreeMap<HServerInfo,List<HRegionInfo>>();
Map<ServerName, List<HRegionInfo>> servers =
new TreeMap<ServerName, List<HRegionInfo>>();
for(int i = 0; i < numServers; i++) {
int numRegions = mockCluster[i];
HServerInfo server = randomServer(numRegions);
LoadBalancer.ServerAndLoad sal = randomServer(0);
List<HRegionInfo> regions = randomRegions(numRegions);
servers.put(server, regions);
servers.put(sal.getServerName(), regions);
}
return servers;
}
@ -426,36 +497,34 @@ public class TestLoadBalancer {
regionQueue.addAll(regions);
}
private Queue<HServerInfo> serverQueue = new LinkedList<HServerInfo>();
private Queue<ServerName> serverQueue = new LinkedList<ServerName>();
private HServerInfo randomServer(int numRegions) {
if(!serverQueue.isEmpty()) {
HServerInfo server = this.serverQueue.poll();
server.getLoad().setNumberOfRegions(numRegions);
return server;
private LoadBalancer.ServerAndLoad randomServer(final int numRegionsPerServer) {
if (!this.serverQueue.isEmpty()) {
ServerName sn = this.serverQueue.poll();
return new LoadBalancer.ServerAndLoad(sn, numRegionsPerServer);
}
String host = "127.0.0.1";
int port = rand.nextInt(60000);
long startCode = rand.nextLong();
HServerInfo hsi =
new HServerInfo(new HServerAddress(host, port), startCode, port, host);
hsi.getLoad().setNumberOfRegions(numRegions);
return hsi;
ServerName sn = new ServerName(host, port, startCode);
return new LoadBalancer.ServerAndLoad(sn, numRegionsPerServer);
}
private List<HServerInfo> randomServers(int numServers, int numRegionsPerServer) {
List<HServerInfo> servers = new ArrayList<HServerInfo>(numServers);
private List<LoadBalancer.ServerAndLoad> randomServers(int numServers, int numRegionsPerServer) {
List<LoadBalancer.ServerAndLoad> servers =
new ArrayList<LoadBalancer.ServerAndLoad>(numServers);
for (int i = 0; i < numServers; i++) {
servers.add(randomServer(numRegionsPerServer));
}
return servers;
}
private void returnServer(HServerInfo server) {
private void returnServer(ServerName server) {
serverQueue.add(server);
}
private void returnServers(List<HServerInfo> servers) {
serverQueue.addAll(servers);
private void returnServers(List<ServerName> servers) {
this.serverQueue.addAll(servers);
}
}

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
import org.apache.hadoop.hbase.replication.regionserver.Replication;
@ -71,7 +72,8 @@ public class TestLogsCleaner {
Path oldLogDir = new Path(HBaseTestingUtility.getTestDir(),
HConstants.HREGION_OLDLOGDIR_NAME);
String fakeMachineName = URLEncoder.encode(server.getServerName(), "UTF8");
String fakeMachineName =
URLEncoder.encode(server.getServerName().toString(), "UTF8");
FileSystem fs = FileSystem.get(conf);
LogCleaner cleaner = new LogCleaner(1000, server, conf, fs, oldLogDir);
@ -146,8 +148,8 @@ public class TestLogsCleaner {
}
@Override
public String getServerName() {
return "regionserver,60020,000000";
public ServerName getServerName() {
return new ServerName("regionserver,60020,000000");
}
@Override

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -75,7 +75,7 @@ public class TestMaster {
TEST_UTIL.loadTable(new HTable(TEST_UTIL.getConfiguration(), TABLENAME),
FAMILYNAME);
List<Pair<HRegionInfo, HServerAddress>> tableRegions =
List<Pair<HRegionInfo, ServerName>> tableRegions =
MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
Bytes.toString(TABLENAME));
LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions));
@ -106,10 +106,10 @@ public class TestMaster {
// We have three regions because one is split-in-progress
assertEquals(3, tableRegions.size());
LOG.info("Making sure we can call getTableRegionClosest while opening");
Pair<HRegionInfo,HServerAddress> pair =
Pair<HRegionInfo, ServerName> pair =
m.getTableRegionForRow(TABLENAME, Bytes.toBytes("cde"));
LOG.info("Result is: " + pair);
Pair<HRegionInfo, HServerAddress> tableRegionFromName =
Pair<HRegionInfo, ServerName> tableRegionFromName =
MetaReader.getRegion(m.getCatalogTracker(),
pair.getFirst().getRegionName());
assertEquals(tableRegionFromName.getFirst(), pair.getFirst());

View File

@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
@ -85,7 +85,7 @@ public class TestMasterFailover {
// verify only one is the active master and we have right number
int numActive = 0;
int activeIndex = -1;
String activeName = null;
ServerName activeName = null;
for (int i = 0; i < masterThreads.size(); i++) {
if (masterThreads.get(i).getMaster().isActiveMaster()) {
numActive++;
@ -278,8 +278,7 @@ public class TestMasterFailover {
// Let's just assign everything to first RS
HRegionServer hrs = cluster.getRegionServer(0);
String serverName = hrs.getServerName();
HServerInfo hsiAlive = hrs.getServerInfo();
ServerName serverName = hrs.getServerName();
// we'll need some regions to already be assigned out properly on live RS
List<HRegionInfo> enabledAndAssignedRegions = new ArrayList<HRegionInfo>();
@ -292,12 +291,12 @@ public class TestMasterFailover {
// now actually assign them
for (HRegionInfo hri : enabledAndAssignedRegions) {
master.assignmentManager.regionPlans.put(hri.getEncodedName(),
new RegionPlan(hri, null, hsiAlive));
new RegionPlan(hri, null, serverName));
master.assignRegion(hri);
}
for (HRegionInfo hri : disabledAndAssignedRegions) {
master.assignmentManager.regionPlans.put(hri.getEncodedName(),
new RegionPlan(hri, null, hsiAlive));
new RegionPlan(hri, null, serverName));
master.assignRegion(hri);
}
@ -583,12 +582,10 @@ public class TestMasterFailover {
// The first RS will stay online
HRegionServer hrs = cluster.getRegionServer(0);
HServerInfo hsiAlive = hrs.getServerInfo();
// The second RS is going to be hard-killed
HRegionServer hrsDead = cluster.getRegionServer(1);
String deadServerName = hrsDead.getServerName();
HServerInfo hsiDead = hrsDead.getServerInfo();
ServerName deadServerName = hrsDead.getServerName();
// we'll need some regions to already be assigned out properly on live RS
List<HRegionInfo> enabledAndAssignedRegions = new ArrayList<HRegionInfo>();
@ -601,12 +598,12 @@ public class TestMasterFailover {
// now actually assign them
for (HRegionInfo hri : enabledAndAssignedRegions) {
master.assignmentManager.regionPlans.put(hri.getEncodedName(),
new RegionPlan(hri, null, hsiAlive));
new RegionPlan(hri, null, hrs.getServerName()));
master.assignRegion(hri);
}
for (HRegionInfo hri : disabledAndAssignedRegions) {
master.assignmentManager.regionPlans.put(hri.getEncodedName(),
new RegionPlan(hri, null, hsiAlive));
new RegionPlan(hri, null, hrs.getServerName()));
master.assignRegion(hri);
}
@ -621,12 +618,12 @@ public class TestMasterFailover {
// set region plan to server to be killed and trigger assign
for (HRegionInfo hri : enabledAndOnDeadRegions) {
master.assignmentManager.regionPlans.put(hri.getEncodedName(),
new RegionPlan(hri, null, hsiDead));
new RegionPlan(hri, null, deadServerName));
master.assignRegion(hri);
}
for (HRegionInfo hri : disabledAndOnDeadRegions) {
master.assignmentManager.regionPlans.put(hri.getEncodedName(),
new RegionPlan(hri, null, hsiDead));
new RegionPlan(hri, null, deadServerName));
master.assignRegion(hri);
}

View File

@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
@ -72,11 +73,11 @@ public class TestRestartCluster {
String unassignedZNode = zooKeeper.assignmentZNode;
ZKUtil.createAndFailSilent(zooKeeper, unassignedZNode);
ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.ROOT_REGIONINFO,
HMaster.MASTER);
ServerName sn = new ServerName(HMaster.MASTER, -1, System.currentTimeMillis());
ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.FIRST_META_REGIONINFO,
HMaster.MASTER);
ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.ROOT_REGIONINFO, sn);
ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.FIRST_META_REGIONINFO, sn);
LOG.debug("Created UNASSIGNED zNode for ROOT and META regions in state " +
EventType.M_ZK_REGION_OFFLINE);

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@ -155,7 +156,7 @@ public class TestRollingRestart {
int num = 1;
int total = regionServers.size();
for (RegionServerThread rst : regionServers) {
String serverName = rst.getRegionServer().getServerName();
ServerName serverName = rst.getRegionServer().getServerName();
log("Stopping region server " + num + " of " + total + " [ " +
serverName + "]");
rst.getRegionServer().stop("Stopping RS during rolling restart");
@ -302,7 +303,7 @@ public class TestRollingRestart {
}
private void waitForRSShutdownToStartAndFinish(MasterThread activeMaster,
String serverName) throws InterruptedException {
ServerName serverName) throws InterruptedException {
ServerManager sm = activeMaster.getMaster().getServerManager();
// First wait for it to be in dead list
while (!sm.getDeadServers().contains(serverName)) {

View File

@ -27,8 +27,8 @@ import java.util.concurrent.Semaphore;
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.MasterAddressTracker;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -75,17 +75,17 @@ public class TestMasterAddressManager {
// Create the master node with a dummy address
String host = "localhost";
int port = 1234;
HServerAddress dummyAddress = new HServerAddress(host, port);
ServerName sn = new ServerName(host, port, System.currentTimeMillis());
LOG.info("Creating master node");
ZKUtil.setAddressAndWatch(zk, zk.masterAddressZNode, dummyAddress);
ZKUtil.createEphemeralNodeAndWatch(zk, zk.masterAddressZNode, sn.getBytes());
// Wait for the node to be created
LOG.info("Waiting for master address manager to be notified");
listener.waitForCreation();
LOG.info("Master node created");
assertTrue(addressManager.hasMaster());
HServerAddress pulledAddress = addressManager.getMasterAddress();
assertTrue(pulledAddress.equals(dummyAddress));
ServerName pulledAddress = addressManager.getMasterAddress();
assertTrue(pulledAddress.equals(sn));
}

View File

@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.UnknownScannerException;
@ -263,11 +262,11 @@ public class TestScanner extends HBaseTestCase {
// Store some new information
HServerAddress address = new HServerAddress("foo.bar.com:1234");
String address = "foo.bar.com:1234";
put = new Put(ROW_KEY, System.currentTimeMillis(), null);
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(address.toString()));
Bytes.toBytes(address));
// put.add(HConstants.COL_STARTCODE, Bytes.toBytes(START_CODE));
@ -301,12 +300,12 @@ public class TestScanner extends HBaseTestCase {
// Now update the information again
address = new HServerAddress("bar.foo.com:4321");
address = "bar.foo.com:4321";
put = new Put(ROW_KEY, System.currentTimeMillis(), null);
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(address.toString()));
Bytes.toBytes(address));
region.put(put);
// Validate again

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Delete;
@ -258,7 +259,7 @@ public class TestSplitTransactionOnCluster {
// Insert into zk a blocking znode, a znode of same name as region
// so it gets in way of our splitting.
ZKAssign.createNodeClosing(t.getConnection().getZooKeeperWatcher(),
hri, "anyOldServer");
hri, new ServerName("any.old.server", 1234, -1));
// Now try splitting.... should fail. And each should successfully
// rollback.
this.admin.split(hri.getRegionNameAsString());
@ -455,7 +456,7 @@ public class TestSplitTransactionOnCluster {
HRegionServer hrs = getOtherRegionServer(cluster, metaRegionServer);
LOG.info("Moving " + hri.getRegionNameAsString() + " to " +
hrs.getServerName() + "; metaServerIndex=" + metaServerIndex);
admin.move(hri.getEncodedNameAsBytes(), Bytes.toBytes(hrs.getServerName()));
admin.move(hri.getEncodedNameAsBytes(), hrs.getServerName().getBytes());
}
// Wait till table region is up on the server that is NOT carrying .META..
while (true) {

View File

@ -31,9 +31,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
@ -72,11 +72,11 @@ public class TestOpenRegionHandler {
*/
static class MockServer implements Server {
boolean stopped = false;
final static String NAME = "MockServer";
final static ServerName NAME = new ServerName("MockServer", 123, -1);
final ZooKeeperWatcher zk;
MockServer() throws ZooKeeperConnectionException, IOException {
this.zk = new ZooKeeperWatcher(HTU.getConfiguration(), NAME, this);
this.zk = new ZooKeeperWatcher(HTU.getConfiguration(), NAME.toString(), this);
}
@Override
@ -113,7 +113,7 @@ public class TestOpenRegionHandler {
}
@Override
public String getServerName() {
public ServerName getServerName() {
return NAME;
}
}
@ -156,11 +156,6 @@ public class TestOpenRegionHandler {
return null;
}
@Override
public HServerInfo getServerInfo() {
return null;
}
@Override
public HBaseRpcMetrics getRpcMetrics() {
return null;
@ -196,7 +191,7 @@ public class TestOpenRegionHandler {
}
@Override
public String getServerName() {
public ServerName getServerName() {
return null;
}

View File

@ -19,6 +19,12 @@
*/
package org.apache.hadoop.hbase.replication.regionserver;
import static org.junit.Assert.assertEquals;
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -32,6 +38,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@ -45,16 +52,8 @@ import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.junit.Assert.assertEquals;
public class TestReplicationSourceManager {
private static final Log LOG =
@ -225,7 +224,7 @@ public class TestReplicationSourceManager {
}
@Override
public String getServerName() {
public ServerName getServerName() {
return null; //To change body of implemented methods use File | Settings | File Templates.
}

View File

@ -26,8 +26,8 @@ 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.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -82,15 +82,15 @@ public class TestHBaseFsck {
for (JVMClusterUtil.RegionServerThread rs :
TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
HServerInfo hsi = rs.getRegionServer().getServerInfo();
ServerName sn = rs.getRegionServer().getServerName();
// When we find a diff RS, change the assignment and break
if (startCode != hsi.getStartCode()) {
if (startCode != sn.getStartcode()) {
Put put = new Put(res.getRow());
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(hsi.getHostnamePort()));
Bytes.toBytes(sn.getHostAndPort()));
put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(hsi.getStartCode()));
Bytes.toBytes(sn.getStartcode()));
meta.put(put);
break resforloop;
}