HBASE-5709 Move active master and backup master znodes to use pbs

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1310553 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-04-06 19:05:42 +00:00
parent 4317892dce
commit 4cc08fa5d6
14 changed files with 749 additions and 183 deletions

View File

@ -1,81 +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 org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
* Manages the location of the current active Master for this RegionServer.
* <p>
* Listens for ZooKeeper events related to the master address. The node
* <code>/master</code> will contain the address of the current master.
* This listener is interested in
* <code>NodeDeleted</code> and <code>NodeCreated</code> events on
* <code>/master</code>.
* <p>
* Utilizes {@link ZooKeeperNodeTracker} for zk interactions.
* <p>
* You can get the current master via {@link #getMasterAddress()}
*/
@InterfaceAudience.Private
public class MasterAddressTracker extends ZooKeeperNodeTracker {
/**
* Construct a master address listener with the specified
* <code>zookeeper</code> reference.
* <p>
* This constructor does not trigger any actions, you must call methods
* explicitly. Normally you will just want to execute {@link #start()} to
* begin tracking of the master address.
*
* @param watcher zk reference and watcher
* @param abortable abortable in case of fatal error
*/
public MasterAddressTracker(ZooKeeperWatcher watcher, Abortable abortable) {
super(watcher, watcher.masterAddressZNode, abortable);
}
/**
* Get the address of the current master if one is available. Returns null
* if no current master.
* @return Server name or null if timed out.
*/
public ServerName getMasterAddress() {
return bytesToServerName(super.getData(false));
}
/**
* Check if there is a master available.
* @return true if there is a master set, false if not.
*/
public boolean hasMaster() {
return super.getData(false) != null;
}
/**
* @param bytes Byte array of {@link ServerName#toString()}
* @return A {@link ServerName} instance.
*/
private ServerName bytesToServerName(final byte [] bytes) {
return bytes == null ? null: ServerName.parseVersionedServerName(bytes);
}
}

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.SoftValueSortedMap;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKTable;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -654,14 +655,7 @@ public class HConnectionManager {
try {
checkIfBaseNodeAvailable(zkw);
byte[] masterAddress = ZKUtil.getData(zkw, zkw.masterAddressZNode);
if (masterAddress == null){
throw new IOException("Can't get master address from ZooKeeper");
}
ServerName sn = ServerName.parseVersionedServerName(masterAddress);
ServerName sn = MasterAddressTracker.getMasterAddress(zkw);
if (sn == null) {
String msg =
"ZooKeeper available but no active master location found";

View File

@ -19,21 +19,21 @@
*/
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
/**
* Handles everything on master-side related to master election.
@ -70,14 +70,16 @@ class ActiveMasterManager extends ZooKeeperListener {
@Override
public void nodeCreated(String path) {
if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
handleMasterNodeChange();
}
handle(path);
}
@Override
public void nodeDeleted(String path) {
if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
handle(path);
}
void handle(final String path) {
if (path.equals(watcher.getMasterAddressZNode()) && !master.isStopped()) {
handleMasterNodeChange();
}
}
@ -99,7 +101,7 @@ class ActiveMasterManager extends ZooKeeperListener {
// Watch the node and check if it exists.
try {
synchronized(clusterHasActiveMaster) {
if(ZKUtil.watchAndCheckExists(watcher, watcher.masterAddressZNode)) {
if (ZKUtil.watchAndCheckExists(watcher, watcher.getMasterAddressZNode())) {
// A master node exists, there is an active master
LOG.debug("A master is now available");
clusterHasActiveMaster.set(true);
@ -136,14 +138,11 @@ class ActiveMasterManager extends ZooKeeperListener {
// Try to become the active master, watch if there is another master.
// Write out our ServerName as versioned bytes.
try {
String backupZNode = ZKUtil.joinZNode(
this.watcher.backupMasterAddressesZNode, this.sn.toString());
if (ZKUtil.createEphemeralNodeAndWatch(this.watcher,
this.watcher.masterAddressZNode, this.sn.getVersionedBytes())) {
String backupZNode = ZKUtil.joinZNode(this.watcher.backupMasterAddressesZNode, this.sn.toString());
if (MasterAddressTracker.setMasterAddress(this.watcher, this.watcher.getMasterAddressZNode(), this.sn)) {
// If we were a backup master before, delete our ZNode from the backup
// master directory since we are the active now
LOG.info("Deleting ZNode for " + backupZNode +
" from backup master directory");
LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory");
ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
// We are the master, return
@ -165,24 +164,22 @@ class ActiveMasterManager extends ZooKeeperListener {
* this node explicitly. If we crash before then, ZooKeeper will delete
* this node for us since it is ephemeral.
*/
LOG.info("Adding ZNode for " + backupZNode +
" in backup master directory");
ZKUtil.createEphemeralNodeAndWatch(this.watcher, backupZNode,
this.sn.getVersionedBytes());
LOG.info("Adding ZNode for " + backupZNode + " in backup master directory");
MasterAddressTracker.setMasterAddress(this.watcher, backupZNode, this.sn);
String msg;
byte[] bytes =
ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
ZKUtil.getDataAndWatch(this.watcher, this.watcher.getMasterAddressZNode());
if (bytes == null) {
msg = ("A master was detected, but went down before its address " +
"could be read. Attempting to become the next active master");
} else {
ServerName currentMaster = ServerName.parseVersionedServerName(bytes);
ServerName currentMaster = ZKUtil.znodeContentToServerName(bytes);
if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
msg = ("Current master has this master's address, " +
currentMaster + "; master was restarted? Deleting node.");
// Hurry along the expiration of the znode.
ZKUtil.deleteNode(this.watcher, this.watcher.masterAddressZNode);
ZKUtil.deleteNode(this.watcher, this.watcher.getMasterAddressZNode());
} else {
msg = "Another master is the active master, " + currentMaster +
"; waiting to become the next active master";
@ -221,10 +218,10 @@ class ActiveMasterManager extends ZooKeeperListener {
*/
public boolean isActiveMaster() {
try {
if (ZKUtil.checkExists(watcher, watcher.masterAddressZNode) >= 0) {
if (ZKUtil.checkExists(watcher, watcher.getMasterAddressZNode()) >= 0) {
return true;
}
}
}
catch (KeeperException ke) {
LOG.info("Received an unexpected KeeperException when checking " +
"isActiveMaster : "+ ke);
@ -235,12 +232,14 @@ class ActiveMasterManager extends ZooKeeperListener {
public void stop() {
try {
// If our address is in ZK, delete it on our way out
byte [] bytes =
ZKUtil.getDataAndWatch(watcher, watcher.masterAddressZNode);
// TODO: redo this to make it atomic (only added for tests)
ServerName master = ServerName.parseVersionedServerName(bytes);
if (master != null && master.equals(this.sn)) {
ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
ServerName activeMaster = null;
try {
activeMaster = MasterAddressTracker.getMasterAddress(this.watcher);
} catch (IOException e) {
LOG.warn("Failed get of master address: " + e.toString());
}
if (activeMaster != null && activeMaster.equals(this.sn)) {
ZKUtil.deleteNode(watcher, watcher.getMasterAddressZNode());
}
} catch (KeeperException e) {
LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);

View File

@ -1430,7 +1430,7 @@ Server {
try {
byte[] bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(this.zooKeeper.backupMasterAddressesZNode, s));
if (bytes != null) {
backupMasters.add(ServerName.parseVersionedServerName(bytes));
backupMasters.add(ZKUtil.znodeContentToServerName(bytes));
}
} catch (KeeperException e) {
LOG.warn(this.zooKeeper.prefix("Unable to get information about " +

View File

@ -481,11 +481,489 @@ public final class ZooKeeperProtos {
// @@protoc_insertion_point(class_scope:RootRegionServer)
}
public interface MasterOrBuilder
extends com.google.protobuf.MessageOrBuilder {
// required .ServerName master = 1;
boolean hasMaster();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getMaster();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getMasterOrBuilder();
}
public static final class Master extends
com.google.protobuf.GeneratedMessage
implements MasterOrBuilder {
// Use Master.newBuilder() to construct.
private Master(Builder builder) {
super(builder);
}
private Master(boolean noInit) {}
private static final Master defaultInstance;
public static Master getDefaultInstance() {
return defaultInstance;
}
public Master getDefaultInstanceForType() {
return defaultInstance;
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Master_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Master_fieldAccessorTable;
}
private int bitField0_;
// required .ServerName master = 1;
public static final int MASTER_FIELD_NUMBER = 1;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName master_;
public boolean hasMaster() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getMaster() {
return master_;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getMasterOrBuilder() {
return master_;
}
private void initFields() {
master_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
byte isInitialized = memoizedIsInitialized;
if (isInitialized != -1) return isInitialized == 1;
if (!hasMaster()) {
memoizedIsInitialized = 0;
return false;
}
if (!getMaster().isInitialized()) {
memoizedIsInitialized = 0;
return false;
}
memoizedIsInitialized = 1;
return true;
}
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeMessage(1, master_);
}
getUnknownFields().writeTo(output);
}
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
size = 0;
if (((bitField0_ & 0x00000001) == 0x00000001)) {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, master_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
}
private static final long serialVersionUID = 0L;
@java.lang.Override
protected java.lang.Object writeReplace()
throws java.io.ObjectStreamException {
return super.writeReplace();
}
@java.lang.Override
public boolean equals(final java.lang.Object obj) {
if (obj == this) {
return true;
}
if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master)) {
return super.equals(obj);
}
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master) obj;
boolean result = true;
result = result && (hasMaster() == other.hasMaster());
if (hasMaster()) {
result = result && getMaster()
.equals(other.getMaster());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
}
@java.lang.Override
public int hashCode() {
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
if (hasMaster()) {
hash = (37 * hash) + MASTER_FIELD_NUMBER;
hash = (53 * hash) + getMaster().hashCode();
}
hash = (29 * hash) + getUnknownFields().hashCode();
return hash;
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(
com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(
byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(java.io.InputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input)) {
return builder.buildParsed();
} else {
return null;
}
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseDelimitedFrom(
java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
return builder.buildParsed();
} else {
return null;
}
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(
com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed();
}
public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master parseFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
@java.lang.Override
protected Builder newBuilderForType(
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
Builder builder = new Builder(parent);
return builder;
}
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder>
implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.MasterOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Master_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_Master_fieldAccessorTable;
}
// Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
private Builder(BuilderParent parent) {
super(parent);
maybeForceBuilderInitialization();
}
private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
getMasterFieldBuilder();
}
}
private static Builder create() {
return new Builder();
}
public Builder clear() {
super.clear();
if (masterBuilder_ == null) {
master_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
} else {
masterBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
public Builder clone() {
return create().mergeFrom(buildPartial());
}
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.getDescriptor();
}
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master getDefaultInstanceForType() {
return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.getDefaultInstance();
}
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master build() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(
result).asInvalidProtocolBufferException();
}
return result;
}
public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
to_bitField0_ |= 0x00000001;
}
if (masterBuilder_ == null) {
result.master_ = master_;
} else {
result.master_ = masterBuilder_.build();
}
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master) {
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master)other);
} else {
super.mergeFrom(other);
return this;
}
}
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master other) {
if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.getDefaultInstance()) return this;
if (other.hasMaster()) {
mergeMaster(other.getMaster());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
public final boolean isInitialized() {
if (!hasMaster()) {
return false;
}
if (!getMaster().isInitialized()) {
return false;
}
return true;
}
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
com.google.protobuf.UnknownFieldSet.newBuilder(
this.getUnknownFields());
while (true) {
int tag = input.readTag();
switch (tag) {
case 0:
this.setUnknownFields(unknownFields.build());
onChanged();
return this;
default: {
if (!parseUnknownField(input, unknownFields,
extensionRegistry, tag)) {
this.setUnknownFields(unknownFields.build());
onChanged();
return this;
}
break;
}
case 10: {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder();
if (hasMaster()) {
subBuilder.mergeFrom(getMaster());
}
input.readMessage(subBuilder, extensionRegistry);
setMaster(subBuilder.buildPartial());
break;
}
}
}
}
private int bitField0_;
// required .ServerName master = 1;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName master_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> masterBuilder_;
public boolean hasMaster() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getMaster() {
if (masterBuilder_ == null) {
return master_;
} else {
return masterBuilder_.getMessage();
}
}
public Builder setMaster(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
if (masterBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
}
master_ = value;
onChanged();
} else {
masterBuilder_.setMessage(value);
}
bitField0_ |= 0x00000001;
return this;
}
public Builder setMaster(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) {
if (masterBuilder_ == null) {
master_ = builderForValue.build();
onChanged();
} else {
masterBuilder_.setMessage(builderForValue.build());
}
bitField0_ |= 0x00000001;
return this;
}
public Builder mergeMaster(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
if (masterBuilder_ == null) {
if (((bitField0_ & 0x00000001) == 0x00000001) &&
master_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) {
master_ =
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(master_).mergeFrom(value).buildPartial();
} else {
master_ = value;
}
onChanged();
} else {
masterBuilder_.mergeFrom(value);
}
bitField0_ |= 0x00000001;
return this;
}
public Builder clearMaster() {
if (masterBuilder_ == null) {
master_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
onChanged();
} else {
masterBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getMasterBuilder() {
bitField0_ |= 0x00000001;
onChanged();
return getMasterFieldBuilder().getBuilder();
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getMasterOrBuilder() {
if (masterBuilder_ != null) {
return masterBuilder_.getMessageOrBuilder();
} else {
return master_;
}
}
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>
getMasterFieldBuilder() {
if (masterBuilder_ == null) {
masterBuilder_ = new com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
master_,
getParentForChildren(),
isClean());
master_ = null;
}
return masterBuilder_;
}
// @@protoc_insertion_point(builder_scope:Master)
}
static {
defaultInstance = new Master(true);
defaultInstance.initFields();
}
// @@protoc_insertion_point(class_scope:Master)
}
private static com.google.protobuf.Descriptors.Descriptor
internal_static_RootRegionServer_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_RootRegionServer_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_Master_descriptor;
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_Master_fieldAccessorTable;
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
@ -496,9 +974,10 @@ public final class ZooKeeperProtos {
static {
java.lang.String[] descriptorData = {
"\n\017ZooKeeper.proto\032\013hbase.proto\"/\n\020RootRe" +
"gionServer\022\033\n\006server\030\001 \002(\0132\013.ServerNameB" +
"E\n*org.apache.hadoop.hbase.protobuf.gene" +
"ratedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
"gionServer\022\033\n\006server\030\001 \002(\0132\013.ServerName\"" +
"%\n\006Master\022\033\n\006master\030\001 \002(\0132\013.ServerNameBE" +
"\n*org.apache.hadoop.hbase.protobuf.gener" +
"atedB\017ZooKeeperProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -513,6 +992,14 @@ public final class ZooKeeperProtos {
new java.lang.String[] { "Server", },
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RootRegionServer.class,
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RootRegionServer.Builder.class);
internal_static_Master_descriptor =
getDescriptor().getMessageTypes().get(1);
internal_static_Master_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_Master_descriptor,
new java.lang.String[] { "Master", },
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.class,
org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Master.Builder.class);
return null;
}
};

View File

@ -72,7 +72,6 @@ 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.MasterAddressTracker;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName;
@ -147,6 +146,7 @@ import org.apache.hadoop.hbase.util.Sleeper;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;

View File

@ -0,0 +1,132 @@
/**
* 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.zookeeper;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.zookeeper.KeeperException;
/**
* Manages the location of the current active Master for the RegionServer.
* <p>
* Listens for ZooKeeper events related to the master address. The node
* <code>/master</code> will contain the address of the current master.
* This listener is interested in
* <code>NodeDeleted</code> and <code>NodeCreated</code> events on
* <code>/master</code>.
* <p>
* Utilizes {@link ZooKeeperNodeTracker} for zk interactions.
* <p>
* You can get the current master via {@link #getMasterAddress()} or via
* {@link #getMasterAddress(ZooKeeperWatcher)} if you do not have a running
* instance of this Tracker in your context.
* <p>
* This class also includes utility for interacting with the master znode, for
* writing and reading the znode content.
*/
@InterfaceAudience.Private
public class MasterAddressTracker extends ZooKeeperNodeTracker {
/**
* Construct a master address listener with the specified
* <code>zookeeper</code> reference.
* <p>
* This constructor does not trigger any actions, you must call methods
* explicitly. Normally you will just want to execute {@link #start()} to
* begin tracking of the master address.
*
* @param watcher zk reference and watcher
* @param abortable abortable in case of fatal error
*/
public MasterAddressTracker(ZooKeeperWatcher watcher, Abortable abortable) {
super(watcher, watcher.getMasterAddressZNode(), abortable);
}
/**
* Get the address of the current master if one is available. Returns null
* if no current master.
* @return Server name or null if timed out.
*/
public ServerName getMasterAddress() {
return ZKUtil.znodeContentToServerName(super.getData(false));
}
/**
* Get master address.
* Use this instead of {@link #getMasterAddress()} if you do not have an
* instance of this tracker in your context.
* @param zkw ZooKeeperWatcher to use
* @return ServerName stored in the the master address znode or null if no
* znode present.
* @throws KeeperException
* @throws IOException
*/
public static ServerName getMasterAddress(final ZooKeeperWatcher zkw)
throws KeeperException, IOException {
byte [] data = ZKUtil.getData(zkw, zkw.getMasterAddressZNode());
if (data == null){
throw new IOException("Can't get master address from ZooKeeper; znode data == null");
}
return ZKUtil.znodeContentToServerName(data);
}
/**
* Set master address into the <code>master</code> znode or into the backup
* subdirectory of backup masters; switch off the passed in <code>znode</code>
* path.
* @param zkw The ZooKeeperWatcher to use.
* @param znode Where to create the znode; could be at the top level or it
* could be under backup masters
* @param master ServerName of the current master
* @return true if node created, false if not; a watch is set in both cases
* @throws KeeperException
*/
public static boolean setMasterAddress(final ZooKeeperWatcher zkw,
final String znode, final ServerName master)
throws KeeperException {
return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, getZNodeData(master));
}
/**
* Check if there is a master available.
* @return true if there is a master set, false if not.
*/
public boolean hasMaster() {
return super.getData(false) != null;
}
/**
* @param sn
* @return Content of the master znode as a serialized pb with the pb
* magic as prefix.
*/
static byte [] getZNodeData(final ServerName sn) {
ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
snbuilder.setHostName(sn.getHostname());
snbuilder.setPort(sn.getPort());
snbuilder.setStartCode(sn.getStartcode());
mbuilder.setMaster(snbuilder.build());
return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray());
}
}

View File

@ -64,7 +64,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
* @throws InterruptedException
*/
public ServerName getRootRegionLocation() throws InterruptedException {
return dataToServerName(super.getData(true));
return ZKUtil.znodeContentToServerName(super.getData(true));
}
/**
@ -76,7 +76,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
*/
public static ServerName getRootRegionLocation(final ZooKeeperWatcher zkw)
throws KeeperException {
return dataToServerName(ZKUtil.getData(zkw, zkw.rootServerZNode));
return ZKUtil.znodeContentToServerName(ZKUtil.getData(zkw, zkw.rootServerZNode));
}
/**
@ -97,7 +97,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
LOG.error(errorMsg);
throw new IllegalArgumentException(errorMsg);
}
return dataToServerName(super.blockUntilAvailable(timeout, true));
return ZKUtil.znodeContentToServerName(super.blockUntilAvailable(timeout, true));
}
/**
@ -164,43 +164,6 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
final long timeout)
throws InterruptedException {
byte [] data = ZKUtil.blockUntilAvailable(zkw, zkw.rootServerZNode, timeout);
return dataToServerName(data);
}
/**
* @param data
* @return Returns null if <code>data</code> is null else converts passed data
* to a ServerName instance.
*/
static ServerName dataToServerName(final byte [] data) {
if (data == null || data.length <= 0) return null;
if (ProtobufUtil.isPBMagicPrefix(data)) {
int prefixLen = ProtobufUtil.lengthOfPBMagic();
try {
RootRegionServer rss =
RootRegionServer.newBuilder().mergeFrom(data, prefixLen, data.length - prefixLen).build();
HBaseProtos.ServerName sn = rss.getServer();
return new ServerName(sn.getHostName(), sn.getPort(), sn.getStartCode());
} catch (InvalidProtocolBufferException e) {
// A failed parse of the znode is pretty catastrophic. Rather than loop
// retrying hoping the bad bytes will changes, and rather than change
// the signature on this method to add an IOE which will send ripples all
// over the code base, throw a RuntimeException. This should "never" happen.
throw new RuntimeException(e);
}
}
// The str returned could be old style -- pre hbase-1502 -- which was
// hostname and port seperated by a colon rather than hostname, port and
// startcode delimited by a ','.
String str = Bytes.toString(data);
int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR);
if (index != -1) {
// Presume its ServerName.toString() format.
return ServerName.parseServerName(str);
}
// Presume it a hostname:port format.
String hostname = Addressing.parseHostname(str);
int port = Addressing.parsePort(str);
return new ServerName(hostname, port, -1L);
return ZKUtil.znodeContentToServerName(data);
}
}

View File

@ -40,6 +40,10 @@ import org.apache.hadoop.hbase.EmptyWatcher;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RootRegionServer;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.zookeeper.AsyncCallback;
@ -52,6 +56,8 @@ import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* Internal HBase utility class for ZooKeeper.
*
@ -724,7 +730,7 @@ public class ZKUtil {
if (isSecureZooKeeper(zkw.getConfiguration())) {
// Certain znodes must be readable by non-authenticated clients
if ((node.equals(zkw.rootServerZNode) == true) ||
(node.equals(zkw.masterAddressZNode) == true) ||
(node.equals(zkw.getMasterAddressZNode()) == true) ||
(node.equals(zkw.clusterIdZNode) == true)) {
return ZooKeeperWatcher.CREATOR_ALL_AND_WORLD_READABLE;
}
@ -1041,8 +1047,12 @@ public class ZKUtil {
StringBuilder sb = new StringBuilder();
try {
sb.append("HBase is rooted at ").append(zkw.baseZNode);
sb.append("\nActive master address: ").append(
ServerName.parseVersionedServerName(getData(zkw, zkw.masterAddressZNode)));
sb.append("\nActive master address: ");
try {
sb.append(MasterAddressTracker.getMasterAddress(zkw));
} catch (IOException e) {
sb.append("<<FAILED LOOKUP: " + e.getMessage() + ">>");
}
sb.append("\nBackup master addresses:");
for (String child : listChildrenNoWatch(zkw,
zkw.backupMasterAddressesZNode)) {
@ -1204,4 +1214,47 @@ public class ZKUtil {
return data;
}
}
/**
* Get a ServerName from the passed in znode data bytes.
* @param data ZNode data with a server name in it; can handle the old style
* servername where servername was host and port. Works too with data that
* begins w/ the pb 'PBUF' magic and that its then followed by a protobuf that
* has a serialized {@link ServerName} in it.
* @return Returns null if <code>data</code> is null else converts passed data
* to a ServerName instance.
*/
public static ServerName znodeContentToServerName(final byte [] data) {
if (data == null || data.length <= 0) return null;
if (ProtobufUtil.isPBMagicPrefix(data)) {
int prefixLen = ProtobufUtil.lengthOfPBMagic();
try {
RootRegionServer rss =
RootRegionServer.newBuilder().mergeFrom(data, prefixLen, data.length - prefixLen).build();
HBaseProtos.ServerName sn = rss.getServer();
return new ServerName(sn.getHostName(), sn.getPort(), sn.getStartCode());
} catch (InvalidProtocolBufferException e) {
// A failed parse of the znode is pretty catastrophic. Rather than loop
// retrying hoping the bad bytes will changes, and rather than change
// the signature on this method to add an IOE which will send ripples all
// over the code base, throw a RuntimeException. This should "never" happen.
// Fail fast if it does.
throw new RuntimeException(e);
}
}
// The str returned could be old style -- pre hbase-1502 -- which was
// hostname and port seperated by a colon rather than hostname, port and
// startcode delimited by a ','.
String str = Bytes.toString(data);
int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR);
if (index != -1) {
// Presume its ServerName serialized with versioned bytes.
return ServerName.parseVersionedServerName(data);
}
// Presume it a hostname:port format.
String hostname = Addressing.parseHostname(str);
int port = Addressing.parsePort(str);
return new ServerName(hostname, port, -1L);
}
}

View File

@ -89,7 +89,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
// znode containing ephemeral nodes of the draining regionservers
public String drainingZNode;
// znode of currently active master
public String masterAddressZNode;
private String masterAddressZNode;
// znode of this master in backup master directory, if not the active master
public String backupMasterAddressesZNode;
// znode containing the current cluster state
@ -455,4 +455,11 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
public boolean isAborted() {
return this.abortable.isAborted();
}
/**
* @return Path to the currently active master.
*/
public String getMasterAddressZNode() {
return this.masterAddressZNode;
}
}

View File

@ -34,3 +34,11 @@ message RootRegionServer {
// The ServerName hosting the root region currently.
required ServerName server = 1;
}
/**
* Content of the master znode.
*/
message Master {
// The ServerName of the current Master
required ServerName master = 1;
}

View File

@ -29,14 +29,17 @@ import java.util.concurrent.Semaphore;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -66,7 +69,7 @@ public class TestActiveMasterManager {
ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
"testActiveMasterManagerFromZK", null, true);
try {
ZKUtil.deleteNode(zk, zk.masterAddressZNode);
ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
ZKUtil.deleteNode(zk, zk.clusterStateZNode);
} catch(KeeperException.NoNodeException nne) {}
@ -108,7 +111,7 @@ public class TestActiveMasterManager {
ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
"testActiveMasterManagerFromZK", null, true);
try {
ZKUtil.deleteNode(zk, zk.masterAddressZNode);
ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
ZKUtil.deleteNode(zk, zk.clusterStateZNode);
} catch(KeeperException.NoNodeException nne) {}
@ -154,11 +157,11 @@ public class TestActiveMasterManager {
ms1.stop("stopping first server");
// Use a listener to capture when the node is actually deleted
NodeDeletionListener listener = new NodeDeletionListener(zk, zk.masterAddressZNode);
NodeDeletionListener listener = new NodeDeletionListener(zk, zk.getMasterAddressZNode());
zk.registerListener(listener);
LOG.info("Deleting master node");
ZKUtil.deleteNode(zk, zk.masterAddressZNode);
ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
// Wait for the node to be deleted
LOG.info("Waiting for active master manager to be notified");
@ -178,7 +181,7 @@ public class TestActiveMasterManager {
assertTrue(t.isActiveMaster);
LOG.info("Deleting master node");
ZKUtil.deleteNode(zk, zk.masterAddressZNode);
ZKUtil.deleteNode(zk, zk.getMasterAddressZNode());
}
/**
@ -186,12 +189,12 @@ public class TestActiveMasterManager {
* @param zk
* @param thisMasterAddress
* @throws KeeperException
* @throws IOException
*/
private void assertMaster(ZooKeeperWatcher zk,
ServerName expectedAddress)
throws KeeperException {
ServerName readAddress =
ServerName.parseVersionedServerName(ZKUtil.getData(zk, zk.masterAddressZNode));
throws KeeperException, IOException {
ServerName readAddress = MasterAddressTracker.getMasterAddress(zk);
assertNotNull(readAddress);
assertTrue(expectedAddress.equals(readAddress));
}

View File

@ -27,6 +27,7 @@ import java.util.concurrent.Semaphore;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -69,7 +70,7 @@ public class TestMasterAddressManager {
zk.registerListener(addressManager);
// Use a listener to capture when the node is actually created
NodeCreationListener listener = new NodeCreationListener(zk, zk.masterAddressZNode);
NodeCreationListener listener = new NodeCreationListener(zk, zk.getMasterAddressZNode());
zk.registerListener(listener);
// Create the master node with a dummy address
@ -77,7 +78,7 @@ public class TestMasterAddressManager {
int port = 1234;
ServerName sn = new ServerName(host, port, System.currentTimeMillis());
LOG.info("Creating master node");
ZKUtil.createEphemeralNodeAndWatch(zk, zk.masterAddressZNode, sn.getVersionedBytes());
MasterAddressTracker.setMasterAddress(zk, zk.getMasterAddressZNode(), sn);
// Wait for the node to be created
LOG.info("Waiting for master address manager to be notified");

View File

@ -28,12 +28,12 @@ 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.MasterAddressTracker;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
import org.apache.hadoop.hbase.tmpl.regionserver.RSStatusTmpl;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Before;
import org.junit.Test;