From d53d8462a62c4640156d5d23497c5e4aff0ef61b Mon Sep 17 00:00:00 2001 From: thiruvel Date: Tue, 18 Oct 2016 19:37:19 -0700 Subject: [PATCH] HBASE-15532: core favored nodes enhancements (draft version) --- .../java/org/apache/hadoop/hbase/HRegionInfo.java | 20 + .../java/org/apache/hadoop/hbase/client/Admin.java | 48 + .../hbase/client/ConnectionImplementation.java | 56 + .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 57 +- .../hbase/shaded/protobuf/RequestConverter.java | 20 + .../java/org/apache/hadoop/hbase/ServerName.java | 10 +- .../shaded/protobuf/generated/HBaseProtos.java | 2456 +++++- .../shaded/protobuf/generated/MasterProtos.java | 8211 +++++++++++++++++++- .../src/main/protobuf/HBase.proto | 16 + .../src/main/protobuf/Master.proto | 70 + .../hbase/protobuf/generated/HBaseProtos.java | 2370 +++++- hbase-protocol/src/main/protobuf/HBase.proto | 16 + hbase-rsgroup/pom.xml | 2 +- .../hadoop/hbase/StartcodeAgnosticServerName.java | 67 + .../hadoop/hbase/favored/FavoredNodesManager.java | 299 + .../hadoop/hbase/master/AssignmentManager.java | 98 +- .../apache/hadoop/hbase/master/CatalogJanitor.java | 11 + .../org/apache/hadoop/hbase/master/HMaster.java | 35 +- .../hadoop/hbase/master/MasterRpcServices.java | 236 + .../apache/hadoop/hbase/master/MasterServices.java | 6 + .../apache/hadoop/hbase/master/RackManager.java | 10 +- .../apache/hadoop/hbase/master/ServerManager.java | 23 + .../master/SnapshotOfRegionAssignmentFromMeta.java | 69 +- .../hbase/master/balancer/BaseLoadBalancer.java | 8 +- .../balancer/FavoredNodeAssignmentHelper.java | 507 +- .../master/balancer/FavoredNodeLoadBalancer.java | 386 +- .../hbase/master/balancer/FavoredNodesPlan.java | 35 +- .../master/balancer/FavoredNodesPromoter.java | 47 + .../master/balancer/FavoredNodesRepairChore.java | 83 + .../master/balancer/FavoredStochasticBalancer.java | 861 ++ .../hbase/master/balancer/ServerAndLoad.java | 4 +- .../master/balancer/StochasticLoadBalancer.java | 47 +- .../master/procedure/DeleteTableProcedure.java | 7 + .../hadoop/hbase/regionserver/HRegionServer.java | 25 +- .../apache/hadoop/hbase/regionserver/HStore.java | 6 +- .../hadoop/hbase/regionserver/RSRpcServices.java | 13 + .../regionserver/RegionMergeTransactionImpl.java | 17 + .../org/apache/hadoop/hbase/MiniHBaseCluster.java | 23 + .../org/apache/hadoop/hbase/TestServerName.java | 29 +- .../org/apache/hadoop/hbase/TestZooKeeper.java | 2 +- .../hbase/master/MockNoopMasterServices.java | 5 + .../master/TestAssignmentManagerOnCluster.java | 7 +- .../hbase/master/balancer/BalancerTestBase.java | 1 + .../balancer/FavoredStochasticBalancerTest.java | 39 + .../balancer/TestFavoredNodeAssignmentHelper.java | 53 +- .../balancer/TestFavoredNodeLoadBalancer.java | 131 + .../balancer/TestFavoredNodesRepairChore.java | 167 + .../TestFavoredStochasticBalancerPickers.java | 451 ++ .../TestFavoredStochasticLoadBalancer.java | 446 ++ .../balancer/TestStochasticLoadBalancer.java | 4 +- hbase-shell/src/main/ruby/hbase/admin.rb | 24 + hbase-shell/src/main/ruby/shell.rb | 4 + .../ruby/shell/commands/check_favored_nodes.rb | 35 + .../ruby/shell/commands/complete_redistribute.rb | 44 + .../src/main/ruby/shell/commands/redistribute.rb | 46 + .../ruby/shell/commands/remove_favored_node.rb | 34 + 56 files changed, 17099 insertions(+), 698 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/StartcodeAgnosticServerName.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPromoter.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesRepairChore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancerTest.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeLoadBalancer.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodesRepairChore.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java create mode 100644 hbase-shell/src/main/ruby/shell/commands/check_favored_nodes.rb create mode 100644 hbase-shell/src/main/ruby/shell/commands/complete_redistribute.rb create mode 100644 hbase-shell/src/main/ruby/shell/commands/redistribute.rb create mode 100644 hbase-shell/src/main/ruby/shell/commands/remove_favored_node.rb diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java index da0d941..660ab60 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java @@ -30,12 +30,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo; import org.apache.hadoop.hbase.util.ByteArrayHashKey; import org.apache.hadoop.hbase.util.Bytes; @@ -44,6 +47,8 @@ import org.apache.hadoop.hbase.util.JenkinsHash; import org.apache.hadoop.hbase.util.MD5Hash; import org.apache.hadoop.io.DataInputBuffer; +import com.google.common.collect.Lists; + /** * Information about a region. A region is a range of keys in the whole keyspace of a table, an * identifier (a timestamp) for differentiating between subset ranges (after region split) @@ -991,6 +996,21 @@ public class HRegionInfo implements Comparable { } } + public static List getFavoredNodes(final Result r) + throws InvalidProtocolBufferException { + byte[] favoredNodes = r.getValue(HConstants.CATALOG_FAMILY, Bytes.toBytes("fn")); + if (favoredNodes != null) { + FavoredNodes f = FavoredNodes.parseFrom(favoredNodes); + List protoNodes = f.getFavoredNodeList(); + List servers = Lists.newArrayList(); + for (HBaseProtos.ServerName node : protoNodes) { + servers.add(ProtobufUtil.toServerName(node)); + } + return servers; + } + return null; + } + /** * Use this instead of {@link #toByteArray()} when writing to a stream and you want to use * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want). diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 34c00ce..888af68 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.ProcedureInfo; @@ -38,6 +39,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.security.SecurityCapability; @@ -46,12 +48,15 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter; import org.apache.hadoop.hbase.quotas.QuotaRetriever; import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; import org.apache.hadoop.hbase.util.Pair; +import com.google.common.net.HostAndPort; + /** * The administrative API for HBase. Obtain an instance from an {@link Connection#getAdmin()} and * call {@link #close()} afterwards. @@ -1790,4 +1795,47 @@ public interface Admin extends Abortable, Closeable { * @return true if the switch is enabled, false otherwise. */ boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException; + + /** + * Redistribute favored nodes without changing existing assignments. + * + * @return true if redistribute ran, false otherwise. + * @throws MasterNotRunningException + * @throws ZooKeeperConnectionException + * @throws ServiceException + */ + boolean redistributeFavoredNodes() + throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException; + + /** + * Generate new favored nodes for all regions and assigns them to new region servers. + * + * @return true if completeRedistributeFavoredNodes ran, false otherwise. + * @throws MasterNotRunningException + * @throws ZooKeeperConnectionException + * @throws ServiceException + */ + boolean completeRedistributeFavoredNodes() + throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException; + + /** + * Removes specified server as favored node from all regions and generates new server as + * replacement. + * + * @throws MasterNotRunningException + * @throws ZooKeeperConnectionException + * @throws ServiceException + */ + void removeFavoredNode(HostAndPort hostAndPort) + throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException; + + /** + * Scans all regions and returns a list of dead favored node servers. + * + * @throws MasterNotRunningException + * @throws ZooKeeperConnectionException + * @throws ServiceException + */ + List checkFavoredNodes() + throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 53eb522..d58ca63 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -72,15 +72,29 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest; @@ -1657,6 +1671,48 @@ class ConnectionImplementation implements ClusterConnection, Closeable { SecurityCapabilitiesRequest request) throws ServiceException { return stub.getSecurityCapabilities(controller, request); } + + @Override + public CheckFavoredNodesResponse checkFavoredNodes(RpcController controller, + CheckFavoredNodesRequest request) throws ServiceException { + return stub.checkFavoredNodes(controller, request); + } + + @Override + public RemoveFavoredNodeResponse removeFavoredNode(RpcController controller, + RemoveFavoredNodeRequest request) throws ServiceException { + return stub.removeFavoredNode(controller, request); + } + + @Override + public GetReplicaLoadResponse getReplicaLoad(RpcController controller, + GetReplicaLoadRequest request) throws ServiceException { + return stub.getReplicaLoad(controller, request); + } + + @Override + public GetFavoredNodesForRegionResponse getFavoredNodesForRegion(RpcController controller, + GetFavoredNodesForRegionRequest request) throws ServiceException { + return stub.getFavoredNodesForRegion(controller, request); + } + + @Override + public CompleteRedistributeFavoredNodesResponse completeRedistributeFavoredNodes(RpcController controller, + CompleteRedistributeFavoredNodesRequest request) throws ServiceException { + return stub.completeRedistributeFavoredNodes(controller, request); + } + + @Override + public RedistributeFavoredNodesResponse redistributeFavoredNodes(RpcController controller, + RedistributeFavoredNodesRequest request) throws ServiceException { + return stub.redistributeFavoredNodes(controller, request); + } + + @Override + public UpdateFavoredNodesResponse updateFavoredNodesForRegion(RpcController controller, + UpdateFavoredNodesRequest request) throws ServiceException { + return stub.updateFavoredNodesForRegion(controller, request); + } }; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 51d07e3..6935224 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -108,6 +108,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateName import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest; @@ -188,6 +189,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; import com.google.protobuf.RpcController; +import com.google.common.collect.Lists; +import com.google.common.net.HostAndPort; /** * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that @@ -3738,4 +3741,56 @@ public class HBaseAdmin implements Admin { private RpcControllerFactory getRpcControllerFactory() { return this.rpcControllerFactory; } -} + + @Override + public boolean redistributeFavoredNodes() throws MasterNotRunningException, + ZooKeeperConnectionException, ServiceException { + MasterKeepAliveConnection stub = connection.getKeepAliveMasterService(); + try { + return stub.redistributeFavoredNodes(null, RequestConverter.buildRedistributeRequest()) + .getResult(); + } finally { + stub.close(); + } + } + + @Override + public boolean completeRedistributeFavoredNodes() throws MasterNotRunningException, + ZooKeeperConnectionException, ServiceException { + MasterKeepAliveConnection stub = connection.getKeepAliveMasterService(); + try { + return stub.completeRedistributeFavoredNodes(null, + RequestConverter.buildCompleteRedistributeRequest()).getResult(); + } finally { + stub.close(); + } + } + + @Override + public void removeFavoredNode(HostAndPort hostAndPort) throws MasterNotRunningException, + ZooKeeperConnectionException, ServiceException { + MasterKeepAliveConnection stub = connection.getKeepAliveMasterService(); + try { + stub.removeFavoredNode(null, RequestConverter.buildRemoveFavoredNodeRequest(ServerName + .valueOf(hostAndPort, ServerName.NON_STARTCODE))); + } finally { + stub.close(); + } + } + + @Override + public List checkFavoredNodes() throws MasterNotRunningException, + ZooKeeperConnectionException, ServiceException { + MasterKeepAliveConnection stub = connection.getKeepAliveMasterService(); + List result = Lists.newArrayList(); + try { + CheckFavoredNodesResponse response = stub.checkFavoredNodes(null, RequestConverter.buildCheckFavoredNodesRequest()); + for (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName sn : response.getServersList()) { + result.add(ProtobufUtil.toServerName(sn)); + } + } finally { + stub.close(); + } + return result; + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java index 7da3727..cfefa92 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java @@ -77,6 +77,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest; @@ -99,6 +101,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTabl import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest; @@ -1495,4 +1499,20 @@ public final class RequestConverter { } throw new UnsupportedOperationException("Unsupport switch type:" + switchType); } + + public static RedistributeFavoredNodesRequest buildRedistributeRequest() { + return RedistributeFavoredNodesRequest.newBuilder().build(); + } + + public static CompleteRedistributeFavoredNodesRequest buildCompleteRedistributeRequest() { + return CompleteRedistributeFavoredNodesRequest.newBuilder().build(); + } + + public static RemoveFavoredNodeRequest buildRemoveFavoredNodeRequest(ServerName sn) { + return RemoveFavoredNodeRequest.newBuilder().setServer(ProtobufUtil.toServerName(sn)).build(); + } + + public static CheckFavoredNodesRequest buildCheckFavoredNodesRequest() { + return CheckFavoredNodesRequest.newBuilder().build(); + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java index 8d18db0..9f49b56 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java @@ -75,10 +75,12 @@ import com.google.common.net.InetAddresses; */ public static final String SERVERNAME_SEPARATOR = ","; + public static final String VALID_START_CODE_REGEX = "(-1|([\\d]+))"; + public static final Pattern SERVERNAME_PATTERN = Pattern.compile("[^" + SERVERNAME_SEPARATOR + "]+" + SERVERNAME_SEPARATOR + Addressing.VALID_PORT_REGEX + - SERVERNAME_SEPARATOR + Addressing.VALID_PORT_REGEX + "$"); + SERVERNAME_SEPARATOR + VALID_START_CODE_REGEX + "$"); /** * What to use if server name is unknown. @@ -98,7 +100,7 @@ import com.google.common.net.InetAddresses; private byte [] bytes; public static final List EMPTY_SERVER_LIST = new ArrayList(0); - private ServerName(final String hostname, final int port, final long startcode) { + protected ServerName(final String hostname, final int port, final long startcode) { // Drop the domain is there is one; no need of it in a local cluster. With it, we get long // unwieldy names. this.hostnameOnly = hostname; @@ -176,6 +178,10 @@ import com.google.common.net.InetAddresses; return new ServerName(hostAndPort, startCode); } + public static ServerName valueOf(final HostAndPort hostAndPort, final long startCode) { + return new ServerName(hostAndPort.getHostText(), hostAndPort.getPort(), startCode); + } + @Override public String toString() { return getServerName(); diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java index 0617426..f5d1e17 100644 --- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/HBaseProtos.java @@ -20307,6 +20307,2404 @@ public final class HBaseProtos { } + public interface ServerReplicaLoadPairOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.ServerReplicaLoadPair) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required .hbase.pb.ServerName server = 1; + */ + boolean hasServer(); + /** + * required .hbase.pb.ServerName server = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServer(); + /** + * required .hbase.pb.ServerName server = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder(); + + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + boolean hasReplicaCount(); + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad getReplicaCount(); + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder getReplicaCountOrBuilder(); + } + /** + * Protobuf type {@code hbase.pb.ServerReplicaLoadPair} + */ + public static final class ServerReplicaLoadPair extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.ServerReplicaLoadPair) + ServerReplicaLoadPairOrBuilder { + // Use ServerReplicaLoadPair.newBuilder() to construct. + private ServerReplicaLoadPair(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private ServerReplicaLoadPair() { + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ServerReplicaLoadPair( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = server_.toBuilder(); + } + server_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(server_); + server_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = replicaCount_.toBuilder(); + } + replicaCount_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(replicaCount_); + replicaCount_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.class, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder.class); + } + + private int bitField0_; + public static final int SERVER_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_; + /** + * required .hbase.pb.ServerName server = 1; + */ + public boolean hasServer() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServer() { + return server_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + return server_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } + + public static final int REPLICACOUNT_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad replicaCount_; + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public boolean hasReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad getReplicaCount() { + return replicaCount_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance() : replicaCount_; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder getReplicaCountOrBuilder() { + return replicaCount_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance() : replicaCount_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasServer()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasReplicaCount()) { + memoizedIsInitialized = 0; + return false; + } + if (!getServer().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getReplicaCount().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, getServer()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, getReplicaCount()); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getServer()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(2, getReplicaCount()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair other = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair) obj; + + boolean result = true; + result = result && (hasServer() == other.hasServer()); + if (hasServer()) { + result = result && getServer() + .equals(other.getServer()); + } + result = result && (hasReplicaCount() == other.hasReplicaCount()); + if (hasReplicaCount()) { + result = result && getReplicaCount() + .equals(other.getReplicaCount()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasServer()) { + hash = (37 * hash) + SERVER_FIELD_NUMBER; + hash = (53 * hash) + getServer().hashCode(); + } + if (hasReplicaCount()) { + hash = (37 * hash) + REPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + getReplicaCount().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.ServerReplicaLoadPair} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.ServerReplicaLoadPair) + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.class, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getServerFieldBuilder(); + getReplicaCountFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (serverBuilder_ == null) { + server_ = null; + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (replicaCountBuilder_ == null) { + replicaCount_ = null; + } else { + replicaCountBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair result = new org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (serverBuilder_ == null) { + result.server_ = server_; + } else { + result.server_ = serverBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (replicaCountBuilder_ == null) { + result.replicaCount_ = replicaCount_; + } else { + result.replicaCount_ = replicaCountBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.getDefaultInstance()) return this; + if (other.hasServer()) { + mergeServer(other.getServer()); + } + if (other.hasReplicaCount()) { + mergeReplicaCount(other.getReplicaCount()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasServer()) { + return false; + } + if (!hasReplicaCount()) { + return false; + } + if (!getServer().isInitialized()) { + return false; + } + if (!getReplicaCount().isInitialized()) { + return false; + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_; + /** + * required .hbase.pb.ServerName server = 1; + */ + public boolean hasServer() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServer() { + if (serverBuilder_ == null) { + return server_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } else { + return serverBuilder_.getMessage(); + } + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder setServer(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + server_ = value; + onChanged(); + } else { + serverBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder setServer( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serverBuilder_ == null) { + server_ = builderForValue.build(); + onChanged(); + } else { + serverBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder mergeServer(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + server_ != null && + server_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) { + server_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.newBuilder(server_).mergeFrom(value).buildPartial(); + } else { + server_ = value; + } + onChanged(); + } else { + serverBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder clearServer() { + if (serverBuilder_ == null) { + server_ = null; + onChanged(); + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServerBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getServerFieldBuilder().getBuilder(); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + if (serverBuilder_ != null) { + return serverBuilder_.getMessageOrBuilder(); + } else { + return server_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } + } + /** + * required .hbase.pb.ServerName server = 1; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServerFieldBuilder() { + if (serverBuilder_ == null) { + serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + getServer(), + getParentForChildren(), + isClean()); + server_ = null; + } + return serverBuilder_; + } + + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad replicaCount_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder> replicaCountBuilder_; + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public boolean hasReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad getReplicaCount() { + if (replicaCountBuilder_ == null) { + return replicaCount_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance() : replicaCount_; + } else { + return replicaCountBuilder_.getMessage(); + } + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder setReplicaCount(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad value) { + if (replicaCountBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + replicaCount_ = value; + onChanged(); + } else { + replicaCountBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder setReplicaCount( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder builderForValue) { + if (replicaCountBuilder_ == null) { + replicaCount_ = builderForValue.build(); + onChanged(); + } else { + replicaCountBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder mergeReplicaCount(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad value) { + if (replicaCountBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + replicaCount_ != null && + replicaCount_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance()) { + replicaCount_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.newBuilder(replicaCount_).mergeFrom(value).buildPartial(); + } else { + replicaCount_ = value; + } + onChanged(); + } else { + replicaCountBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder clearReplicaCount() { + if (replicaCountBuilder_ == null) { + replicaCount_ = null; + onChanged(); + } else { + replicaCountBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder getReplicaCountBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getReplicaCountFieldBuilder().getBuilder(); + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder getReplicaCountOrBuilder() { + if (replicaCountBuilder_ != null) { + return replicaCountBuilder_.getMessageOrBuilder(); + } else { + return replicaCount_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance() : replicaCount_; + } + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder> + getReplicaCountFieldBuilder() { + if (replicaCountBuilder_ == null) { + replicaCountBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder>( + getReplicaCount(), + getParentForChildren(), + isClean()); + replicaCount_ = null; + } + return replicaCountBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.ServerReplicaLoadPair) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.ServerReplicaLoadPair) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public ServerReplicaLoadPair parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new ServerReplicaLoadPair(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface ReplicaLoadOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicaLoad) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required int64 primaryReplicaCount = 1; + */ + boolean hasPrimaryReplicaCount(); + /** + * required int64 primaryReplicaCount = 1; + */ + long getPrimaryReplicaCount(); + + /** + * required int64 secondaryReplicaCount = 2; + */ + boolean hasSecondaryReplicaCount(); + /** + * required int64 secondaryReplicaCount = 2; + */ + long getSecondaryReplicaCount(); + + /** + * required int64 tertiaryReplicaCount = 3; + */ + boolean hasTertiaryReplicaCount(); + /** + * required int64 tertiaryReplicaCount = 3; + */ + long getTertiaryReplicaCount(); + } + /** + * Protobuf type {@code hbase.pb.ReplicaLoad} + */ + public static final class ReplicaLoad extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.ReplicaLoad) + ReplicaLoadOrBuilder { + // Use ReplicaLoad.newBuilder() to construct. + private ReplicaLoad(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private ReplicaLoad() { + primaryReplicaCount_ = 0L; + secondaryReplicaCount_ = 0L; + tertiaryReplicaCount_ = 0L; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ReplicaLoad( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + primaryReplicaCount_ = input.readInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + secondaryReplicaCount_ = input.readInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + tertiaryReplicaCount_ = input.readInt64(); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.class, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder.class); + } + + private int bitField0_; + public static final int PRIMARYREPLICACOUNT_FIELD_NUMBER = 1; + private long primaryReplicaCount_; + /** + * required int64 primaryReplicaCount = 1; + */ + public boolean hasPrimaryReplicaCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required int64 primaryReplicaCount = 1; + */ + public long getPrimaryReplicaCount() { + return primaryReplicaCount_; + } + + public static final int SECONDARYREPLICACOUNT_FIELD_NUMBER = 2; + private long secondaryReplicaCount_; + /** + * required int64 secondaryReplicaCount = 2; + */ + public boolean hasSecondaryReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public long getSecondaryReplicaCount() { + return secondaryReplicaCount_; + } + + public static final int TERTIARYREPLICACOUNT_FIELD_NUMBER = 3; + private long tertiaryReplicaCount_; + /** + * required int64 tertiaryReplicaCount = 3; + */ + public boolean hasTertiaryReplicaCount() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public long getTertiaryReplicaCount() { + return tertiaryReplicaCount_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasPrimaryReplicaCount()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSecondaryReplicaCount()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTertiaryReplicaCount()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt64(1, primaryReplicaCount_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt64(2, secondaryReplicaCount_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt64(3, tertiaryReplicaCount_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeInt64Size(1, primaryReplicaCount_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeInt64Size(2, secondaryReplicaCount_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeInt64Size(3, tertiaryReplicaCount_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad other = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad) obj; + + boolean result = true; + result = result && (hasPrimaryReplicaCount() == other.hasPrimaryReplicaCount()); + if (hasPrimaryReplicaCount()) { + result = result && (getPrimaryReplicaCount() + == other.getPrimaryReplicaCount()); + } + result = result && (hasSecondaryReplicaCount() == other.hasSecondaryReplicaCount()); + if (hasSecondaryReplicaCount()) { + result = result && (getSecondaryReplicaCount() + == other.getSecondaryReplicaCount()); + } + result = result && (hasTertiaryReplicaCount() == other.hasTertiaryReplicaCount()); + if (hasTertiaryReplicaCount()) { + result = result && (getTertiaryReplicaCount() + == other.getTertiaryReplicaCount()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasPrimaryReplicaCount()) { + hash = (37 * hash) + PRIMARYREPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getPrimaryReplicaCount()); + } + if (hasSecondaryReplicaCount()) { + hash = (37 * hash) + SECONDARYREPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getSecondaryReplicaCount()); + } + if (hasTertiaryReplicaCount()) { + hash = (37 * hash) + TERTIARYREPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong( + getTertiaryReplicaCount()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.ReplicaLoad} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicaLoad) + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.class, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + primaryReplicaCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + secondaryReplicaCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + tertiaryReplicaCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad result = new org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.primaryReplicaCount_ = primaryReplicaCount_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.secondaryReplicaCount_ = secondaryReplicaCount_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.tertiaryReplicaCount_ = tertiaryReplicaCount_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance()) return this; + if (other.hasPrimaryReplicaCount()) { + setPrimaryReplicaCount(other.getPrimaryReplicaCount()); + } + if (other.hasSecondaryReplicaCount()) { + setSecondaryReplicaCount(other.getSecondaryReplicaCount()); + } + if (other.hasTertiaryReplicaCount()) { + setTertiaryReplicaCount(other.getTertiaryReplicaCount()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasPrimaryReplicaCount()) { + return false; + } + if (!hasSecondaryReplicaCount()) { + return false; + } + if (!hasTertiaryReplicaCount()) { + return false; + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private long primaryReplicaCount_ ; + /** + * required int64 primaryReplicaCount = 1; + */ + public boolean hasPrimaryReplicaCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required int64 primaryReplicaCount = 1; + */ + public long getPrimaryReplicaCount() { + return primaryReplicaCount_; + } + /** + * required int64 primaryReplicaCount = 1; + */ + public Builder setPrimaryReplicaCount(long value) { + bitField0_ |= 0x00000001; + primaryReplicaCount_ = value; + onChanged(); + return this; + } + /** + * required int64 primaryReplicaCount = 1; + */ + public Builder clearPrimaryReplicaCount() { + bitField0_ = (bitField0_ & ~0x00000001); + primaryReplicaCount_ = 0L; + onChanged(); + return this; + } + + private long secondaryReplicaCount_ ; + /** + * required int64 secondaryReplicaCount = 2; + */ + public boolean hasSecondaryReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public long getSecondaryReplicaCount() { + return secondaryReplicaCount_; + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public Builder setSecondaryReplicaCount(long value) { + bitField0_ |= 0x00000002; + secondaryReplicaCount_ = value; + onChanged(); + return this; + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public Builder clearSecondaryReplicaCount() { + bitField0_ = (bitField0_ & ~0x00000002); + secondaryReplicaCount_ = 0L; + onChanged(); + return this; + } + + private long tertiaryReplicaCount_ ; + /** + * required int64 tertiaryReplicaCount = 3; + */ + public boolean hasTertiaryReplicaCount() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public long getTertiaryReplicaCount() { + return tertiaryReplicaCount_; + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public Builder setTertiaryReplicaCount(long value) { + bitField0_ |= 0x00000004; + tertiaryReplicaCount_ = value; + onChanged(); + return this; + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public Builder clearTertiaryReplicaCount() { + bitField0_ = (bitField0_ & ~0x00000004); + tertiaryReplicaCount_ = 0L; + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicaLoad) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.ReplicaLoad) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public ReplicaLoad parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new ReplicaLoad(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface FavoredNodesInfoPairOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.FavoredNodesInfoPair) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required string regionName = 1; + */ + boolean hasRegionName(); + /** + * required string regionName = 1; + */ + java.lang.String getRegionName(); + /** + * required string regionName = 1; + */ + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getRegionNameBytes(); + + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + java.util.List + getServersList(); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + int getServersCount(); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + java.util.List + getServersOrBuilderList(); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index); + } + /** + * Protobuf type {@code hbase.pb.FavoredNodesInfoPair} + */ + public static final class FavoredNodesInfoPair extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.FavoredNodesInfoPair) + FavoredNodesInfoPairOrBuilder { + // Use FavoredNodesInfoPair.newBuilder() to construct. + private FavoredNodesInfoPair(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private FavoredNodesInfoPair() { + regionName_ = ""; + servers_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private FavoredNodesInfoPair( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + regionName_ = bs; + break; + } + case 18: { + if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000002; + } + servers_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry)); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.class, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.Builder.class); + } + + private int bitField0_; + public static final int REGIONNAME_FIELD_NUMBER = 1; + private volatile java.lang.Object regionName_; + /** + * required string regionName = 1; + */ + public boolean hasRegionName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string regionName = 1; + */ + public java.lang.String getRegionName() { + java.lang.Object ref = regionName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + regionName_ = s; + } + return s; + } + } + /** + * required string regionName = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getRegionNameBytes() { + java.lang.Object ref = regionName_; + if (ref instanceof java.lang.String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + regionName_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + + public static final int SERVERS_FIELD_NUMBER = 2; + private java.util.List servers_; + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List getServersList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List + getServersOrBuilderList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public int getServersCount() { + return servers_.size(); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + return servers_.get(index); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + return servers_.get(index); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasRegionName()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, regionName_); + } + for (int i = 0; i < servers_.size(); i++) { + output.writeMessage(2, servers_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, regionName_); + } + for (int i = 0; i < servers_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(2, servers_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair other = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair) obj; + + boolean result = true; + result = result && (hasRegionName() == other.hasRegionName()); + if (hasRegionName()) { + result = result && getRegionName() + .equals(other.getRegionName()); + } + result = result && getServersList() + .equals(other.getServersList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRegionName()) { + hash = (37 * hash) + REGIONNAME_FIELD_NUMBER; + hash = (53 * hash) + getRegionName().hashCode(); + } + if (getServersCount() > 0) { + hash = (37 * hash) + SERVERS_FIELD_NUMBER; + hash = (53 * hash) + getServersList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.FavoredNodesInfoPair} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.FavoredNodesInfoPair) + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPairOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.class, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getServersFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + regionName_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + } else { + serversBuilder_.clear(); + } + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair result = new org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.regionName_ = regionName_; + if (serversBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.servers_ = servers_; + } else { + result.servers_ = serversBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.getDefaultInstance()) return this; + if (other.hasRegionName()) { + bitField0_ |= 0x00000001; + regionName_ = other.regionName_; + onChanged(); + } + if (serversBuilder_ == null) { + if (!other.servers_.isEmpty()) { + if (servers_.isEmpty()) { + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureServersIsMutable(); + servers_.addAll(other.servers_); + } + onChanged(); + } + } else { + if (!other.servers_.isEmpty()) { + if (serversBuilder_.isEmpty()) { + serversBuilder_.dispose(); + serversBuilder_ = null; + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000002); + serversBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getServersFieldBuilder() : null; + } else { + serversBuilder_.addAllMessages(other.servers_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasRegionName()) { + return false; + } + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.lang.Object regionName_ = ""; + /** + * required string regionName = 1; + */ + public boolean hasRegionName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string regionName = 1; + */ + public java.lang.String getRegionName() { + java.lang.Object ref = regionName_; + if (!(ref instanceof java.lang.String)) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = + (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + regionName_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string regionName = 1; + */ + public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString + getRegionNameBytes() { + java.lang.Object ref = regionName_; + if (ref instanceof String) { + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + regionName_ = b; + return b; + } else { + return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref; + } + } + /** + * required string regionName = 1; + */ + public Builder setRegionName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + regionName_ = value; + onChanged(); + return this; + } + /** + * required string regionName = 1; + */ + public Builder clearRegionName() { + bitField0_ = (bitField0_ & ~0x00000001); + regionName_ = getDefaultInstance().getRegionName(); + onChanged(); + return this; + } + /** + * required string regionName = 1; + */ + public Builder setRegionNameBytes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + regionName_ = value; + onChanged(); + return this; + } + + private java.util.List servers_ = + java.util.Collections.emptyList(); + private void ensureServersIsMutable() { + if (!((bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = new java.util.ArrayList(servers_); + bitField0_ |= 0x00000002; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serversBuilder_; + + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List getServersList() { + if (serversBuilder_ == null) { + return java.util.Collections.unmodifiableList(servers_); + } else { + return serversBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public int getServersCount() { + if (serversBuilder_ == null) { + return servers_.size(); + } else { + return serversBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + if (serversBuilder_ == null) { + return servers_.get(index); + } else { + return serversBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.set(index, value); + onChanged(); + } else { + serversBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.set(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(value); + onChanged(); + } else { + serversBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(index, value); + onChanged(); + } else { + serversBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addAllServers( + java.lang.Iterable values) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, servers_); + onChanged(); + } else { + serversBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder clearServers() { + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + } else { + serversBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder removeServers(int index) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.remove(index); + onChanged(); + } else { + serversBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServersBuilder( + int index) { + return getServersFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + if (serversBuilder_ == null) { + return servers_.get(index); } else { + return serversBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List + getServersOrBuilderList() { + if (serversBuilder_ != null) { + return serversBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(servers_); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder() { + return getServersFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder( + int index) { + return getServersFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List + getServersBuilderList() { + return getServersFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServersFieldBuilder() { + if (serversBuilder_ == null) { + serversBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + servers_, + ((bitField0_ & 0x00000002) == 0x00000002), + getParentForChildren(), + isClean()); + servers_ = null; + } + return serversBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.FavoredNodesInfoPair) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.FavoredNodesInfoPair) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public FavoredNodesInfoPair parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new FavoredNodesInfoPair(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodesInfoPair getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_TableName_descriptor; private static final @@ -20432,6 +22830,21 @@ public final class HBaseProtos { private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_ReplicaLoad_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable; public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -20495,15 +22908,22 @@ public final class HBaseProtos { "rc_checksum\030\006 \002(\t\022\025\n\rversion_major\030\007 \001(\r" + "\022\025\n\rversion_minor\030\010 \001(\r\"Q\n\020RegionServerI" + "nfo\022\020\n\010infoPort\030\001 \001(\005\022+\n\014version_info\030\002 " + - "\001(\0132\025.hbase.pb.VersionInfo*r\n\013CompareTyp" + - "e\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020" + - "\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013" + - "\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NA" + - "NOSECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISE" + - "CONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOU", - "RS\020\006\022\010\n\004DAYS\020\007BE\n1org.apache.hadoop.hbas" + - "e.shaded.protobuf.generatedB\013HBaseProtos" + - "H\001\240\001\001" + "\001(\0132\025.hbase.pb.VersionInfo\"j\n\025ServerRepl" + + "icaLoadPair\022$\n\006server\030\001 \002(\0132\024.hbase.pb.S" + + "erverName\022+\n\014replicaCount\030\002 \002(\0132\025.hbase." + + "pb.ReplicaLoad\"g\n\013ReplicaLoad\022\033\n\023primary" + + "ReplicaCount\030\001 \002(\003\022\035\n\025secondaryReplicaCo" + + "unt\030\002 \002(\003\022\034\n\024tertiaryReplicaCount\030\003 \002(\003\"", + "Q\n\024FavoredNodesInfoPair\022\022\n\nregionName\030\001 " + + "\002(\t\022%\n\007servers\030\002 \003(\0132\024.hbase.pb.ServerNa" + + "me*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_E" + + "QUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREA" + + "TER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n" + + "\n\010TimeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECO" + + "NDS\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007" + + "MINUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007BE\n1org.ap" + + "ache.hadoop.hbase.shaded.protobuf.genera" + + "tedB\013HBaseProtosH\001\240\001\001" }; org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -20667,6 +23087,24 @@ public final class HBaseProtos { org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_hbase_pb_RegionServerInfo_descriptor, new java.lang.String[] { "InfoPort", "VersionInfo", }); + internal_static_hbase_pb_ServerReplicaLoadPair_descriptor = + getDescriptor().getMessageTypes().get(25); + internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_ServerReplicaLoadPair_descriptor, + new java.lang.String[] { "Server", "ReplicaCount", }); + internal_static_hbase_pb_ReplicaLoad_descriptor = + getDescriptor().getMessageTypes().get(26); + internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_ReplicaLoad_descriptor, + new java.lang.String[] { "PrimaryReplicaCount", "SecondaryReplicaCount", "TertiaryReplicaCount", }); + internal_static_hbase_pb_FavoredNodesInfoPair_descriptor = + getDescriptor().getMessageTypes().get(27); + internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_FavoredNodesInfoPair_descriptor, + new java.lang.String[] { "RegionName", "Servers", }); } // @@protoc_insertion_point(outer_class_scope) diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java index 03ef208..4b4e5ec 100644 --- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java +++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java @@ -64113,6 +64113,6740 @@ public final class MasterProtos { } + public interface RedistributeFavoredNodesRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.RedistributeFavoredNodesRequest) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + } + /** + * Protobuf type {@code hbase.pb.RedistributeFavoredNodesRequest} + */ + public static final class RedistributeFavoredNodesRequest extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.RedistributeFavoredNodesRequest) + RedistributeFavoredNodesRequestOrBuilder { + // Use RedistributeFavoredNodesRequest.newBuilder() to construct. + private RedistributeFavoredNodesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private RedistributeFavoredNodesRequest() { + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RedistributeFavoredNodesRequest( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.Builder.class); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest) obj; + + boolean result = true; + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.RedistributeFavoredNodesRequest} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.RedistributeFavoredNodesRequest) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequestOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesRequest_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest(this); + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.getDefaultInstance()) return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.RedistributeFavoredNodesRequest) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.RedistributeFavoredNodesRequest) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public RedistributeFavoredNodesRequest parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new RedistributeFavoredNodesRequest(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface RedistributeFavoredNodesResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.RedistributeFavoredNodesResponse) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required bool result = 1; + */ + boolean hasResult(); + /** + * required bool result = 1; + */ + boolean getResult(); + } + /** + * Protobuf type {@code hbase.pb.RedistributeFavoredNodesResponse} + */ + public static final class RedistributeFavoredNodesResponse extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.RedistributeFavoredNodesResponse) + RedistributeFavoredNodesResponseOrBuilder { + // Use RedistributeFavoredNodesResponse.newBuilder() to construct. + private RedistributeFavoredNodesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private RedistributeFavoredNodesResponse() { + result_ = false; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RedistributeFavoredNodesResponse( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + result_ = input.readBool(); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.Builder.class); + } + + private int bitField0_; + public static final int RESULT_FIELD_NUMBER = 1; + private boolean result_; + /** + * required bool result = 1; + */ + public boolean hasResult() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bool result = 1; + */ + public boolean getResult() { + return result_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasResult()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBool(1, result_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeBoolSize(1, result_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse) obj; + + boolean result = true; + result = result && (hasResult() == other.hasResult()); + if (hasResult()) { + result = result && (getResult() + == other.getResult()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasResult()) { + hash = (37 * hash) + RESULT_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean( + getResult()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.RedistributeFavoredNodesResponse} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.RedistributeFavoredNodesResponse) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponseOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + result_ = false; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RedistributeFavoredNodesResponse_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.result_ = result_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.getDefaultInstance()) return this; + if (other.hasResult()) { + setResult(other.getResult()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasResult()) { + return false; + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private boolean result_ ; + /** + * required bool result = 1; + */ + public boolean hasResult() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bool result = 1; + */ + public boolean getResult() { + return result_; + } + /** + * required bool result = 1; + */ + public Builder setResult(boolean value) { + bitField0_ |= 0x00000001; + result_ = value; + onChanged(); + return this; + } + /** + * required bool result = 1; + */ + public Builder clearResult() { + bitField0_ = (bitField0_ & ~0x00000001); + result_ = false; + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.RedistributeFavoredNodesResponse) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.RedistributeFavoredNodesResponse) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public RedistributeFavoredNodesResponse parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new RedistributeFavoredNodesResponse(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface CompleteRedistributeFavoredNodesRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.CompleteRedistributeFavoredNodesRequest) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + } + /** + * Protobuf type {@code hbase.pb.CompleteRedistributeFavoredNodesRequest} + */ + public static final class CompleteRedistributeFavoredNodesRequest extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.CompleteRedistributeFavoredNodesRequest) + CompleteRedistributeFavoredNodesRequestOrBuilder { + // Use CompleteRedistributeFavoredNodesRequest.newBuilder() to construct. + private CompleteRedistributeFavoredNodesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CompleteRedistributeFavoredNodesRequest() { + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CompleteRedistributeFavoredNodesRequest( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.Builder.class); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest) obj; + + boolean result = true; + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.CompleteRedistributeFavoredNodesRequest} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.CompleteRedistributeFavoredNodesRequest) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequestOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest(this); + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.getDefaultInstance()) return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.CompleteRedistributeFavoredNodesRequest) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.CompleteRedistributeFavoredNodesRequest) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public CompleteRedistributeFavoredNodesRequest parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new CompleteRedistributeFavoredNodesRequest(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface CompleteRedistributeFavoredNodesResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.CompleteRedistributeFavoredNodesResponse) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required bool result = 1; + */ + boolean hasResult(); + /** + * required bool result = 1; + */ + boolean getResult(); + } + /** + * Protobuf type {@code hbase.pb.CompleteRedistributeFavoredNodesResponse} + */ + public static final class CompleteRedistributeFavoredNodesResponse extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.CompleteRedistributeFavoredNodesResponse) + CompleteRedistributeFavoredNodesResponseOrBuilder { + // Use CompleteRedistributeFavoredNodesResponse.newBuilder() to construct. + private CompleteRedistributeFavoredNodesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CompleteRedistributeFavoredNodesResponse() { + result_ = false; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CompleteRedistributeFavoredNodesResponse( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + result_ = input.readBool(); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.Builder.class); + } + + private int bitField0_; + public static final int RESULT_FIELD_NUMBER = 1; + private boolean result_; + /** + * required bool result = 1; + */ + public boolean hasResult() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bool result = 1; + */ + public boolean getResult() { + return result_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasResult()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBool(1, result_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeBoolSize(1, result_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse) obj; + + boolean result = true; + result = result && (hasResult() == other.hasResult()); + if (hasResult()) { + result = result && (getResult() + == other.getResult()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasResult()) { + hash = (37 * hash) + RESULT_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean( + getResult()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.CompleteRedistributeFavoredNodesResponse} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.CompleteRedistributeFavoredNodesResponse) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponseOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + result_ = false; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.result_ = result_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.getDefaultInstance()) return this; + if (other.hasResult()) { + setResult(other.getResult()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasResult()) { + return false; + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private boolean result_ ; + /** + * required bool result = 1; + */ + public boolean hasResult() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bool result = 1; + */ + public boolean getResult() { + return result_; + } + /** + * required bool result = 1; + */ + public Builder setResult(boolean value) { + bitField0_ |= 0x00000001; + result_ = value; + onChanged(); + return this; + } + /** + * required bool result = 1; + */ + public Builder clearResult() { + bitField0_ = (bitField0_ & ~0x00000001); + result_ = false; + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.CompleteRedistributeFavoredNodesResponse) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.CompleteRedistributeFavoredNodesResponse) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public CompleteRedistributeFavoredNodesResponse parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new CompleteRedistributeFavoredNodesResponse(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface GetFavoredNodesForRegionRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.GetFavoredNodesForRegionRequest) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + boolean hasRegionInfo(); + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(); + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(); + } + /** + * Protobuf type {@code hbase.pb.GetFavoredNodesForRegionRequest} + */ + public static final class GetFavoredNodesForRegionRequest extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.GetFavoredNodesForRegionRequest) + GetFavoredNodesForRegionRequestOrBuilder { + // Use GetFavoredNodesForRegionRequest.newBuilder() to construct. + private GetFavoredNodesForRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private GetFavoredNodesForRegionRequest() { + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetFavoredNodesForRegionRequest( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = regionInfo_.toBuilder(); + } + regionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(regionInfo_); + regionInfo_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.Builder.class); + } + + private int bitField0_; + public static final int REGIONINFO_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_; + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public boolean hasRegionInfo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() { + return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_; + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() { + return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasRegionInfo()) { + memoizedIsInitialized = 0; + return false; + } + if (!getRegionInfo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, getRegionInfo()); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getRegionInfo()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest) obj; + + boolean result = true; + result = result && (hasRegionInfo() == other.hasRegionInfo()); + if (hasRegionInfo()) { + result = result && getRegionInfo() + .equals(other.getRegionInfo()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRegionInfo()) { + hash = (37 * hash) + REGIONINFO_FIELD_NUMBER; + hash = (53 * hash) + getRegionInfo().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.GetFavoredNodesForRegionRequest} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.GetFavoredNodesForRegionRequest) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequestOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getRegionInfoFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (regionInfoBuilder_ == null) { + regionInfo_ = null; + } else { + regionInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionRequest_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (regionInfoBuilder_ == null) { + result.regionInfo_ = regionInfo_; + } else { + result.regionInfo_ = regionInfoBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.getDefaultInstance()) return this; + if (other.hasRegionInfo()) { + mergeRegionInfo(other.getRegionInfo()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasRegionInfo()) { + return false; + } + if (!getRegionInfo().isInitialized()) { + return false; + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_; + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public boolean hasRegionInfo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() { + if (regionInfoBuilder_ == null) { + return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_; + } else { + return regionInfoBuilder_.getMessage(); + } + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public Builder setRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) { + if (regionInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + regionInfo_ = value; + onChanged(); + } else { + regionInfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public Builder setRegionInfo( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) { + if (regionInfoBuilder_ == null) { + regionInfo_ = builderForValue.build(); + onChanged(); + } else { + regionInfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public Builder mergeRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) { + if (regionInfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + regionInfo_ != null && + regionInfo_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) { + regionInfo_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(regionInfo_).mergeFrom(value).buildPartial(); + } else { + regionInfo_ = value; + } + onChanged(); + } else { + regionInfoBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public Builder clearRegionInfo() { + if (regionInfoBuilder_ == null) { + regionInfo_ = null; + onChanged(); + } else { + regionInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getRegionInfoFieldBuilder().getBuilder(); + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() { + if (regionInfoBuilder_ != null) { + return regionInfoBuilder_.getMessageOrBuilder(); + } else { + return regionInfo_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_; + } + } + /** + * required .hbase.pb.RegionInfo regionInfo = 1; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> + getRegionInfoFieldBuilder() { + if (regionInfoBuilder_ == null) { + regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>( + getRegionInfo(), + getParentForChildren(), + isClean()); + regionInfo_ = null; + } + return regionInfoBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.GetFavoredNodesForRegionRequest) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.GetFavoredNodesForRegionRequest) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public GetFavoredNodesForRegionRequest parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new GetFavoredNodesForRegionRequest(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface GetFavoredNodesForRegionResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.GetFavoredNodesForRegionResponse) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + java.util.List + getServersList(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + int getServersCount(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + java.util.List + getServersOrBuilderList(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index); + } + /** + * Protobuf type {@code hbase.pb.GetFavoredNodesForRegionResponse} + */ + public static final class GetFavoredNodesForRegionResponse extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.GetFavoredNodesForRegionResponse) + GetFavoredNodesForRegionResponseOrBuilder { + // Use GetFavoredNodesForRegionResponse.newBuilder() to construct. + private GetFavoredNodesForRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private GetFavoredNodesForRegionResponse() { + servers_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetFavoredNodesForRegionResponse( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + servers_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry)); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.Builder.class); + } + + public static final int SERVERS_FIELD_NUMBER = 1; + private java.util.List servers_; + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List getServersList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersOrBuilderList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public int getServersCount() { + return servers_.size(); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + return servers_.get(index); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + return servers_.get(index); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < servers_.size(); i++) { + output.writeMessage(1, servers_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < servers_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, servers_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse) obj; + + boolean result = true; + result = result && getServersList() + .equals(other.getServersList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getServersCount() > 0) { + hash = (37 * hash) + SERVERS_FIELD_NUMBER; + hash = (53 * hash) + getServersList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.GetFavoredNodesForRegionResponse} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.GetFavoredNodesForRegionResponse) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponseOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getServersFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + serversBuilder_.clear(); + } + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetFavoredNodesForRegionResponse_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse(this); + int from_bitField0_ = bitField0_; + if (serversBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.servers_ = servers_; + } else { + result.servers_ = serversBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.getDefaultInstance()) return this; + if (serversBuilder_ == null) { + if (!other.servers_.isEmpty()) { + if (servers_.isEmpty()) { + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureServersIsMutable(); + servers_.addAll(other.servers_); + } + onChanged(); + } + } else { + if (!other.servers_.isEmpty()) { + if (serversBuilder_.isEmpty()) { + serversBuilder_.dispose(); + serversBuilder_ = null; + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000001); + serversBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getServersFieldBuilder() : null; + } else { + serversBuilder_.addAllMessages(other.servers_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.util.List servers_ = + java.util.Collections.emptyList(); + private void ensureServersIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = new java.util.ArrayList(servers_); + bitField0_ |= 0x00000001; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serversBuilder_; + + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List getServersList() { + if (serversBuilder_ == null) { + return java.util.Collections.unmodifiableList(servers_); + } else { + return serversBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public int getServersCount() { + if (serversBuilder_ == null) { + return servers_.size(); + } else { + return serversBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + if (serversBuilder_ == null) { + return servers_.get(index); + } else { + return serversBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.set(index, value); + onChanged(); + } else { + serversBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.set(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(value); + onChanged(); + } else { + serversBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(index, value); + onChanged(); + } else { + serversBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addAllServers( + java.lang.Iterable values) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, servers_); + onChanged(); + } else { + serversBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder clearServers() { + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + serversBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder removeServers(int index) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.remove(index); + onChanged(); + } else { + serversBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServersBuilder( + int index) { + return getServersFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + if (serversBuilder_ == null) { + return servers_.get(index); } else { + return serversBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersOrBuilderList() { + if (serversBuilder_ != null) { + return serversBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(servers_); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder() { + return getServersFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder( + int index) { + return getServersFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersBuilderList() { + return getServersFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServersFieldBuilder() { + if (serversBuilder_ == null) { + serversBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + servers_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + servers_ = null; + } + return serversBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.GetFavoredNodesForRegionResponse) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.GetFavoredNodesForRegionResponse) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public GetFavoredNodesForRegionResponse parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new GetFavoredNodesForRegionResponse(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface GetReplicaLoadRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.GetReplicaLoadRequest) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + java.util.List + getServersList(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + int getServersCount(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + java.util.List + getServersOrBuilderList(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index); + } + /** + * Protobuf type {@code hbase.pb.GetReplicaLoadRequest} + */ + public static final class GetReplicaLoadRequest extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.GetReplicaLoadRequest) + GetReplicaLoadRequestOrBuilder { + // Use GetReplicaLoadRequest.newBuilder() to construct. + private GetReplicaLoadRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private GetReplicaLoadRequest() { + servers_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetReplicaLoadRequest( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + servers_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry)); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.Builder.class); + } + + public static final int SERVERS_FIELD_NUMBER = 1; + private java.util.List servers_; + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List getServersList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersOrBuilderList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public int getServersCount() { + return servers_.size(); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + return servers_.get(index); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + return servers_.get(index); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < servers_.size(); i++) { + output.writeMessage(1, servers_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < servers_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, servers_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest) obj; + + boolean result = true; + result = result && getServersList() + .equals(other.getServersList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getServersCount() > 0) { + hash = (37 * hash) + SERVERS_FIELD_NUMBER; + hash = (53 * hash) + getServersList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.GetReplicaLoadRequest} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.GetReplicaLoadRequest) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequestOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getServersFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + serversBuilder_.clear(); + } + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadRequest_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest(this); + int from_bitField0_ = bitField0_; + if (serversBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.servers_ = servers_; + } else { + result.servers_ = serversBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.getDefaultInstance()) return this; + if (serversBuilder_ == null) { + if (!other.servers_.isEmpty()) { + if (servers_.isEmpty()) { + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureServersIsMutable(); + servers_.addAll(other.servers_); + } + onChanged(); + } + } else { + if (!other.servers_.isEmpty()) { + if (serversBuilder_.isEmpty()) { + serversBuilder_.dispose(); + serversBuilder_ = null; + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000001); + serversBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getServersFieldBuilder() : null; + } else { + serversBuilder_.addAllMessages(other.servers_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.util.List servers_ = + java.util.Collections.emptyList(); + private void ensureServersIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = new java.util.ArrayList(servers_); + bitField0_ |= 0x00000001; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serversBuilder_; + + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List getServersList() { + if (serversBuilder_ == null) { + return java.util.Collections.unmodifiableList(servers_); + } else { + return serversBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public int getServersCount() { + if (serversBuilder_ == null) { + return servers_.size(); + } else { + return serversBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + if (serversBuilder_ == null) { + return servers_.get(index); + } else { + return serversBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.set(index, value); + onChanged(); + } else { + serversBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.set(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(value); + onChanged(); + } else { + serversBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(index, value); + onChanged(); + } else { + serversBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addAllServers( + java.lang.Iterable values) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, servers_); + onChanged(); + } else { + serversBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder clearServers() { + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + serversBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder removeServers(int index) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.remove(index); + onChanged(); + } else { + serversBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServersBuilder( + int index) { + return getServersFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + if (serversBuilder_ == null) { + return servers_.get(index); } else { + return serversBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersOrBuilderList() { + if (serversBuilder_ != null) { + return serversBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(servers_); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder() { + return getServersFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder( + int index) { + return getServersFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersBuilderList() { + return getServersFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServersFieldBuilder() { + if (serversBuilder_ == null) { + serversBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + servers_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + servers_ = null; + } + return serversBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.GetReplicaLoadRequest) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.GetReplicaLoadRequest) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public GetReplicaLoadRequest parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new GetReplicaLoadRequest(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface GetReplicaLoadResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.GetReplicaLoadResponse) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + java.util.List + getReplicaLoadList(); + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair getReplicaLoad(int index); + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + int getReplicaLoadCount(); + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + java.util.List + getReplicaLoadOrBuilderList(); + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder getReplicaLoadOrBuilder( + int index); + } + /** + * Protobuf type {@code hbase.pb.GetReplicaLoadResponse} + */ + public static final class GetReplicaLoadResponse extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.GetReplicaLoadResponse) + GetReplicaLoadResponseOrBuilder { + // Use GetReplicaLoadResponse.newBuilder() to construct. + private GetReplicaLoadResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private GetReplicaLoadResponse() { + replicaLoad_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetReplicaLoadResponse( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + replicaLoad_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + replicaLoad_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.PARSER, extensionRegistry)); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + replicaLoad_ = java.util.Collections.unmodifiableList(replicaLoad_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.Builder.class); + } + + public static final int REPLICA_LOAD_FIELD_NUMBER = 1; + private java.util.List replicaLoad_; + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public java.util.List getReplicaLoadList() { + return replicaLoad_; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public java.util.List + getReplicaLoadOrBuilderList() { + return replicaLoad_; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public int getReplicaLoadCount() { + return replicaLoad_.size(); + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair getReplicaLoad(int index) { + return replicaLoad_.get(index); + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder getReplicaLoadOrBuilder( + int index) { + return replicaLoad_.get(index); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getReplicaLoadCount(); i++) { + if (!getReplicaLoad(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < replicaLoad_.size(); i++) { + output.writeMessage(1, replicaLoad_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < replicaLoad_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, replicaLoad_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse) obj; + + boolean result = true; + result = result && getReplicaLoadList() + .equals(other.getReplicaLoadList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getReplicaLoadCount() > 0) { + hash = (37 * hash) + REPLICA_LOAD_FIELD_NUMBER; + hash = (53 * hash) + getReplicaLoadList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.GetReplicaLoadResponse} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.GetReplicaLoadResponse) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponseOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getReplicaLoadFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (replicaLoadBuilder_ == null) { + replicaLoad_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + replicaLoadBuilder_.clear(); + } + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetReplicaLoadResponse_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse(this); + int from_bitField0_ = bitField0_; + if (replicaLoadBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + replicaLoad_ = java.util.Collections.unmodifiableList(replicaLoad_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.replicaLoad_ = replicaLoad_; + } else { + result.replicaLoad_ = replicaLoadBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.getDefaultInstance()) return this; + if (replicaLoadBuilder_ == null) { + if (!other.replicaLoad_.isEmpty()) { + if (replicaLoad_.isEmpty()) { + replicaLoad_ = other.replicaLoad_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureReplicaLoadIsMutable(); + replicaLoad_.addAll(other.replicaLoad_); + } + onChanged(); + } + } else { + if (!other.replicaLoad_.isEmpty()) { + if (replicaLoadBuilder_.isEmpty()) { + replicaLoadBuilder_.dispose(); + replicaLoadBuilder_ = null; + replicaLoad_ = other.replicaLoad_; + bitField0_ = (bitField0_ & ~0x00000001); + replicaLoadBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getReplicaLoadFieldBuilder() : null; + } else { + replicaLoadBuilder_.addAllMessages(other.replicaLoad_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getReplicaLoadCount(); i++) { + if (!getReplicaLoad(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.util.List replicaLoad_ = + java.util.Collections.emptyList(); + private void ensureReplicaLoadIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + replicaLoad_ = new java.util.ArrayList(replicaLoad_); + bitField0_ |= 0x00000001; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder> replicaLoadBuilder_; + + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public java.util.List getReplicaLoadList() { + if (replicaLoadBuilder_ == null) { + return java.util.Collections.unmodifiableList(replicaLoad_); + } else { + return replicaLoadBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public int getReplicaLoadCount() { + if (replicaLoadBuilder_ == null) { + return replicaLoad_.size(); + } else { + return replicaLoadBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair getReplicaLoad(int index) { + if (replicaLoadBuilder_ == null) { + return replicaLoad_.get(index); + } else { + return replicaLoadBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder setReplicaLoad( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair value) { + if (replicaLoadBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureReplicaLoadIsMutable(); + replicaLoad_.set(index, value); + onChanged(); + } else { + replicaLoadBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder setReplicaLoad( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder builderForValue) { + if (replicaLoadBuilder_ == null) { + ensureReplicaLoadIsMutable(); + replicaLoad_.set(index, builderForValue.build()); + onChanged(); + } else { + replicaLoadBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder addReplicaLoad(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair value) { + if (replicaLoadBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureReplicaLoadIsMutable(); + replicaLoad_.add(value); + onChanged(); + } else { + replicaLoadBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder addReplicaLoad( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair value) { + if (replicaLoadBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureReplicaLoadIsMutable(); + replicaLoad_.add(index, value); + onChanged(); + } else { + replicaLoadBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder addReplicaLoad( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder builderForValue) { + if (replicaLoadBuilder_ == null) { + ensureReplicaLoadIsMutable(); + replicaLoad_.add(builderForValue.build()); + onChanged(); + } else { + replicaLoadBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder addReplicaLoad( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder builderForValue) { + if (replicaLoadBuilder_ == null) { + ensureReplicaLoadIsMutable(); + replicaLoad_.add(index, builderForValue.build()); + onChanged(); + } else { + replicaLoadBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder addAllReplicaLoad( + java.lang.Iterable values) { + if (replicaLoadBuilder_ == null) { + ensureReplicaLoadIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, replicaLoad_); + onChanged(); + } else { + replicaLoadBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder clearReplicaLoad() { + if (replicaLoadBuilder_ == null) { + replicaLoad_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + replicaLoadBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public Builder removeReplicaLoad(int index) { + if (replicaLoadBuilder_ == null) { + ensureReplicaLoadIsMutable(); + replicaLoad_.remove(index); + onChanged(); + } else { + replicaLoadBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder getReplicaLoadBuilder( + int index) { + return getReplicaLoadFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder getReplicaLoadOrBuilder( + int index) { + if (replicaLoadBuilder_ == null) { + return replicaLoad_.get(index); } else { + return replicaLoadBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public java.util.List + getReplicaLoadOrBuilderList() { + if (replicaLoadBuilder_ != null) { + return replicaLoadBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(replicaLoad_); + } + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder addReplicaLoadBuilder() { + return getReplicaLoadFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder addReplicaLoadBuilder( + int index) { + return getReplicaLoadFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerReplicaLoadPair replica_load = 1; + */ + public java.util.List + getReplicaLoadBuilderList() { + return getReplicaLoadFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder> + getReplicaLoadFieldBuilder() { + if (replicaLoadBuilder_ == null) { + replicaLoadBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder>( + replicaLoad_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + replicaLoad_ = null; + } + return replicaLoadBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.GetReplicaLoadResponse) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.GetReplicaLoadResponse) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public GetReplicaLoadResponse parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new GetReplicaLoadResponse(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface RemoveFavoredNodeRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.RemoveFavoredNodeRequest) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * required .hbase.pb.ServerName server = 1; + */ + boolean hasServer(); + /** + * required .hbase.pb.ServerName server = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServer(); + /** + * required .hbase.pb.ServerName server = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder(); + } + /** + * Protobuf type {@code hbase.pb.RemoveFavoredNodeRequest} + */ + public static final class RemoveFavoredNodeRequest extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.RemoveFavoredNodeRequest) + RemoveFavoredNodeRequestOrBuilder { + // Use RemoveFavoredNodeRequest.newBuilder() to construct. + private RemoveFavoredNodeRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private RemoveFavoredNodeRequest() { + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RemoveFavoredNodeRequest( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = server_.toBuilder(); + } + server_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(server_); + server_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.Builder.class); + } + + private int bitField0_; + public static final int SERVER_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_; + /** + * required .hbase.pb.ServerName server = 1; + */ + public boolean hasServer() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServer() { + return server_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + return server_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasServer()) { + memoizedIsInitialized = 0; + return false; + } + if (!getServer().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, getServer()); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getServer()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest) obj; + + boolean result = true; + result = result && (hasServer() == other.hasServer()); + if (hasServer()) { + result = result && getServer() + .equals(other.getServer()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasServer()) { + hash = (37 * hash) + SERVER_FIELD_NUMBER; + hash = (53 * hash) + getServer().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.RemoveFavoredNodeRequest} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.RemoveFavoredNodeRequest) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequestOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getServerFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (serverBuilder_ == null) { + server_ = null; + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeRequest_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (serverBuilder_ == null) { + result.server_ = server_; + } else { + result.server_ = serverBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.getDefaultInstance()) return this; + if (other.hasServer()) { + mergeServer(other.getServer()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasServer()) { + return false; + } + if (!getServer().isInitialized()) { + return false; + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName server_ = null; + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_; + /** + * required .hbase.pb.ServerName server = 1; + */ + public boolean hasServer() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServer() { + if (serverBuilder_ == null) { + return server_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } else { + return serverBuilder_.getMessage(); + } + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder setServer(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + server_ = value; + onChanged(); + } else { + serverBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder setServer( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serverBuilder_ == null) { + server_ = builderForValue.build(); + onChanged(); + } else { + serverBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder mergeServer(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + server_ != null && + server_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) { + server_ = + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.newBuilder(server_).mergeFrom(value).buildPartial(); + } else { + server_ = value; + } + onChanged(); + } else { + serverBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder clearServer() { + if (serverBuilder_ == null) { + server_ = null; + onChanged(); + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServerBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getServerFieldBuilder().getBuilder(); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + if (serverBuilder_ != null) { + return serverBuilder_.getMessageOrBuilder(); + } else { + return server_ == null ? + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : server_; + } + } + /** + * required .hbase.pb.ServerName server = 1; + */ + private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServerFieldBuilder() { + if (serverBuilder_ == null) { + serverBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + getServer(), + getParentForChildren(), + isClean()); + server_ = null; + } + return serverBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.RemoveFavoredNodeRequest) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.RemoveFavoredNodeRequest) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public RemoveFavoredNodeRequest parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new RemoveFavoredNodeRequest(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface RemoveFavoredNodeResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.RemoveFavoredNodeResponse) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + } + /** + * Protobuf type {@code hbase.pb.RemoveFavoredNodeResponse} + */ + public static final class RemoveFavoredNodeResponse extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.RemoveFavoredNodeResponse) + RemoveFavoredNodeResponseOrBuilder { + // Use RemoveFavoredNodeResponse.newBuilder() to construct. + private RemoveFavoredNodeResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private RemoveFavoredNodeResponse() { + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RemoveFavoredNodeResponse( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.Builder.class); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse) obj; + + boolean result = true; + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.RemoveFavoredNodeResponse} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.RemoveFavoredNodeResponse) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponseOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_RemoveFavoredNodeResponse_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse(this); + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.RemoveFavoredNodeResponse) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.RemoveFavoredNodeResponse) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public RemoveFavoredNodeResponse parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new RemoveFavoredNodeResponse(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface CheckFavoredNodesRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.CheckFavoredNodesRequest) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * optional bool detailed = 1; + */ + boolean hasDetailed(); + /** + * optional bool detailed = 1; + */ + boolean getDetailed(); + } + /** + * Protobuf type {@code hbase.pb.CheckFavoredNodesRequest} + */ + public static final class CheckFavoredNodesRequest extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.CheckFavoredNodesRequest) + CheckFavoredNodesRequestOrBuilder { + // Use CheckFavoredNodesRequest.newBuilder() to construct. + private CheckFavoredNodesRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CheckFavoredNodesRequest() { + detailed_ = false; + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CheckFavoredNodesRequest( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + detailed_ = input.readBool(); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.Builder.class); + } + + private int bitField0_; + public static final int DETAILED_FIELD_NUMBER = 1; + private boolean detailed_; + /** + * optional bool detailed = 1; + */ + public boolean hasDetailed() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional bool detailed = 1; + */ + public boolean getDetailed() { + return detailed_; + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBool(1, detailed_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeBoolSize(1, detailed_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest) obj; + + boolean result = true; + result = result && (hasDetailed() == other.hasDetailed()); + if (hasDetailed()) { + result = result && (getDetailed() + == other.getDetailed()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasDetailed()) { + hash = (37 * hash) + DETAILED_FIELD_NUMBER; + hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean( + getDetailed()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.CheckFavoredNodesRequest} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.CheckFavoredNodesRequest) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequestOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesRequest_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + public Builder clear() { + super.clear(); + detailed_ = false; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesRequest_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.detailed_ = detailed_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.getDefaultInstance()) return this; + if (other.hasDetailed()) { + setDetailed(other.getDetailed()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private boolean detailed_ ; + /** + * optional bool detailed = 1; + */ + public boolean hasDetailed() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional bool detailed = 1; + */ + public boolean getDetailed() { + return detailed_; + } + /** + * optional bool detailed = 1; + */ + public Builder setDetailed(boolean value) { + bitField0_ |= 0x00000001; + detailed_ = value; + onChanged(); + return this; + } + /** + * optional bool detailed = 1; + */ + public Builder clearDetailed() { + bitField0_ = (bitField0_ & ~0x00000001); + detailed_ = false; + onChanged(); + return this; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.CheckFavoredNodesRequest) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.CheckFavoredNodesRequest) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public CheckFavoredNodesRequest parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new CheckFavoredNodesRequest(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface CheckFavoredNodesResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:hbase.pb.CheckFavoredNodesResponse) + org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder { + + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + java.util.List + getServersList(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + int getServersCount(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + java.util.List + getServersOrBuilderList(); + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index); + } + /** + * Protobuf type {@code hbase.pb.CheckFavoredNodesResponse} + */ + public static final class CheckFavoredNodesResponse extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:hbase.pb.CheckFavoredNodesResponse) + CheckFavoredNodesResponseOrBuilder { + // Use CheckFavoredNodesResponse.newBuilder() to construct. + private CheckFavoredNodesResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + private CheckFavoredNodesResponse() { + servers_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CheckFavoredNodesResponse( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + this(); + int mutable_bitField0_ = 0; + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields = + org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + servers_.add( + input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry)); + break; + } + } + } + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.Builder.class); + } + + public static final int SERVERS_FIELD_NUMBER = 1; + private java.util.List servers_; + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List getServersList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersOrBuilderList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public int getServersCount() { + return servers_.size(); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + return servers_.get(index); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + return servers_.get(index); + } + + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < servers_.size(); i++) { + output.writeMessage(1, servers_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < servers_.size(); i++) { + size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream + .computeMessageSize(1, servers_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse) obj; + + boolean result = true; + result = result && getServersList() + .equals(other.getServersList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getServersCount() > 0) { + hash = (37 * hash) + SERVERS_FIELD_NUMBER; + hash = (53 * hash) + getServersList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom(byte[] data) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom( + byte[] data, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseDelimitedFrom( + java.io.InputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parseFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.CheckFavoredNodesResponse} + */ + public static final class Builder extends + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:hbase.pb.CheckFavoredNodesResponse) + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponseOrBuilder { + public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesResponse_descriptor; + } + + protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getServersFieldBuilder(); + } + } + public Builder clear() { + super.clear(); + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + serversBuilder_.clear(); + } + return this; + } + + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_CheckFavoredNodesResponse_descriptor; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse build() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse buildPartial() { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse(this); + int from_bitField0_ = bitField0_; + if (serversBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.servers_ = servers_; + } else { + result.servers_ = serversBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + public Builder setField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.setField(field, value); + } + public Builder clearField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + public Builder clearOneof( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + public Builder setRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + public Builder addRepeatedField( + org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field, + Object value) { + return (Builder) super.addRepeatedField(field, value); + } + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse) { + return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse other) { + if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.getDefaultInstance()) return this; + if (serversBuilder_ == null) { + if (!other.servers_.isEmpty()) { + if (servers_.isEmpty()) { + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureServersIsMutable(); + servers_.addAll(other.servers_); + } + onChanged(); + } + } else { + if (!other.servers_.isEmpty()) { + if (serversBuilder_.isEmpty()) { + serversBuilder_.dispose(); + serversBuilder_ = null; + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000001); + serversBuilder_ = + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getServersFieldBuilder() : null; + } else { + serversBuilder_.addAllMessages(other.servers_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + private java.util.List servers_ = + java.util.Collections.emptyList(); + private void ensureServersIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + servers_ = new java.util.ArrayList(servers_); + bitField0_ |= 0x00000001; + } + } + + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serversBuilder_; + + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List getServersList() { + if (serversBuilder_ == null) { + return java.util.Collections.unmodifiableList(servers_); + } else { + return serversBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public int getServersCount() { + if (serversBuilder_ == null) { + return servers_.size(); + } else { + return serversBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + if (serversBuilder_ == null) { + return servers_.get(index); + } else { + return serversBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.set(index, value); + onChanged(); + } else { + serversBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.set(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(value); + onChanged(); + } else { + serversBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(index, value); + onChanged(); + } else { + serversBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder addAllServers( + java.lang.Iterable values) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, servers_); + onChanged(); + } else { + serversBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder clearServers() { + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + serversBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public Builder removeServers(int index) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.remove(index); + onChanged(); + } else { + serversBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServersBuilder( + int index) { + return getServersFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + if (serversBuilder_ == null) { + return servers_.get(index); } else { + return serversBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersOrBuilderList() { + if (serversBuilder_ != null) { + return serversBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(servers_); + } + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder() { + return getServersFieldBuilder().addBuilder( + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder( + int index) { + return getServersFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 1; + */ + public java.util.List + getServersBuilderList() { + return getServersFieldBuilder().getBuilderList(); + } + private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getServersFieldBuilder() { + if (serversBuilder_ == null) { + serversBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3< + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + servers_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + servers_ = null; + } + return serversBuilder_; + } + public final Builder setUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:hbase.pb.CheckFavoredNodesResponse) + } + + // @@protoc_insertion_point(class_scope:hbase.pb.CheckFavoredNodesResponse) + private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse(); + } + + public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser + PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser() { + public CheckFavoredNodesResponse parsePartialFrom( + org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input, + org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException { + return new CheckFavoredNodesResponse(input, extensionRegistry); + } + }; + + public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + /** * Protobuf service {@code hbase.pb.MasterService} */ @@ -64744,6 +71478,74 @@ public final class MasterProtos { /** *
+       ** Updated favoredNodes for a region 
+       * 
+ * + * rpc UpdateFavoredNodesForRegion(.hbase.pb.UpdateFavoredNodesRequest) returns (.hbase.pb.UpdateFavoredNodesResponse); + */ + public abstract void updateFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + *
+       ** Redistribute all favored node replicas of region. This API does not move current assignments
+       * 
+ * + * rpc redistributeFavoredNodes(.hbase.pb.RedistributeFavoredNodesRequest) returns (.hbase.pb.RedistributeFavoredNodesResponse); + */ + public abstract void redistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + *
+       ** Redistribute all favored node replicas of region. This API moves current assignments.
+       * 
+ * + * rpc completeRedistributeFavoredNodes(.hbase.pb.CompleteRedistributeFavoredNodesRequest) returns (.hbase.pb.CompleteRedistributeFavoredNodesResponse); + */ + public abstract void completeRedistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc getFavoredNodesForRegion(.hbase.pb.GetFavoredNodesForRegionRequest) returns (.hbase.pb.GetFavoredNodesForRegionResponse); + */ + public abstract void getFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc getReplicaLoad(.hbase.pb.GetReplicaLoadRequest) returns (.hbase.pb.GetReplicaLoadResponse); + */ + public abstract void getReplicaLoad( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc removeFavoredNode(.hbase.pb.RemoveFavoredNodeRequest) returns (.hbase.pb.RemoveFavoredNodeResponse); + */ + public abstract void removeFavoredNode( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc checkFavoredNodes(.hbase.pb.CheckFavoredNodesRequest) returns (.hbase.pb.CheckFavoredNodesResponse); + */ + public abstract void checkFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + *
        ** returns table state 
        * 
* @@ -65232,6 +72034,62 @@ public final class MasterProtos { } @java.lang.Override + public void updateFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + impl.updateFavoredNodesForRegion(controller, request, done); + } + + @java.lang.Override + public void redistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + impl.redistributeFavoredNodes(controller, request, done); + } + + @java.lang.Override + public void completeRedistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + impl.completeRedistributeFavoredNodes(controller, request, done); + } + + @java.lang.Override + public void getFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + impl.getFavoredNodesForRegion(controller, request, done); + } + + @java.lang.Override + public void getReplicaLoad( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + impl.getReplicaLoad(controller, request, done); + } + + @java.lang.Override + public void removeFavoredNode( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + impl.removeFavoredNode(controller, request, done); + } + + @java.lang.Override + public void checkFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + impl.checkFavoredNodes(controller, request, done); + } + + @java.lang.Override public void getTableState( org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request, @@ -65416,20 +72274,34 @@ public final class MasterProtos { case 48: return impl.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request); case 49: - return impl.getTableState(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest)request); + return impl.updateFavoredNodesForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)request); case 50: - return impl.setQuota(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest)request); + return impl.redistributeFavoredNodes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest)request); case 51: - return impl.getLastMajorCompactionTimestamp(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest)request); + return impl.completeRedistributeFavoredNodes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest)request); case 52: - return impl.getLastMajorCompactionTimestampForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest)request); + return impl.getFavoredNodesForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest)request); case 53: - return impl.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request); + return impl.getReplicaLoad(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest)request); case 54: - return impl.getSecurityCapabilities(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest)request); + return impl.removeFavoredNode(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest)request); case 55: - return impl.abortProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest)request); + return impl.checkFavoredNodes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest)request); case 56: + return impl.getTableState(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest)request); + case 57: + return impl.setQuota(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest)request); + case 58: + return impl.getLastMajorCompactionTimestamp(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest)request); + case 59: + return impl.getLastMajorCompactionTimestampForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest)request); + case 60: + return impl.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request); + case 61: + return impl.getSecurityCapabilities(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest)request); + case 62: + return impl.abortProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest)request); + case 63: return impl.listProcedures(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest)request); default: throw new java.lang.AssertionError("Can't get here."); @@ -65544,20 +72416,34 @@ public final class MasterProtos { case 48: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance(); case 49: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance(); case 50: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.getDefaultInstance(); case 51: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.getDefaultInstance(); case 52: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.getDefaultInstance(); case 53: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.getDefaultInstance(); case 54: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.getDefaultInstance(); case 55: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.getDefaultInstance(); case 56: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance(); + case 57: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest.getDefaultInstance(); + case 58: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest.getDefaultInstance(); + case 59: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest.getDefaultInstance(); + case 60: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance(); + case 61: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest.getDefaultInstance(); + case 62: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest.getDefaultInstance(); + case 63: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -65672,20 +72558,34 @@ public final class MasterProtos { case 48: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance(); case 49: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(); case 50: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.getDefaultInstance(); case 51: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.getDefaultInstance(); case 52: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.getDefaultInstance(); case 53: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.getDefaultInstance(); case 54: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.getDefaultInstance(); case 55: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.getDefaultInstance(); case 56: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(); + case 57: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance(); + case 58: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + case 59: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + case 60: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance(); + case 61: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance(); + case 62: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance(); + case 63: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -66318,6 +73218,74 @@ public final class MasterProtos { /** *
+     ** Updated favoredNodes for a region 
+     * 
+ * + * rpc UpdateFavoredNodesForRegion(.hbase.pb.UpdateFavoredNodesRequest) returns (.hbase.pb.UpdateFavoredNodesResponse); + */ + public abstract void updateFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + *
+     ** Redistribute all favored node replicas of region. This API does not move current assignments
+     * 
+ * + * rpc redistributeFavoredNodes(.hbase.pb.RedistributeFavoredNodesRequest) returns (.hbase.pb.RedistributeFavoredNodesResponse); + */ + public abstract void redistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + *
+     ** Redistribute all favored node replicas of region. This API moves current assignments.
+     * 
+ * + * rpc completeRedistributeFavoredNodes(.hbase.pb.CompleteRedistributeFavoredNodesRequest) returns (.hbase.pb.CompleteRedistributeFavoredNodesResponse); + */ + public abstract void completeRedistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc getFavoredNodesForRegion(.hbase.pb.GetFavoredNodesForRegionRequest) returns (.hbase.pb.GetFavoredNodesForRegionResponse); + */ + public abstract void getFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc getReplicaLoad(.hbase.pb.GetReplicaLoadRequest) returns (.hbase.pb.GetReplicaLoadResponse); + */ + public abstract void getReplicaLoad( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc removeFavoredNode(.hbase.pb.RemoveFavoredNodeRequest) returns (.hbase.pb.RemoveFavoredNodeResponse); + */ + public abstract void removeFavoredNode( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + * rpc checkFavoredNodes(.hbase.pb.CheckFavoredNodesRequest) returns (.hbase.pb.CheckFavoredNodesResponse); + */ + public abstract void checkFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done); + + /** + *
      ** returns table state 
      * 
* @@ -66665,52 +73633,87 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 47: - this.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request, - org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + case 47: + this.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 48: + this.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 49: + this.updateFavoredNodesForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 50: + this.redistributeFavoredNodes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 51: + this.completeRedistributeFavoredNodes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 52: + this.getFavoredNodesForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 53: + this.getReplicaLoad(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 54: + this.removeFavoredNode(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 48: - this.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request, - org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( + case 55: + this.checkFavoredNodes(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest)request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 49: + case 56: this.getTableState(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 50: + case 57: this.setQuota(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 51: + case 58: this.getLastMajorCompactionTimestamp(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 52: + case 59: this.getLastMajorCompactionTimestampForRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 53: + case 60: this.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 54: + case 61: this.getSecurityCapabilities(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 55: + case 62: this.abortProcedure(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 56: + case 63: this.listProcedures(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest)request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.specializeCallback( done)); @@ -66828,20 +73831,34 @@ public final class MasterProtos { case 48: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance(); case 49: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance(); case 50: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest.getDefaultInstance(); case 51: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest.getDefaultInstance(); case 52: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest.getDefaultInstance(); case 53: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest.getDefaultInstance(); case 54: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest.getDefaultInstance(); case 55: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest.getDefaultInstance(); case 56: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance(); + case 57: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest.getDefaultInstance(); + case 58: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest.getDefaultInstance(); + case 59: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest.getDefaultInstance(); + case 60: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance(); + case 61: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest.getDefaultInstance(); + case 62: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest.getDefaultInstance(); + case 63: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -66956,20 +73973,34 @@ public final class MasterProtos { case 48: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance(); case 49: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(); case 50: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.getDefaultInstance(); case 51: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.getDefaultInstance(); case 52: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.getDefaultInstance(); case 53: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.getDefaultInstance(); case 54: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.getDefaultInstance(); case 55: - return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.getDefaultInstance(); case 56: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(); + case 57: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance(); + case 58: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + case 59: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(); + case 60: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance(); + case 61: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance(); + case 62: + return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance(); + case 63: return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -67727,12 +74758,117 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance())); } + public void updateFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(49), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(), + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.class, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance())); + } + + public void redistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(50), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.getDefaultInstance(), + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.class, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.getDefaultInstance())); + } + + public void completeRedistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(51), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.getDefaultInstance(), + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.class, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.getDefaultInstance())); + } + + public void getFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(52), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.getDefaultInstance(), + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.class, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.getDefaultInstance())); + } + + public void getReplicaLoad( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(53), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.getDefaultInstance(), + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.class, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.getDefaultInstance())); + } + + public void removeFavoredNode( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(54), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.getDefaultInstance(), + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.class, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.getDefaultInstance())); + } + + public void checkFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest request, + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(55), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.getDefaultInstance(), + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.class, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.getDefaultInstance())); + } + public void getTableState( org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(49), + getDescriptor().getMethods().get(56), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(), @@ -67747,7 +74883,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(50), + getDescriptor().getMethods().get(57), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance(), @@ -67762,7 +74898,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(51), + getDescriptor().getMethods().get(58), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(), @@ -67777,7 +74913,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(52), + getDescriptor().getMethods().get(59), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance(), @@ -67792,7 +74928,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(53), + getDescriptor().getMethods().get(60), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance(), @@ -67807,7 +74943,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(54), + getDescriptor().getMethods().get(61), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance(), @@ -67822,7 +74958,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(55), + getDescriptor().getMethods().get(62), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance(), @@ -67837,7 +74973,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request, org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(56), + getDescriptor().getMethods().get(63), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance(), @@ -68099,6 +75235,41 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse updateFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse redistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse completeRedistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse getFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse getReplicaLoad( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse removeFavoredNode( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse checkFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse getTableState( org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request) @@ -68735,12 +75906,96 @@ public final class MasterProtos { } + public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse updateFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(49), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse redistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(50), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RedistributeFavoredNodesResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse completeRedistributeFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(51), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CompleteRedistributeFavoredNodesResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse getFavoredNodesForRegion( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(52), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetFavoredNodesForRegionResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse getReplicaLoad( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(53), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetReplicaLoadResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse removeFavoredNode( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(54), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveFavoredNodeResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse checkFavoredNodes( + org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesRequest request) + throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(55), + controller, + request, + org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CheckFavoredNodesResponse.getDefaultInstance()); + } + + public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse getTableState( org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(49), + getDescriptor().getMethods().get(56), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance()); @@ -68752,7 +76007,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(50), + getDescriptor().getMethods().get(57), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse.getDefaultInstance()); @@ -68764,7 +76019,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(51), + getDescriptor().getMethods().get(58), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance()); @@ -68776,7 +76031,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(52), + getDescriptor().getMethods().get(59), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse.getDefaultInstance()); @@ -68788,7 +76043,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(53), + getDescriptor().getMethods().get(60), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance()); @@ -68800,7 +76055,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(54), + getDescriptor().getMethods().get(61), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance()); @@ -68812,7 +76067,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(55), + getDescriptor().getMethods().get(62), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse.getDefaultInstance()); @@ -68824,7 +76079,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest request) throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(56), + getDescriptor().getMethods().get(63), controller, request, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse.getDefaultInstance()); @@ -69390,6 +76645,66 @@ public final class MasterProtos { private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_hbase_pb_SecurityCapabilitiesResponse_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_RedistributeFavoredNodesRequest_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_RedistributeFavoredNodesRequest_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_RedistributeFavoredNodesResponse_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_RedistributeFavoredNodesResponse_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_GetFavoredNodesForRegionRequest_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_GetFavoredNodesForRegionRequest_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_GetFavoredNodesForRegionResponse_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_GetFavoredNodesForRegionResponse_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_GetReplicaLoadRequest_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_GetReplicaLoadRequest_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_GetReplicaLoadResponse_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_GetReplicaLoadResponse_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_RemoveFavoredNodeRequest_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_RemoveFavoredNodeRequest_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_RemoveFavoredNodeResponse_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_RemoveFavoredNodeResponse_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_CheckFavoredNodesRequest_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_CheckFavoredNodesRequest_fieldAccessorTable; + private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_CheckFavoredNodesResponse_descriptor; + private static final + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_hbase_pb_CheckFavoredNodesResponse_fieldAccessorTable; public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -69400,335 +76715,371 @@ public final class MasterProtos { static { java.lang.String[] descriptorData = { "\n\014Master.proto\022\010hbase.pb\032\013HBase.proto\032\014C" + - "lient.proto\032\023ClusterStatus.proto\032\023ErrorH" + - "andling.proto\032\017Procedure.proto\032\013Quota.pr" + - "oto\"\234\001\n\020AddColumnRequest\022\'\n\ntable_name\030\001" + - " \002(\0132\023.hbase.pb.TableName\0225\n\017column_fami" + - "lies\030\002 \002(\0132\034.hbase.pb.ColumnFamilySchema" + - "\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:" + - "\0010\"$\n\021AddColumnResponse\022\017\n\007proc_id\030\001 \001(\004" + - "\"}\n\023DeleteColumnRequest\022\'\n\ntable_name\030\001 " + - "\002(\0132\023.hbase.pb.TableName\022\023\n\013column_name\030", - "\002 \002(\014\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004" + - " \001(\004:\0010\"\'\n\024DeleteColumnResponse\022\017\n\007proc_" + - "id\030\001 \001(\004\"\237\001\n\023ModifyColumnRequest\022\'\n\ntabl" + - "e_name\030\001 \002(\0132\023.hbase.pb.TableName\0225\n\017col" + - "umn_families\030\002 \002(\0132\034.hbase.pb.ColumnFami" + - "lySchema\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005nonc" + - "e\030\004 \001(\004:\0010\"\'\n\024ModifyColumnResponse\022\017\n\007pr" + - "oc_id\030\001 \001(\004\"n\n\021MoveRegionRequest\022)\n\006regi" + - "on\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022.\n\020d" + - "est_server_name\030\002 \001(\0132\024.hbase.pb.ServerN", - "ame\"\024\n\022MoveRegionResponse\"\274\001\n\035DispatchMe" + - "rgingRegionsRequest\022+\n\010region_a\030\001 \002(\0132\031." + - "hbase.pb.RegionSpecifier\022+\n\010region_b\030\002 \002" + - "(\0132\031.hbase.pb.RegionSpecifier\022\027\n\010forcibl" + - "e\030\003 \001(\010:\005false\022\026\n\013nonce_group\030\004 \001(\004:\0010\022\020" + - "\n\005nonce\030\005 \001(\004:\0010\"1\n\036DispatchMergingRegio" + - "nsResponse\022\017\n\007proc_id\030\001 \001(\004\"@\n\023AssignReg" + - "ionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re" + - "gionSpecifier\"\026\n\024AssignRegionResponse\"X\n" + - "\025UnassignRegionRequest\022)\n\006region\030\001 \002(\0132\031", - ".hbase.pb.RegionSpecifier\022\024\n\005force\030\002 \001(\010" + - ":\005false\"\030\n\026UnassignRegionResponse\"A\n\024Off" + - "lineRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbas" + - "e.pb.RegionSpecifier\"\027\n\025OfflineRegionRes" + - "ponse\"\177\n\022CreateTableRequest\022+\n\014table_sch" + - "ema\030\001 \002(\0132\025.hbase.pb.TableSchema\022\022\n\nspli" + - "t_keys\030\002 \003(\014\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n\005" + - "nonce\030\004 \001(\004:\0010\"&\n\023CreateTableResponse\022\017\n" + - "\007proc_id\030\001 \001(\004\"g\n\022DeleteTableRequest\022\'\n\n" + - "table_name\030\001 \002(\0132\023.hbase.pb.TableName\022\026\n", - "\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"" + - "&\n\023DeleteTableResponse\022\017\n\007proc_id\030\001 \001(\004\"" + - "\207\001\n\024TruncateTableRequest\022&\n\ttableName\030\001 " + - "\002(\0132\023.hbase.pb.TableName\022\035\n\016preserveSpli" + - "ts\030\002 \001(\010:\005false\022\026\n\013nonce_group\030\003 \001(\004:\0010\022" + - "\020\n\005nonce\030\004 \001(\004:\0010\"(\n\025TruncateTableRespon" + - "se\022\017\n\007proc_id\030\001 \001(\004\"g\n\022EnableTableReques" + - "t\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableNa" + - "me\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(" + - "\004:\0010\"&\n\023EnableTableResponse\022\017\n\007proc_id\030\001", - " \001(\004\"h\n\023DisableTableRequest\022\'\n\ntable_nam" + - "e\030\001 \002(\0132\023.hbase.pb.TableName\022\026\n\013nonce_gr" + - "oup\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"\'\n\024Disabl" + - "eTableResponse\022\017\n\007proc_id\030\001 \001(\004\"\224\001\n\022Modi" + - "fyTableRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hba" + - "se.pb.TableName\022+\n\014table_schema\030\002 \002(\0132\025." + - "hbase.pb.TableSchema\022\026\n\013nonce_group\030\003 \001(" + - "\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"&\n\023ModifyTableRes" + - "ponse\022\017\n\007proc_id\030\001 \001(\004\"~\n\026CreateNamespac" + - "eRequest\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.", + "lient.proto\032\023ClusterStatus.proto\032\013Admin." + + "proto\032\023ErrorHandling.proto\032\017Procedure.pr" + + "oto\032\013Quota.proto\"\234\001\n\020AddColumnRequest\022\'\n" + + "\ntable_name\030\001 \002(\0132\023.hbase.pb.TableName\0225" + + "\n\017column_families\030\002 \002(\0132\034.hbase.pb.Colum" + + "nFamilySchema\022\026\n\013nonce_group\030\003 \001(\004:\0010\022\020\n" + + "\005nonce\030\004 \001(\004:\0010\"$\n\021AddColumnResponse\022\017\n\007" + + "proc_id\030\001 \001(\004\"}\n\023DeleteColumnRequest\022\'\n\n" + + "table_name\030\001 \002(\0132\023.hbase.pb.TableName\022\023\n", + "\013column_name\030\002 \002(\014\022\026\n\013nonce_group\030\003 \001(\004:" + + "\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\'\n\024DeleteColumnResp" + + "onse\022\017\n\007proc_id\030\001 \001(\004\"\237\001\n\023ModifyColumnRe" + + "quest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.Tab" + + "leName\0225\n\017column_families\030\002 \002(\0132\034.hbase." + + "pb.ColumnFamilySchema\022\026\n\013nonce_group\030\003 \001" + + "(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"\'\n\024ModifyColumnR" + + "esponse\022\017\n\007proc_id\030\001 \001(\004\"n\n\021MoveRegionRe" + + "quest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionS" + + "pecifier\022.\n\020dest_server_name\030\002 \001(\0132\024.hba", + "se.pb.ServerName\"\024\n\022MoveRegionResponse\"\274" + + "\001\n\035DispatchMergingRegionsRequest\022+\n\010regi" + + "on_a\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022+\n" + + "\010region_b\030\002 \002(\0132\031.hbase.pb.RegionSpecifi" + + "er\022\027\n\010forcible\030\003 \001(\010:\005false\022\026\n\013nonce_gro" + + "up\030\004 \001(\004:\0010\022\020\n\005nonce\030\005 \001(\004:\0010\"1\n\036Dispatc" + + "hMergingRegionsResponse\022\017\n\007proc_id\030\001 \001(\004" + + "\"@\n\023AssignRegionRequest\022)\n\006region\030\001 \002(\0132" + + "\031.hbase.pb.RegionSpecifier\"\026\n\024AssignRegi" + + "onResponse\"X\n\025UnassignRegionRequest\022)\n\006r", + "egion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\024" + + "\n\005force\030\002 \001(\010:\005false\"\030\n\026UnassignRegionRe" + + "sponse\"A\n\024OfflineRegionRequest\022)\n\006region" + + "\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\"\027\n\025Off" + + "lineRegionResponse\"\177\n\022CreateTableRequest" + + "\022+\n\014table_schema\030\001 \002(\0132\025.hbase.pb.TableS" + + "chema\022\022\n\nsplit_keys\030\002 \003(\014\022\026\n\013nonce_group" + + "\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"&\n\023CreateTab" + + "leResponse\022\017\n\007proc_id\030\001 \001(\004\"g\n\022DeleteTab" + + "leRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb", + ".TableName\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005no" + + "nce\030\003 \001(\004:\0010\"&\n\023DeleteTableResponse\022\017\n\007p" + + "roc_id\030\001 \001(\004\"\207\001\n\024TruncateTableRequest\022&\n" + + "\ttableName\030\001 \002(\0132\023.hbase.pb.TableName\022\035\n" + + "\016preserveSplits\030\002 \001(\010:\005false\022\026\n\013nonce_gr" + + "oup\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"(\n\025Trunca" + + "teTableResponse\022\017\n\007proc_id\030\001 \001(\004\"g\n\022Enab" + + "leTableRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hba" + + "se.pb.TableName\022\026\n\013nonce_group\030\002 \001(\004:\0010\022" + + "\020\n\005nonce\030\003 \001(\004:\0010\"&\n\023EnableTableResponse", + "\022\017\n\007proc_id\030\001 \001(\004\"h\n\023DisableTableRequest" + + "\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableNam" + + "e\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004" + + ":\0010\"\'\n\024DisableTableResponse\022\017\n\007proc_id\030\001" + + " \001(\004\"\224\001\n\022ModifyTableRequest\022\'\n\ntable_nam" + + "e\030\001 \002(\0132\023.hbase.pb.TableName\022+\n\014table_sc" + + "hema\030\002 \002(\0132\025.hbase.pb.TableSchema\022\026\n\013non" + + "ce_group\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"&\n\023M" + + "odifyTableResponse\022\017\n\007proc_id\030\001 \001(\004\"~\n\026C" + + "reateNamespaceRequest\022:\n\023namespaceDescri", + "ptor\030\001 \002(\0132\035.hbase.pb.NamespaceDescripto" + + "r\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004" + + ":\0010\"*\n\027CreateNamespaceResponse\022\017\n\007proc_i" + + "d\030\001 \001(\004\"Y\n\026DeleteNamespaceRequest\022\025\n\rnam" + + "espaceName\030\001 \002(\t\022\026\n\013nonce_group\030\002 \001(\004:\0010" + + "\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027DeleteNamespaceRes" + + "ponse\022\017\n\007proc_id\030\001 \001(\004\"~\n\026ModifyNamespac" + + "eRequest\022:\n\023namespaceDescriptor\030\001 \002(\0132\035." + "hbase.pb.NamespaceDescriptor\022\026\n\013nonce_gr" + - "oup\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027Create" + - "NamespaceResponse\022\017\n\007proc_id\030\001 \001(\004\"Y\n\026De" + - "leteNamespaceRequest\022\025\n\rnamespaceName\030\001 " + - "\002(\t\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001" + - "(\004:\0010\"*\n\027DeleteNamespaceResponse\022\017\n\007proc" + - "_id\030\001 \001(\004\"~\n\026ModifyNamespaceRequest\022:\n\023n" + - "amespaceDescriptor\030\001 \002(\0132\035.hbase.pb.Name" + - "spaceDescriptor\022\026\n\013nonce_group\030\002 \001(\004:\0010\022" + - "\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027ModifyNamespaceResp", - "onse\022\017\n\007proc_id\030\001 \001(\004\"6\n\035GetNamespaceDes" + - "criptorRequest\022\025\n\rnamespaceName\030\001 \002(\t\"\\\n" + - "\036GetNamespaceDescriptorResponse\022:\n\023names" + - "paceDescriptor\030\001 \002(\0132\035.hbase.pb.Namespac" + - "eDescriptor\"!\n\037ListNamespaceDescriptorsR" + - "equest\"^\n ListNamespaceDescriptorsRespon" + - "se\022:\n\023namespaceDescriptor\030\001 \003(\0132\035.hbase." + - "pb.NamespaceDescriptor\"?\n&ListTableDescr" + - "iptorsByNamespaceRequest\022\025\n\rnamespaceNam" + - "e\030\001 \002(\t\"U\n\'ListTableDescriptorsByNamespa", - "ceResponse\022*\n\013tableSchema\030\001 \003(\0132\025.hbase." + - "pb.TableSchema\"9\n ListTableNamesByNamesp" + - "aceRequest\022\025\n\rnamespaceName\030\001 \002(\t\"K\n!Lis" + - "tTableNamesByNamespaceResponse\022&\n\ttableN" + - "ame\030\001 \003(\0132\023.hbase.pb.TableName\"\021\n\017Shutdo" + - "wnRequest\"\022\n\020ShutdownResponse\"\023\n\021StopMas" + - "terRequest\"\024\n\022StopMasterResponse\"\034\n\032IsIn" + - "MaintenanceModeRequest\"8\n\033IsInMaintenanc" + - "eModeResponse\022\031\n\021inMaintenanceMode\030\001 \002(\010" + - "\"\037\n\016BalanceRequest\022\r\n\005force\030\001 \001(\010\"\'\n\017Bal", - "anceResponse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031Se" + - "tBalancerRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013s" + - "ynchronous\030\002 \001(\010\"8\n\032SetBalancerRunningRe" + - "sponse\022\032\n\022prev_balance_value\030\001 \001(\010\"\032\n\030Is" + - "BalancerEnabledRequest\",\n\031IsBalancerEnab" + - "ledResponse\022\017\n\007enabled\030\001 \002(\010\"w\n\035SetSplit" + - "OrMergeEnabledRequest\022\017\n\007enabled\030\001 \002(\010\022\023" + - "\n\013synchronous\030\002 \001(\010\0220\n\014switch_types\030\003 \003(" + - "\0162\032.hbase.pb.MasterSwitchType\"4\n\036SetSpli" + - "tOrMergeEnabledResponse\022\022\n\nprev_value\030\001 ", - "\003(\010\"O\n\034IsSplitOrMergeEnabledRequest\022/\n\013s" + - "witch_type\030\001 \002(\0162\032.hbase.pb.MasterSwitch" + - "Type\"0\n\035IsSplitOrMergeEnabledResponse\022\017\n" + - "\007enabled\030\001 \002(\010\"\022\n\020NormalizeRequest\"+\n\021No" + - "rmalizeResponse\022\026\n\016normalizer_ran\030\001 \002(\010\"" + - ")\n\033SetNormalizerRunningRequest\022\n\n\002on\030\001 \002" + - "(\010\"=\n\034SetNormalizerRunningResponse\022\035\n\025pr" + - "ev_normalizer_value\030\001 \001(\010\"\034\n\032IsNormalize" + - "rEnabledRequest\".\n\033IsNormalizerEnabledRe" + - "sponse\022\017\n\007enabled\030\001 \002(\010\"\027\n\025RunCatalogSca", - "nRequest\"-\n\026RunCatalogScanResponse\022\023\n\013sc" + - "an_result\030\001 \001(\005\"-\n\033EnableCatalogJanitorR" + - "equest\022\016\n\006enable\030\001 \002(\010\"2\n\034EnableCatalogJ" + - "anitorResponse\022\022\n\nprev_value\030\001 \001(\010\" \n\036Is" + - "CatalogJanitorEnabledRequest\"0\n\037IsCatalo" + - "gJanitorEnabledResponse\022\r\n\005value\030\001 \002(\010\"B" + - "\n\017SnapshotRequest\022/\n\010snapshot\030\001 \002(\0132\035.hb" + - "ase.pb.SnapshotDescription\",\n\020SnapshotRe" + - "sponse\022\030\n\020expected_timeout\030\001 \002(\003\"\036\n\034GetC" + - "ompletedSnapshotsRequest\"Q\n\035GetCompleted", - "SnapshotsResponse\0220\n\tsnapshots\030\001 \003(\0132\035.h" + - "base.pb.SnapshotDescription\"H\n\025DeleteSna" + - "pshotRequest\022/\n\010snapshot\030\001 \002(\0132\035.hbase.p" + - "b.SnapshotDescription\"\030\n\026DeleteSnapshotR" + - "esponse\"s\n\026RestoreSnapshotRequest\022/\n\010sna" + - "pshot\030\001 \002(\0132\035.hbase.pb.SnapshotDescripti" + - "on\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(" + - "\004:\0010\"*\n\027RestoreSnapshotResponse\022\017\n\007proc_" + - "id\030\001 \002(\004\"H\n\025IsSnapshotDoneRequest\022/\n\010sna" + - "pshot\030\001 \001(\0132\035.hbase.pb.SnapshotDescripti", - "on\"^\n\026IsSnapshotDoneResponse\022\023\n\004done\030\001 \001" + - "(\010:\005false\022/\n\010snapshot\030\002 \001(\0132\035.hbase.pb.S" + - "napshotDescription\"O\n\034IsRestoreSnapshotD" + - "oneRequest\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb." + - "SnapshotDescription\"4\n\035IsRestoreSnapshot" + - "DoneResponse\022\023\n\004done\030\001 \001(\010:\005false\"F\n\033Get" + - "SchemaAlterStatusRequest\022\'\n\ntable_name\030\001" + - " \002(\0132\023.hbase.pb.TableName\"T\n\034GetSchemaAl" + - "terStatusResponse\022\035\n\025yet_to_update_regio" + - "ns\030\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"\213\001\n\032GetT", - "ableDescriptorsRequest\022(\n\013table_names\030\001 " + - "\003(\0132\023.hbase.pb.TableName\022\r\n\005regex\030\002 \001(\t\022" + - "!\n\022include_sys_tables\030\003 \001(\010:\005false\022\021\n\tna" + - "mespace\030\004 \001(\t\"J\n\033GetTableDescriptorsResp" + - "onse\022+\n\014table_schema\030\001 \003(\0132\025.hbase.pb.Ta" + - "bleSchema\"[\n\024GetTableNamesRequest\022\r\n\005reg" + - "ex\030\001 \001(\t\022!\n\022include_sys_tables\030\002 \001(\010:\005fa" + - "lse\022\021\n\tnamespace\030\003 \001(\t\"A\n\025GetTableNamesR" + - "esponse\022(\n\013table_names\030\001 \003(\0132\023.hbase.pb." + - "TableName\"?\n\024GetTableStateRequest\022\'\n\ntab", - "le_name\030\001 \002(\0132\023.hbase.pb.TableName\"B\n\025Ge" + - "tTableStateResponse\022)\n\013table_state\030\001 \002(\013" + - "2\024.hbase.pb.TableState\"\031\n\027GetClusterStat" + - "usRequest\"K\n\030GetClusterStatusResponse\022/\n" + - "\016cluster_status\030\001 \002(\0132\027.hbase.pb.Cluster" + - "Status\"\030\n\026IsMasterRunningRequest\"4\n\027IsMa" + - "sterRunningResponse\022\031\n\021is_master_running" + - "\030\001 \002(\010\"I\n\024ExecProcedureRequest\0221\n\tproced" + - "ure\030\001 \002(\0132\036.hbase.pb.ProcedureDescriptio" + - "n\"F\n\025ExecProcedureResponse\022\030\n\020expected_t", - "imeout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"K\n\026IsP" + - "rocedureDoneRequest\0221\n\tprocedure\030\001 \001(\0132\036" + - ".hbase.pb.ProcedureDescription\"`\n\027IsProc" + - "edureDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\0220" + - "\n\010snapshot\030\002 \001(\0132\036.hbase.pb.ProcedureDes" + - "cription\",\n\031GetProcedureResultRequest\022\017\n" + - "\007proc_id\030\001 \002(\004\"\371\001\n\032GetProcedureResultRes" + - "ponse\0229\n\005state\030\001 \002(\0162*.hbase.pb.GetProce" + - "dureResultResponse.State\022\022\n\nstart_time\030\002" + - " \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006result\030\004 \001(", - "\014\0224\n\texception\030\005 \001(\0132!.hbase.pb.ForeignE" + - "xceptionMessage\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022" + - "\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"M\n\025AbortProce" + - "dureRequest\022\017\n\007proc_id\030\001 \002(\004\022#\n\025mayInter" + - "ruptIfRunning\030\002 \001(\010:\004true\"6\n\026AbortProced" + - "ureResponse\022\034\n\024is_procedure_aborted\030\001 \002(" + - "\010\"\027\n\025ListProceduresRequest\"@\n\026ListProced" + - "uresResponse\022&\n\tprocedure\030\001 \003(\0132\023.hbase." + - "pb.Procedure\"\315\001\n\017SetQuotaRequest\022\021\n\tuser" + - "_name\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tnames", - "pace\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hbase.p" + - "b.TableName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypas" + - "s_globals\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031.hbas" + - "e.pb.ThrottleRequest\"\022\n\020SetQuotaResponse" + - "\"J\n\037MajorCompactionTimestampRequest\022\'\n\nt" + - "able_name\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(" + - "MajorCompactionTimestampForRegionRequest" + - "\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecif" + - "ier\"@\n MajorCompactionTimestampResponse\022" + - "\034\n\024compaction_timestamp\030\001 \002(\003\"\035\n\033Securit", - "yCapabilitiesRequest\"\354\001\n\034SecurityCapabil" + - "itiesResponse\022G\n\014capabilities\030\001 \003(\01621.hb" + - "ase.pb.SecurityCapabilitiesResponse.Capa" + - "bility\"\202\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTI" + - "CATION\020\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rA" + - "UTHORIZATION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023" + - "\n\017CELL_VISIBILITY\020\004*(\n\020MasterSwitchType\022" + - "\t\n\005SPLIT\020\000\022\t\n\005MERGE\020\0012\323(\n\rMasterService\022" + - "e\n\024GetSchemaAlterStatus\022%.hbase.pb.GetSc" + - "hemaAlterStatusRequest\032&.hbase.pb.GetSch", - "emaAlterStatusResponse\022b\n\023GetTableDescri" + - "ptors\022$.hbase.pb.GetTableDescriptorsRequ" + - "est\032%.hbase.pb.GetTableDescriptorsRespon" + - "se\022P\n\rGetTableNames\022\036.hbase.pb.GetTableN" + - "amesRequest\032\037.hbase.pb.GetTableNamesResp" + - "onse\022Y\n\020GetClusterStatus\022!.hbase.pb.GetC" + - "lusterStatusRequest\032\".hbase.pb.GetCluste" + - "rStatusResponse\022V\n\017IsMasterRunning\022 .hba" + - "se.pb.IsMasterRunningRequest\032!.hbase.pb." + - "IsMasterRunningResponse\022D\n\tAddColumn\022\032.h", - "base.pb.AddColumnRequest\032\033.hbase.pb.AddC" + - "olumnResponse\022M\n\014DeleteColumn\022\035.hbase.pb" + - ".DeleteColumnRequest\032\036.hbase.pb.DeleteCo" + - "lumnResponse\022M\n\014ModifyColumn\022\035.hbase.pb." + - "ModifyColumnRequest\032\036.hbase.pb.ModifyCol" + - "umnResponse\022G\n\nMoveRegion\022\033.hbase.pb.Mov" + - "eRegionRequest\032\034.hbase.pb.MoveRegionResp" + - "onse\022k\n\026DispatchMergingRegions\022\'.hbase.p" + - "b.DispatchMergingRegionsRequest\032(.hbase." + - "pb.DispatchMergingRegionsResponse\022M\n\014Ass", - "ignRegion\022\035.hbase.pb.AssignRegionRequest" + - "\032\036.hbase.pb.AssignRegionResponse\022S\n\016Unas" + - "signRegion\022\037.hbase.pb.UnassignRegionRequ" + - "est\032 .hbase.pb.UnassignRegionResponse\022P\n" + - "\rOfflineRegion\022\036.hbase.pb.OfflineRegionR" + - "equest\032\037.hbase.pb.OfflineRegionResponse\022" + - "J\n\013DeleteTable\022\034.hbase.pb.DeleteTableReq" + - "uest\032\035.hbase.pb.DeleteTableResponse\022P\n\rt" + - "runcateTable\022\036.hbase.pb.TruncateTableReq" + - "uest\032\037.hbase.pb.TruncateTableResponse\022J\n", - "\013EnableTable\022\034.hbase.pb.EnableTableReque" + - "st\032\035.hbase.pb.EnableTableResponse\022M\n\014Dis" + - "ableTable\022\035.hbase.pb.DisableTableRequest" + - "\032\036.hbase.pb.DisableTableResponse\022J\n\013Modi" + - "fyTable\022\034.hbase.pb.ModifyTableRequest\032\035." + - "hbase.pb.ModifyTableResponse\022J\n\013CreateTa" + - "ble\022\034.hbase.pb.CreateTableRequest\032\035.hbas" + - "e.pb.CreateTableResponse\022A\n\010Shutdown\022\031.h" + - "base.pb.ShutdownRequest\032\032.hbase.pb.Shutd" + - "ownResponse\022G\n\nStopMaster\022\033.hbase.pb.Sto", - "pMasterRequest\032\034.hbase.pb.StopMasterResp" + - "onse\022h\n\031IsMasterInMaintenanceMode\022$.hbas" + - "e.pb.IsInMaintenanceModeRequest\032%.hbase." + - "pb.IsInMaintenanceModeResponse\022>\n\007Balanc" + - "e\022\030.hbase.pb.BalanceRequest\032\031.hbase.pb.B" + - "alanceResponse\022_\n\022SetBalancerRunning\022#.h" + - "base.pb.SetBalancerRunningRequest\032$.hbas" + - "e.pb.SetBalancerRunningResponse\022\\\n\021IsBal" + - "ancerEnabled\022\".hbase.pb.IsBalancerEnable" + - "dRequest\032#.hbase.pb.IsBalancerEnabledRes", - "ponse\022k\n\026SetSplitOrMergeEnabled\022\'.hbase." + - "pb.SetSplitOrMergeEnabledRequest\032(.hbase" + - ".pb.SetSplitOrMergeEnabledResponse\022h\n\025Is" + - "SplitOrMergeEnabled\022&.hbase.pb.IsSplitOr" + - "MergeEnabledRequest\032\'.hbase.pb.IsSplitOr" + - "MergeEnabledResponse\022D\n\tNormalize\022\032.hbas" + - "e.pb.NormalizeRequest\032\033.hbase.pb.Normali" + - "zeResponse\022e\n\024SetNormalizerRunning\022%.hba" + - "se.pb.SetNormalizerRunningRequest\032&.hbas" + - "e.pb.SetNormalizerRunningResponse\022b\n\023IsN", - "ormalizerEnabled\022$.hbase.pb.IsNormalizer" + - "EnabledRequest\032%.hbase.pb.IsNormalizerEn" + - "abledResponse\022S\n\016RunCatalogScan\022\037.hbase." + - "pb.RunCatalogScanRequest\032 .hbase.pb.RunC" + - "atalogScanResponse\022e\n\024EnableCatalogJanit" + - "or\022%.hbase.pb.EnableCatalogJanitorReques" + - "t\032&.hbase.pb.EnableCatalogJanitorRespons" + - "e\022n\n\027IsCatalogJanitorEnabled\022(.hbase.pb." + - "IsCatalogJanitorEnabledRequest\032).hbase.p" + - "b.IsCatalogJanitorEnabledResponse\022^\n\021Exe", - "cMasterService\022#.hbase.pb.CoprocessorSer" + - "viceRequest\032$.hbase.pb.CoprocessorServic" + - "eResponse\022A\n\010Snapshot\022\031.hbase.pb.Snapsho" + - "tRequest\032\032.hbase.pb.SnapshotResponse\022h\n\025" + - "GetCompletedSnapshots\022&.hbase.pb.GetComp" + - "letedSnapshotsRequest\032\'.hbase.pb.GetComp" + - "letedSnapshotsResponse\022S\n\016DeleteSnapshot" + - "\022\037.hbase.pb.DeleteSnapshotRequest\032 .hbas" + - "e.pb.DeleteSnapshotResponse\022S\n\016IsSnapsho" + - "tDone\022\037.hbase.pb.IsSnapshotDoneRequest\032 ", - ".hbase.pb.IsSnapshotDoneResponse\022V\n\017Rest" + - "oreSnapshot\022 .hbase.pb.RestoreSnapshotRe" + - "quest\032!.hbase.pb.RestoreSnapshotResponse" + - "\022P\n\rExecProcedure\022\036.hbase.pb.ExecProcedu" + - "reRequest\032\037.hbase.pb.ExecProcedureRespon" + - "se\022W\n\024ExecProcedureWithRet\022\036.hbase.pb.Ex" + - "ecProcedureRequest\032\037.hbase.pb.ExecProced" + - "ureResponse\022V\n\017IsProcedureDone\022 .hbase.p" + - "b.IsProcedureDoneRequest\032!.hbase.pb.IsPr" + - "ocedureDoneResponse\022V\n\017ModifyNamespace\022 ", - ".hbase.pb.ModifyNamespaceRequest\032!.hbase" + - ".pb.ModifyNamespaceResponse\022V\n\017CreateNam" + - "espace\022 .hbase.pb.CreateNamespaceRequest" + - "\032!.hbase.pb.CreateNamespaceResponse\022V\n\017D" + - "eleteNamespace\022 .hbase.pb.DeleteNamespac" + - "eRequest\032!.hbase.pb.DeleteNamespaceRespo" + - "nse\022k\n\026GetNamespaceDescriptor\022\'.hbase.pb" + - ".GetNamespaceDescriptorRequest\032(.hbase.p" + - "b.GetNamespaceDescriptorResponse\022q\n\030List" + - "NamespaceDescriptors\022).hbase.pb.ListName", - "spaceDescriptorsRequest\032*.hbase.pb.ListN" + - "amespaceDescriptorsResponse\022\206\001\n\037ListTabl" + - "eDescriptorsByNamespace\0220.hbase.pb.ListT" + - "ableDescriptorsByNamespaceRequest\0321.hbas" + - "e.pb.ListTableDescriptorsByNamespaceResp" + - "onse\022t\n\031ListTableNamesByNamespace\022*.hbas" + - "e.pb.ListTableNamesByNamespaceRequest\032+." + - "hbase.pb.ListTableNamesByNamespaceRespon" + - "se\022P\n\rGetTableState\022\036.hbase.pb.GetTableS" + - "tateRequest\032\037.hbase.pb.GetTableStateResp", - "onse\022A\n\010SetQuota\022\031.hbase.pb.SetQuotaRequ" + - "est\032\032.hbase.pb.SetQuotaResponse\022x\n\037getLa" + - "stMajorCompactionTimestamp\022).hbase.pb.Ma" + - "jorCompactionTimestampRequest\032*.hbase.pb" + - ".MajorCompactionTimestampResponse\022\212\001\n(ge" + - "tLastMajorCompactionTimestampForRegion\0222" + - ".hbase.pb.MajorCompactionTimestampForReg" + - "ionRequest\032*.hbase.pb.MajorCompactionTim" + - "estampResponse\022_\n\022getProcedureResult\022#.h" + - "base.pb.GetProcedureResultRequest\032$.hbas", - "e.pb.GetProcedureResultResponse\022h\n\027getSe" + - "curityCapabilities\022%.hbase.pb.SecurityCa" + - "pabilitiesRequest\032&.hbase.pb.SecurityCap" + - "abilitiesResponse\022S\n\016AbortProcedure\022\037.hb" + - "ase.pb.AbortProcedureRequest\032 .hbase.pb." + - "AbortProcedureResponse\022S\n\016ListProcedures" + - "\022\037.hbase.pb.ListProceduresRequest\032 .hbas" + - "e.pb.ListProceduresResponseBI\n1org.apach" + - "e.hadoop.hbase.shaded.protobuf.generated" + - "B\014MasterProtosH\001\210\001\001\240\001\001" + "oup\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027Modify", + "NamespaceResponse\022\017\n\007proc_id\030\001 \001(\004\"6\n\035Ge" + + "tNamespaceDescriptorRequest\022\025\n\rnamespace" + + "Name\030\001 \002(\t\"\\\n\036GetNamespaceDescriptorResp" + + "onse\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.hbas" + + "e.pb.NamespaceDescriptor\"!\n\037ListNamespac" + + "eDescriptorsRequest\"^\n ListNamespaceDesc" + + "riptorsResponse\022:\n\023namespaceDescriptor\030\001" + + " \003(\0132\035.hbase.pb.NamespaceDescriptor\"?\n&L" + + "istTableDescriptorsByNamespaceRequest\022\025\n" + + "\rnamespaceName\030\001 \002(\t\"U\n\'ListTableDescrip", + "torsByNamespaceResponse\022*\n\013tableSchema\030\001" + + " \003(\0132\025.hbase.pb.TableSchema\"9\n ListTable" + + "NamesByNamespaceRequest\022\025\n\rnamespaceName" + + "\030\001 \002(\t\"K\n!ListTableNamesByNamespaceRespo" + + "nse\022&\n\ttableName\030\001 \003(\0132\023.hbase.pb.TableN" + + "ame\"\021\n\017ShutdownRequest\"\022\n\020ShutdownRespon" + + "se\"\023\n\021StopMasterRequest\"\024\n\022StopMasterRes" + + "ponse\"\034\n\032IsInMaintenanceModeRequest\"8\n\033I" + + "sInMaintenanceModeResponse\022\031\n\021inMaintena" + + "nceMode\030\001 \002(\010\"\037\n\016BalanceRequest\022\r\n\005force", + "\030\001 \001(\010\"\'\n\017BalanceResponse\022\024\n\014balancer_ra" + + "n\030\001 \002(\010\"<\n\031SetBalancerRunningRequest\022\n\n\002" + + "on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8\n\032SetBala" + + "ncerRunningResponse\022\032\n\022prev_balance_valu" + + "e\030\001 \001(\010\"\032\n\030IsBalancerEnabledRequest\",\n\031I" + + "sBalancerEnabledResponse\022\017\n\007enabled\030\001 \002(" + + "\010\"w\n\035SetSplitOrMergeEnabledRequest\022\017\n\007en" + + "abled\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\0220\n\014swit" + + "ch_types\030\003 \003(\0162\032.hbase.pb.MasterSwitchTy" + + "pe\"4\n\036SetSplitOrMergeEnabledResponse\022\022\n\n", + "prev_value\030\001 \003(\010\"O\n\034IsSplitOrMergeEnable" + + "dRequest\022/\n\013switch_type\030\001 \002(\0162\032.hbase.pb" + + ".MasterSwitchType\"0\n\035IsSplitOrMergeEnabl" + + "edResponse\022\017\n\007enabled\030\001 \002(\010\"\022\n\020Normalize" + + "Request\"+\n\021NormalizeResponse\022\026\n\016normaliz" + + "er_ran\030\001 \002(\010\")\n\033SetNormalizerRunningRequ" + + "est\022\n\n\002on\030\001 \002(\010\"=\n\034SetNormalizerRunningR" + + "esponse\022\035\n\025prev_normalizer_value\030\001 \001(\010\"\034" + + "\n\032IsNormalizerEnabledRequest\".\n\033IsNormal" + + "izerEnabledResponse\022\017\n\007enabled\030\001 \002(\010\"\027\n\025", + "RunCatalogScanRequest\"-\n\026RunCatalogScanR" + + "esponse\022\023\n\013scan_result\030\001 \001(\005\"-\n\033EnableCa" + + "talogJanitorRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034E" + + "nableCatalogJanitorResponse\022\022\n\nprev_valu" + + "e\030\001 \001(\010\" \n\036IsCatalogJanitorEnabledReques" + + "t\"0\n\037IsCatalogJanitorEnabledResponse\022\r\n\005" + + "value\030\001 \002(\010\"B\n\017SnapshotRequest\022/\n\010snapsh" + + "ot\030\001 \002(\0132\035.hbase.pb.SnapshotDescription\"" + + ",\n\020SnapshotResponse\022\030\n\020expected_timeout\030" + + "\001 \002(\003\"\036\n\034GetCompletedSnapshotsRequest\"Q\n", + "\035GetCompletedSnapshotsResponse\0220\n\tsnapsh" + + "ots\030\001 \003(\0132\035.hbase.pb.SnapshotDescription" + + "\"H\n\025DeleteSnapshotRequest\022/\n\010snapshot\030\001 " + + "\002(\0132\035.hbase.pb.SnapshotDescription\"\030\n\026De" + + "leteSnapshotResponse\"s\n\026RestoreSnapshotR" + + "equest\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.Snap" + + "shotDescription\022\026\n\013nonce_group\030\002 \001(\004:\0010\022" + + "\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027RestoreSnapshotResp" + + "onse\022\017\n\007proc_id\030\001 \002(\004\"H\n\025IsSnapshotDoneR" + + "equest\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.Snap", + "shotDescription\"^\n\026IsSnapshotDoneRespons" + + "e\022\023\n\004done\030\001 \001(\010:\005false\022/\n\010snapshot\030\002 \001(\013" + + "2\035.hbase.pb.SnapshotDescription\"O\n\034IsRes" + + "toreSnapshotDoneRequest\022/\n\010snapshot\030\001 \001(" + + "\0132\035.hbase.pb.SnapshotDescription\"4\n\035IsRe" + + "storeSnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:" + + "\005false\"F\n\033GetSchemaAlterStatusRequest\022\'\n" + + "\ntable_name\030\001 \002(\0132\023.hbase.pb.TableName\"T" + + "\n\034GetSchemaAlterStatusResponse\022\035\n\025yet_to" + + "_update_regions\030\001 \001(\r\022\025\n\rtotal_regions\030\002", + " \001(\r\"\213\001\n\032GetTableDescriptorsRequest\022(\n\013t" + + "able_names\030\001 \003(\0132\023.hbase.pb.TableName\022\r\n" + + "\005regex\030\002 \001(\t\022!\n\022include_sys_tables\030\003 \001(\010" + + ":\005false\022\021\n\tnamespace\030\004 \001(\t\"J\n\033GetTableDe" + + "scriptorsResponse\022+\n\014table_schema\030\001 \003(\0132" + + "\025.hbase.pb.TableSchema\"[\n\024GetTableNamesR" + + "equest\022\r\n\005regex\030\001 \001(\t\022!\n\022include_sys_tab" + + "les\030\002 \001(\010:\005false\022\021\n\tnamespace\030\003 \001(\t\"A\n\025G" + + "etTableNamesResponse\022(\n\013table_names\030\001 \003(" + + "\0132\023.hbase.pb.TableName\"?\n\024GetTableStateR", + "equest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.Ta" + + "bleName\"B\n\025GetTableStateResponse\022)\n\013tabl" + + "e_state\030\001 \002(\0132\024.hbase.pb.TableState\"\031\n\027G" + + "etClusterStatusRequest\"K\n\030GetClusterStat" + + "usResponse\022/\n\016cluster_status\030\001 \002(\0132\027.hba" + + "se.pb.ClusterStatus\"\030\n\026IsMasterRunningRe" + + "quest\"4\n\027IsMasterRunningResponse\022\031\n\021is_m" + + "aster_running\030\001 \002(\010\"I\n\024ExecProcedureRequ" + + "est\0221\n\tprocedure\030\001 \002(\0132\036.hbase.pb.Proced" + + "ureDescription\"F\n\025ExecProcedureResponse\022", + "\030\n\020expected_timeout\030\001 \001(\003\022\023\n\013return_data" + + "\030\002 \001(\014\"K\n\026IsProcedureDoneRequest\0221\n\tproc" + + "edure\030\001 \001(\0132\036.hbase.pb.ProcedureDescript" + + "ion\"`\n\027IsProcedureDoneResponse\022\023\n\004done\030\001" + + " \001(\010:\005false\0220\n\010snapshot\030\002 \001(\0132\036.hbase.pb" + + ".ProcedureDescription\",\n\031GetProcedureRes" + + "ultRequest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032GetProce" + + "dureResultResponse\0229\n\005state\030\001 \002(\0162*.hbas" + + "e.pb.GetProcedureResultResponse.State\022\022\n" + + "\nstart_time\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016", + "\n\006result\030\004 \001(\014\0224\n\texception\030\005 \001(\0132!.hbas" + + "e.pb.ForeignExceptionMessage\"1\n\005State\022\r\n" + + "\tNOT_FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"" + + "M\n\025AbortProcedureRequest\022\017\n\007proc_id\030\001 \002(" + + "\004\022#\n\025mayInterruptIfRunning\030\002 \001(\010:\004true\"6" + + "\n\026AbortProcedureResponse\022\034\n\024is_procedure" + + "_aborted\030\001 \002(\010\"\027\n\025ListProceduresRequest\"" + + "@\n\026ListProceduresResponse\022&\n\tprocedure\030\001" + + " \003(\0132\023.hbase.pb.Procedure\"\315\001\n\017SetQuotaRe" + + "quest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser_group\030\002", + " \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\'\n\ntable_name\030\004 " + + "\001(\0132\023.hbase.pb.TableName\022\022\n\nremove_all\030\005" + + " \001(\010\022\026\n\016bypass_globals\030\006 \001(\010\022+\n\010throttle" + + "\030\007 \001(\0132\031.hbase.pb.ThrottleRequest\"\022\n\020Set" + + "QuotaResponse\"J\n\037MajorCompactionTimestam" + + "pRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb." + + "TableName\"U\n(MajorCompactionTimestampFor" + + "RegionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb" + + ".RegionSpecifier\"@\n MajorCompactionTimes" + + "tampResponse\022\034\n\024compaction_timestamp\030\001 \002", + "(\003\"\035\n\033SecurityCapabilitiesRequest\"\354\001\n\034Se" + + "curityCapabilitiesResponse\022G\n\014capabiliti" + + "es\030\001 \003(\01621.hbase.pb.SecurityCapabilities" + + "Response.Capability\"\202\001\n\nCapability\022\031\n\025SI" + + "MPLE_AUTHENTICATION\020\000\022\031\n\025SECURE_AUTHENTI" + + "CATION\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022CELL_AUTH" + + "ORIZATION\020\003\022\023\n\017CELL_VISIBILITY\020\004\"!\n\037Redi" + + "stributeFavoredNodesRequest\"2\n Redistrib" + + "uteFavoredNodesResponse\022\016\n\006result\030\001 \002(\010\"" + + ")\n\'CompleteRedistributeFavoredNodesReque", + "st\":\n(CompleteRedistributeFavoredNodesRe" + + "sponse\022\016\n\006result\030\001 \002(\010\"K\n\037GetFavoredNode" + + "sForRegionRequest\022(\n\nregionInfo\030\001 \002(\0132\024." + + "hbase.pb.RegionInfo\"I\n GetFavoredNodesFo" + + "rRegionResponse\022%\n\007servers\030\001 \003(\0132\024.hbase" + + ".pb.ServerName\">\n\025GetReplicaLoadRequest\022" + + "%\n\007servers\030\001 \003(\0132\024.hbase.pb.ServerName\"O" + + "\n\026GetReplicaLoadResponse\0225\n\014replica_load" + + "\030\001 \003(\0132\037.hbase.pb.ServerReplicaLoadPair\"" + + "@\n\030RemoveFavoredNodeRequest\022$\n\006server\030\001 ", + "\002(\0132\024.hbase.pb.ServerName\"\033\n\031RemoveFavor" + + "edNodeResponse\",\n\030CheckFavoredNodesReque" + + "st\022\020\n\010detailed\030\001 \001(\010\"B\n\031CheckFavoredNode" + + "sResponse\022%\n\007servers\030\001 \003(\0132\024.hbase.pb.Se" + + "rverName*(\n\020MasterSwitchType\022\t\n\005SPLIT\020\000\022" + + "\t\n\005MERGE\020\0012\300.\n\rMasterService\022e\n\024GetSchem" + + "aAlterStatus\022%.hbase.pb.GetSchemaAlterSt" + + "atusRequest\032&.hbase.pb.GetSchemaAlterSta" + + "tusResponse\022b\n\023GetTableDescriptors\022$.hba" + + "se.pb.GetTableDescriptorsRequest\032%.hbase", + ".pb.GetTableDescriptorsResponse\022P\n\rGetTa" + + "bleNames\022\036.hbase.pb.GetTableNamesRequest" + + "\032\037.hbase.pb.GetTableNamesResponse\022Y\n\020Get" + + "ClusterStatus\022!.hbase.pb.GetClusterStatu" + + "sRequest\032\".hbase.pb.GetClusterStatusResp" + + "onse\022V\n\017IsMasterRunning\022 .hbase.pb.IsMas" + + "terRunningRequest\032!.hbase.pb.IsMasterRun" + + "ningResponse\022D\n\tAddColumn\022\032.hbase.pb.Add" + + "ColumnRequest\032\033.hbase.pb.AddColumnRespon" + + "se\022M\n\014DeleteColumn\022\035.hbase.pb.DeleteColu", + "mnRequest\032\036.hbase.pb.DeleteColumnRespons" + + "e\022M\n\014ModifyColumn\022\035.hbase.pb.ModifyColum" + + "nRequest\032\036.hbase.pb.ModifyColumnResponse" + + "\022G\n\nMoveRegion\022\033.hbase.pb.MoveRegionRequ" + + "est\032\034.hbase.pb.MoveRegionResponse\022k\n\026Dis" + + "patchMergingRegions\022\'.hbase.pb.DispatchM" + + "ergingRegionsRequest\032(.hbase.pb.Dispatch" + + "MergingRegionsResponse\022M\n\014AssignRegion\022\035" + + ".hbase.pb.AssignRegionRequest\032\036.hbase.pb" + + ".AssignRegionResponse\022S\n\016UnassignRegion\022", + "\037.hbase.pb.UnassignRegionRequest\032 .hbase" + + ".pb.UnassignRegionResponse\022P\n\rOfflineReg" + + "ion\022\036.hbase.pb.OfflineRegionRequest\032\037.hb" + + "ase.pb.OfflineRegionResponse\022J\n\013DeleteTa" + + "ble\022\034.hbase.pb.DeleteTableRequest\032\035.hbas" + + "e.pb.DeleteTableResponse\022P\n\rtruncateTabl" + + "e\022\036.hbase.pb.TruncateTableRequest\032\037.hbas" + + "e.pb.TruncateTableResponse\022J\n\013EnableTabl" + + "e\022\034.hbase.pb.EnableTableRequest\032\035.hbase." + + "pb.EnableTableResponse\022M\n\014DisableTable\022\035", + ".hbase.pb.DisableTableRequest\032\036.hbase.pb" + + ".DisableTableResponse\022J\n\013ModifyTable\022\034.h" + + "base.pb.ModifyTableRequest\032\035.hbase.pb.Mo" + + "difyTableResponse\022J\n\013CreateTable\022\034.hbase" + + ".pb.CreateTableRequest\032\035.hbase.pb.Create" + + "TableResponse\022A\n\010Shutdown\022\031.hbase.pb.Shu" + + "tdownRequest\032\032.hbase.pb.ShutdownResponse" + + "\022G\n\nStopMaster\022\033.hbase.pb.StopMasterRequ" + + "est\032\034.hbase.pb.StopMasterResponse\022h\n\031IsM" + + "asterInMaintenanceMode\022$.hbase.pb.IsInMa", + "intenanceModeRequest\032%.hbase.pb.IsInMain" + + "tenanceModeResponse\022>\n\007Balance\022\030.hbase.p" + + "b.BalanceRequest\032\031.hbase.pb.BalanceRespo" + + "nse\022_\n\022SetBalancerRunning\022#.hbase.pb.Set" + + "BalancerRunningRequest\032$.hbase.pb.SetBal" + + "ancerRunningResponse\022\\\n\021IsBalancerEnable" + + "d\022\".hbase.pb.IsBalancerEnabledRequest\032#." + + "hbase.pb.IsBalancerEnabledResponse\022k\n\026Se" + + "tSplitOrMergeEnabled\022\'.hbase.pb.SetSplit" + + "OrMergeEnabledRequest\032(.hbase.pb.SetSpli", + "tOrMergeEnabledResponse\022h\n\025IsSplitOrMerg" + + "eEnabled\022&.hbase.pb.IsSplitOrMergeEnable" + + "dRequest\032\'.hbase.pb.IsSplitOrMergeEnable" + + "dResponse\022D\n\tNormalize\022\032.hbase.pb.Normal" + + "izeRequest\032\033.hbase.pb.NormalizeResponse\022" + + "e\n\024SetNormalizerRunning\022%.hbase.pb.SetNo" + + "rmalizerRunningRequest\032&.hbase.pb.SetNor" + + "malizerRunningResponse\022b\n\023IsNormalizerEn" + + "abled\022$.hbase.pb.IsNormalizerEnabledRequ" + + "est\032%.hbase.pb.IsNormalizerEnabledRespon", + "se\022S\n\016RunCatalogScan\022\037.hbase.pb.RunCatal" + + "ogScanRequest\032 .hbase.pb.RunCatalogScanR" + + "esponse\022e\n\024EnableCatalogJanitor\022%.hbase." + + "pb.EnableCatalogJanitorRequest\032&.hbase.p" + + "b.EnableCatalogJanitorResponse\022n\n\027IsCata" + + "logJanitorEnabled\022(.hbase.pb.IsCatalogJa" + + "nitorEnabledRequest\032).hbase.pb.IsCatalog" + + "JanitorEnabledResponse\022^\n\021ExecMasterServ" + + "ice\022#.hbase.pb.CoprocessorServiceRequest" + + "\032$.hbase.pb.CoprocessorServiceResponse\022A", + "\n\010Snapshot\022\031.hbase.pb.SnapshotRequest\032\032." + + "hbase.pb.SnapshotResponse\022h\n\025GetComplete" + + "dSnapshots\022&.hbase.pb.GetCompletedSnapsh" + + "otsRequest\032\'.hbase.pb.GetCompletedSnapsh" + + "otsResponse\022S\n\016DeleteSnapshot\022\037.hbase.pb" + + ".DeleteSnapshotRequest\032 .hbase.pb.Delete" + + "SnapshotResponse\022S\n\016IsSnapshotDone\022\037.hba" + + "se.pb.IsSnapshotDoneRequest\032 .hbase.pb.I" + + "sSnapshotDoneResponse\022V\n\017RestoreSnapshot" + + "\022 .hbase.pb.RestoreSnapshotRequest\032!.hba", + "se.pb.RestoreSnapshotResponse\022P\n\rExecPro" + + "cedure\022\036.hbase.pb.ExecProcedureRequest\032\037" + + ".hbase.pb.ExecProcedureResponse\022W\n\024ExecP" + + "rocedureWithRet\022\036.hbase.pb.ExecProcedure" + + "Request\032\037.hbase.pb.ExecProcedureResponse" + + "\022V\n\017IsProcedureDone\022 .hbase.pb.IsProcedu" + + "reDoneRequest\032!.hbase.pb.IsProcedureDone" + + "Response\022V\n\017ModifyNamespace\022 .hbase.pb.M" + + "odifyNamespaceRequest\032!.hbase.pb.ModifyN" + + "amespaceResponse\022V\n\017CreateNamespace\022 .hb", + "ase.pb.CreateNamespaceRequest\032!.hbase.pb" + + ".CreateNamespaceResponse\022V\n\017DeleteNamesp" + + "ace\022 .hbase.pb.DeleteNamespaceRequest\032!." + + "hbase.pb.DeleteNamespaceResponse\022k\n\026GetN" + + "amespaceDescriptor\022\'.hbase.pb.GetNamespa" + + "ceDescriptorRequest\032(.hbase.pb.GetNamesp" + + "aceDescriptorResponse\022q\n\030ListNamespaceDe" + + "scriptors\022).hbase.pb.ListNamespaceDescri" + + "ptorsRequest\032*.hbase.pb.ListNamespaceDes" + + "criptorsResponse\022\206\001\n\037ListTableDescriptor", + "sByNamespace\0220.hbase.pb.ListTableDescrip" + + "torsByNamespaceRequest\0321.hbase.pb.ListTa" + + "bleDescriptorsByNamespaceResponse\022t\n\031Lis" + + "tTableNamesByNamespace\022*.hbase.pb.ListTa" + + "bleNamesByNamespaceRequest\032+.hbase.pb.Li" + + "stTableNamesByNamespaceResponse\022h\n\033Updat" + + "eFavoredNodesForRegion\022#.hbase.pb.Update" + + "FavoredNodesRequest\032$.hbase.pb.UpdateFav" + + "oredNodesResponse\022q\n\030redistributeFavored" + + "Nodes\022).hbase.pb.RedistributeFavoredNode", + "sRequest\032*.hbase.pb.RedistributeFavoredN" + + "odesResponse\022\211\001\n completeRedistributeFav" + + "oredNodes\0221.hbase.pb.CompleteRedistribut" + + "eFavoredNodesRequest\0322.hbase.pb.Complete" + + "RedistributeFavoredNodesResponse\022q\n\030getF" + + "avoredNodesForRegion\022).hbase.pb.GetFavor" + + "edNodesForRegionRequest\032*.hbase.pb.GetFa" + + "voredNodesForRegionResponse\022S\n\016getReplic" + + "aLoad\022\037.hbase.pb.GetReplicaLoadRequest\032 " + + ".hbase.pb.GetReplicaLoadResponse\022\\\n\021remo", + "veFavoredNode\022\".hbase.pb.RemoveFavoredNo" + + "deRequest\032#.hbase.pb.RemoveFavoredNodeRe" + + "sponse\022\\\n\021checkFavoredNodes\022\".hbase.pb.C" + + "heckFavoredNodesRequest\032#.hbase.pb.Check" + + "FavoredNodesResponse\022P\n\rGetTableState\022\036." + + "hbase.pb.GetTableStateRequest\032\037.hbase.pb" + + ".GetTableStateResponse\022A\n\010SetQuota\022\031.hba" + + "se.pb.SetQuotaRequest\032\032.hbase.pb.SetQuot" + + "aResponse\022x\n\037getLastMajorCompactionTimes" + + "tamp\022).hbase.pb.MajorCompactionTimestamp", + "Request\032*.hbase.pb.MajorCompactionTimest" + + "ampResponse\022\212\001\n(getLastMajorCompactionTi" + + "mestampForRegion\0222.hbase.pb.MajorCompact" + + "ionTimestampForRegionRequest\032*.hbase.pb." + + "MajorCompactionTimestampResponse\022_\n\022getP" + + "rocedureResult\022#.hbase.pb.GetProcedureRe" + + "sultRequest\032$.hbase.pb.GetProcedureResul" + + "tResponse\022h\n\027getSecurityCapabilities\022%.h" + + "base.pb.SecurityCapabilitiesRequest\032&.hb" + + "ase.pb.SecurityCapabilitiesResponse\022S\n\016A", + "bortProcedure\022\037.hbase.pb.AbortProcedureR" + + "equest\032 .hbase.pb.AbortProcedureResponse" + + "\022S\n\016ListProcedures\022\037.hbase.pb.ListProced" + + "uresRequest\032 .hbase.pb.ListProceduresRes" + + "ponseBI\n1org.apache.hadoop.hbase.shaded." + + "protobuf.generatedB\014MasterProtosH\001\210\001\001\240\001\001" }; org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -69744,6 +77095,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(), org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.getDescriptor(), org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor(), + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.getDescriptor(), org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.getDescriptor(), org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.getDescriptor(), org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor(), @@ -70414,9 +77766,82 @@ public final class MasterProtos { org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor, new java.lang.String[] { "Capabilities", }); + internal_static_hbase_pb_RedistributeFavoredNodesRequest_descriptor = + getDescriptor().getMessageTypes().get(111); + internal_static_hbase_pb_RedistributeFavoredNodesRequest_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_RedistributeFavoredNodesRequest_descriptor, + new java.lang.String[] { }); + internal_static_hbase_pb_RedistributeFavoredNodesResponse_descriptor = + getDescriptor().getMessageTypes().get(112); + internal_static_hbase_pb_RedistributeFavoredNodesResponse_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_RedistributeFavoredNodesResponse_descriptor, + new java.lang.String[] { "Result", }); + internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_descriptor = + getDescriptor().getMessageTypes().get(113); + internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_CompleteRedistributeFavoredNodesRequest_descriptor, + new java.lang.String[] { }); + internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_descriptor = + getDescriptor().getMessageTypes().get(114); + internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_CompleteRedistributeFavoredNodesResponse_descriptor, + new java.lang.String[] { "Result", }); + internal_static_hbase_pb_GetFavoredNodesForRegionRequest_descriptor = + getDescriptor().getMessageTypes().get(115); + internal_static_hbase_pb_GetFavoredNodesForRegionRequest_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_GetFavoredNodesForRegionRequest_descriptor, + new java.lang.String[] { "RegionInfo", }); + internal_static_hbase_pb_GetFavoredNodesForRegionResponse_descriptor = + getDescriptor().getMessageTypes().get(116); + internal_static_hbase_pb_GetFavoredNodesForRegionResponse_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_GetFavoredNodesForRegionResponse_descriptor, + new java.lang.String[] { "Servers", }); + internal_static_hbase_pb_GetReplicaLoadRequest_descriptor = + getDescriptor().getMessageTypes().get(117); + internal_static_hbase_pb_GetReplicaLoadRequest_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_GetReplicaLoadRequest_descriptor, + new java.lang.String[] { "Servers", }); + internal_static_hbase_pb_GetReplicaLoadResponse_descriptor = + getDescriptor().getMessageTypes().get(118); + internal_static_hbase_pb_GetReplicaLoadResponse_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_GetReplicaLoadResponse_descriptor, + new java.lang.String[] { "ReplicaLoad", }); + internal_static_hbase_pb_RemoveFavoredNodeRequest_descriptor = + getDescriptor().getMessageTypes().get(119); + internal_static_hbase_pb_RemoveFavoredNodeRequest_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_RemoveFavoredNodeRequest_descriptor, + new java.lang.String[] { "Server", }); + internal_static_hbase_pb_RemoveFavoredNodeResponse_descriptor = + getDescriptor().getMessageTypes().get(120); + internal_static_hbase_pb_RemoveFavoredNodeResponse_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_RemoveFavoredNodeResponse_descriptor, + new java.lang.String[] { }); + internal_static_hbase_pb_CheckFavoredNodesRequest_descriptor = + getDescriptor().getMessageTypes().get(121); + internal_static_hbase_pb_CheckFavoredNodesRequest_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_CheckFavoredNodesRequest_descriptor, + new java.lang.String[] { "Detailed", }); + internal_static_hbase_pb_CheckFavoredNodesResponse_descriptor = + getDescriptor().getMessageTypes().get(122); + internal_static_hbase_pb_CheckFavoredNodesResponse_fieldAccessorTable = new + org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_hbase_pb_CheckFavoredNodesResponse_descriptor, + new java.lang.String[] { "Servers", }); org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(); org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.getDescriptor(); org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor(); + org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.getDescriptor(); org.apache.hadoop.hbase.shaded.protobuf.generated.ErrorHandlingProtos.getDescriptor(); org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.getDescriptor(); org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.getDescriptor(); diff --git a/hbase-protocol-shaded/src/main/protobuf/HBase.proto b/hbase-protocol-shaded/src/main/protobuf/HBase.proto index 9b6b556..8e689c5 100644 --- a/hbase-protocol-shaded/src/main/protobuf/HBase.proto +++ b/hbase-protocol-shaded/src/main/protobuf/HBase.proto @@ -250,3 +250,19 @@ message RegionServerInfo { optional int32 infoPort = 1; optional VersionInfo version_info = 2; } + +message ServerReplicaLoadPair { + required ServerName server = 1; + required ReplicaLoad replicaCount = 2; +} + +message ReplicaLoad { + required int64 primaryReplicaCount = 1; + required int64 secondaryReplicaCount = 2; + required int64 tertiaryReplicaCount = 3; +} + +message FavoredNodesInfoPair { + required string regionName = 1; + repeated ServerName servers = 2; +} diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto index 9e6d1ed..d3af77c 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto @@ -29,6 +29,7 @@ option optimize_for = SPEED; import "HBase.proto"; import "Client.proto"; import "ClusterStatus.proto"; +import "Admin.proto"; import "ErrorHandling.proto"; import "Procedure.proto"; import "Quota.proto"; @@ -553,6 +554,51 @@ message SecurityCapabilitiesResponse { repeated Capability capabilities = 1; } +message RedistributeFavoredNodesRequest { +} + +message RedistributeFavoredNodesResponse { + required bool result = 1; +} + +message CompleteRedistributeFavoredNodesRequest { +} + +message CompleteRedistributeFavoredNodesResponse { + required bool result = 1; +} + +message GetFavoredNodesForRegionRequest { + required RegionInfo regionInfo = 1; +} + +message GetFavoredNodesForRegionResponse { + repeated ServerName servers = 1; +} + +message GetReplicaLoadRequest { + repeated ServerName servers = 1; +} + +message GetReplicaLoadResponse { + repeated ServerReplicaLoadPair replica_load = 1; +} + +message RemoveFavoredNodeRequest { + required ServerName server = 1; +} + +message RemoveFavoredNodeResponse { +} + +message CheckFavoredNodesRequest { + optional bool detailed = 1; +} + +message CheckFavoredNodesResponse { + repeated ServerName servers = 1; +} + service MasterService { /** Used by the client to get the number of regions that have received the updated schema */ rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest) @@ -799,6 +845,30 @@ service MasterService { rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest) returns(ListTableNamesByNamespaceResponse); + /** Updated favoredNodes for a region */ + rpc UpdateFavoredNodesForRegion(UpdateFavoredNodesRequest) + returns(UpdateFavoredNodesResponse); + + /** Redistribute all favored node replicas of region. This API does not move current assignments*/ + rpc redistributeFavoredNodes(RedistributeFavoredNodesRequest) + returns(RedistributeFavoredNodesResponse); + + /** Redistribute all favored node replicas of region. This API moves current assignments.*/ + rpc completeRedistributeFavoredNodes(CompleteRedistributeFavoredNodesRequest) + returns(CompleteRedistributeFavoredNodesResponse); + + rpc getFavoredNodesForRegion(GetFavoredNodesForRegionRequest) + returns (GetFavoredNodesForRegionResponse); + + rpc getReplicaLoad(GetReplicaLoadRequest) + returns (GetReplicaLoadResponse); + + rpc removeFavoredNode(RemoveFavoredNodeRequest) + returns (RemoveFavoredNodeResponse); + + rpc checkFavoredNodes(CheckFavoredNodesRequest) + returns (CheckFavoredNodesResponse); + /** returns table state */ rpc GetTableState(GetTableStateRequest) returns(GetTableStateResponse); diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java index 72fc7b7..e64df4e 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java @@ -19528,6 +19528,2319 @@ public final class HBaseProtos { // @@protoc_insertion_point(class_scope:hbase.pb.RegionServerInfo) } + public interface ServerReplicaLoadPairOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .hbase.pb.ServerName server = 1; + /** + * required .hbase.pb.ServerName server = 1; + */ + boolean hasServer(); + /** + * required .hbase.pb.ServerName server = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer(); + /** + * required .hbase.pb.ServerName server = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder(); + + // required .hbase.pb.ReplicaLoad replicaCount = 2; + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + boolean hasReplicaCount(); + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad getReplicaCount(); + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder getReplicaCountOrBuilder(); + } + /** + * Protobuf type {@code hbase.pb.ServerReplicaLoadPair} + */ + public static final class ServerReplicaLoadPair extends + com.google.protobuf.GeneratedMessage + implements ServerReplicaLoadPairOrBuilder { + // Use ServerReplicaLoadPair.newBuilder() to construct. + private ServerReplicaLoadPair(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ServerReplicaLoadPair(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ServerReplicaLoadPair defaultInstance; + public static ServerReplicaLoadPair getDefaultInstance() { + return defaultInstance; + } + + public ServerReplicaLoadPair getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ServerReplicaLoadPair( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = server_.toBuilder(); + } + server_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(server_); + server_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = replicaCount_.toBuilder(); + } + replicaCount_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(replicaCount_); + replicaCount_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ServerReplicaLoadPair parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ServerReplicaLoadPair(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .hbase.pb.ServerName server = 1; + public static final int SERVER_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_; + /** + * required .hbase.pb.ServerName server = 1; + */ + public boolean hasServer() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() { + return server_; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + return server_; + } + + // required .hbase.pb.ReplicaLoad replicaCount = 2; + public static final int REPLICACOUNT_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad replicaCount_; + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public boolean hasReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad getReplicaCount() { + return replicaCount_; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder getReplicaCountOrBuilder() { + return replicaCount_; + } + + private void initFields() { + server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + replicaCount_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasServer()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasReplicaCount()) { + memoizedIsInitialized = 0; + return false; + } + if (!getServer().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getReplicaCount().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, server_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, replicaCount_); + } + 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, server_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, replicaCount_); + } + 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.HBaseProtos.ServerReplicaLoadPair)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair) obj; + + boolean result = true; + result = result && (hasServer() == other.hasServer()); + if (hasServer()) { + result = result && getServer() + .equals(other.getServer()); + } + result = result && (hasReplicaCount() == other.hasReplicaCount()); + if (hasReplicaCount()) { + result = result && getReplicaCount() + .equals(other.getReplicaCount()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasServer()) { + hash = (37 * hash) + SERVER_FIELD_NUMBER; + hash = (53 * hash) + getServer().hashCode(); + } + if (hasReplicaCount()) { + hash = (37 * hash) + REPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + getReplicaCount().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair 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; + } + /** + * Protobuf type {@code hbase.pb.ServerReplicaLoadPair} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPairOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getServerFieldBuilder(); + getReplicaCountFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (serverBuilder_ == null) { + server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (replicaCountBuilder_ == null) { + replicaCount_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance(); + } else { + replicaCountBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair build() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (serverBuilder_ == null) { + result.server_ = server_; + } else { + result.server_ = serverBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (replicaCountBuilder_ == null) { + result.replicaCount_ = replicaCount_; + } else { + result.replicaCount_ = replicaCountBuilder_.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.HBaseProtos.ServerReplicaLoadPair) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair.getDefaultInstance()) return this; + if (other.hasServer()) { + mergeServer(other.getServer()); + } + if (other.hasReplicaCount()) { + mergeReplicaCount(other.getReplicaCount()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasServer()) { + + return false; + } + if (!hasReplicaCount()) { + + return false; + } + if (!getServer().isInitialized()) { + + return false; + } + if (!getReplicaCount().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerReplicaLoadPair) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .hbase.pb.ServerName server = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_ = 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> serverBuilder_; + /** + * required .hbase.pb.ServerName server = 1; + */ + public boolean hasServer() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() { + if (serverBuilder_ == null) { + return server_; + } else { + return serverBuilder_.getMessage(); + } + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder setServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + server_ = value; + onChanged(); + } else { + serverBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder setServer( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serverBuilder_ == null) { + server_ = builderForValue.build(); + onChanged(); + } else { + serverBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder mergeServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (serverBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + server_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) { + server_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(server_).mergeFrom(value).buildPartial(); + } else { + server_ = value; + } + onChanged(); + } else { + serverBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public Builder clearServer() { + if (serverBuilder_ == null) { + server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + onChanged(); + } else { + serverBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServerBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getServerFieldBuilder().getBuilder(); + } + /** + * required .hbase.pb.ServerName server = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() { + if (serverBuilder_ != null) { + return serverBuilder_.getMessageOrBuilder(); + } else { + return server_; + } + } + /** + * required .hbase.pb.ServerName server = 1; + */ + 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> + getServerFieldBuilder() { + if (serverBuilder_ == null) { + serverBuilder_ = 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>( + server_, + getParentForChildren(), + isClean()); + server_ = null; + } + return serverBuilder_; + } + + // required .hbase.pb.ReplicaLoad replicaCount = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad replicaCount_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder> replicaCountBuilder_; + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public boolean hasReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad getReplicaCount() { + if (replicaCountBuilder_ == null) { + return replicaCount_; + } else { + return replicaCountBuilder_.getMessage(); + } + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder setReplicaCount(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad value) { + if (replicaCountBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + replicaCount_ = value; + onChanged(); + } else { + replicaCountBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder setReplicaCount( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder builderForValue) { + if (replicaCountBuilder_ == null) { + replicaCount_ = builderForValue.build(); + onChanged(); + } else { + replicaCountBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder mergeReplicaCount(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad value) { + if (replicaCountBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + replicaCount_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance()) { + replicaCount_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.newBuilder(replicaCount_).mergeFrom(value).buildPartial(); + } else { + replicaCount_ = value; + } + onChanged(); + } else { + replicaCountBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public Builder clearReplicaCount() { + if (replicaCountBuilder_ == null) { + replicaCount_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance(); + onChanged(); + } else { + replicaCountBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder getReplicaCountBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getReplicaCountFieldBuilder().getBuilder(); + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder getReplicaCountOrBuilder() { + if (replicaCountBuilder_ != null) { + return replicaCountBuilder_.getMessageOrBuilder(); + } else { + return replicaCount_; + } + } + /** + * required .hbase.pb.ReplicaLoad replicaCount = 2; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder> + getReplicaCountFieldBuilder() { + if (replicaCountBuilder_ == null) { + replicaCountBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder>( + replicaCount_, + getParentForChildren(), + isClean()); + replicaCount_ = null; + } + return replicaCountBuilder_; + } + + // @@protoc_insertion_point(builder_scope:hbase.pb.ServerReplicaLoadPair) + } + + static { + defaultInstance = new ServerReplicaLoadPair(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:hbase.pb.ServerReplicaLoadPair) + } + + public interface ReplicaLoadOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required int64 primaryReplicaCount = 1; + /** + * required int64 primaryReplicaCount = 1; + */ + boolean hasPrimaryReplicaCount(); + /** + * required int64 primaryReplicaCount = 1; + */ + long getPrimaryReplicaCount(); + + // required int64 secondaryReplicaCount = 2; + /** + * required int64 secondaryReplicaCount = 2; + */ + boolean hasSecondaryReplicaCount(); + /** + * required int64 secondaryReplicaCount = 2; + */ + long getSecondaryReplicaCount(); + + // required int64 tertiaryReplicaCount = 3; + /** + * required int64 tertiaryReplicaCount = 3; + */ + boolean hasTertiaryReplicaCount(); + /** + * required int64 tertiaryReplicaCount = 3; + */ + long getTertiaryReplicaCount(); + } + /** + * Protobuf type {@code hbase.pb.ReplicaLoad} + */ + public static final class ReplicaLoad extends + com.google.protobuf.GeneratedMessage + implements ReplicaLoadOrBuilder { + // Use ReplicaLoad.newBuilder() to construct. + private ReplicaLoad(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ReplicaLoad(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ReplicaLoad defaultInstance; + public static ReplicaLoad getDefaultInstance() { + return defaultInstance; + } + + public ReplicaLoad getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ReplicaLoad( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + primaryReplicaCount_ = input.readInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + secondaryReplicaCount_ = input.readInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + tertiaryReplicaCount_ = input.readInt64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ReplicaLoad parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ReplicaLoad(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required int64 primaryReplicaCount = 1; + public static final int PRIMARYREPLICACOUNT_FIELD_NUMBER = 1; + private long primaryReplicaCount_; + /** + * required int64 primaryReplicaCount = 1; + */ + public boolean hasPrimaryReplicaCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required int64 primaryReplicaCount = 1; + */ + public long getPrimaryReplicaCount() { + return primaryReplicaCount_; + } + + // required int64 secondaryReplicaCount = 2; + public static final int SECONDARYREPLICACOUNT_FIELD_NUMBER = 2; + private long secondaryReplicaCount_; + /** + * required int64 secondaryReplicaCount = 2; + */ + public boolean hasSecondaryReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public long getSecondaryReplicaCount() { + return secondaryReplicaCount_; + } + + // required int64 tertiaryReplicaCount = 3; + public static final int TERTIARYREPLICACOUNT_FIELD_NUMBER = 3; + private long tertiaryReplicaCount_; + /** + * required int64 tertiaryReplicaCount = 3; + */ + public boolean hasTertiaryReplicaCount() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public long getTertiaryReplicaCount() { + return tertiaryReplicaCount_; + } + + private void initFields() { + primaryReplicaCount_ = 0L; + secondaryReplicaCount_ = 0L; + tertiaryReplicaCount_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasPrimaryReplicaCount()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSecondaryReplicaCount()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTertiaryReplicaCount()) { + 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.writeInt64(1, primaryReplicaCount_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt64(2, secondaryReplicaCount_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt64(3, tertiaryReplicaCount_); + } + 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 + .computeInt64Size(1, primaryReplicaCount_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, secondaryReplicaCount_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, tertiaryReplicaCount_); + } + 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.HBaseProtos.ReplicaLoad)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad) obj; + + boolean result = true; + result = result && (hasPrimaryReplicaCount() == other.hasPrimaryReplicaCount()); + if (hasPrimaryReplicaCount()) { + result = result && (getPrimaryReplicaCount() + == other.getPrimaryReplicaCount()); + } + result = result && (hasSecondaryReplicaCount() == other.hasSecondaryReplicaCount()); + if (hasSecondaryReplicaCount()) { + result = result && (getSecondaryReplicaCount() + == other.getSecondaryReplicaCount()); + } + result = result && (hasTertiaryReplicaCount() == other.hasTertiaryReplicaCount()); + if (hasTertiaryReplicaCount()) { + result = result && (getTertiaryReplicaCount() + == other.getTertiaryReplicaCount()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasPrimaryReplicaCount()) { + hash = (37 * hash) + PRIMARYREPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getPrimaryReplicaCount()); + } + if (hasSecondaryReplicaCount()) { + hash = (37 * hash) + SECONDARYREPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getSecondaryReplicaCount()); + } + if (hasTertiaryReplicaCount()) { + hash = (37 * hash) + TERTIARYREPLICACOUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getTertiaryReplicaCount()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad 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; + } + /** + * Protobuf type {@code hbase.pb.ReplicaLoad} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoadOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + primaryReplicaCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + secondaryReplicaCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + tertiaryReplicaCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_ReplicaLoad_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad build() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.primaryReplicaCount_ = primaryReplicaCount_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.secondaryReplicaCount_ = secondaryReplicaCount_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.tertiaryReplicaCount_ = tertiaryReplicaCount_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad.getDefaultInstance()) return this; + if (other.hasPrimaryReplicaCount()) { + setPrimaryReplicaCount(other.getPrimaryReplicaCount()); + } + if (other.hasSecondaryReplicaCount()) { + setSecondaryReplicaCount(other.getSecondaryReplicaCount()); + } + if (other.hasTertiaryReplicaCount()) { + setTertiaryReplicaCount(other.getTertiaryReplicaCount()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasPrimaryReplicaCount()) { + + return false; + } + if (!hasSecondaryReplicaCount()) { + + return false; + } + if (!hasTertiaryReplicaCount()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ReplicaLoad) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required int64 primaryReplicaCount = 1; + private long primaryReplicaCount_ ; + /** + * required int64 primaryReplicaCount = 1; + */ + public boolean hasPrimaryReplicaCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required int64 primaryReplicaCount = 1; + */ + public long getPrimaryReplicaCount() { + return primaryReplicaCount_; + } + /** + * required int64 primaryReplicaCount = 1; + */ + public Builder setPrimaryReplicaCount(long value) { + bitField0_ |= 0x00000001; + primaryReplicaCount_ = value; + onChanged(); + return this; + } + /** + * required int64 primaryReplicaCount = 1; + */ + public Builder clearPrimaryReplicaCount() { + bitField0_ = (bitField0_ & ~0x00000001); + primaryReplicaCount_ = 0L; + onChanged(); + return this; + } + + // required int64 secondaryReplicaCount = 2; + private long secondaryReplicaCount_ ; + /** + * required int64 secondaryReplicaCount = 2; + */ + public boolean hasSecondaryReplicaCount() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public long getSecondaryReplicaCount() { + return secondaryReplicaCount_; + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public Builder setSecondaryReplicaCount(long value) { + bitField0_ |= 0x00000002; + secondaryReplicaCount_ = value; + onChanged(); + return this; + } + /** + * required int64 secondaryReplicaCount = 2; + */ + public Builder clearSecondaryReplicaCount() { + bitField0_ = (bitField0_ & ~0x00000002); + secondaryReplicaCount_ = 0L; + onChanged(); + return this; + } + + // required int64 tertiaryReplicaCount = 3; + private long tertiaryReplicaCount_ ; + /** + * required int64 tertiaryReplicaCount = 3; + */ + public boolean hasTertiaryReplicaCount() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public long getTertiaryReplicaCount() { + return tertiaryReplicaCount_; + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public Builder setTertiaryReplicaCount(long value) { + bitField0_ |= 0x00000004; + tertiaryReplicaCount_ = value; + onChanged(); + return this; + } + /** + * required int64 tertiaryReplicaCount = 3; + */ + public Builder clearTertiaryReplicaCount() { + bitField0_ = (bitField0_ & ~0x00000004); + tertiaryReplicaCount_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicaLoad) + } + + static { + defaultInstance = new ReplicaLoad(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:hbase.pb.ReplicaLoad) + } + + public interface FavoredNodesInfoPairOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string regionName = 1; + /** + * required string regionName = 1; + */ + boolean hasRegionName(); + /** + * required string regionName = 1; + */ + java.lang.String getRegionName(); + /** + * required string regionName = 1; + */ + com.google.protobuf.ByteString + getRegionNameBytes(); + + // repeated .hbase.pb.ServerName servers = 2; + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + java.util.List + getServersList(); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServers(int index); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + int getServersCount(); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + java.util.List + getServersOrBuilderList(); + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index); + } + /** + * Protobuf type {@code hbase.pb.FavoredNodesInfoPair} + */ + public static final class FavoredNodesInfoPair extends + com.google.protobuf.GeneratedMessage + implements FavoredNodesInfoPairOrBuilder { + // Use FavoredNodesInfoPair.newBuilder() to construct. + private FavoredNodesInfoPair(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private FavoredNodesInfoPair(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final FavoredNodesInfoPair defaultInstance; + public static FavoredNodesInfoPair getDefaultInstance() { + return defaultInstance; + } + + public FavoredNodesInfoPair getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private FavoredNodesInfoPair( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + regionName_ = input.readBytes(); + break; + } + case 18: { + if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000002; + } + servers_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry)); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public FavoredNodesInfoPair parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new FavoredNodesInfoPair(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required string regionName = 1; + public static final int REGIONNAME_FIELD_NUMBER = 1; + private java.lang.Object regionName_; + /** + * required string regionName = 1; + */ + public boolean hasRegionName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string regionName = 1; + */ + public java.lang.String getRegionName() { + java.lang.Object ref = regionName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + regionName_ = s; + } + return s; + } + } + /** + * required string regionName = 1; + */ + public com.google.protobuf.ByteString + getRegionNameBytes() { + java.lang.Object ref = regionName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + regionName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // repeated .hbase.pb.ServerName servers = 2; + public static final int SERVERS_FIELD_NUMBER = 2; + private java.util.List servers_; + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List getServersList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List + getServersOrBuilderList() { + return servers_; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public int getServersCount() { + return servers_.size(); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + return servers_.get(index); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + return servers_.get(index); + } + + private void initFields() { + regionName_ = ""; + servers_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRegionName()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).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.writeBytes(1, getRegionNameBytes()); + } + for (int i = 0; i < servers_.size(); i++) { + output.writeMessage(2, servers_.get(i)); + } + 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 + .computeBytesSize(1, getRegionNameBytes()); + } + for (int i = 0; i < servers_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, servers_.get(i)); + } + 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.HBaseProtos.FavoredNodesInfoPair)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair) obj; + + boolean result = true; + result = result && (hasRegionName() == other.hasRegionName()); + if (hasRegionName()) { + result = result && getRegionName() + .equals(other.getRegionName()); + } + result = result && getServersList() + .equals(other.getServersList()); + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRegionName()) { + hash = (37 * hash) + REGIONNAME_FIELD_NUMBER; + hash = (53 * hash) + getRegionName().hashCode(); + } + if (getServersCount() > 0) { + hash = (37 * hash) + SERVERS_FIELD_NUMBER; + hash = (53 * hash) + getServersList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair 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; + } + /** + * Protobuf type {@code hbase.pb.FavoredNodesInfoPair} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPairOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getServersFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + regionName_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + } else { + serversBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair build() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.regionName_ = regionName_; + if (serversBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = java.util.Collections.unmodifiableList(servers_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.servers_ = servers_; + } else { + result.servers_ = serversBuilder_.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.HBaseProtos.FavoredNodesInfoPair) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair.getDefaultInstance()) return this; + if (other.hasRegionName()) { + bitField0_ |= 0x00000001; + regionName_ = other.regionName_; + onChanged(); + } + if (serversBuilder_ == null) { + if (!other.servers_.isEmpty()) { + if (servers_.isEmpty()) { + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureServersIsMutable(); + servers_.addAll(other.servers_); + } + onChanged(); + } + } else { + if (!other.servers_.isEmpty()) { + if (serversBuilder_.isEmpty()) { + serversBuilder_.dispose(); + serversBuilder_ = null; + servers_ = other.servers_; + bitField0_ = (bitField0_ & ~0x00000002); + serversBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getServersFieldBuilder() : null; + } else { + serversBuilder_.addAllMessages(other.servers_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasRegionName()) { + + return false; + } + for (int i = 0; i < getServersCount(); i++) { + if (!getServers(i).isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FavoredNodesInfoPair) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required string regionName = 1; + private java.lang.Object regionName_ = ""; + /** + * required string regionName = 1; + */ + public boolean hasRegionName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string regionName = 1; + */ + public java.lang.String getRegionName() { + java.lang.Object ref = regionName_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + regionName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string regionName = 1; + */ + public com.google.protobuf.ByteString + getRegionNameBytes() { + java.lang.Object ref = regionName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + regionName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string regionName = 1; + */ + public Builder setRegionName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + regionName_ = value; + onChanged(); + return this; + } + /** + * required string regionName = 1; + */ + public Builder clearRegionName() { + bitField0_ = (bitField0_ & ~0x00000001); + regionName_ = getDefaultInstance().getRegionName(); + onChanged(); + return this; + } + /** + * required string regionName = 1; + */ + public Builder setRegionNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + regionName_ = value; + onChanged(); + return this; + } + + // repeated .hbase.pb.ServerName servers = 2; + private java.util.List servers_ = + java.util.Collections.emptyList(); + private void ensureServersIsMutable() { + if (!((bitField0_ & 0x00000002) == 0x00000002)) { + servers_ = new java.util.ArrayList(servers_); + bitField0_ |= 0x00000002; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + 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> serversBuilder_; + + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List getServersList() { + if (serversBuilder_ == null) { + return java.util.Collections.unmodifiableList(servers_); + } else { + return serversBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public int getServersCount() { + if (serversBuilder_ == null) { + return servers_.size(); + } else { + return serversBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServers(int index) { + if (serversBuilder_ == null) { + return servers_.get(index); + } else { + return serversBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.set(index, value); + onChanged(); + } else { + serversBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder setServers( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.set(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(value); + onChanged(); + } else { + serversBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (serversBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureServersIsMutable(); + servers_.add(index, value); + onChanged(); + } else { + serversBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addServers( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.add(index, builderForValue.build()); + onChanged(); + } else { + serversBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder addAllServers( + java.lang.Iterable values) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + super.addAll(values, servers_); + onChanged(); + } else { + serversBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder clearServers() { + if (serversBuilder_ == null) { + servers_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + } else { + serversBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public Builder removeServers(int index) { + if (serversBuilder_ == null) { + ensureServersIsMutable(); + servers_.remove(index); + onChanged(); + } else { + serversBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServersBuilder( + int index) { + return getServersFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServersOrBuilder( + int index) { + if (serversBuilder_ == null) { + return servers_.get(index); } else { + return serversBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List + getServersOrBuilderList() { + if (serversBuilder_ != null) { + return serversBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(servers_); + } + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder() { + return getServersFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder addServersBuilder( + int index) { + return getServersFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()); + } + /** + * repeated .hbase.pb.ServerName servers = 2; + */ + public java.util.List + getServersBuilderList() { + return getServersFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + 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> + getServersFieldBuilder() { + if (serversBuilder_ == null) { + serversBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + 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>( + servers_, + ((bitField0_ & 0x00000002) == 0x00000002), + getParentForChildren(), + isClean()); + servers_ = null; + } + return serversBuilder_; + } + + // @@protoc_insertion_point(builder_scope:hbase.pb.FavoredNodesInfoPair) + } + + static { + defaultInstance = new FavoredNodesInfoPair(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:hbase.pb.FavoredNodesInfoPair) + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_TableName_descriptor; private static @@ -19653,6 +21966,21 @@ public final class HBaseProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_ServerReplicaLoadPair_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_ReplicaLoad_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_FavoredNodesInfoPair_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -19716,14 +22044,22 @@ public final class HBaseProtos { "rc_checksum\030\006 \002(\t\022\025\n\rversion_major\030\007 \001(\r" + "\022\025\n\rversion_minor\030\010 \001(\r\"Q\n\020RegionServerI" + "nfo\022\020\n\010infoPort\030\001 \001(\005\022+\n\014version_info\030\002 " + - "\001(\0132\025.hbase.pb.VersionInfo*r\n\013CompareTyp" + - "e\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020" + - "\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013" + - "\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NA" + - "NOSECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISE" + - "CONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOU", - "RS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache.hadoop.hbas" + - "e.protobuf.generatedB\013HBaseProtosH\001\240\001\001" + "\001(\0132\025.hbase.pb.VersionInfo\"j\n\025ServerRepl" + + "icaLoadPair\022$\n\006server\030\001 \002(\0132\024.hbase.pb.S" + + "erverName\022+\n\014replicaCount\030\002 \002(\0132\025.hbase." + + "pb.ReplicaLoad\"g\n\013ReplicaLoad\022\033\n\023primary" + + "ReplicaCount\030\001 \002(\003\022\035\n\025secondaryReplicaCo" + + "unt\030\002 \002(\003\022\034\n\024tertiaryReplicaCount\030\003 \002(\003\"", + "Q\n\024FavoredNodesInfoPair\022\022\n\nregionName\030\001 " + + "\002(\t\022%\n\007servers\030\002 \003(\0132\024.hbase.pb.ServerNa" + + "me*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_E" + + "QUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREA" + + "TER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n" + + "\n\010TimeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECO" + + "NDS\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007" + + "MINUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.ap" + + "ache.hadoop.hbase.protobuf.generatedB\013HB" + + "aseProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -19880,6 +22216,24 @@ public final class HBaseProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_RegionServerInfo_descriptor, new java.lang.String[] { "InfoPort", "VersionInfo", }); + internal_static_hbase_pb_ServerReplicaLoadPair_descriptor = + getDescriptor().getMessageTypes().get(25); + internal_static_hbase_pb_ServerReplicaLoadPair_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_hbase_pb_ServerReplicaLoadPair_descriptor, + new java.lang.String[] { "Server", "ReplicaCount", }); + internal_static_hbase_pb_ReplicaLoad_descriptor = + getDescriptor().getMessageTypes().get(26); + internal_static_hbase_pb_ReplicaLoad_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_hbase_pb_ReplicaLoad_descriptor, + new java.lang.String[] { "PrimaryReplicaCount", "SecondaryReplicaCount", "TertiaryReplicaCount", }); + internal_static_hbase_pb_FavoredNodesInfoPair_descriptor = + getDescriptor().getMessageTypes().get(27); + internal_static_hbase_pb_FavoredNodesInfoPair_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_hbase_pb_FavoredNodesInfoPair_descriptor, + new java.lang.String[] { "RegionName", "Servers", }); return null; } }; diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto index 7d064f3..f872fa0 100644 --- a/hbase-protocol/src/main/protobuf/HBase.proto +++ b/hbase-protocol/src/main/protobuf/HBase.proto @@ -250,3 +250,19 @@ message RegionServerInfo { optional int32 infoPort = 1; optional VersionInfo version_info = 2; } + +message ServerReplicaLoadPair { + required ServerName server = 1; + required ReplicaLoad replicaCount = 2; +} + +message ReplicaLoad { + required int64 primaryReplicaCount = 1; + required int64 secondaryReplicaCount = 2; + required int64 tertiaryReplicaCount = 3; +} + +message FavoredNodesInfoPair { + required string regionName = 1; + repeated ServerName servers = 2; +} diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml index 58802cf..e056e6b 100644 --- a/hbase-rsgroup/pom.xml +++ b/hbase-rsgroup/pom.xml @@ -200,7 +200,7 @@ log4j log4j - test + ${log4j.version} io.dropwizard.metrics diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/StartcodeAgnosticServerName.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/StartcodeAgnosticServerName.java new file mode 100644 index 0000000..81920e2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/StartcodeAgnosticServerName.java @@ -0,0 +1,67 @@ +/** + * + * 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 com.google.common.net.HostAndPort; +import org.apache.hadoop.hbase.util.Addressing; + +public class StartcodeAgnosticServerName extends ServerName { + + public StartcodeAgnosticServerName(final String hostname, final int port, long startcode) { + super(hostname, port, startcode); + } + + public static StartcodeAgnosticServerName valueOf(final ServerName serverName) { + return new StartcodeAgnosticServerName(serverName.getHostname(), serverName.getPort(), + serverName.getStartcode()); + } + + public static StartcodeAgnosticServerName valueOf(final String hostnameAndPort, long startcode) { + return new StartcodeAgnosticServerName(Addressing.parseHostname(hostnameAndPort), + Addressing.parsePort(hostnameAndPort), startcode); + } + + public static StartcodeAgnosticServerName valueOf(final HostAndPort hostnameAndPort, long startcode) { + return new StartcodeAgnosticServerName(hostnameAndPort.getHostText(), + hostnameAndPort.getPort(), startcode); + } + + @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 0; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null) return false; + if (!(o instanceof StartcodeAgnosticServerName)) return false; + return this.compareTo((StartcodeAgnosticServerName)o) == 0; + } + + @Override + public int hashCode() { + return getHostAndPort().hashCode(); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java new file mode 100644 index 0000000..3b40a91 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesManager.java @@ -0,0 +1,299 @@ +/** + * + * 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.favored; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.master.RackManager; +import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.master.RegionStates; +import org.apache.hadoop.hbase.master.SnapshotOfRegionAssignmentFromMeta; +import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; +import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +public class FavoredNodesManager { + private static final Log LOG = LogFactory.getLog(FavoredNodesManager.class); + + private FavoredNodesPlan globalFavoredNodesAssignmentPlan; + private RackManager rackManager; + private Configuration conf; + private Map> primaryRSToRegionMap; + private Map> secondaryRSToRegionMap; + private Map> teritiaryRSToRegionMap; + + private MasterServices masterServices; + + public FavoredNodesManager(MasterServices masterServices) { + this.masterServices = masterServices; + conf = masterServices.getConfiguration(); + globalFavoredNodesAssignmentPlan = new FavoredNodesPlan(); + rackManager = new RackManager(conf); + primaryRSToRegionMap = new HashMap>(); + secondaryRSToRegionMap = new HashMap>(); + teritiaryRSToRegionMap = new HashMap>(); + } + + public void initialize() throws HBaseIOException { + SnapshotOfRegionAssignmentFromMeta snaphotOfRegionAssignment = + new SnapshotOfRegionAssignmentFromMeta(masterServices.getConnection()); + try { + snaphotOfRegionAssignment.initialize(); + } catch (IOException e) { + throw new HBaseIOException(e); + } + globalFavoredNodesAssignmentPlan = snaphotOfRegionAssignment.getExistingAssignmentPlan(); + primaryRSToRegionMap = snaphotOfRegionAssignment.getPrimaryToRegionInfoMap(); + secondaryRSToRegionMap = snaphotOfRegionAssignment.getSecondaryToRegionInfoMap(); + teritiaryRSToRegionMap = snaphotOfRegionAssignment.getTertiaryToRegionInfoMap(); + } + + public List getFavoredNodes(HRegionInfo regionInfo) { + return this.globalFavoredNodesAssignmentPlan.getFavoredNodes(regionInfo); + } + + public void generateFavoredNodes(FavoredNodeAssignmentHelper assignmentHelper, + Map> assignmentMap, List regions, + List servers) throws IOException { + if (regions.size() > 0) { + if (assignmentHelper.canPlaceFavoredNodes()) { + Map primaryRSMap = new HashMap(); + assignmentHelper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions); + Map> generatedFavNodes = + assignmentHelper.generateFavoredNodes(primaryRSMap); + for (HRegionInfo hri : generatedFavNodes.keySet()) { + updateFavoredNodes(hri, generatedFavNodes.get(hri)); + } + } else { + throw new HBaseIOException(" Not enough nodes to do RR assignment"); + } + } + } + + public synchronized void updateFavoredNodes( + HRegionInfo regionInfo, List servers) throws IOException { + if (servers.size() != Sets.newHashSet(servers).size()) { + throw new IOException("Duplicates found: "+servers); + } + if (servers.size() == FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) { + Map> regionToFavoredNodes = new HashMap>(); + List serversWithNoStartCodes = Lists.newArrayList(); + for (ServerName sn : servers) { + if (sn.getStartcode() == ServerName.NON_STARTCODE) { + serversWithNoStartCodes.add(sn); + } else { + serversWithNoStartCodes.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), + ServerName.NON_STARTCODE)); + } + } + regionToFavoredNodes.put(regionInfo, serversWithNoStartCodes); + FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo( + regionToFavoredNodes, + masterServices.getConnection()); + if (getFavoredNodes(regionInfo) != null) { + deleteFavoredNodesForRegion(Lists.newArrayList(regionInfo)); + } + globalFavoredNodesAssignmentPlan.updateFavoredNodesMap(regionInfo, serversWithNoStartCodes); + addToReplicaLoad(regionInfo, serversWithNoStartCodes); + } else { + throw new IOException("At least " + FavoredNodeAssignmentHelper.FAVORED_NODES_NUM + + " favored nodes should be present for region : " + regionInfo.getEncodedName() + + " current FN servers:" + servers); + } + } + + private synchronized void addToReplicaLoad(HRegionInfo hri, List servers) { + ServerName serverToUse = ServerName.valueOf(servers.get(0).getHostAndPort(), + ServerName.NON_STARTCODE); + List regionList = primaryRSToRegionMap.get(serverToUse); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(hri); + primaryRSToRegionMap.put(serverToUse, regionList); + + serverToUse = ServerName + .valueOf(servers.get(1).getHostAndPort(), ServerName.NON_STARTCODE); + regionList = secondaryRSToRegionMap.get(serverToUse); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(hri); + secondaryRSToRegionMap.put(serverToUse, regionList); + + serverToUse = ServerName.valueOf(servers.get(2).getHostAndPort(), ServerName.NON_STARTCODE); + regionList = teritiaryRSToRegionMap.get(serverToUse); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(hri); + teritiaryRSToRegionMap.put(serverToUse, regionList); + } + + public synchronized void deleteFavoredNodesForRegion(List regionInfoList) { + for (HRegionInfo hri : regionInfoList) { + List favNodes = getFavoredNodes(hri); + if (favNodes != null) { + if (primaryRSToRegionMap.containsKey(favNodes.get(0))) { + primaryRSToRegionMap.get(favNodes.get(0)).remove(hri); + } + if (secondaryRSToRegionMap.containsKey(favNodes.get(1))) { + secondaryRSToRegionMap.get(favNodes.get(1)).remove(hri); + } + if (teritiaryRSToRegionMap.containsKey(favNodes.get(2))) { + teritiaryRSToRegionMap.get(favNodes.get(2)).remove(hri); + } + globalFavoredNodesAssignmentPlan.removeFavoredNodes(hri); + } + } + } + + public synchronized Map> getReplicaLoad(List servers) { + Map> result = new HashMap>(); + for (ServerName sn : servers) { + ServerName serverWithNoStartCode = ServerName.valueOf(sn.getHostAndPort(), + ServerName.NON_STARTCODE); + List countList = Lists.newArrayList(); + if (primaryRSToRegionMap.containsKey(serverWithNoStartCode)) { + countList.add(primaryRSToRegionMap.get(serverWithNoStartCode).size()); + } else { + countList.add(0); + } + if (secondaryRSToRegionMap.containsKey(serverWithNoStartCode)) { + countList.add(secondaryRSToRegionMap.get(serverWithNoStartCode).size()); + } else { + countList.add(0); + } + if (teritiaryRSToRegionMap.containsKey(serverWithNoStartCode)) { + countList.add(teritiaryRSToRegionMap.get(serverWithNoStartCode).size()); + } else { + countList.add(0); + } + result.put(sn, countList); + } + return result; + } + + public synchronized void removeFavoredNode( + ServerName decommissionedServer, List servers) throws IOException { + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, this.conf); + helper.initialize(); + Set regions = Sets.newHashSet(); + ServerName sn = ServerName.valueOf(decommissionedServer.getHostAndPort(), + ServerName.NON_STARTCODE); + if (primaryRSToRegionMap.containsKey(sn)) { + regions.addAll(primaryRSToRegionMap.get(sn)); + } + if (secondaryRSToRegionMap.containsKey(sn)) { + regions.addAll(secondaryRSToRegionMap.get(sn)); + } + if (teritiaryRSToRegionMap.containsKey(sn)) { + regions.addAll(teritiaryRSToRegionMap.get(sn)); + } + + Iterator itr = regions.iterator(); + RegionStates regionStates = masterServices.getAssignmentManager().getRegionStates(); + while (itr.hasNext()) { + HRegionInfo hri = itr.next(); + if (regionStates.isRegionOffline(hri) + || regionStates.isRegionInState(hri, RegionState.State.SPLIT, RegionState.State.MERGED, + RegionState.State.MERGING_NEW, RegionState.State.SPLITTING_NEW)) { + itr.remove(); + } + } + Map> newFavoredNodes = + fixFavoredNodes(helper, decommissionedServer, regions); + for (HRegionInfo hri : newFavoredNodes.keySet()) { + updateFavoredNodes(hri, newFavoredNodes.get(hri)); + } + updateFavoredNodesInRegionServer(newFavoredNodes); + primaryRSToRegionMap.remove(sn); + secondaryRSToRegionMap.remove(sn); + teritiaryRSToRegionMap.remove(sn); + } + + public void updateFavoredNodesInRegionServer(Map> favoredNodes) + throws IOException { + Map> regionsGroupedByServer = new HashMap>(); + for (HRegionInfo hri : favoredNodes.keySet()) { + ServerName sn = + masterServices.getAssignmentManager().getRegionStates().getRegionServerOfRegion(hri); + if (sn != null) { + List regionsOfServer = regionsGroupedByServer.get(sn); + if (regionsOfServer == null) { + regionsOfServer = Lists.newArrayList(); + } + regionsOfServer.add(hri); + regionsGroupedByServer.put(sn, regionsOfServer); + } else { + LOG.warn("Server could not be found for region = " + hri.getRegionNameAsString()); + } + } + Map> favNodesByServer = new HashMap>(); + for (Entry> entry : regionsGroupedByServer.entrySet()) { + for (HRegionInfo hri : entry.getValue()) { + favNodesByServer.put(hri, favoredNodes.get(hri)); + } + masterServices.getServerManager().sendFavoredNodes(entry.getKey(), favNodesByServer); + favNodesByServer.clear(); + } + } + + private Map> fixFavoredNodes(FavoredNodeAssignmentHelper helper, + ServerName decommissionedServer, Set regions) throws IOException { + Map> onlineFavoredNodes = new HashMap>(); + for (HRegionInfo hri : regions) { + if (!hri.getTable().isSystemTable()) { + Set favNodeWithoutStartCode = Sets.newHashSet(getFavoredNodes(hri)); + favNodeWithoutStartCode.remove(ServerName.valueOf(decommissionedServer.getHostAndPort(), + ServerName.NON_STARTCODE)); + while (favNodeWithoutStartCode.size() < FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) { + ServerName sn = helper.generateMissingFavoredNode(Lists + .newArrayList(favNodeWithoutStartCode)); + favNodeWithoutStartCode.add(ServerName.valueOf(sn.getHostAndPort(), + ServerName.NON_STARTCODE)); + } + LOG.debug("Generated one missing favored nodes for " + hri.getEncodedName() + " : " + + favNodeWithoutStartCode); + onlineFavoredNodes.put(hri, Lists.newArrayList(favNodeWithoutStartCode)); + } + } + return onlineFavoredNodes; + } + + public RackManager getRackManager() { + return rackManager; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 2715aed..6bd2bff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -74,8 +74,7 @@ import org.apache.hadoop.hbase.ipc.FailedServerException; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.master.RegionState.State; -import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; -import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer; +import org.apache.hadoop.hbase.master.balancer.FavoredNodesPromoter; import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; @@ -230,10 +229,6 @@ public class AssignmentManager { this.regionsToReopen = Collections.synchronizedMap (new HashMap ()); Configuration conf = server.getConfiguration(); - // Only read favored nodes if using the favored nodes load balancer. - this.shouldAssignRegionsWithFavoredNodes = conf.getClass( - HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals( - FavoredNodeLoadBalancer.class); this.tableStateManager = tableStateManager; @@ -243,6 +238,8 @@ public class AssignmentManager { this.sleepTimeBeforeRetryingMetaAssignment = this.server.getConfiguration().getLong( "hbase.meta.assignment.retry.sleeptime", 1000l); this.balancer = balancer; + // Only read favored nodes if using the favored nodes load balancer. + this.shouldAssignRegionsWithFavoredNodes = this.balancer instanceof FavoredNodesPromoter; int maxThreads = conf.getInt("hbase.assignment.threads.max", 30); this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool( @@ -630,23 +627,17 @@ public class AssignmentManager { } } - // TODO: processFavoredNodes might throw an exception, for e.g., if the - // meta could not be contacted/updated. We need to see how seriously to treat - // this problem as. Should we fail the current assignment. We should be able - // to recover from this problem eventually (if the meta couldn't be updated - // things should work normally and eventually get fixed up). - void processFavoredNodes(List regions) throws IOException { - if (!shouldAssignRegionsWithFavoredNodes) return; - // The AM gets the favored nodes info for each region and updates the meta - // table with that info - Map> regionToFavoredNodes = - new HashMap>(); - for (HRegionInfo region : regions) { - regionToFavoredNodes.put(region, - ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region)); + Map> processFavoredNodesForDaughters(HRegionInfo parent, + HRegionInfo regionA, HRegionInfo regionB) throws IOException { + return ((FavoredNodesPromoter) this.balancer).generateFavoredNodesForDaughter( + this.serverManager.getOnlineServersList(), parent, regionA, regionB); + } + + void processFavoredNodesForMerge(HRegionInfo merged, HRegionInfo a, HRegionInfo b) + throws IOException { + if (shouldAssignRegionsWithFavoredNodes) { + ((FavoredNodesPromoter) this.balancer).generateFavoredNodesForMergedRegion(merged, a, b); } - FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(regionToFavoredNodes, - this.server.getConnection()); } /** @@ -806,8 +797,8 @@ public class AssignmentManager { regionStates.updateRegionState( region, State.PENDING_OPEN, destination); List favoredNodes = ServerName.EMPTY_SERVER_LIST; - if (this.shouldAssignRegionsWithFavoredNodes) { - favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region); + if (!region.isSystemTable() && this.shouldAssignRegionsWithFavoredNodes) { + favoredNodes = server.getFavoredNodesManager().getFavoredNodes(region); } regionOpenInfos.add(new Pair>( region, favoredNodes)); @@ -1114,8 +1105,10 @@ public class AssignmentManager { " to " + plan.getDestination(); try { List favoredNodes = ServerName.EMPTY_SERVER_LIST; - if (this.shouldAssignRegionsWithFavoredNodes) { - favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region); + if (!region.isSystemTable() && shouldAssignRegionsWithFavoredNodes) { + List regions = new ArrayList(1); + regions.add(region); + favoredNodes = server.getFavoredNodesManager().getFavoredNodes(region); } serverManager.sendRegionOpen(plan.getDestination(), region, favoredNodes); return; // we're done @@ -1300,15 +1293,6 @@ public class AssignmentManager { LOG.warn("Failed to create new plan.",ex); return null; } - if (!region.isMetaTable() && shouldAssignRegionsWithFavoredNodes) { - List regions = new ArrayList(1); - regions.add(region); - try { - processFavoredNodes(regions); - } catch (IOException ie) { - LOG.warn("Ignoring exception in processFavoredNodes " + ie); - } - } this.regionPlans.put(encodedName, randomPlan); } } @@ -1580,7 +1564,6 @@ public class AssignmentManager { processBogusAssignments(bulkPlan); - processFavoredNodes(regions); assign(regions.size(), servers.size(), "round-robin=true", bulkPlan); } @@ -1869,8 +1852,8 @@ public class AssignmentManager { return; // Region is not in the expected state any more } List favoredNodes = ServerName.EMPTY_SERVER_LIST; - if (shouldAssignRegionsWithFavoredNodes) { - favoredNodes = ((FavoredNodeLoadBalancer)balancer).getFavoredNodes(hri); + if (!hri.isSystemTable() && shouldAssignRegionsWithFavoredNodes) { + favoredNodes = ((MasterServices)server).getFavoredNodesManager().getFavoredNodes(hri); } serverManager.sendRegionOpen(serverName, hri, favoredNodes); return; // we're done @@ -2188,16 +2171,23 @@ public class AssignmentManager { String encodedName = hri.getEncodedName(); ReentrantLock lock = locker.acquireLock(encodedName); try { - if (!regionStates.isRegionOnline(hri)) { - RegionState state = regionStates.getRegionState(encodedName); - LOG.info("Ignored moving region not assigned: " + hri + ", " - + (state == null ? "not in region states" : state)); - return; - } - synchronized (this.regionPlans) { - this.regionPlans.put(plan.getRegionName(), plan); + if (LoadBalancer.BOGUS_SERVER_NAME.equals(plan.getDestination())) { + this.unassign(plan.getRegionInfo()); + + } else { + + if (!regionStates.isRegionOnline(hri)) { + RegionState state = regionStates.getRegionState(encodedName); + LOG.info("Ignored moving region not assigned: " + hri + ", " + (state == null ? + "not in region states" : + state)); + return; + } + synchronized (this.regionPlans) { + this.regionPlans.put(plan.getRegionName(), plan); + } + unassign(hri, plan.getDestination()); } - unassign(hri, plan.getDestination()); } finally { lock.unlock(); } @@ -2398,6 +2388,7 @@ public class AssignmentManager { return hri.getShortNameToLog() + " is not splitting on " + serverName; } + Map> favoredNodes = null; final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1)); final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2)); RegionState rs_a = regionStates.getRegionState(a); @@ -2425,6 +2416,10 @@ public class AssignmentManager { try { regionStates.splitRegion(hri, a, b, serverName); + if (!hri.getTable().isSystemTable() && shouldAssignRegionsWithFavoredNodes) { + favoredNodes = processFavoredNodesForDaughters(hri, a ,b); + this.serverManager.sendFavoredNodes(serverName, favoredNodes); + } } catch (IOException ioe) { LOG.info("Failed to record split region " + hri.getShortNameToLog()); return "Failed to record the splitting in meta"; @@ -2627,6 +2622,15 @@ public class AssignmentManager { regionOffline(b, State.MERGED); regionOnline(hri, serverName, 1); + try { + if (!a.getTable().isSystemTable() && shouldAssignRegionsWithFavoredNodes) { + processFavoredNodesForMerge(hri, a, b); + } + } catch (IOException e) { + LOG.warn("Error while processing favored nodes after merge.", e); + return StringUtils.stringifyException(e); + } + // User could disable the table before master knows the new region. if (getTableStateManager().isTableState(hri.getTable(), TableState.State.DISABLED, TableState.State.DISABLING)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java index e748c3b..227d0bf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -50,6 +51,8 @@ import org.apache.hadoop.hbase.util.PairOfSameType; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Triple; +import com.google.common.collect.Lists; + /** * A janitor for the catalog tables. Scans the hbase:meta catalog * table on a period looking for unused regions to garbage collect. @@ -213,6 +216,10 @@ public class CatalogJanitor extends ScheduledChore { HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionA); HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionB); MetaTableAccessor.deleteMergeQualifiers(services.getConnection(), mergedRegion); + FavoredNodesManager fnm = this.services.getFavoredNodesManager(); + if (fnm != null) { + fnm.deleteFavoredNodesForRegion(Lists.newArrayList(regionA, regionB)); + } services.getServerManager().removeRegion(regionA); services.getServerManager().removeRegion(regionB); return true; @@ -359,6 +366,10 @@ public class CatalogJanitor extends ScheduledChore { if (LOG.isTraceEnabled()) LOG.trace("Archiving parent region: " + parent); HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent); MetaTableAccessor.deleteRegion(this.connection, parent); + FavoredNodesManager fnm = this.services.getFavoredNodesManager(); + if (fnm != null) { + fnm.deleteFavoredNodesForRegion(Lists.newArrayList(parent)); + } services.getServerManager().removeRegion(parent); result = true; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index a8329e3..27447ec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.MergeRegionException; import org.apache.hadoop.hbase.executor.ExecutorType; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; @@ -90,6 +91,8 @@ import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode; import org.apache.hadoop.hbase.master.balancer.BalancerChore; import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer; import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore; +import org.apache.hadoop.hbase.master.balancer.FavoredNodesPromoter; +import org.apache.hadoop.hbase.master.balancer.FavoredNodesRepairChore; import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.LogCleaner; @@ -279,7 +282,7 @@ public class HMaster extends HRegionServer implements MasterServices { private volatile ServerManager serverManager; // manager of assignment nodes in zookeeper - private AssignmentManager assignmentManager; + protected AssignmentManager assignmentManager; // buffer for "fatal error" notices from region servers // in the cluster. This is only used for assisting @@ -308,6 +311,7 @@ public class HMaster extends HRegionServer implements MasterServices { private ClusterStatusChore clusterStatusChore; private ClusterStatusPublisher clusterStatusPublisherChore = null; private PeriodicDoMetrics periodicDoMetricsChore = null; + private FavoredNodesRepairChore favoredNodesRepairChore = null; CatalogJanitor catalogJanitorChore; private ReplicationMetaCleaner replicationMetaCleaner; @@ -507,6 +511,10 @@ public class HMaster extends HRegionServer implements MasterServices { return super.getFsTableDescriptors(); } + private boolean isFavoredNodesPromoter; + + private FavoredNodesManager favoredNodesManager; + /** * For compatibility, if failed with regionserver credentials, try the master one */ @@ -736,6 +744,10 @@ public class HMaster extends HRegionServer implements MasterServices { this.initializationBeforeMetaAssignment = true; + if (this.balancer instanceof FavoredNodesPromoter) { + isFavoredNodesPromoter = true; + favoredNodesManager = new FavoredNodesManager(this); + } // Wait for regionserver to finish initialization. if (BaseLoadBalancer.tablesOnMaster(conf)) { waitForServerOnline(); @@ -745,7 +757,6 @@ public class HMaster extends HRegionServer implements MasterServices { this.balancer.setClusterStatus(getClusterStatus()); this.balancer.setMasterServices(this); this.balancer.initialize(); - // Check if master is shutting down because of some issue // in initializing the regionserver or the balancer. if (isStopped()) return; @@ -758,6 +769,11 @@ public class HMaster extends HRegionServer implements MasterServices { // assigned when master is shutting down if (isStopped()) return; + //Initialize after meta as it scans meta + if (isFavoredNodesPromoter) { + favoredNodesManager.initialize(); + } + // migrating existent table state from zk, so splitters // and recovery process treat states properly. for (Map.Entry entry : ZKDataMigrator @@ -787,6 +803,8 @@ public class HMaster extends HRegionServer implements MasterServices { getChoreService().scheduleChore(normalizerChore); this.catalogJanitorChore = new CatalogJanitor(this); getChoreService().scheduleChore(catalogJanitorChore); + this.favoredNodesRepairChore = new FavoredNodesRepairChore(this); + getChoreService().scheduleChore(favoredNodesRepairChore); // Do Metrics periodically periodicDoMetricsChore = new PeriodicDoMetrics(msgInterval, this); @@ -1087,6 +1105,9 @@ public class HMaster extends HRegionServer implements MasterServices { if (this.periodicDoMetricsChore != null) { periodicDoMetricsChore.cancel(); } + if (this.favoredNodesRepairChore != null) { + this.favoredNodesRepairChore.cancel(); + } } /** @@ -1195,15 +1216,15 @@ public class HMaster extends HRegionServer implements MasterServices { long balStartTime = System.currentTimeMillis(); //TODO: bulk assign this.assignmentManager.balance(plan); - totalRegPlanExecTime += System.currentTimeMillis()-balStartTime; + totalRegPlanExecTime += System.currentTimeMillis() - balStartTime; rpCount++; if (rpCount < plans.size() && // if performing next balance exceeds cutoff time, exit the loop (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) { //TODO: After balance, there should not be a cutoff time (keeping it as // a security net for now) - LOG.debug("No more balancing till next balance run; maximumBalanceTime=" + - maximumBalanceTime); + LOG.debug("No more balancing till next balance run; maximumBalanceTime=" + + maximumBalanceTime); break; } } @@ -2860,4 +2881,8 @@ public class HMaster extends HRegionServer implements MasterServices { public LoadBalancer getLoadBalancer() { return balancer; } + + @Override public FavoredNodesManager getFavoredNodesManager() { + return favoredNodesManager; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 7cc58aa..a5e46d5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -19,17 +19,22 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; +import java.io.InterruptedIOException; import java.net.InetAddress; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; +import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; @@ -50,6 +55,7 @@ import org.apache.hadoop.hbase.ipc.PriorityFunction; import org.apache.hadoop.hbase.ipc.QosPriority; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.master.balancer.FavoredNodesPromoter; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.procedure.MasterProcedureManager; import org.apache.hadoop.hbase.procedure2.Procedure; @@ -60,6 +66,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionIn import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; @@ -67,6 +75,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ReplicaLoad; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerReplicaLoadPair; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*; @@ -94,10 +104,13 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ForeignExceptionUtil; import org.apache.hadoop.hbase.util.Pair; import org.apache.zookeeper.KeeperException; +import com.google.common.collect.Lists; + /** * Implements the master RPC services. */ @@ -1586,4 +1599,227 @@ public class MasterRpcServices extends RSRpcServices } return null; } + + @Override + public UpdateFavoredNodesResponse updateFavoredNodesForRegion(RpcController controller, + UpdateFavoredNodesRequest request) throws ServiceException { + try { + for (UpdateFavoredNodesRequest.RegionUpdateInfo updateInfo : request.getUpdateInfoList()) { + List sNames = new ArrayList(); + for (HBaseProtos.ServerName sn : updateInfo.getFavoredNodesList()) { + sNames.add(ProtobufUtil.toServerName(sn)); + } + master.getFavoredNodesManager().updateFavoredNodes( + HRegionInfo.convert(updateInfo.getRegion()), sNames); + } + return UpdateFavoredNodesResponse.newBuilder().setResponse(request.getUpdateInfoCount()) + .build(); + } catch (IOException exp) { + throw new ServiceException(exp); + } + } + + @Override + public RedistributeFavoredNodesResponse redistributeFavoredNodes(RpcController controller, + RedistributeFavoredNodesRequest request) throws ServiceException { + try { + return RedistributeFavoredNodesResponse.newBuilder() + .setResult(redistributeFavoredNodes()).build(); + } catch (HBaseIOException ex) { + throw new ServiceException(ex); + } + } + + @Override + public CompleteRedistributeFavoredNodesResponse completeRedistributeFavoredNodes( + RpcController controller, CompleteRedistributeFavoredNodesRequest request) + throws ServiceException { + try { + return CompleteRedistributeFavoredNodesResponse.newBuilder() + .setResult(completeRedistributeFavoredNodes()).build(); + } catch (HBaseIOException ex) { + throw new ServiceException(ex); + } catch (InterruptedIOException e) { + throw new ServiceException(e); + } + } + + public boolean redistributeFavoredNodes() throws HBaseIOException { + if (!master.isInitialized()) { + throw new HBaseIOException("Master has not been initialized, cannot run redistribute."); + } + if (master.getFavoredNodesManager() == null) { + LOG.debug("FavoredNodes not enabled, skipping"); + return false; + } + if (master.assignmentManager.getRegionStates().isRegionsInTransition()) { + Set regionsInTransition = + master.assignmentManager.getRegionStates().getRegionsInTransition(); + LOG.debug("Not running redistribute because " + regionsInTransition.size() + + " region(s) in transition: " + org.apache.commons.lang.StringUtils. + abbreviate(regionsInTransition.toString(), 256)); + return false; + } + if (master.getServerManager().areDeadServersInProgress()) { + LOG.debug("Not running redistribute because processing dead regionserver(s): " + + master.getServerManager().getDeadServers()); + return false; + } + long startTime = System.currentTimeMillis(); + synchronized (master.getLoadBalancer()) { + Map>> assignmentsByTable = master.assignmentManager + .getRegionStates().getAssignmentsByTable(); + Map> updatedFavoredNodes = new HashMap>(); + try { + for (Map> assignments : assignmentsByTable.values()) { + Map> newFavNodes = ((FavoredNodesPromoter) master.getLoadBalancer()) + .redistribute(assignments); + updatedFavoredNodes.putAll(newFavNodes); + } + for (HRegionInfo hri : updatedFavoredNodes.keySet()) { + master.getFavoredNodesManager().updateFavoredNodes(hri, updatedFavoredNodes.get(hri)); + } + master.getFavoredNodesManager().updateFavoredNodesInRegionServer(updatedFavoredNodes); + } catch (IOException exp) { + LOG.error("Error while redistributing favored nodes.", exp); + return false; + } + } + long duration = System.currentTimeMillis() - startTime; + LOG.info("Redistribute took " + duration + " millisecs."); + return true; + } + + public boolean completeRedistributeFavoredNodes() throws HBaseIOException, + InterruptedIOException { + if (!master.isInitialized()) { + throw new HBaseIOException( + "Master has not been initialized, cannot run completeRedistribute."); + } + if (master.getFavoredNodesManager() == null) { + LOG.debug("FavoredNodes not enabled, skipping"); + return false; + } + long startTime = EnvironmentEdgeManager.currentTime(); + synchronized (master.getLoadBalancer()) { + if (master.getServerManager().areDeadServersInProgress()) { + LOG.debug("Not running completeRedistribute because processing dead regionserver(s): " + + master.getServerManager().getDeadServers()); + return false; + } + Map>> assignmentsByTable = master.assignmentManager + .getRegionStates().getAssignmentsByTable(); + master.getLoadBalancer().setClusterStatus(master.getClusterStatus()); + List regionPlans = Lists.newArrayList(); + try { + for (Map> assignments : assignmentsByTable.values()) { + List partialResult = ((FavoredNodesPromoter) master.getLoadBalancer()) + .completeRedistribute(assignments); + if (partialResult != null) { + regionPlans.addAll(partialResult); + } + } + if (!regionPlans.isEmpty()) { + for (RegionPlan plan: regionPlans) { + master.assignmentManager.balance(plan); + } + } + } catch (IOException exp) { + LOG.error("Error while completeRedistribute favored nodes.", exp); + return false; + } + } + + LOG.info("Complete redistribute took " + (EnvironmentEdgeManager.currentTime() - startTime) + " millisecs."); + return true; + } + + @Override + public GetFavoredNodesForRegionResponse getFavoredNodesForRegion(RpcController controller, + GetFavoredNodesForRegionRequest request) throws ServiceException { + GetFavoredNodesForRegionResponse.Builder response = + GetFavoredNodesForRegionResponse.newBuilder(); + if (master.getFavoredNodesManager() != null) { + List favoredNodes = + master.getFavoredNodesManager().getFavoredNodes(HRegionInfo.convert(request + .getRegionInfo())); + if (favoredNodes != null) { + for (ServerName sn : favoredNodes) { + response.addServers(ProtobufUtil.toServerName(sn)); + } + return response.build(); + } + } else { + throw new ServiceException("FavoredNodes not enabled"); + } + return response.build(); + } + + @Override + public GetReplicaLoadResponse getReplicaLoad(RpcController controller, + GetReplicaLoadRequest request) throws ServiceException { + GetReplicaLoadResponse.Builder response = GetReplicaLoadResponse.newBuilder(); + if (master.getFavoredNodesManager() != null) { + List servers = Lists.newArrayList(); + for (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName sn : request + .getServersList()) { + servers.add(ProtobufUtil.toServerName(sn)); + } + Map> replicaLoad = getReplicaLoad(servers); + for (Map.Entry> entry : replicaLoad.entrySet()) { + HBaseProtos.ServerReplicaLoadPair.Builder pair = ServerReplicaLoadPair.newBuilder(); + pair.setServer(ProtobufUtil.toServerName(entry.getKey())); + ReplicaLoad.Builder rl = ReplicaLoad.newBuilder(); + rl.setPrimaryReplicaCount(entry.getValue().get(0)); + rl.setSecondaryReplicaCount(entry.getValue().get(1)); + rl.setTertiaryReplicaCount(entry.getValue().get(2)); + pair.setReplicaCount(rl.build()); + response.addReplicaLoad(pair.build()); + } + } + return response.build(); + } + + Map> getReplicaLoad(List servers) { + if (master.getFavoredNodesManager() != null) { + return master.getFavoredNodesManager().getReplicaLoad(servers); + } else { + return new HashMap>(); + } + } + + @Override + public RemoveFavoredNodeResponse removeFavoredNode(RpcController controller, + RemoveFavoredNodeRequest request) throws ServiceException { + RemoveFavoredNodeResponse.Builder response = RemoveFavoredNodeResponse.newBuilder(); + if (master.getFavoredNodesManager() != null) { + ServerName sn = ProtobufUtil.toServerName(request.getServer()); + try { + master.getFavoredNodesManager().removeFavoredNode(sn, + master.getServerManager().getOnlineServersList()); + } catch (IOException e) { + LOG.warn("Exception while removing favored node.", e); + throw new ServiceException(e); + } + } + return response.build(); + } + + @Override + public CheckFavoredNodesResponse checkFavoredNodes(RpcController controller, + CheckFavoredNodesRequest request) throws ServiceException { + CheckFavoredNodesResponse.Builder response = CheckFavoredNodesResponse.newBuilder(); + if (master.getFavoredNodesManager() != null) { + Map> deadServers = ((FavoredNodesPromoter) master.getLoadBalancer()) + .checkFavoredNodes(master.getServerManager().getOnlineServersList(), + Lists.newArrayList(master.assignmentManager.getRegionStates().getRegionAssignments() + .keySet())); + for (Map.Entry> entry : deadServers.entrySet()) { + LOG.info(entry.getKey() + " is dead and referenced by the regions = " + + StringUtils.join(entry.getValue().iterator(), ",")); + response.addServers(ProtobufUtil.toServerName(entry.getKey())); + } + } + return response.build(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 9bdcf76..ec901d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.executor.ExecutorService; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; @@ -379,4 +380,9 @@ public interface MasterServices extends Server { * @return True if this master is stopping. */ boolean isStopping(); + + /** + * @return Favored Nodes Manager + */ + public FavoredNodesManager getFavoredNodesManager(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java index ea4612a..6e600e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RackManager.java @@ -57,13 +57,13 @@ public class RackManager { * @param server the server for which to get the rack name * @return the rack name of the server */ - public String getRack(ServerName server) { - if (server == null) { + public String getRack(String hostname) { + if (hostname == null) { return UNKNOWN_RACK; } // just a note - switchMapping caches results (at least the implementation should unless the // resolution is really a lightweight process) - List racks = switchMapping.resolve(Arrays.asList(server.getHostname())); + List racks = switchMapping.resolve(Arrays.asList(hostname)); if (racks != null && !racks.isEmpty()) { return racks.get(0); } @@ -71,6 +71,10 @@ public class RackManager { return UNKNOWN_RACK; } + public String getRack(ServerName sn) { + return getRack(sn.getHostname()); + } + /** * Same as {@link #getRack(ServerName)} except that a list is passed * @param servers list of servers we're requesting racks information for diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index 437c787..7bc78e7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminServic import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; @@ -1204,4 +1205,26 @@ public class ServerManager { removeRegion(hri); } } + + public void sendFavoredNodes(final ServerName server, + Map> favoredNodes) throws IOException { + AdminService.BlockingInterface admin = getRsAdmin(server); + if (admin == null) { + LOG.warn("Attempting to send favored nodes update rpc to server " + server.toString() + + " failed because no RPC connection found to this server"); + } + List>> regionUpdateInfos = + new ArrayList>>(); + for (Entry> entry : favoredNodes.entrySet()) { + regionUpdateInfos.add(new Pair>(entry.getKey(), entry + .getValue())); + } + UpdateFavoredNodesRequest request = RequestConverter + .buildUpdateFavoredNodesRequest(regionUpdateInfos); + try { + admin.updateFavoredNodes(null, request); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java index 39beba8..0b44b5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java @@ -66,7 +66,10 @@ public class SnapshotOfRegionAssignmentFromMeta { private final Map regionNameToRegionInfoMap; /** the regionServer to region map */ - private final Map> regionServerToRegionMap; + private final Map> currentRSToRegionMap; + private final Map> secondaryRSToRegionMap; + private final Map> teritiaryRSToRegionMap; + private final Map> primaryRSToRegionMap; /** the existing assignment plan in the hbase:meta region */ private final FavoredNodesPlan existingAssignmentPlan; private final Set disabledTables; @@ -81,7 +84,10 @@ public class SnapshotOfRegionAssignmentFromMeta { this.connection = connection; tableToRegionMap = new HashMap>(); regionToRegionServerMap = new HashMap(); - regionServerToRegionMap = new HashMap>(); + currentRSToRegionMap = new HashMap>(); + primaryRSToRegionMap = new HashMap>(); + secondaryRSToRegionMap = new HashMap>(); + teritiaryRSToRegionMap = new HashMap>(); regionNameToRegionInfoMap = new TreeMap(); existingAssignmentPlan = new FavoredNodesPlan(); this.disabledTables = disabledTables; @@ -122,6 +128,7 @@ public class SnapshotOfRegionAssignmentFromMeta { addRegion(hri); } + hri = rl.getRegionLocation(0).getRegionInfo(); // the code below is to handle favored nodes byte[] favoredNodes = result.getValue(HConstants.CATALOG_FAMILY, FavoredNodeAssignmentHelper.FAVOREDNODES_QUALIFIER); @@ -132,6 +139,15 @@ public class SnapshotOfRegionAssignmentFromMeta { // Add the favored nodes into assignment plan existingAssignmentPlan.updateFavoredNodesMap(hri, Arrays.asList(favoredServerList)); + for (int i = 0; i < FavoredNodeAssignmentHelper.FAVORED_NODES_NUM; i++) { + if (i == 0) addPrimaryAssignment(hri, favoredServerList[i]); + if (i == 1) addSecondaryAssignment(hri, favoredServerList[i]); + if (i == 2) addTeritiaryAssignment(hri, favoredServerList[i]); + } + if (favoredServerList.length != FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) { + LOG.warn("Insufficient favored nodes for region " + hri + " fn: " + Arrays + .toString(favoredServerList)); + } return true; } catch (RuntimeException e) { LOG.error("Catche remote exception " + e.getMessage() + @@ -169,12 +185,42 @@ public class SnapshotOfRegionAssignmentFromMeta { if (server == null) return; // Process the region server to region map - List regionList = regionServerToRegionMap.get(server); + List regionList = currentRSToRegionMap.get(server); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(regionInfo); + currentRSToRegionMap.put(server, regionList); + } + + private void addPrimaryAssignment(HRegionInfo regionInfo, ServerName server) { + // Process the region server to region map + List regionList = primaryRSToRegionMap.get(server); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(regionInfo); + primaryRSToRegionMap.put(server, regionList); + } + + private void addSecondaryAssignment(HRegionInfo regionInfo, ServerName server) { + // Process the region server to region map + List regionList = secondaryRSToRegionMap.get(server); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(regionInfo); + secondaryRSToRegionMap.put(server, regionList); + } + + private void addTeritiaryAssignment(HRegionInfo regionInfo, ServerName server) { + // Process the region server to region map + List regionList = teritiaryRSToRegionMap.get(server); if (regionList == null) { regionList = new ArrayList(); } regionList.add(regionInfo); - regionServerToRegionMap.put(server, regionList); + teritiaryRSToRegionMap.put(server, regionList); } /** @@ -206,7 +252,7 @@ public class SnapshotOfRegionAssignmentFromMeta { * @return regionserver to region map */ public Map> getRegionServerToRegionMap() { - return regionServerToRegionMap; + return currentRSToRegionMap; } /** @@ -224,4 +270,17 @@ public class SnapshotOfRegionAssignmentFromMeta { public Set getTableSet() { return this.tableToRegionMap.keySet(); } + + public Map> getSecondaryToRegionInfoMap() { + return this.secondaryRSToRegionMap; + } + + public Map> getTertiaryToRegionInfoMap() { + return this.teritiaryRSToRegionMap; + } + + public Map> getPrimaryToRegionInfoMap() { + return this.primaryRSToRegionMap; + } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java index 2b13b21..421b37c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java @@ -1003,7 +1003,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer { protected float slop; protected Configuration config; protected RackManager rackManager; - private static final Random RANDOM = new Random(System.currentTimeMillis()); + static final Random RANDOM = new Random(System.currentTimeMillis()); private static final Log LOG = LogFactory.getLog(BaseLoadBalancer.class); // Regions of these tables are put on the master by default. @@ -1241,7 +1241,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer { */ @Override public Map> roundRobinAssignment(List regions, - List servers) { + List servers) throws HBaseIOException { metricsBalancer.incrMiscInvocations(); Map> assignments = assignMasterRegions(regions, servers); if (assignments != null && !assignments.isEmpty()) { @@ -1344,7 +1344,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer { * Used to assign a single region to a random server. */ @Override - public ServerName randomAssignment(HRegionInfo regionInfo, List servers) { + public ServerName randomAssignment(HRegionInfo regionInfo, List servers) throws HBaseIOException { metricsBalancer.incrMiscInvocations(); if (servers != null && servers.contains(masterServerName)) { if (shouldBeOnMaster(regionInfo)) { @@ -1388,7 +1388,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer { */ @Override public Map> retainAssignment(Map regions, - List servers) { + List servers) throws HBaseIOException { // Update metrics metricsBalancer.incrMiscInvocations(); Map> assignments diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java index 355339e..edc4731 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeAssignmentHelper.java @@ -21,21 +21,26 @@ package org.apache.hadoop.hbase.master.balancer; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Random; import java.util.Set; +import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.StartcodeAgnosticServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Connection; @@ -46,9 +51,14 @@ import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes; +import org.apache.hadoop.hbase.util.Addressing; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.net.HostAndPort; import com.google.protobuf.InvalidProtocolBufferException; /** @@ -64,7 +74,9 @@ public class FavoredNodeAssignmentHelper { private RackManager rackManager; private Map> rackToRegionServerMap; private List uniqueRackList; - private Map regionServerToRackMap; + // This map serves as a cache for rack to sn lookups. The num of + // region server entries might not match with that is in servers. + private Map regionServerToRackMap; private Random random; private List servers; public static final byte [] FAVOREDNODES_QUALIFIER = Bytes.toBytes("fn"); @@ -79,7 +91,7 @@ public class FavoredNodeAssignmentHelper { this.servers = servers; this.rackManager = rackManager; this.rackToRegionServerMap = new HashMap>(); - this.regionServerToRackMap = new HashMap(); + this.regionServerToRackMap = new HashMap(); this.uniqueRackList = new ArrayList(); this.random = new Random(); } @@ -148,8 +160,8 @@ public class FavoredNodeAssignmentHelper { byte[] favoredNodes = getFavoredNodes(favoredNodeList); put.addImmutable(HConstants.CATALOG_FAMILY, FAVOREDNODES_QUALIFIER, EnvironmentEdgeManager.currentTime(), favoredNodes); - LOG.info("Create the region " + regionInfo.getRegionNameAsString() + - " with favored nodes " + Bytes.toString(favoredNodes)); + LOG.debug("Create the region " + regionInfo.getRegionNameAsString() + + " with favored nodes " + favoredNodeList); } return put; } @@ -180,7 +192,7 @@ public class FavoredNodeAssignmentHelper { HBaseProtos.ServerName.Builder b = HBaseProtos.ServerName.newBuilder(); b.setHostName(s.getHostname()); b.setPort(s.getPort()); - b.setStartCode(s.getStartcode()); + b.setStartCode(ServerName.NON_STARTCODE); f.addFavoredNode(b.build()); } return f.build().toByteArray(); @@ -195,7 +207,7 @@ public class FavoredNodeAssignmentHelper { // If there were fewer servers in one rack, say r3, which had 3 servers, one possible // placement could be r2:s5, , r4:s5, r1:s5, r2:s6, ... // The regions should be distributed proportionately to the racksizes - void placePrimaryRSAsRoundRobin(Map> assignmentMap, + public void placePrimaryRSAsRoundRobin(Map> assignmentMap, Map primaryRSMap, List regions) { List rackList = new ArrayList(rackToRegionServerMap.size()); rackList.addAll(rackToRegionServerMap.keySet()); @@ -234,12 +246,14 @@ public class FavoredNodeAssignmentHelper { // Place the current region with the current primary region server primaryRSMap.put(regionInfo, currentServer); - List regionsForServer = assignmentMap.get(currentServer); - if (regionsForServer == null) { - regionsForServer = new ArrayList(); - assignmentMap.put(currentServer, regionsForServer); + if (assignmentMap != null) { + List regionsForServer = assignmentMap.get(currentServer); + if (regionsForServer == null) { + regionsForServer = new ArrayList(); + assignmentMap.put(currentServer, regionsForServer); + } + regionsForServer.add(regionInfo); } - regionsForServer.add(regionInfo); // Set the next processing index if (numIterations % rackList.size() == 0) { @@ -263,7 +277,7 @@ public class FavoredNodeAssignmentHelper { // Create the secondary and tertiary region server pair object. ServerName[] favoredNodes; // Get the rack for the primary region server - String primaryRack = rackManager.getRack(primaryRS); + String primaryRack = getRackOfServer(primaryRS); if (getTotalNumberOfRacks() == 1) { favoredNodes = singleRackCase(regionInfo, primaryRS, primaryRack); @@ -284,6 +298,46 @@ public class FavoredNodeAssignmentHelper { return secondaryAndTertiaryMap; } + /** + * Generates new secondary and tertiary randomly, without + * knowledge of primary. This API is used by the RegionPlacementMaintainer tool + * for migrating tables between groups. + * + * @return List of ServerName + * @throws IOException Signals that an I/O exception has occurred. + */ + public List getNewSecondaryAndTertiary() throws IOException { + List nodes = Lists.newArrayList(); + ServerName secondaryRS = this.servers.get(random.nextInt(this.servers.size())); + String secondaryRack = getRackOfServer(secondaryRS); + ServerName tertiaryRS = null; + List serverList = getServersFromRack(secondaryRack); + if (serverList.size() >= 2) { + Set skipServerSet = new HashSet(); + skipServerSet.add(secondaryRS); + tertiaryRS = getOneRandomServer(secondaryRack, skipServerSet); + } else { + SetrackSkipSet = new HashSet(); + rackSkipSet.add(secondaryRack); + String tertiaryRandomRack = getOneRandomRack(rackSkipSet); + tertiaryRS = getOneRandomServer(tertiaryRandomRack); + } + if (secondaryRS == null || tertiaryRS == null) { + throw new IOException("Cannot place the secondary and terinary nodes"); + } + nodes.add(secondaryRS); + nodes.add(tertiaryRS); + return nodes; + } + + public Map getNewTertiary(List regions) throws IOException { + Map tertiaryHosts = new HashMap(); + Map> assignmentMap = new HashMap>(); + // Using the primary host placement API for good RR distribution + placePrimaryRSAsRoundRobin(assignmentMap, tertiaryHosts, regions); + return tertiaryHosts; + } + private Map> mapRSToPrimaries( Map primaryRSMap) { Map> primaryServerMap = @@ -319,7 +373,7 @@ public class FavoredNodeAssignmentHelper { ServerName primaryRS = entry.getValue(); try { // Get the rack for the primary region server - String primaryRack = rackManager.getRack(primaryRS); + String primaryRack = getRackOfServer(primaryRS); ServerName[] favoredNodes = null; if (getTotalNumberOfRacks() == 1) { // Single rack case: have to pick the secondary and tertiary @@ -370,10 +424,10 @@ public class FavoredNodeAssignmentHelper { for (HRegionInfo primary : primaries) { secondaryAndTertiary = secondaryAndTertiaryMap.get(primary); if (secondaryAndTertiary != null) { - if (regionServerToRackMap.get(secondaryAndTertiary[0]).equals(secondaryRack)) { + if (getRackOfServer(secondaryAndTertiary[0]).equals(secondaryRack)) { skipServerSet.add(secondaryAndTertiary[0]); } - if (regionServerToRackMap.get(secondaryAndTertiary[1]).equals(secondaryRack)) { + if (getRackOfServer(secondaryAndTertiary[1]).equals(secondaryRack)) { skipServerSet.add(secondaryAndTertiary[1]); } } @@ -440,7 +494,7 @@ public class FavoredNodeAssignmentHelper { // Single rack case: have to pick the secondary and tertiary // from the same rack List serverList = getServersFromRack(primaryRack); - if (serverList.size() <= 2) { + if ((serverList == null) || (serverList.size() <= 2)) { // Single region server case: cannot not place the favored nodes // on any server; return null; @@ -454,14 +508,10 @@ public class FavoredNodeAssignmentHelper { ServerName secondaryRS = getOneRandomServer(primaryRack, serverSkipSet); // Skip the secondary for the tertiary placement serverSkipSet.add(secondaryRS); - - // Place the tertiary RS - ServerName tertiaryRS = - getOneRandomServer(primaryRack, serverSkipSet); + ServerName tertiaryRS = getOneRandomServer(primaryRack, serverSkipSet); if (secondaryRS == null || tertiaryRS == null) { - LOG.error("Cannot place the secondary and terinary" + - "region server for region " + + LOG.error("Cannot place the secondary, tertiary favored node for region " + regionInfo.getRegionNameAsString()); } // Create the secondary and tertiary pair @@ -472,63 +522,56 @@ public class FavoredNodeAssignmentHelper { } } + /** + * Place secondary and tertiary nodes in a multi rack case. + * If there are only two racks, then we try the place the secondary + * and tertiary on different rack than primary. But if the other rack has + * only one region server, then we place primary and tertiary on one rack + * and secondary on another. The aim is two distribute the three favored nodes + * on >= 2 racks. + * TODO: see how we can use generateMissingFavoredNodeMultiRack API here + * @param An instance of HRegionInfo. + * @param primaryRS The primary favored node. + * @param primaryRack The racj of the primary favored node. + * @return Array containing secondary and tertiary favored nodes. + * @throws IOException Signals that an I/O exception has occurred. + */ private ServerName[] multiRackCase(HRegionInfo regionInfo, ServerName primaryRS, String primaryRack) throws IOException { - // Random to choose the secondary and tertiary region server - // from another rack to place the secondary and tertiary - - // Random to choose one rack except for the current rack - Set rackSkipSet = new HashSet(); - rackSkipSet.add(primaryRack); - ServerName[] favoredNodes = new ServerName[2]; - String secondaryRack = getOneRandomRack(rackSkipSet); - List serverList = getServersFromRack(secondaryRack); - if (serverList.size() >= 2) { - // Randomly pick up two servers from this secondary rack - - // Place the secondary RS - ServerName secondaryRS = getOneRandomServer(secondaryRack); - - // Skip the secondary for the tertiary placement - Set skipServerSet = new HashSet(); - skipServerSet.add(secondaryRS); - // Place the tertiary RS - ServerName tertiaryRS = getOneRandomServer(secondaryRack, skipServerSet); - - if (secondaryRS == null || tertiaryRS == null) { - LOG.error("Cannot place the secondary and terinary" + - "region server for region " + - regionInfo.getRegionNameAsString()); - } - // Create the secondary and tertiary pair - favoredNodes[0] = secondaryRS; - favoredNodes[1] = tertiaryRS; + ListfavoredNodes = Lists.newArrayList(primaryRS); + // Create the secondary and tertiary pair + ServerName secondaryRS = generateMissingFavoredNodeMultiRack(favoredNodes); + favoredNodes.add(secondaryRS); + String secondaryRack = getRackOfServer(secondaryRS); + ServerName tertiaryRS; + if (primaryRack.equals(secondaryRack)) { + tertiaryRS = generateMissingFavoredNodeMultiRack(favoredNodes); } else { - // Pick the secondary rs from this secondary rack - // and pick the tertiary from another random rack - favoredNodes[0] = getOneRandomServer(secondaryRack); - - // Pick the tertiary - if (getTotalNumberOfRacks() == 2) { - // Pick the tertiary from the same rack of the primary RS - Set serverSkipSet = new HashSet(); - serverSkipSet.add(primaryRS); - favoredNodes[1] = getOneRandomServer(primaryRack, serverSkipSet); - } else { - // Pick the tertiary from another rack - rackSkipSet.add(secondaryRack); - String tertiaryRandomRack = getOneRandomRack(rackSkipSet); - favoredNodes[1] = getOneRandomServer(tertiaryRandomRack); + // Try to place tertiary in secondary RS rack else place on primary rack. + String tertiaryRack = secondaryRack; + tertiaryRS = getOneRandomServer(tertiaryRack, Sets.newHashSet(secondaryRS)); + if (tertiaryRS == null) { + tertiaryRS = getOneRandomServer(primaryRack, Sets.newHashSet(primaryRS)); + } + if (tertiaryRS == null) { + tertiaryRack = getOneRandomRack(Sets.newHashSet(primaryRack,secondaryRack)); + tertiaryRS = getOneRandomServer(tertiaryRack, Sets.newHashSet(favoredNodes)); } } - return favoredNodes; + + if (secondaryRS != null && tertiaryRS != null) { + ServerName[] result = { secondaryRS, tertiaryRS }; + return result; + } else { + throw new IOException("Primary RS = " + primaryRS + "Secondary RS = " + secondaryRS + " Tertiary RS = " + tertiaryRS + + " could not place favored nodes"); + } } - boolean canPlaceFavoredNodes() { - int serverSize = this.regionServerToRackMap.size(); - return (serverSize >= FAVORED_NODES_NUM); + public boolean canPlaceFavoredNodes() { + return (this.servers.size() >= FAVORED_NODES_NUM); } public void initialize() { @@ -543,37 +586,47 @@ public class FavoredNodeAssignmentHelper { if (!serverList.contains(sn)) { serverList.add(sn); this.rackToRegionServerMap.put(rackName, serverList); - this.regionServerToRackMap.put(sn, rackName); + this.regionServerToRackMap.put(sn.getHostname(), rackName); } } } - private int getTotalNumberOfRacks() { + public int getTotalNumberOfRacks() { return this.uniqueRackList.size(); } - private List getServersFromRack(String rack) { + public List getServersFromRack(String rack) { return this.rackToRegionServerMap.get(rack); } - private ServerName getOneRandomServer(String rack, - Set skipServerSet) throws IOException { - if(rack == null) return null; - List serverList = this.rackToRegionServerMap.get(rack); - if (serverList == null) return null; - - // Get a random server except for any servers from the skip set - if (skipServerSet != null && serverList.size() <= skipServerSet.size()) { - throw new IOException("Cannot randomly pick another random server"); + private ServerName getOneRandomServer(String rack, Set skipServerSet) + throws IOException { + if (rack == null) return null; + if (this.rackToRegionServerMap.get(rack) == null) return null; + Set serverList = Sets.newHashSet(); + for (ServerName sn : this.rackToRegionServerMap.get(rack)) { + serverList.add(StartcodeAgnosticServerName.valueOf(sn)); } - - ServerName randomServer; - do { - int randomIndex = random.nextInt(serverList.size()); - randomServer = serverList.get(randomIndex); - } while (skipServerSet != null && skipServerSet.contains(randomServer)); - - return randomServer; + ServerName randomServer = null; + if (skipServerSet != null && skipServerSet.size() > 0) { + for (ServerName sn : skipServerSet) { + StartcodeAgnosticServerName temp = StartcodeAgnosticServerName.valueOf(sn); + serverList.remove(temp); + } + if (serverList.size() == 0) { + return null; + } + } + int randomIndex = random.nextInt(serverList.size()); + int j = 0; + for (StartcodeAgnosticServerName sn : serverList) { + if (j == randomIndex) { + randomServer = sn; + break; + } + j++; + } + return ServerName.valueOf(randomServer.getHostAndPort(), randomServer.getStartcode()); } private ServerName getOneRandomServer(String rack) throws IOException { @@ -603,4 +656,276 @@ public class FavoredNodeAssignmentHelper { } return strBuf.toString(); } -} \ No newline at end of file + + public Map> getRackToRegionServerMap() { + return this.rackToRegionServerMap; + } + + public ServerName generateMissingFavoredNode(List favoredNodes) throws IOException { + if (this.uniqueRackList.size() == 1) { + return generateMissingFavoredNodeSingleRack(favoredNodes, null); + } else { + return generateMissingFavoredNodeMultiRack(favoredNodes, null); + } + } + + public ServerName generateMissingFavoredNode(List favoredNodes, + List excludeNodes) throws IOException { + if (this.uniqueRackList.size() == 1) { + return generateMissingFavoredNodeSingleRack(favoredNodes, excludeNodes); + } else { + return generateMissingFavoredNodeMultiRack(favoredNodes, excludeNodes); + } + } + + private ServerName generateMissingFavoredNodeSingleRack(List favoredNodes, + List excludeNodes) throws IOException { + ServerName newServer = null; + Set favoredNodeSet = Sets.newHashSet(favoredNodes); + if (excludeNodes != null && excludeNodes.size() > 0) { + favoredNodeSet.addAll(excludeNodes); + } + if (favoredNodes.size() < FAVORED_NODES_NUM) { + newServer = this + .getOneRandomServer(this.uniqueRackList.get(0), favoredNodeSet); + } + return newServer; + } + + private ServerName generateMissingFavoredNodeMultiRack(List favoredNodes) + throws IOException { + return generateMissingFavoredNodeMultiRack(favoredNodes, null); + } + + private ServerName generateMissingFavoredNodeMultiRack(List favoredNodes, + List excludeNodes) throws IOException { + ServerName newServer = null; + Set skipRackSet = Sets.newHashSet(); + Set racks = Sets.newHashSet(); + Map> fnRackToRSMap = new HashMap>(); + for (ServerName sn : favoredNodes) { + String rack = getRackOfServer(sn); + racks.add(rack); + Set serversInRack = fnRackToRSMap.get(rack); + if (serversInRack == null) { + serversInRack = Sets.newHashSet(); + } + serversInRack.add(sn); + fnRackToRSMap.put(rack, serversInRack); + } + if (racks.size() == 1 && favoredNodes.size() > 1) { + skipRackSet.add(racks.iterator().next()); + } + // If there are no free nodes on the existing racks, we should just skip those racks + for (String rack : racks) { + if (this.rackToRegionServerMap.get(rack) != null && + fnRackToRSMap.get(rack).size() == this.rackToRegionServerMap.get(rack).size()) { + skipRackSet.add(rack); + } + } + + Set favoredNodeSet = Sets.newHashSet(favoredNodes); + if (excludeNodes != null && excludeNodes.size() > 0) { + favoredNodeSet.addAll(excludeNodes); + } + int i = 0; + Set randomRacks = Sets.newHashSet(); + do { + String randomRack = this.getOneRandomRack(skipRackSet); + newServer = this.getOneRandomServer(randomRack, favoredNodeSet); + randomRacks.add(randomRack); + i++; + } while ((i < 10) && (newServer == null)); + + if (newServer == null) { + if (LOG.isTraceEnabled()) { + LOG.trace(String.format("Unable to generate additional favored nodes for %s after " + + "considering racks %s and skip rack %s with a unique rack list of %s and rack " + + "to RS map of %s and RS to rack map of %s", + StringUtils.join(favoredNodes, ","), randomRacks, skipRackSet, uniqueRackList, + rackToRegionServerMap, regionServerToRackMap)); + } + throw new IOException(" Unable to generate additional favored nodes for " + + StringUtils.join(favoredNodes, ",")); + } + return newServer; + } + + /** + * Try Replace a given favored node with some other node. + * + * @param toBeReplaced the favored node intended to be replaced + * @param replacement the replacement node + * @param favNodes favored nodes + * @return List of favored node if the passed fn can be replaced, if not, null + */ + public List replaceFavoredNode(ServerName toBeReplaced, ServerName replacement, + List favNodes) { + Set stringFN = Sets.newHashSet(); + boolean shouldReplace = false; + for (ServerName sn : favNodes) { + stringFN.add(sn.getHostAndPort()); + } + if (stringFN.contains(toBeReplaced.getHostAndPort()) && + !stringFN.contains(replacement.getHostAndPort())) { + + stringFN.remove(toBeReplaced.getHostAndPort()); + stringFN.add(replacement.getHostAndPort()); + Set racks = Sets.newHashSet(); + for (String sn : stringFN) { + racks.add(this.rackManager.getRack(Addressing.parseHostname(sn))); + } + if (this.uniqueRackList.size() == 1) { + if (racks.size() > 0) { + shouldReplace = true; + } + } else { + if (racks.size() >= 2) { + shouldReplace = true; + } + } + } else { + LOG.warn("Not replacing favored node. Either node to be replaced " + toBeReplaced + + " not found in favored nodes " + favNodes + " or replacement " + replacement + + " is already present"); + shouldReplace = false; + } + if (shouldReplace) { + List result = Lists.newArrayList(); + for (String sn : stringFN) { + result.add(ServerName.valueOf(sn, ServerName.NON_STARTCODE)); + } + return result; + } + return null; + } + + Map> generateRRSecondaryAndTertiary( + Map primaryRSMap, String primaryRack) throws IOException { + List sNames = Lists.newArrayList(); + Map> assignments = new HashMap>(); + for (String rk : this.rackToRegionServerMap.keySet()) { + if (!rk.equals(primaryRack)) { + sNames.addAll(this.rackToRegionServerMap.get(rk)); + } + } + if (sNames.isEmpty()) { + throw new IOException("Unable to find server in rack different than " + primaryRack); + } + int randomIndexToStart = random.nextInt(sNames.size()); + Iterator circularItr = Iterables.cycle(sNames).iterator(); + for (int i = 0; i < randomIndexToStart; i++) { + circularItr.next(); + } + for (HRegionInfo hri : primaryRSMap.keySet()) { + List favoredNodes = new ArrayList(FAVORED_NODES_NUM); + favoredNodes.add(ServerName.valueOf(primaryRSMap.get(hri).getHostAndPort(), + ServerName.NON_STARTCODE)); + ServerName secondary = ServerName.valueOf(circularItr.next().getHostAndPort(), + ServerName.NON_STARTCODE); + ServerName tertiary = ServerName.valueOf(circularItr.next().getHostAndPort(), + ServerName.NON_STARTCODE); + favoredNodes.add(secondary); + favoredNodes.add(tertiary); + assignments.put(hri, favoredNodes); + } + return assignments; + } + + Map> generateRRPrimaryAndSecondary(List regions, + List servers) { + if (regions.isEmpty() || servers.isEmpty()) { + return null; + } + Map> assignments = new HashMap>(); + int serverIndex = random.nextInt(servers.size()); + for (HRegionInfo hri : regions) { + List favoredNodes = new ArrayList(FAVORED_NODES_NUM); + while (favoredNodes.size() < FAVORED_NODES_NUM) { + favoredNodes.add(ServerName.valueOf(servers.get(serverIndex).getHostAndPort(), + ServerName.NON_STARTCODE)); + serverIndex = (serverIndex + 1) % servers.size(); + } + assignments.put(hri, favoredNodes); + } + return assignments; + } + + public Map> generateFavoredNodes( + Map primaryRSMap) { + Map> generatedFavNodes = new HashMap>(); + Map secondaryAndTertiaryRSMap = placeSecondaryAndTertiaryRS(primaryRSMap); + for (HRegionInfo region : primaryRSMap.keySet()) { + List favoredNodesForRegion = new ArrayList(FAVORED_NODES_NUM); + ServerName sn = primaryRSMap.get(region); + favoredNodesForRegion.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), + ServerName.NON_STARTCODE)); + ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(region); + if (secondaryAndTertiaryNodes != null) { + favoredNodesForRegion.add(ServerName.valueOf( + secondaryAndTertiaryNodes[0].getHostname(), secondaryAndTertiaryNodes[0].getPort(), + ServerName.NON_STARTCODE)); + favoredNodesForRegion.add(ServerName.valueOf( + secondaryAndTertiaryNodes[1].getHostname(), secondaryAndTertiaryNodes[1].getPort(), + ServerName.NON_STARTCODE)); + } + generatedFavNodes.put(region, favoredNodesForRegion); + } + return generatedFavNodes; + } + + public List generateFavoredNodes(HRegionInfo hri) throws IOException { + List favoredNodesForRegion = new ArrayList(FAVORED_NODES_NUM); + ServerName primary = servers.get(random.nextInt(servers.size())); + favoredNodesForRegion.add(ServerName.valueOf(primary.getHostAndPort(), ServerName.NON_STARTCODE)); + Map primaryRSMap = new HashMap(1); + primaryRSMap.put(hri, primary); + Map secondaryAndTertiaryRSMap = placeSecondaryAndTertiaryRS(primaryRSMap); + ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(hri); + if (secondaryAndTertiaryNodes != null) { + favoredNodesForRegion.add(ServerName.valueOf( + secondaryAndTertiaryNodes[0].getHostAndPort(), ServerName.NON_STARTCODE)); + favoredNodesForRegion.add(ServerName.valueOf( + secondaryAndTertiaryNodes[1].getHostAndPort(), ServerName.NON_STARTCODE)); + return favoredNodesForRegion; + } else { + throw new HBaseIOException("Unable to generate secondary and tertiary favored nodes."); + } + } + + public static List filterServers(Collection servers, + Collection onlineServers) { + Set online = Sets.newHashSet(); + List result = Lists.newArrayList(); + for (ServerName curr : onlineServers) { + online.add(StartcodeAgnosticServerName.valueOf(curr)); + } + for (HostAndPort server : servers) { + StartcodeAgnosticServerName groupServer = StartcodeAgnosticServerName.valueOf(server, + ServerName.NON_STARTCODE); + if (online.contains(groupServer)) { + result.add(groupServer); + } + } + return result; +} + + public String getRackOfServer(ServerName sn) { + if (this.regionServerToRackMap.containsKey(sn.getHostname())) { + return this.regionServerToRackMap.get(sn.getHostname()); + } else { + String rack = this.rackManager.getRack(sn); + this.regionServerToRackMap.put(sn.getHostname(), rack); + return rack; + } + } + + public static Set convertToStartCodeAgnosticSN( + List servers) { + Set result = Sets.newHashSet(); + for (ServerName sn : servers) { + result.add(StartcodeAgnosticServerName.valueOf(sn)); + } + return result; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java index 7e4fecf..3df4f56 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodeLoadBalancer.java @@ -21,8 +21,10 @@ package org.apache.hadoop.hbase.master.balancer; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -31,17 +33,21 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.RackManager; import org.apache.hadoop.hbase.master.RegionPlan; +import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.SnapshotOfRegionAssignmentFromMeta; import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan.Position; import org.apache.hadoop.hbase.util.Pair; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + /** * An implementation of the {@link org.apache.hadoop.hbase.master.LoadBalancer} that * assigns favored nodes for each region. There is a Primary RegionServer that hosts @@ -56,18 +62,32 @@ import org.apache.hadoop.hbase.util.Pair; * primary region servers die. * */ +//TODO This is a dead class. We should remove this class once reviewer takes a look and agrees. @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) -public class FavoredNodeLoadBalancer extends BaseLoadBalancer { +public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements FavoredNodesPromoter { private static final Log LOG = LogFactory.getLog(FavoredNodeLoadBalancer.class); private FavoredNodesPlan globalFavoredNodesAssignmentPlan; private RackManager rackManager; + private Configuration conf; + private Map> primaryRSToRegionMap; + private Map> secondaryRSToRegionMap; + private Map> teritiaryRSToRegionMap; @Override public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public synchronized void initialize() throws HBaseIOException { + super.initialize(); super.setConf(conf); globalFavoredNodesAssignmentPlan = new FavoredNodesPlan(); this.rackManager = new RackManager(conf); + primaryRSToRegionMap = new HashMap>(); + secondaryRSToRegionMap = new HashMap>(); + teritiaryRSToRegionMap = new HashMap>(); super.setConf(conf); } @@ -102,8 +122,7 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer { currentServer.getPort(), ServerName.NON_STARTCODE); List list = entry.getValue(); for (HRegionInfo region : list) { - if(region.getTable().getNamespaceAsString() - .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { + if(region.getTable().isSystemTable()) { continue; } List favoredNodes = globalFavoredNodesAssignmentPlan.getFavoredNodes(region); @@ -149,15 +168,13 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer { @Override public Map> roundRobinAssignment(List regions, - List servers) { + List servers) throws HBaseIOException { Map> assignmentMap; try { FavoredNodeAssignmentHelper assignmentHelper = new FavoredNodeAssignmentHelper(servers, rackManager); assignmentHelper.initialize(); - if (!assignmentHelper.canPlaceFavoredNodes()) { - return super.roundRobinAssignment(regions, servers); - } + // Segregate the regions into two types: // 1. The regions that have favored node assignment, and where at least // one of the favored node is still alive. In this case, try to adhere @@ -179,26 +196,26 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer { Map> regionsWithFavoredNodesMap = segregatedRegions.getFirst(); List regionsWithNoFavoredNodes = segregatedRegions.getSecond(); assignmentMap = new HashMap>(); - roundRobinAssignmentImpl(assignmentHelper, assignmentMap, regionsWithNoFavoredNodes, + generateFavoredNodes(assignmentHelper, assignmentMap, regionsWithNoFavoredNodes, servers); // merge the assignment maps assignmentMap.putAll(regionsWithFavoredNodesMap); } catch (Exception ex) { - LOG.warn("Encountered exception while doing favored-nodes assignment " + ex + - " Falling back to regular assignment"); - assignmentMap = super.roundRobinAssignment(regions, servers); + throw new HBaseIOException("Encountered exception while doing favored-nodes assignment " + ex + + " Falling back to regular assignment", ex); } return assignmentMap; } @Override - public ServerName randomAssignment(HRegionInfo regionInfo, List servers) { + public ServerName randomAssignment(HRegionInfo regionInfo, List servers) + throws HBaseIOException { + ServerName primary = super.randomAssignment(regionInfo, servers); try { FavoredNodeAssignmentHelper assignmentHelper = new FavoredNodeAssignmentHelper(servers, rackManager); assignmentHelper.initialize(); - ServerName primary = super.randomAssignment(regionInfo, servers); - if (!assignmentHelper.canPlaceFavoredNodes()) { + if (!assignmentHelper.canPlaceFavoredNodes() || regionInfo.getTable().isSystemTable()) { return primary; } List favoredNodes = globalFavoredNodesAssignmentPlan.getFavoredNodes(regionInfo); @@ -213,16 +230,14 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer { } } } - List regions = new ArrayList(1); - regions.add(regionInfo); - Map primaryRSMap = new HashMap(1); - primaryRSMap.put(regionInfo, primary); - assignSecondaryAndTertiaryNodesForRegion(assignmentHelper, regions, primaryRSMap); - return primary; + List newFavoredNodes = assignmentHelper.generateFavoredNodes(regionInfo); + services.getFavoredNodesManager().updateFavoredNodes(regionInfo, newFavoredNodes); + updateFavoredNodesMap(regionInfo, newFavoredNodes); + return newFavoredNodes.get(this.RANDOM.nextInt(newFavoredNodes.size())); } catch (Exception ex) { LOG.warn("Encountered exception while doing favored-nodes (random)assignment " + ex + " Falling back to regular assignment"); - return super.randomAssignment(regionInfo, servers); + return primary; } } @@ -233,31 +248,38 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer { new HashMap>(regions.size() / 2); List regionsWithNoFavoredNodes = new ArrayList(regions.size()/2); for (HRegionInfo region : regions) { - List favoredNodes = globalFavoredNodesAssignmentPlan.getFavoredNodes(region); - ServerName primaryHost = null; - ServerName secondaryHost = null; - ServerName tertiaryHost = null; - if (favoredNodes != null) { - for (ServerName s : favoredNodes) { - ServerName serverWithLegitStartCode = availableServersContains(availableServers, s); - if (serverWithLegitStartCode != null) { - FavoredNodesPlan.Position position = - FavoredNodesPlan.getFavoredServerPosition(favoredNodes, s); - if (Position.PRIMARY.equals(position)) { - primaryHost = serverWithLegitStartCode; - } else if (Position.SECONDARY.equals(position)) { - secondaryHost = serverWithLegitStartCode; - } else if (Position.TERTIARY.equals(position)) { - tertiaryHost = serverWithLegitStartCode; + if (region.getTable().isSystemTable()) { + try { + ServerName destination = super.randomAssignment(region, availableServers); + addRegionToMap(assignmentMapForFavoredNodes, region, destination); + } catch (HBaseIOException e) { + LOG.error(e); + } + } else { + List favoredNodes = getFavoredNodes(region); + ServerName primaryHost = null; + ServerName secondaryHost = null; + ServerName tertiaryHost = null; + if (favoredNodes != null) { + for (ServerName s : favoredNodes) { + ServerName serverWithLegitStartCode = availableServersContains(availableServers, s); + if (serverWithLegitStartCode != null) { + FavoredNodesPlan.Position position = + FavoredNodesPlan.getFavoredServerPosition(favoredNodes, s); + if (Position.PRIMARY.equals(position)) { + primaryHost = serverWithLegitStartCode; + } else if (Position.SECONDARY.equals(position)) { + secondaryHost = serverWithLegitStartCode; + } else if (Position.TERTIARY.equals(position)) { + tertiaryHost = serverWithLegitStartCode; + } } } + assignRegionToAvailableFavoredNode(assignmentMapForFavoredNodes, region, primaryHost, + secondaryHost, tertiaryHost); + } else { + regionsWithNoFavoredNodes.add(region); } - assignRegionToAvailableFavoredNode(assignmentMapForFavoredNodes, region, - primaryHost, secondaryHost, tertiaryHost); - } - if (primaryHost == null && secondaryHost == null && tertiaryHost == null) { - //all favored nodes unavailable - regionsWithNoFavoredNodes.add(region); } } return new Pair>, List>( @@ -287,10 +309,18 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer { ServerName s; ServerLoad tertiaryLoad = super.services.getServerManager().getLoad(tertiaryHost); ServerLoad secondaryLoad = super.services.getServerManager().getLoad(secondaryHost); - if (secondaryLoad.getLoad() < tertiaryLoad.getLoad()) { - s = secondaryHost; + if (secondaryLoad != null && tertiaryLoad != null) { + if (secondaryLoad.getLoad() < tertiaryLoad.getLoad()) { + s = secondaryHost; + } else { + s = tertiaryHost; + } } else { - s = tertiaryHost; + if (this.RANDOM.nextBoolean()) { + s = secondaryHost; + } else { + s = tertiaryHost; + } } addRegionToMap(assignmentMapForFavoredNodes, region, s); } else if (secondaryHost != null) { @@ -310,42 +340,248 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer { regionsOnServer.add(region); } - public List getFavoredNodes(HRegionInfo regionInfo) { + public synchronized List getFavoredNodes(HRegionInfo regionInfo) { return this.globalFavoredNodesAssignmentPlan.getFavoredNodes(regionInfo); } - private void roundRobinAssignmentImpl(FavoredNodeAssignmentHelper assignmentHelper, - Map> assignmentMap, - List regions, List servers) { - Map primaryRSMap = new HashMap(); - // figure the primary RSs - assignmentHelper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions); - assignSecondaryAndTertiaryNodesForRegion(assignmentHelper, regions, primaryRSMap); + private void generateFavoredNodes(FavoredNodeAssignmentHelper assignmentHelper, + Map> assignmentMap, List regions, + List servers) throws IOException { + if (regions.size() > 0) { + if (assignmentHelper.canPlaceFavoredNodes()) { + Map primaryRSMap = new HashMap(); + assignmentHelper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions); + Map> generatedFavNodes = assignmentHelper + .generateFavoredNodes(primaryRSMap); + for (HRegionInfo hri : generatedFavNodes.keySet()) { + services.getFavoredNodesManager().updateFavoredNodes(hri, generatedFavNodes.get(hri)); + } + } else { + throw new HBaseIOException(" Not enough nodes to do RR assignment"); + } + } } - private void assignSecondaryAndTertiaryNodesForRegion( - FavoredNodeAssignmentHelper assignmentHelper, - List regions, Map primaryRSMap) { - // figure the secondary and tertiary RSs - Map secondaryAndTertiaryRSMap = - assignmentHelper.placeSecondaryAndTertiaryRS(primaryRSMap); - // now record all the assignments so that we can serve queries later - for (HRegionInfo region : regions) { - // Store the favored nodes without startCode for the ServerName objects - // We don't care about the startcode; but only the hostname really - List favoredNodesForRegion = new ArrayList(3); - ServerName sn = primaryRSMap.get(region); - favoredNodesForRegion.add(ServerName.valueOf(sn.getHostname(), sn.getPort(), +// @Override + public synchronized void updateFavoredNodesMap(HRegionInfo region, List servers) { + if (getFavoredNodes(region) != null) { + deleteFavoredNodesForRegion(Lists.newArrayList(region)); + } + globalFavoredNodesAssignmentPlan.updateFavoredNodesMap(region, servers); + addToReplicaLoad(region, servers); + } + + public void initializeFavoredNodes() throws IOException { + SnapshotOfRegionAssignmentFromMeta snaphotOfRegionAssignment = + new SnapshotOfRegionAssignmentFromMeta(super.services.getConnection()); + try { + snaphotOfRegionAssignment.initialize(); + } catch (IOException e) { + throw new HBaseIOException(e); + } + globalFavoredNodesAssignmentPlan = snaphotOfRegionAssignment.getExistingAssignmentPlan(); + primaryRSToRegionMap = snaphotOfRegionAssignment.getPrimaryToRegionInfoMap(); + secondaryRSToRegionMap = snaphotOfRegionAssignment.getSecondaryToRegionInfoMap(); + teritiaryRSToRegionMap = snaphotOfRegionAssignment.getTertiaryToRegionInfoMap(); + } + + @Override + public Map> redistribute( + Map> clusterState) throws IOException { + throw new IOException("Not implemented"); + } + + @Override + public List completeRedistribute( + Map> clusterState) throws IOException { + throw new IOException("Not implemented"); + } + + private synchronized void addToReplicaLoad(HRegionInfo hri, List servers) { + ServerName serverToUse = ServerName.valueOf(servers.get(0).getHostAndPort(), + ServerName.NON_STARTCODE); + List regionList = primaryRSToRegionMap.get(serverToUse); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(hri); + primaryRSToRegionMap.put(serverToUse, regionList); + + serverToUse = ServerName + .valueOf(servers.get(1).getHostAndPort(), ServerName.NON_STARTCODE); + regionList = secondaryRSToRegionMap.get(serverToUse); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(hri); + secondaryRSToRegionMap.put(serverToUse, regionList); + + serverToUse = ServerName.valueOf(servers.get(2).getHostAndPort(), ServerName.NON_STARTCODE); + regionList = teritiaryRSToRegionMap.get(serverToUse); + if (regionList == null) { + regionList = new ArrayList(); + } + regionList.add(hri); + teritiaryRSToRegionMap.put(serverToUse, regionList); + } + +// @Override + public synchronized void deleteFavoredNodesForRegion(List regionInfoList) { + for (HRegionInfo hri : regionInfoList) { + List favNodes = getFavoredNodes(hri); + if (favNodes != null) { + if (primaryRSToRegionMap.containsKey(favNodes.get(0))) { + primaryRSToRegionMap.get(favNodes.get(0)).remove(hri); + } + if (secondaryRSToRegionMap.containsKey(favNodes.get(1))) { + secondaryRSToRegionMap.get(favNodes.get(1)).remove(hri); + } + if (teritiaryRSToRegionMap.containsKey(favNodes.get(2))) { + teritiaryRSToRegionMap.get(favNodes.get(2)).remove(hri); + } + globalFavoredNodesAssignmentPlan.removeFavoredNodes(hri); + } + } + } + +// @Override + public synchronized Map> getReplicaLoad(List servers) { + Map> result = new HashMap>(); + for (ServerName sn : servers) { + ServerName serverWithNoStartCode = ServerName.valueOf(sn.getHostAndPort(), + ServerName.NON_STARTCODE); + List countList = Lists.newArrayList(); + if (primaryRSToRegionMap.containsKey(serverWithNoStartCode)) { + countList.add(primaryRSToRegionMap.get(serverWithNoStartCode).size()); + } else { + countList.add(0); + } + if (secondaryRSToRegionMap.containsKey(serverWithNoStartCode)) { + countList.add(secondaryRSToRegionMap.get(serverWithNoStartCode).size()); + } else { + countList.add(0); + } + if (teritiaryRSToRegionMap.containsKey(serverWithNoStartCode)) { + countList.add(teritiaryRSToRegionMap.get(serverWithNoStartCode).size()); + } else { + countList.add(0); + } + result.put(sn, countList); + } + return result; + } + +// @Override + public synchronized void removeFavoredNode( + ServerName decommissionedServer, List servers) throws IOException { + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, this.conf); + helper.initialize(); + Set regions = Sets.newHashSet(); + ServerName sn = ServerName.valueOf(decommissionedServer.getHostAndPort(), + ServerName.NON_STARTCODE); + if (primaryRSToRegionMap.containsKey(sn)) { + regions.addAll(primaryRSToRegionMap.get(sn)); + } + if (secondaryRSToRegionMap.containsKey(sn)) { + regions.addAll(secondaryRSToRegionMap.get(sn)); + } + if (teritiaryRSToRegionMap.containsKey(sn)) { + regions.addAll(teritiaryRSToRegionMap.get(sn)); + } + + Iterator itr = regions.iterator(); + RegionStates regionStates = this.services.getAssignmentManager().getRegionStates(); + while (itr.hasNext()) { + HRegionInfo hri = itr.next(); + if (regionStates.isRegionOffline(hri) + || regionStates.isRegionInState(hri, RegionState.State.SPLIT, RegionState.State.MERGED, + RegionState.State.MERGING_NEW, RegionState.State.SPLITTING_NEW)) { + itr.remove(); + } + } + Map> newFavoredNodes = fixFavoredNodes(helper, + decommissionedServer, regions); + for (HRegionInfo hri : newFavoredNodes.keySet()) { + services.getFavoredNodesManager().updateFavoredNodes(hri, newFavoredNodes.get(hri)); + } + services.getFavoredNodesManager().updateFavoredNodesInRegionServer(newFavoredNodes); + primaryRSToRegionMap.remove(sn); + secondaryRSToRegionMap.remove(sn); + teritiaryRSToRegionMap.remove(sn); + } + + private Map> fixFavoredNodes(FavoredNodeAssignmentHelper helper, + ServerName decommissionedServer, Set regions) throws IOException { + Map> onlineFavoredNodes = new HashMap>(); + for (HRegionInfo hri : regions) { + if (!hri.getTable().isSystemTable()) { + Set favNodeWithoutStartCode = Sets.newHashSet(getFavoredNodes(hri)); + favNodeWithoutStartCode.remove(ServerName.valueOf(decommissionedServer.getHostAndPort(), ServerName.NON_STARTCODE)); - ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(region); - if (secondaryAndTertiaryNodes != null) { - favoredNodesForRegion.add(ServerName.valueOf(secondaryAndTertiaryNodes[0].getHostname(), - secondaryAndTertiaryNodes[0].getPort(), ServerName.NON_STARTCODE)); - favoredNodesForRegion.add(ServerName.valueOf(secondaryAndTertiaryNodes[1].getHostname(), - secondaryAndTertiaryNodes[1].getPort(), ServerName.NON_STARTCODE)); + while (favNodeWithoutStartCode.size() < 3) { + ServerName sn = helper.generateMissingFavoredNode(Lists + .newArrayList(favNodeWithoutStartCode)); + favNodeWithoutStartCode.add(ServerName.valueOf(sn.getHostAndPort(), + ServerName.NON_STARTCODE)); + } + LOG.debug("Generated one missing favored nodes for " + hri.getEncodedName() + " : " + + favNodeWithoutStartCode); + onlineFavoredNodes.put(hri, Lists.newArrayList(favNodeWithoutStartCode)); } - globalFavoredNodesAssignmentPlan.updateFavoredNodesMap(region, favoredNodesForRegion); } + return onlineFavoredNodes; + } + + @Override + public Map> checkFavoredNodes(List servers, + List regions) { + return null; + } + + @Override + public Map> generateFavoredNodesForDaughter( + List servers, HRegionInfo parent, HRegionInfo hri_a, HRegionInfo hri_b) + throws IOException { + Map> result = new HashMap>(); + FavoredNodeAssignmentHelper assignmentHelper = new FavoredNodeAssignmentHelper(servers, + rackManager); + assignmentHelper.initialize(); + List parentFavoredNodes = getFavoredNodes(parent); + if (parentFavoredNodes == null) { + LOG.debug("Unable to find favored nodes for parent, " + parent + + " generating new favored nodes for daughter"); + result.put(hri_a, assignmentHelper.generateFavoredNodes(hri_a)); + result.put(hri_b, assignmentHelper.generateFavoredNodes(hri_b)); + } else { + Set existingFavNodes = Sets.newHashSet(); + existingFavNodes.add(parentFavoredNodes.get(0)); + existingFavNodes.add(parentFavoredNodes.get(1)); + while (existingFavNodes.size() < 3) { + ServerName newNode = assignmentHelper.generateMissingFavoredNode(Lists + .newArrayList(existingFavNodes)); + existingFavNodes.add(newNode); + } + result.put(hri_a, Lists.newArrayList(existingFavNodes)); + existingFavNodes.clear(); + existingFavNodes.add(parentFavoredNodes.get(0)); + existingFavNodes.add(parentFavoredNodes.get(2)); + while (existingFavNodes.size() < 3) { + ServerName newNode = assignmentHelper.generateMissingFavoredNode(Lists + .newArrayList(existingFavNodes)); + existingFavNodes.add(newNode); + } + result.put(hri_b, Lists.newArrayList(existingFavNodes)); + } + services.getFavoredNodesManager().updateFavoredNodes(hri_a, result.get(hri_a)); + services.getFavoredNodesManager().updateFavoredNodes(hri_b, result.get(hri_b)); + return result; + } + + @Override + public void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo hriA, + HRegionInfo hriB) throws IOException { + services.getFavoredNodesManager().updateFavoredNodes(merged, getFavoredNodes(hriA)); + services.getFavoredNodesManager().deleteFavoredNodesForRegion(Lists.newArrayList(hriA, hriB)); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPlan.java index 17be833..9a87626 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPlan.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPlan.java @@ -18,15 +18,17 @@ */ package org.apache.hadoop.hbase.master.balancer; +import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.util.Bytes; /** * This class contains the mapping information between each region and @@ -46,11 +48,18 @@ public class FavoredNodesPlan { public static enum Position { PRIMARY, SECONDARY, - TERTIARY; - }; + TERTIARY + } public FavoredNodesPlan() { - favoredNodesMap = new ConcurrentHashMap>(); + favoredNodesMap = new ConcurrentSkipListMap>( + new Comparator() { + @Override + public int compare(HRegionInfo o1, HRegionInfo o2) { + return Bytes.compareTo(o1.getRegionName(), o2.getRegionName()); + } + } + ); } /** @@ -58,10 +67,11 @@ public class FavoredNodesPlan { * @param region * @param servers */ - public synchronized void updateFavoredNodesMap(HRegionInfo region, + public void updateFavoredNodesMap(HRegionInfo region, List servers) { - if (region == null || servers == null || servers.size() ==0) + if (region == null || servers == null || servers.size() ==0) { return; + } this.favoredNodesMap.put(region, servers); } @@ -69,7 +79,7 @@ public class FavoredNodesPlan { * @param region * @return the list of favored region server for this region based on the plan */ - public synchronized List getFavoredNodes(HRegionInfo region) { + public List getFavoredNodes(HRegionInfo region) { return favoredNodesMap.get(region); } @@ -97,8 +107,8 @@ public class FavoredNodesPlan { /** * @return the mapping between each region to its favored region server list */ - public synchronized Map> getAssignmentMap() { - return this.favoredNodesMap; + public Map> getAssignmentMap() { + return favoredNodesMap; } /** @@ -128,8 +138,7 @@ public class FavoredNodesPlan { return false; } // To compare the map from objec o is identical to current assignment map. - Map> comparedMap= - ((FavoredNodesPlan)o).getAssignmentMap(); + Map> comparedMap = ((FavoredNodesPlan)o).getAssignmentMap(); // compare the size if (comparedMap.size() != this.favoredNodesMap.size()) @@ -152,4 +161,8 @@ public class FavoredNodesPlan { public int hashCode() { return favoredNodesMap.hashCode(); } + + public List removeFavoredNodes(HRegionInfo region) { + return favoredNodesMap.remove(region); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPromoter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPromoter.java new file mode 100644 index 0000000..0ccc1ea --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesPromoter.java @@ -0,0 +1,47 @@ +/** + * 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.balancer; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.master.RegionPlan; + +public interface FavoredNodesPromoter { + + String MIN_LOCALITY_FOR_REDISTRIBUTE = "hbase.min.locality.redistribute"; + String REDISTRIBUTE_ON_SAME_RACK = "hbase.redistribute.even.on.same.rack"; + String ALWAYS_ASSIGN_REGIONS = "hbase.assignment.always.assign"; + + Map> generateFavoredNodesForDaughter(List servers, + HRegionInfo parent, HRegionInfo hriA, HRegionInfo hriB) throws IOException; + + void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo hriA, + HRegionInfo hriB) throws IOException; + + Map> redistribute(Map> clusterState) + throws IOException; + + List completeRedistribute( + Map> clusterState) throws IOException; + + Map> checkFavoredNodes(List servers, List regions); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesRepairChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesRepairChore.java new file mode 100644 index 0000000..f2b72d8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredNodesRepairChore.java @@ -0,0 +1,83 @@ +/** + * 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.balancer; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +public class FavoredNodesRepairChore extends ScheduledChore { + private static final Log LOG = LogFactory.getLog(FavoredNodesRepairChore.class); + public static final String FAVORED_NODE_REPAIR_CHORE_FREQ = "hbase.favorednodes.repairPeriod"; + private final HMaster master; + private FavoredNodesManager favoredNodesManager; + + public FavoredNodesRepairChore(HMaster master) { + super(master.getServerName() + "-FavoredNodesRepairChore", master, master.getConfiguration().getInt( + FAVORED_NODE_REPAIR_CHORE_FREQ, 1 * 60 * 60000)); + this.master = master; + favoredNodesManager = master.getFavoredNodesManager(); + } + + @Override + protected void chore() { + LOG.info("Started to sync favored nodes between master and regionservers."); + long startTime = EnvironmentEdgeManager.currentTime(); + syncFavoredNodesWithRegionServers(); + LOG.info("Finished syncing favored nodes, took " + + (EnvironmentEdgeManager.currentTime() - startTime) + " ms to finish."); + } + + void syncFavoredNodesWithRegionServers() { + List onlineServers = master.getServerManager().getOnlineServersList(); + for (ServerName sn : onlineServers) { + Set regionsOfServer = master.getAssignmentManager().getRegionStates() + .getServerRegions(sn); + // Some region servers might be empty/just starting, lets ignore them + if (regionsOfServer != null && regionsOfServer.size() > 0) { + syncFavoredNodesForRS(sn, regionsOfServer); + } + } + } + + void syncFavoredNodesForRS(ServerName server, Collection regions) { + Map> favoredNodesMap = new HashMap>(); + for (HRegionInfo hri : regions) { + if (!hri.getTable().isSystemTable()) { + favoredNodesMap.put(hri, favoredNodesManager.getFavoredNodes(hri)); + } + } + try { + master.getServerManager().sendFavoredNodes(server, favoredNodesMap); + } catch (IOException e) { + LOG.warn("Exception while updating favored nodes on server " + server, e); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java new file mode 100644 index 0000000..c4b80da --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java @@ -0,0 +1,861 @@ +/** + * 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.balancer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.ClusterStatus; +import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.HDFSBlocksDistribution; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.ServerLoad; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.StartcodeAgnosticServerName; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; +import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.master.RackManager; +import org.apache.hadoop.hbase.master.RegionPlan; +import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan.Position; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Pair; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +public class FavoredStochasticBalancer extends StochasticLoadBalancer implements + FavoredNodesPromoter { + + private static final Log LOG = LogFactory.getLog(FavoredStochasticBalancer.class); + private ClusterStatus status; + private FavoredNodesManager fnm; + + + @Override + public void initialize() throws HBaseIOException { + configureGenerators(); + super.initialize(); + } + + protected void configureGenerators() { + List fnPickers = new ArrayList(2); + fnPickers.add(new FavoredNodeLoadPicker()); + fnPickers.add(new FavoredNodeLocalityPicker()); + setCandidateGenerators(fnPickers); + } + + @Override + public void setMasterServices(MasterServices masterServices) { + super.setMasterServices(masterServices); + fnm = masterServices.getFavoredNodesManager(); + } + + @Override + public void setClusterStatus(ClusterStatus st) { + super.setClusterStatus(st); + this.status = st; + } + + @Override + public Map> roundRobinAssignment(List regions, + List servers) throws HBaseIOException { + //TODO: Use complete redistribute API + Map> assignmentMap; + try { + FavoredNodeAssignmentHelper assignmentHelper = + new FavoredNodeAssignmentHelper(servers, fnm.getRackManager()); + assignmentHelper.initialize(); + + // Segregate the regions into two types: + // 1. The regions that have favored node assignment, and where at least + // one of the favored node is still alive. In this case, try to adhere + // to the current favored nodes assignment as much as possible - i.e., + // if the current primary is gone, then make the secondary or tertiary + // as the new host for the region (based on their current load). + // Note that we don't change the favored + // node assignments here (even though one or more favored node is currently + // down). It is up to the balanceCluster to do this hard work. The HDFS + // can handle the fact that some nodes in the favored nodes hint is down + // It'd allocate some other DNs. In combination with stale settings for HDFS, + // we should be just fine. + // 2. The regions that currently don't have favored node assignment. We will + // need to come up with favored nodes assignments for them. The corner case + // in (1) above is that all the nodes are unavailable and in that case, we + // will note that this region doesn't have favored nodes. + Pair>, List> segregatedRegions = + segregateRegionsAndAssignRegionsWithFavoredNodes(regions, servers); + Map> regionsWithFavoredNodesMap = segregatedRegions.getFirst(); + List regionsWithNoFavoredNodes = segregatedRegions.getSecond(); + assignmentMap = new HashMap>(); + fnm.generateFavoredNodes(assignmentHelper, assignmentMap, regionsWithNoFavoredNodes, + servers); + // merge the assignment maps + for (ServerName sn : regionsWithFavoredNodesMap.keySet()) { + if (assignmentMap.get(sn) == null) { + assignmentMap.put(sn, Lists.newArrayList()); + } + assignmentMap.get(sn).addAll(regionsWithFavoredNodesMap.get(sn)); + } + assignmentMap.putAll(regionsWithFavoredNodesMap); + } catch (Exception ex) { + throw new HBaseIOException("Encountered exception while doing favored-nodes assignment " + ex + + " Falling back to regular assignment", ex); + } + return assignmentMap; + } + + private Pair>, List> + segregateRegionsAndAssignRegionsWithFavoredNodes(List regions, + List availableServers) { + Map> assignmentMapForFavoredNodes = + new HashMap>(regions.size() / 2); + List regionsWithNoFavoredNodes = new ArrayList(regions.size()/2); + for (HRegionInfo region : regions) { + if (region.getTable().isSystemTable()) { + try { + ServerName destination = super.randomAssignment(region, availableServers); + addRegionToMap(assignmentMapForFavoredNodes, region, destination); + } catch (HBaseIOException e) { + LOG.error("Failed to assign region: " + region.getRegionNameAsString(), e); + } + } else { + List favoredNodes = fnm.getFavoredNodes(region); + ServerName primaryHost = null; + ServerName secondaryHost = null; + ServerName tertiaryHost = null; + if (favoredNodes != null) { + for (ServerName s : favoredNodes) { + ServerName serverWithLegitStartCode = availableServersContains(availableServers, s); + if (serverWithLegitStartCode != null) { + FavoredNodesPlan.Position position = FavoredNodesPlan.getFavoredServerPosition( + favoredNodes, s); + if (Position.PRIMARY.equals(position)) { + primaryHost = serverWithLegitStartCode; + } else if (Position.SECONDARY.equals(position)) { + secondaryHost = serverWithLegitStartCode; + } else if (Position.TERTIARY.equals(position)) { + tertiaryHost = serverWithLegitStartCode; + } + } + } + assignRegionToAvailableFavoredNode(assignmentMapForFavoredNodes, region, primaryHost, + secondaryHost, tertiaryHost); + } else { + regionsWithNoFavoredNodes.add(region); + } + } + } + return new Pair>, List>( + assignmentMapForFavoredNodes, regionsWithNoFavoredNodes); + } + + private void addRegionToMap(Map> assignmentMapForFavoredNodes, + HRegionInfo region, ServerName host) { + List regionsOnServer = null; + if ((regionsOnServer = assignmentMapForFavoredNodes.get(host)) == null) { + regionsOnServer = new ArrayList(); + assignmentMapForFavoredNodes.put(host, regionsOnServer); + } + regionsOnServer.add(region); + } + + // Do a check of the hostname and port and return the servername from the servers list + // that matched (the favoredNode will have a startcode of -1 but we want the real + // server with the legit startcode + private ServerName availableServersContains(List servers, ServerName favoredNode) { + for (ServerName server : servers) { + if (ServerName.isSameHostnameAndPort(favoredNode, server)) { + return server; + } + } + return null; + } + + private void assignRegionToAvailableFavoredNode(Map> assignmentMapForFavoredNodes, HRegionInfo region, ServerName primaryHost, + ServerName secondaryHost, ServerName tertiaryHost) { + if (primaryHost != null) { + addRegionToMap(assignmentMapForFavoredNodes, region, primaryHost); + } else if (secondaryHost != null && tertiaryHost != null) { + // assign the region to the one with a lower load + // (both have the desired hdfs blocks) + ServerName s; + ServerLoad tertiaryLoad = super.services.getServerManager().getLoad(tertiaryHost); + ServerLoad secondaryLoad = super.services.getServerManager().getLoad(secondaryHost); + if (secondaryLoad != null && tertiaryLoad != null) { + if (secondaryLoad.getLoad() < tertiaryLoad.getLoad()) { + s = secondaryHost; + } else { + s = tertiaryHost; + } + } else { + if (this.RANDOM.nextBoolean()) { + s = secondaryHost; + } else { + s = tertiaryHost; + } + } + addRegionToMap(assignmentMapForFavoredNodes, region, s); + } else if (secondaryHost != null) { + addRegionToMap(assignmentMapForFavoredNodes, region, secondaryHost); + } else if (tertiaryHost != null) { + addRegionToMap(assignmentMapForFavoredNodes, region, tertiaryHost); + } + } + + @Override + public ServerName randomAssignment(HRegionInfo regionInfo, List servers) throws HBaseIOException { + ServerName destination = null; + // TODO: Decide strategy for assigning system tables. + if (regionInfo.getTable().isSystemTable()) { + destination = super.randomAssignment(regionInfo, servers); + return destination; + } + // We don't use FavoredNodeBalancer's random assignment, as that method generates + // new favored nodes if none of the favored nodes are online. We want to return null + // in that case. + List favoredNodes = fnm.getFavoredNodes(regionInfo); + if (favoredNodes == null || favoredNodes.isEmpty()) { + // Generate new favored nodes and return primary, don't use FavoredNodeBalancer + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, getConf()); + helper.initialize(); + try { + List newFavoredNodes = helper.generateFavoredNodes(regionInfo); + fnm.updateFavoredNodes(regionInfo, newFavoredNodes); + List onlineServers = getOnlineFavoredNodes(servers, newFavoredNodes); + destination = onlineServers.get(RANDOM.nextInt(onlineServers.size())); + } catch (IOException e) { + LOG.warn("Encountered exception while doing favored-nodes (random)assignment " + e + + " Falling back to regular assignment"); + //return super.randomAssignment(regionInfo, servers); + throw new HBaseIOException(e); + } + } else { + List onlineServers = getOnlineFavoredNodes(servers, favoredNodes); + if (onlineServers.size() > 0) { + destination = onlineServers.get(RANDOM.nextInt(onlineServers.size())); + } + } + boolean alwaysAssign = getConf().getBoolean(ALWAYS_ASSIGN_REGIONS, true); + if (destination == null && alwaysAssign) { + destination = super.randomAssignment(regionInfo, servers); + } + return destination; + } + + @Override + public Map> retainAssignment(Map regions, + List servers) throws HBaseIOException { + Map> result = super.retainAssignment(regions, servers); + + // Lets check if favored nodes info is in META, if not generate now. + FavoredNodeAssignmentHelper assignmentHelper = new FavoredNodeAssignmentHelper(servers, + new RackManager(getConf())); + assignmentHelper.initialize(); + LOG.debug("Generating favored nodes for regions missing them."); + for (Entry> entry : result.entrySet()) { + Map primaryRSMap = new HashMap(); + ServerName current = ServerName.valueOf(entry.getKey().getHostAndPort(), + ServerName.NON_STARTCODE); + try { + for (HRegionInfo region : entry.getValue()) { + List favoredNodes = fnm.getFavoredNodes(region); + if (!region.getTable().isSystemTable()) { + if (favoredNodes == null + || favoredNodes.size() < FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) { + primaryRSMap.put(region, current); + LOG.debug("Generating favored nodes for region " + region); + } + } + } + Map secondaryAndTertiaryRSMap = assignmentHelper + .placeSecondaryAndTertiaryRS(primaryRSMap); + for (HRegionInfo hri : primaryRSMap.keySet()) { + ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(hri); + if (secondaryAndTertiaryNodes != null) { + List newFavoredNodes = Lists.newArrayList(); + newFavoredNodes.add(primaryRSMap.get(hri)); + newFavoredNodes.add(ServerName.valueOf(secondaryAndTertiaryNodes[0].getHostname(), + secondaryAndTertiaryNodes[0].getPort(), ServerName.NON_STARTCODE)); + newFavoredNodes.add(ServerName.valueOf(secondaryAndTertiaryNodes[1].getHostname(), + secondaryAndTertiaryNodes[1].getPort(), ServerName.NON_STARTCODE)); + fnm.updateFavoredNodes(hri, newFavoredNodes); + } else { + throw new HBaseIOException("Favored nodes not updated for region " + hri); + } + } + } catch (Exception ex) { + throw new HBaseIOException("Encountered exception while generating favored nodes. ", ex); + } + } + return result; + } + + @Override + public Map> redistribute( + Map> clusterState) throws IOException { + long startTime = System.currentTimeMillis(); + boolean redistributeOnSameRack = getConf().getBoolean( + FavoredNodeLoadBalancer.REDISTRIBUTE_ON_SAME_RACK, true); + NavigableMap> replicaLoadAndCandidateRegionMap + = new TreeMap>(); + List serversForHelper = Lists.newArrayList(); + for (ServerName sn : clusterState.keySet()) { + ServerName temp = ServerName.valueOf(sn.getHostAndPort(), ServerName.NON_STARTCODE); + serversForHelper.add(temp); + } + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(serversForHelper, + getConf()); + helper.initialize(); + boolean multiRack = helper.getTotalNumberOfRacks() > 1 ? true : false; + if (!redistributeOnSameRack && !multiRack) { + throw new IOException( + "All the region servers belong to only one rack. Exiting redistribute."); + } + Map> replicaLoad = fnm.getReplicaLoad(serversForHelper); + Map serverLoadLookUp = new HashMap(); + for (Entry> entry : replicaLoad.entrySet()) { + int tempLoad = 0; + for (Integer load : entry.getValue()) { + tempLoad = tempLoad + load; + } + serverLoadLookUp.put(entry.getKey(), new ServerAndLoad(entry.getKey(), tempLoad)); + List empty = Lists.newArrayList(); + replicaLoadAndCandidateRegionMap.put(new ServerAndLoad(entry.getKey(), tempLoad), empty); + } + for (Entry> entry : clusterState.entrySet()) { + ServerName currentServer = entry.getKey(); + for (HRegionInfo hri : entry.getValue()) { + try { + if (!hri.getTable().isSystemTable()) { + List favNodes = fnm.getFavoredNodes(hri); + ServerName fnToBeReplaced = applyConstraints(currentServer, hri, favNodes, multiRack); + if (fnToBeReplaced != null) { + ServerAndLoad key = serverLoadLookUp.get(fnToBeReplaced); + // key would be null if the server is dead. + if (key != null) { + replicaLoadAndCandidateRegionMap.get(key).add(hri); + } + } + } + } catch (Exception e) { + LOG.error("Ignore, Could not redistribute region: " + hri.getRegionNameAsString(), e); + } + } + } + Map> newFavoredNodes = distributeReplicas(replicaLoadAndCandidateRegionMap, helper); + long endTime = System.currentTimeMillis(); + LOG.debug("Finished computing new load distribute plan. Computation took " + + (endTime - startTime) + ".Found a solution that moves favored nodes of " + + newFavoredNodes.size() + " regions."); + return newFavoredNodes; + } + + + @Override + public List completeRedistribute( + Map> clusterState) throws IOException { + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper( + Lists.newArrayList(clusterState.keySet()), getConf()); + helper.initialize(); + Map assignmentMap = new HashMap(); + Map> newFavoredNodes = + new HashMap>(); + List regionsToRR = Lists.newArrayList(); + for (Entry> entry : clusterState.entrySet()) { + ServerName current = entry.getKey(); + for( HRegionInfo hri : entry.getValue()) { + if(!hri.getTable().isSystemTable()){ + regionsToRR.add(hri); + assignmentMap.put(hri.getRegionNameAsString(), new RegionPlan(hri, current, null)); + } + } + } + + if (helper.getTotalNumberOfRacks() == 1) { + newFavoredNodes = helper.generateRRPrimaryAndSecondary(regionsToRR, + Lists.newArrayList(clusterState.keySet())); + } else { + Map primaryRSMap = new HashMap(); + helper.placePrimaryRSAsRoundRobin(null, primaryRSMap, regionsToRR); + Map> regionsByRack = new HashMap>(); + for (Entry entry : primaryRSMap.entrySet()) { + String rack = helper.getRackOfServer(entry.getValue()); + if (regionsByRack.containsKey(rack)){ + regionsByRack.get(rack).put(entry.getKey(), entry.getValue()); + } else { + Map map = Maps.newHashMap(); + map.put(entry.getKey(), entry.getValue()); + regionsByRack.put(rack, map); + } + } + for (String rack : helper.getRackToRegionServerMap().keySet()) { + Map> partialResult = helper.generateRRSecondaryAndTertiary( + regionsByRack.get(rack), rack); + newFavoredNodes.putAll(partialResult); + } + } + + for (HRegionInfo hri : newFavoredNodes.keySet()) { + ServerName primary = newFavoredNodes.get(hri).get(0); + RegionPlan rp = assignmentMap.get(hri.getRegionNameAsString()); + rp.setDestination(primary); + assignmentMap.put(hri.getRegionNameAsString(), rp); + } + + for (HRegionInfo hri : newFavoredNodes.keySet()) { + fnm.updateFavoredNodes(hri, newFavoredNodes.get(hri)); + } + return Lists.newArrayList(assignmentMap.values()); + } + + /** + * This method applies constraints to the three replicas of a region and + * chooses candidate replica to move. The constraints are as follows: + * 1. The replica on the current assignment cannot be a candidate. + * 2. In the single rack case, since one the favored nodes is a bogus server, + * the remaining replica becomes the candidate. + * 3. In the multi rack case, we check if the remaining two replica's are above + * threshold locality. If not, then we do not choose any replica for redistribute. + * If both the replicas are above threshold locality, then we choose random server + * as candidate. + * @param currentServer -- The current region server of the region + * @param hri instance of HRegionInfo + * @param favoredNodes nodes of the region + * @param multiRack indicating if the region servers belong to atleast two racks. + * @return The servername of the candidate replica which can be moved via redistribute. + * @throws IOException Signals that an I/O exception has occurred. + */ + private ServerName applyConstraints(ServerName currentServer, HRegionInfo hri, + List favoredNodes, boolean multiRack) throws IOException { + List result = Lists.newArrayList(favoredNodes); + ServerName currentAssignment = null; + for (ServerName rp : result) { + if (ServerName.isSameHostnameAndPort(currentServer, rp)) { + currentAssignment = rp; + break; + } + } + if (currentAssignment != null) { + result.remove(currentAssignment); + } + + float threshold = getConf().getFloat(FavoredNodeLoadBalancer.MIN_LOCALITY_FOR_REDISTRIBUTE, + 0.8f); + // We will remove the server with high locality and let the server with lower + // locality be moved. + ArrayListMultimap localityMap = ArrayListMultimap.create(); + for (ServerName rp : result) { + float locality = getLocalityOfRegion(hri, rp); + if (locality >= threshold) { + localityMap.put(Float.valueOf(locality), rp); + } + } + if (localityMap.size() == 0) { + // no servers with locality > threshold. Do not move any replica. + result.clear(); + } else if (localityMap.size() == 1) { + result.remove(localityMap.values().iterator().next()); + } else if (localityMap.size() == 2) { + Random rand = new Random(); + ServerName toBeRemoved = result.get(rand.nextInt(result.size())); + result.remove(toBeRemoved); + } + if (result.size() == 1) { + return result.get(0); + } else { + return null; + } + } + + float getLocalityOfRegion(HRegionInfo region, ServerName sn) throws IOException { + HTableDescriptor desc = this.services.getTableDescriptors().get(region.getTable()); + HDFSBlocksDistribution localityDistribution = HRegion.computeHDFSBlocksDistribution(getConf(), + desc, region); + return localityDistribution.getBlockLocalityIndex(sn.getHostname()); + } + + Map> groupRegionPlanByRegionInfo(List outcomeOfBalance) { + Map> result = new HashMap>(); + for (RegionPlan plan : outcomeOfBalance) { + List list = result.get(plan.getRegionInfo()); + if (list == null) { + list = new ArrayList(); + } + list.add(plan); + result.put(plan.getRegionInfo(), list); + } + return result; + } + + /** + * Distribute replicas from overloaded servers to underloaded servers adhering to + * the favored nodes constraint ( 3 favored nodes spread on 2 racks). + * + * @param replicaLoadAndCandidateMap Map of current replica load and candidate regions that could be potentially moved + * @param helper of FavoredNodeAssignmentHelper + * @return Map of region to list of new favored nodes. + */ + Map> distributeReplicas( + NavigableMap> replicaLoadAndCandidateMap, FavoredNodeAssignmentHelper helper) { + int numServers = replicaLoadAndCandidateMap.keySet().size(); + int numReplicas = 0; + for (ServerAndLoad sal : replicaLoadAndCandidateMap.keySet()) { + numReplicas = numReplicas + sal.getLoad(); + } + int min = numReplicas / numServers; + int max = numReplicas % numServers == 0 ? min : min + 1; + Map> newFavoredNodes = new HashMap>(); + + LOG.debug("Replica distribute parameter: numReplicas = " + numReplicas + ", numServers = " + + numServers + ", max = " + max + ", min = " + min); + + NavigableMap> overLoadedServers = new TreeMap>(); + Map underLoadedServers = new HashMap(); + for (Map.Entry> server : replicaLoadAndCandidateMap.descendingMap() + .entrySet()) { + ServerAndLoad sal = server.getKey(); + int replicaCount = sal.getLoad(); + if (replicaCount > max) { + int numToOffload = replicaCount - max; + overLoadedServers.put(new ServerAndLoad(sal.getServerName(), numToOffload), server.getValue()); + } else if (replicaCount < min) { + int regionsToPut = min - replicaCount; + underLoadedServers.put(sal.getServerName(), new AtomicInteger(regionsToPut)); + } + } + // Lets get some randomness + List shuffledUnderloadedServers = Lists.newArrayList(underLoadedServers.keySet()); + Collections.shuffle(shuffledUnderloadedServers); + Iterator underloadedServersItr = Iterables.cycle(shuffledUnderloadedServers).iterator(); + int underloadedServers = underLoadedServers.keySet().size(); + // Iterate on overloaded servers + for (Entry> entry : overLoadedServers.descendingMap().entrySet()) { + ServerName overloadedServer = entry.getKey().getServerName(); + int numToOffload = entry.getKey().getLoad(); + Iterator regionItr = entry.getValue().iterator(); + while (regionItr.hasNext() && (numToOffload > 0)) { + HRegionInfo hri = regionItr.next(); + int serversToVisit = underloadedServers; + while (underloadedServersItr.hasNext() && serversToVisit > 0) { + ServerName ulServer = underloadedServersItr.next(); + AtomicInteger regionsToTake = underLoadedServers.get(ulServer); + try { + if (regionsToTake.get() > 0) { + List favoredNodes = fnm.getFavoredNodes(hri); + List possibleFNs = + helper.replaceFavoredNode(overloadedServer, ulServer, favoredNodes); + if (possibleFNs != null) { + regionsToTake.decrementAndGet(); + numToOffload--; + newFavoredNodes.put(hri, possibleFNs); + break; + } + } + } catch (Exception e) { + LOG.error("Ignore, exception trying to distribute regions in server: " + ulServer, e); + } + serversToVisit--; + } + } + } + return newFavoredNodes; + } + + @Override + public Map> checkFavoredNodes(List servers, + List regions) { + Map> result = new HashMap>(); + Set onlineServers = + FavoredNodeAssignmentHelper.convertToStartCodeAgnosticSN(servers); + for (HRegionInfo hri : regions) { + if (hri.getTable().isSystemTable()) continue; + List favNodes = fnm.getFavoredNodes(hri); + if (favNodes != null) { + for (ServerName sn : favNodes) { + if (!onlineServers.contains(StartcodeAgnosticServerName.valueOf(sn))) { + if (result.containsKey(sn)) { + result.get(sn).add(hri.getRegionNameAsString()); + } else { + result.put(sn, Lists.newArrayList(hri.getRegionNameAsString())); + } + } + } + } else { + LOG.warn("No favored nodes found for " + hri.getRegionNameAsString()); + } + } + return result; + } + + public List getOnlineFavoredNodes(List onlineServers, + List serversWithoutStartCodes) { + if (serversWithoutStartCodes == null) { + return null; + } else { + List result = Lists.newArrayList(); + for (ServerName sn : serversWithoutStartCodes) { + for (ServerName online : onlineServers) { + if (ServerName.isSameHostnameAndPort(sn, online)) { + result.add(online); + } + } + } + return result; + } + } + + /* + * This should only be called for system tables. + */ + @Override + public Map> generateFavoredNodesForDaughter( + List servers, HRegionInfo parent, HRegionInfo hri_a, HRegionInfo hri_b) + throws IOException { + Map> result = new HashMap>(); + FavoredNodeAssignmentHelper assignmentHelper = new FavoredNodeAssignmentHelper(servers, + fnm.getRackManager()); + assignmentHelper.initialize(); + List parentFavoredNodes = fnm.getFavoredNodes(parent); + if (parentFavoredNodes == null) { + LOG.debug("Unable to find favored nodes for parent, " + parent + + " generating new favored nodes for daughter"); + result.put(hri_a, assignmentHelper.generateFavoredNodes(hri_a)); + result.put(hri_b, assignmentHelper.generateFavoredNodes(hri_b)); + } else { + Set existingFavNodes = Sets.newHashSet(); + existingFavNodes.add(parentFavoredNodes.get(0)); + existingFavNodes.add(parentFavoredNodes.get(1)); + while (existingFavNodes.size() < FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) { + ServerName newNode = assignmentHelper.generateMissingFavoredNode(Lists + .newArrayList(existingFavNodes)); + existingFavNodes.add(newNode); + } + result.put(hri_a, Lists.newArrayList(existingFavNodes)); + existingFavNodes.clear(); + existingFavNodes.add(parentFavoredNodes.get(0)); + existingFavNodes.add(parentFavoredNodes.get(2)); + while (existingFavNodes.size() < FavoredNodeAssignmentHelper.FAVORED_NODES_NUM) { + ServerName newNode = assignmentHelper.generateMissingFavoredNode(Lists + .newArrayList(existingFavNodes)); + existingFavNodes.add(newNode); + } + result.put(hri_b, Lists.newArrayList(existingFavNodes)); + } + fnm.updateFavoredNodes(hri_a, result.get(hri_a)); + fnm.updateFavoredNodes(hri_b, result.get(hri_b)); + return result; + } + + @Override + public void generateFavoredNodesForMergedRegion(HRegionInfo merged, HRegionInfo hriA, + HRegionInfo hriB) throws IOException { + //TODO probably should inherit the bigger region's FN + services.getFavoredNodesManager().updateFavoredNodes(merged, fnm.getFavoredNodes(hriA)); + } + + class FavoredNodeLocalityPicker extends CandidateGenerator { + + @Override + Cluster.Action generate(Cluster cluster) { + cluster.calculateRegionServerLocalities(); + // Pick lowest local region server + int thisServer = pickLowestLocalityServer(cluster); + int thisRegion; + if (thisServer == -1) { + LOG.trace("Could not pick lowest local region server"); + return Cluster.NullAction; + } else { + // Pick lowest local region on this server + thisRegion = pickLowestLocalRegionOnServer(cluster, thisServer); + } + if (thisRegion == -1) { + if (cluster.regionsPerServer[thisServer].length > 0) { + LOG.trace("Could not pick lowest local region even when region server held " + + cluster.regionsPerServer[thisServer].length + " regions"); + } + return Cluster.NullAction; + } + HRegionInfo hri = cluster.regions[thisRegion]; + List favoredNodes = fnm.getFavoredNodes(hri); + int otherServer; + if (favoredNodes == null) { + if (hri.getTable().isSystemTable()) { + otherServer = pickOtherRandomServer(cluster, thisServer); + } else { + return Cluster.NullAction; + } + } else { + // Pick other favored node with the highest locality + otherServer = getDifferentFavoredNode(cluster, favoredNodes, thisServer); + } + return getAction(thisServer, thisRegion, otherServer, -1); + } + + private int pickLowestLocalityServer(Cluster cluster) { + return cluster.getLowestLocalityRegionServer(); + } + + private int getDifferentFavoredNode(Cluster cluster, List favoredNodes, + int currentServer) { + List fnIndex = new ArrayList(); + for (ServerName sn : favoredNodes) { + if (cluster.serversToIndex.containsKey(sn.getHostAndPort())) { + fnIndex.add(cluster.serversToIndex.get(sn.getHostAndPort())); + } + } + float locality = 0; + int highestLocalRSIndex = -1; + for (Integer index : fnIndex) { + if (index != currentServer) { + float temp = cluster.localityPerServer[index]; + if (temp >= locality) { + locality = temp; + highestLocalRSIndex = index; + } + } + } + return highestLocalRSIndex; + } + + private int pickLowestLocalRegionOnServer(Cluster cluster, int server) { + return cluster.getLowestLocalityRegionOnServer(server); + } + } + + class FavoredNodeLoadPicker extends CandidateGenerator { + + @Override + Cluster.Action generate(Cluster cluster) { + cluster.sortServersByRegionCount(); + int thisServer = pickMostLoadedServer(cluster); + int thisRegion = pickRandomRegion(cluster, thisServer, 0); + HRegionInfo hri = cluster.regions[thisRegion]; + int otherServer; + List favoredNodes = fnm.getFavoredNodes(hri); + if (favoredNodes == null) { + otherServer = pickLeastLoadedServer(cluster, thisServer); + } else { + otherServer = pickLeastLoadedFNServer(cluster, favoredNodes, thisServer); + } + return getAction(thisServer, thisRegion, otherServer, -1); + } + + private int pickLeastLoadedServer(final Cluster cluster, int thisServer) { + Integer[] servers = cluster.serverIndicesSortedByRegionCount; + int index; + for (index = 0; index < servers.length ; index++) { + if ((servers[index] != null) && servers[index] != thisServer) { + break; + } + } + return servers[index]; + } + + private int pickLeastLoadedFNServer(final Cluster cluster, List favoredNodes, + int currentServerIndex) { + List fnIndex = new ArrayList(); + for (ServerName sn : favoredNodes) { + if (cluster.serversToIndex.containsKey(sn.getHostAndPort())) { + fnIndex.add(cluster.serversToIndex.get(sn.getHostAndPort())); + } + } + int leastLoadedFN = -1; + int load = Integer.MAX_VALUE; + for (Integer index : fnIndex) { + if (index != currentServerIndex) { + int temp = cluster.getNumRegions(index); + if (temp < load) { + load = temp; + leastLoadedFN = index; + } + } + } + return leastLoadedFN; + } + + /** + * Pick most loaded server. + * + * @param cluster + * @return index of the region server picked. + */ + private int pickMostLoadedServer(final Cluster cluster) { + Integer[] servers = cluster.serverIndicesSortedByRegionCount; + int index; + for (index = servers.length - 1; index > 0 ; index--) { + if (servers[index] != null) { + break; + } + } + return servers[index]; + } + } + + @Override + public List balanceCluster(Map> clusterState) { + if (this.services != null) { + List regionPlans = Lists.newArrayList(); + Map> correctAssignments = new HashMap>(); + int misplacedRegions = 0; + for (Entry> entry : clusterState.entrySet()) { + ServerName current = entry.getKey(); + List regions = Lists.newArrayList(); + correctAssignments.put(current, regions); + for (HRegionInfo hri : entry.getValue()) { + List favoredNodes = fnm.getFavoredNodes(hri); + //TODO: we might need this lookup to be o(1) + if (FavoredNodesPlan.getFavoredServerPosition(favoredNodes, current) != null + || hri.getTable().isSystemTable()) { + correctAssignments.get(current).add(hri); + } else { + RegionPlan rp = new RegionPlan(hri, current, BOGUS_SERVER_NAME); + regionPlans.add(rp); + misplacedRegions++; + } + } + } + LOG.debug("Found " + misplacedRegions + " number of regions not on favored nodes."); + List regionPlansFromBalance = super.balanceCluster(correctAssignments); + if (regionPlansFromBalance != null) { + regionPlans.addAll(regionPlansFromBalance); + } + return regionPlans; + } else { + return super.balanceCluster(clusterState); + } + } +} + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java index cceaf87..6e495b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/ServerAndLoad.java @@ -26,12 +26,12 @@ import org.apache.hadoop.hbase.ServerName; * Data structure that holds servername and 'load'. */ @InterfaceAudience.Private -class ServerAndLoad implements Comparable, Serializable { +public class ServerAndLoad implements Comparable, Serializable { private static final long serialVersionUID = 2735470854607296965L; private final ServerName sn; private final int load; - ServerAndLoad(final ServerName sn, final int load) { + public ServerAndLoad(final ServerName sn, final int load) { this.sn = sn; this.load = load; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java index 345fbf3..efebc99 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionLoad; @@ -51,6 +52,8 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegi import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import com.google.common.collect.Lists; + /** *

This is a best effort load balancer. Given a Cost function F(C) => x It will * randomly try and mutate the cluster to Cprime. If F(Cprime) < F(C) then the @@ -107,11 +110,13 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { "hbase.master.balancer.stochastic.maxRunningTime"; protected static final String KEEP_REGION_LOADS = "hbase.master.balancer.stochastic.numRegionLoadsToRemember"; + private static final String RUN_MAXIMUM_STEPS = + "hbase.master.balancer.stochastic.execute.maxSteps"; private static final String TABLE_FUNCTION_SEP = "_"; protected static final String MIN_COST_NEED_BALANCE_KEY = "hbase.master.balancer.stochastic.minCostNeedBalance"; - private static final Random RANDOM = new Random(System.currentTimeMillis()); + protected static final Random RANDOM = new Random(System.currentTimeMillis()); private static final Log LOG = LogFactory.getLog(StochasticLoadBalancer.class); Map> loads = new HashMap>(); @@ -123,7 +128,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { private int numRegionLoadsToRemember = 15; private float minCostNeedBalance = 0.05f; - private CandidateGenerator[] candidateGenerators; + private List candidateGenerators; private CostFromRegionLoadFunction[] regionLoadFunctions; private CostFunction[] costFunctions; // FindBugs: Wants this protected; IS2_INCONSISTENT_SYNC @@ -136,6 +141,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { // when new services are offered private LocalityBasedCandidateGenerator localityCandidateGenerator; private LocalityCostFunction localityCost; + private boolean executeMaximumSteps; private RegionReplicaHostCostFunction regionReplicaHostCostFunction; private RegionReplicaRackCostFunction regionReplicaRackCostFunction; private boolean isByTable = false; @@ -163,7 +169,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { stepsPerRegion = conf.getInt(STEPS_PER_REGION_KEY, stepsPerRegion); maxRunningTime = conf.getLong(MAX_RUNNING_TIME_KEY, maxRunningTime); - + executeMaximumSteps = conf.getBoolean(RUN_MAXIMUM_STEPS, false); numRegionLoadsToRemember = conf.getInt(KEEP_REGION_LOADS, numRegionLoadsToRemember); isByTable = conf.getBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, isByTable); @@ -174,13 +180,12 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { } localityCost = new LocalityCostFunction(conf, services); - if (candidateGenerators == null) { - candidateGenerators = new CandidateGenerator[] { - new RandomCandidateGenerator(), - new LoadCandidateGenerator(), - localityCandidateGenerator, - new RegionReplicaRackCandidateGenerator(), - }; + if (this.candidateGenerators == null) { + candidateGenerators = Lists.newArrayList(); + candidateGenerators.add(new RandomCandidateGenerator()); + candidateGenerators.add(new LoadCandidateGenerator()); + candidateGenerators.add(localityCandidateGenerator); + candidateGenerators.add(new RegionReplicaRackCandidateGenerator()); } regionLoadFunctions = new CostFromRegionLoadFunction[] { @@ -213,6 +218,11 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { } @Override + public void initialize() throws HBaseIOException { + super.initialize(); + } + + @Override protected void setSlop(Configuration conf) { this.slop = conf.getFloat("hbase.regions.slop", 0.001F); } @@ -254,6 +264,10 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { } + public void setCandidateGenerators(List customCandidateGenerators) { + this.candidateGenerators = customCandidateGenerators; + } + @Override protected synchronized boolean areSomeRegionReplicasColocated(Cluster c) { regionReplicaHostCostFunction.init(c); @@ -364,8 +378,12 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { long step; for (step = 0; step < computedMaxSteps; step++) { - int generatorIdx = RANDOM.nextInt(candidateGenerators.length); - CandidateGenerator p = candidateGenerators[generatorIdx]; + if (EnvironmentEdgeManager.currentTime() - startTime > maxRunningTime) { + break; + } + + int generatorIdx = RANDOM.nextInt(candidateGenerators.size()); + CandidateGenerator p = candidateGenerators.get(generatorIdx); Cluster.Action action = p.generate(cluster); if (action.type == Type.NULL) { @@ -393,11 +411,6 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { cluster.doAction(undoAction); updateCostsWithAction(cluster, undoAction); } - - if (EnvironmentEdgeManager.currentTime() - startTime > - maxRunningTime) { - break; - } } long endTime = EnvironmentEdgeManager.currentTime(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index ca44902..61a0566 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.exceptions.HBaseException; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; @@ -365,6 +366,12 @@ public class DeleteTableProcedure // clean region references from the server manager env.getMasterServices().getServerManager().removeRegions(regions); + + // Clear Favored Nodes for this table + FavoredNodesManager fnm = env.getMasterServices().getFavoredNodesManager(); + if (fnm != null) { + fnm.deleteFavoredNodesForRegion(regions); + } } protected static void deleteAssignmentState(final MasterProcedureEnv env, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 8bea130..fdb544a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -179,8 +179,11 @@ import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.metrics2.util.MBeans; +import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.zookeeper.KeeperException; @@ -479,6 +482,7 @@ public class HRegionServer extends HasThread implements final ServerNonceManager nonceManager; private UserProvider userProvider; + private int defaultDatanodeDataTransferPort; protected final RSRpcServices rpcServices; @@ -621,6 +625,7 @@ public class HRegionServer extends HasThread implements rpcServices.start(); putUpWebUI(); + setupDatanodePort(conf); this.walRoller = new LogRoller(this, this); this.choreService = new ChoreService(getServerName().toString(), true); this.flushThroughputController = FlushThroughputControllerFactory.create(this, conf); @@ -646,6 +651,18 @@ public class HRegionServer extends HasThread implements choreService.scheduleChore(compactedFileDischarger); } + private void setupDatanodePort(Configuration conf) { + HdfsConfiguration.init(); + + Configuration dnConf = new HdfsConfiguration(conf); + this.defaultDatanodeDataTransferPort = NetUtils.createSocketAddr( + dnConf.get(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, + DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT)).getPort(); + if (LOG.isDebugEnabled()) { + LOG.debug("Loaded default datanode port for FN: " + defaultDatanodeDataTransferPort); + } + } + protected TableDescriptors getFsTableDescriptors() throws IOException { return new FSTableDescriptors(this.conf, this.fs, this.rootDir, !canUpdateTableDescriptor(), false); @@ -3083,13 +3100,15 @@ public class HRegionServer extends HasThread implements @Override public void updateRegionFavoredNodesMapping(String encodedRegionName, List favoredNodes) { - InetSocketAddress[] addr = new InetSocketAddress[favoredNodes.size()]; + List favoredNodesList = new ArrayList(); // Refer to the comment on the declaration of regionFavoredNodesMap on why // it is a map of region name to InetSocketAddress[] for (int i = 0; i < favoredNodes.size(); i++) { - addr[i] = InetSocketAddress.createUnresolved(favoredNodes.get(i).getHostName(), - favoredNodes.get(i).getPort()); + favoredNodesList.add(InetSocketAddress.createUnresolved(favoredNodes.get(i).getHostName(), + this.defaultDatanodeDataTransferPort)); } + InetSocketAddress[] addr = new InetSocketAddress[favoredNodesList.size()]; + addr = favoredNodesList.toArray(addr); regionFavoredNodesMap.put(encodedRegionName, addr); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index d7b0d36..74489fc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -1024,9 +1025,10 @@ public class HStore implements Store { writerCacheConf = cacheConf; } InetSocketAddress[] favoredNodes = null; - if (region.getRegionServerServices() != null) { + if (region.getRegionServerServices() != null + && !region.getRegionInfo().getTable().isSystemTable()) { favoredNodes = region.getRegionServerServices().getFavoredNodesForRegion( - region.getRegionInfo().getEncodedName()); + region.getRegionInfo().getEncodedName()); } HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag, cryptoContext); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 09c7fb9..ce86ee4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -27,6 +27,7 @@ import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -1749,6 +1750,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler, regionServer.service.submit(new OpenMetaHandler( regionServer, regionServer, region, htd, masterSystemTime)); } else { + if (regionOpenInfo.getFavoredNodesList() != null) { + LOG.debug("Updating region server with favored nodes of region " + + region.getEncodedName() + " with favored nodes "); + for (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName sn + : regionOpenInfo.getFavoredNodesList()) { + LOG.debug(ProtobufUtil.toServerName(sn)); + } + } else { + LOG.debug("Favored nodes for region " + region.getEncodedName() + " found empty"); + } regionServer.updateRegionFavoredNodesMapping(region.getEncodedName(), regionOpenInfo.getFavoredNodesList()); if (htd.getPriority() >= HConstants.ADMIN_QOS || region.getTable().isSystemTable()) { @@ -2048,6 +2059,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, UpdateFavoredNodesResponse.Builder respBuilder = UpdateFavoredNodesResponse.newBuilder(); for (UpdateFavoredNodesRequest.RegionUpdateInfo regionUpdateInfo : openInfoList) { HRegionInfo hri = HRegionInfo.convert(regionUpdateInfo.getRegion()); + LOG.debug("Updating " + hri.getRegionNameAsString() + " favored nodes = " + + Arrays.toString(regionUpdateInfo.getFavoredNodesList().toArray())); regionServer.updateRegionFavoredNodesMapping(hri.getEncodedName(), regionUpdateInfo.getFavoredNodesList()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java index 5c86429..a583830 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.io.InterruptedIOException; +import java.net.InetSocketAddress; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; @@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl.LoggingProgressable; import org.apache.hadoop.hbase.security.User; @@ -47,6 +49,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; @InterfaceAudience.Private public class RegionMergeTransactionImpl implements RegionMergeTransaction { @@ -62,6 +65,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { // We only merge adjacent regions if forcible is false private final boolean forcible; private final long masterSystemTime; + private InetSocketAddress[] favNodes; /* * Transaction state for listener, only valid during execute and @@ -223,6 +227,9 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { this.mergedRegionInfo = getMergedRegionInfo(region_a.getRegionInfo(), region_b.getRegionInfo()); + if (services != null) { + favNodes = services.getFavoredNodesForRegion(region_a.getRegionInfo().getEncodedName()); + } transition(RegionMergeTransactionPhase.PREPARED); return true; } @@ -542,6 +549,16 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { + mergedRegionInfo.getShortNameToLog()); } services.addToOnlineRegions(merged); + List favoredNodes = Lists + .newArrayList(); + if (favNodes != null) { + for (InetSocketAddress addr : favNodes) { + favoredNodes.add(ProtobufUtil.toServerName(ServerName.valueOf(addr.getHostName(), + addr.getPort(), -1))); + } + services.updateRegionFavoredNodesMapping(merged.getRegionInfo().getEncodedName(), + favoredNodes); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java index 190de4e..f7b2e5f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -357,6 +357,29 @@ public class MiniHBaseCluster extends HBaseCluster { } /** + * Starts a region server thread on a specific port + * + * @param specficConf + * @throws IOException + * @return New RegionServerThread + */ + public JVMClusterUtil.RegionServerThread startRegionServer(Configuration specficConf) + throws IOException { + User rsUser = + HBaseTestingUtility.getDifferentUser(specficConf, ".hfs."+index++); + JVMClusterUtil.RegionServerThread t = null; + try { + t = hbaseCluster.addRegionServer( + specficConf, hbaseCluster.getRegionServers().size(), rsUser); + t.start(); + t.waitForServerOnline(); + } catch (InterruptedException ie) { + throw new IOException("Interrupted adding regionserver to cluster", ie); + } + return t; + } + + /** * Cause a region server to exit doing basic clean up only on its way out. * @param serverNumber Used as index into a list. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java index cde26e6..6bceec8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerName.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; @@ -62,10 +63,30 @@ public class TestServerName { assertTrue(Pattern.matches(Addressing.VALID_PORT_REGEX, "123")); assertFalse(Pattern.matches(Addressing.VALID_PORT_REGEX, "")); assertTrue(ServerName.SERVERNAME_PATTERN.matcher("www1.example.org,1234,567").matches()); - ServerName.parseServerName("a.b.c,58102,1319771740322"); - ServerName.parseServerName("192.168.1.199,58102,1319771740322"); - ServerName.parseServerName("a.b.c:58102"); - ServerName.parseServerName("192.168.1.199:58102"); + assertNotNull(ServerName.parseServerName("a.b.c,58102,1319771740322")); + assertNotNull(ServerName.parseServerName("192.168.1.199,58102,1319771740322")); + assertNotNull(ServerName.parseServerName("a.b.c:58102")); + assertNotNull(ServerName.parseServerName("192.168.1.199:58102")); + assertTrue(ServerName.SERVERNAME_PATTERN.matcher("192.168.1.199,58102,-1").matches()); + assertFalse(ServerName.SERVERNAME_PATTERN.matcher("192.168.1.199,58102,abc").matches()); + assertFalse(ServerName.SERVERNAME_PATTERN.matcher("192.168.1.199,58102,-455").matches()); + boolean exceptionCaught = false; + try { + ServerName.parseServerName("192.168.1.199,58102"); + } catch (IllegalArgumentException exp) { + exceptionCaught = true; + } finally { + assertTrue(exceptionCaught); + } + exceptionCaught = false; + try { + ServerName.parseServerName("192.168.1.199,-58102,10000"); + } catch (IllegalArgumentException exp) { + exceptionCaught = true; + } finally { + assertTrue(exceptionCaught); + } + } @Test public void testParseOfBytes() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index 4feebcc..d760519 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -576,7 +576,7 @@ public class TestZooKeeper { @Override public Map> retainAssignment( - Map regions, List servers) { + Map regions, List servers) throws HBaseIOException { retainAssignCalled = true; return super.retainAssignment(regions, servers); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java index 87fb169..2c33b91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.executor.ExecutorService; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; @@ -347,6 +348,10 @@ public class MockNoopMasterServices implements MasterServices, Server { return null; } + @Override public FavoredNodesManager getFavoredNodesManager() { + return null; + } + @Override public SnapshotManager getSnapshotManager() { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java index e473b75..17e8c91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java @@ -40,6 +40,7 @@ import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -1268,7 +1269,7 @@ public class TestAssignmentManagerOnCluster { @Override public ServerName randomAssignment(HRegionInfo regionInfo, - List servers) { + List servers) throws HBaseIOException { if (regionInfo.equals(controledRegion)) { return null; } @@ -1277,7 +1278,7 @@ public class TestAssignmentManagerOnCluster { @Override public Map> roundRobinAssignment( - List regions, List servers) { + List regions, List servers) throws HBaseIOException { if (countRegionServers != null && services != null) { int regionServers = services.getServerManager().countOfRegionServers(); if (regionServers < countRegionServers.intValue()) { @@ -1297,7 +1298,7 @@ public class TestAssignmentManagerOnCluster { @Override public Map> retainAssignment( - Map regions, List servers) { + Map regions, List servers) throws HBaseIOException { for (HRegionInfo hri : regions.keySet()) { if (hri.equals(controledRegion)) { Map> m = Maps.newHashMap(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java index 047cf0f..36d3d23 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java @@ -95,6 +95,7 @@ public class BalancerTestBase { conf.setFloat("hbase.master.balancer.stochastic.minCostNeedBalance", 0.0f); loadBalancer = new StochasticLoadBalancer(); loadBalancer.setConf(conf); + loadBalancer.initialize(); } protected int[] largeCluster = new int[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancerTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancerTest.java new file mode 100644 index 0000000..bcd0d62 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancerTest.java @@ -0,0 +1,39 @@ +/** + * 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.balancer; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +/** + * Used for tests + */ +public class FavoredStochasticBalancerTest extends FavoredStochasticBalancer { + + private static final Log LOG = LogFactory.getLog(FavoredStochasticBalancerTest.class); + + @Override + protected void configureGenerators() { + List fnPickers = new ArrayList(); + fnPickers.add(new FavoredNodeLoadPicker()); + setCandidateGenerators(fnPickers); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java index 522b072..15ecdb0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeAssignmentHelper.java @@ -18,13 +18,16 @@ package org.apache.hadoop.hbase.master.balancer; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -43,6 +46,9 @@ import org.junit.Ignore; import org.junit.experimental.categories.Category; import org.mockito.Mockito; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + @Category({MasterTests.class, SmallTests.class}) public class TestFavoredNodeAssignmentHelper { @@ -155,6 +161,8 @@ public class TestFavoredNodeAssignmentHelper { // primary/secondary/tertiary for any given region for (HRegionInfo region : regions) { ServerName[] secondaryAndTertiaryServers = secondaryAndTertiaryMap.get(region); + assertNotNull(secondaryAndTertiaryServers); + assertTrue(primaryRSMap.containsKey(region)); assertTrue(!secondaryAndTertiaryServers[0].equals(primaryRSMap.get(region))); assertTrue(!secondaryAndTertiaryServers[1].equals(primaryRSMap.get(region))); assertTrue(!secondaryAndTertiaryServers[0].equals(secondaryAndTertiaryServers[1])); @@ -203,8 +211,10 @@ public class TestFavoredNodeAssignmentHelper { String primaryRSRack = rackManager.getRack(primaryRSMap.get(entry.getKey())); String secondaryRSRack = rackManager.getRack(allServersForRegion[0]); String tertiaryRSRack = rackManager.getRack(allServersForRegion[1]); - assertTrue(!primaryRSRack.equals(secondaryRSRack)); - assertTrue(secondaryRSRack.equals(tertiaryRSRack)); + Set racks = Sets.newHashSet(primaryRSRack); + racks.add(secondaryRSRack); + racks.add(tertiaryRSRack); + assertTrue(racks.size() >= 2); } } @@ -247,18 +257,15 @@ public class TestFavoredNodeAssignmentHelper { assertTrue(primaryRSMap.size() == 6); Map secondaryAndTertiaryMap = helper.placeSecondaryAndTertiaryRS(primaryRSMap); + assertTrue(secondaryAndTertiaryMap.size() == regions.size()); for (HRegionInfo region : regions) { ServerName s = primaryRSMap.get(region); ServerName secondaryRS = secondaryAndTertiaryMap.get(region)[0]; ServerName tertiaryRS = secondaryAndTertiaryMap.get(region)[1]; - if (rackManager.getRack(s).equals("rack1")) { - assertTrue(rackManager.getRack(secondaryRS).equals("rack2") && - rackManager.getRack(tertiaryRS).equals("rack1")); - } - if (rackManager.getRack(s).equals("rack2")) { - assertTrue(rackManager.getRack(secondaryRS).equals("rack1") && - rackManager.getRack(tertiaryRS).equals("rack1")); - } + Set racks = Sets.newHashSet(rackManager.getRack(s)); + racks.add(rackManager.getRack(secondaryRS)); + racks.add(rackManager.getRack(tertiaryRS)); + assertTrue(racks.size() >= 2); } } @@ -361,4 +368,30 @@ public class TestFavoredNodeAssignmentHelper { + " " + thirdRackSize + " " + regionsOnRack1 + " " + regionsOnRack2 + " " + regionsOnRack3; } + + @Test + public void testConstrainedPlacement() throws Exception { + List servers = Lists.newArrayList(); + servers.add(ServerName.valueOf("foo" + 1 + ":1234", -1)); + servers.add(ServerName.valueOf("foo" + 2 + ":1234", -1)); + servers.add(ServerName.valueOf("foo" + 15 + ":1234", -1)); + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(servers, rackManager); + helper.initialize(); + assertTrue(helper.canPlaceFavoredNodes()); + + List regions = new ArrayList(20); + for (int i = 0; i < 20; i++) { + HRegionInfo region = new HRegionInfo(TableName.valueOf("foobar"), + Bytes.toBytes(i), Bytes.toBytes(i + 1)); + regions.add(region); + } + Map> assignmentMap = + new HashMap>(); + Map primaryRSMap = new HashMap(); + helper.placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions); + assertTrue(primaryRSMap.size() == regions.size()); + Map secondaryAndTertiary = + helper.placeSecondaryAndTertiaryRS(primaryRSMap); + assertEquals(regions.size(), secondaryAndTertiary.size()); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeLoadBalancer.java new file mode 100644 index 0000000..bb2a4a8 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeLoadBalancer.java @@ -0,0 +1,131 @@ +/** + * 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.balancer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.util.ArrayList; +import java.util.HashMap; +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.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.ServerLoad; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.master.DeadServer; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.LoadBalancer; +import org.apache.hadoop.hbase.master.ServerManager; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.RegionSplitter.HexStringSplit; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; + +import com.google.common.collect.Lists; + + +@Category(MediumTests.class) +public class TestFavoredNodeLoadBalancer { + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + protected static Configuration conf; + private static final ServerName SERVERNAME_A = + ServerName.valueOf("example.org1", 1234, 5678); + private static final ServerName SERVERNAME_B = + ServerName.valueOf("example.org2", 0, 5678); + private static final ServerName SERVERNAME_C = + ServerName.valueOf("example.org3", 6789, 6000); + + @BeforeClass + public static void setupBeforeClass() throws Exception { + setUpOnce(); + } + + static void setUpOnce() throws Exception { + conf = TEST_UTIL.getConfiguration(); + // Enable the favored nodes based load balancer + conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, + FavoredNodeLoadBalancer.class, LoadBalancer.class); + conf.setInt("hbase.assignment.maximum.attempts", 3); + } + + @Test + public void testWhenRSDoesNotReportLoad() throws Exception { + ServerManager serverManager = Mockito.mock(ServerManager.class); + Mockito.when(serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true); + Mockito.when(serverManager.isServerOnline(SERVERNAME_B)).thenReturn(true); + Mockito.when(serverManager.isServerOnline(SERVERNAME_C)).thenReturn(true); + Mockito.when(serverManager.getDeadServers()).thenReturn(new DeadServer()); + final List onlineServers = Lists.newArrayList(SERVERNAME_A, SERVERNAME_B, + SERVERNAME_C); + Mockito.when(serverManager.getOnlineServersList()).thenReturn( + new ArrayList(onlineServers)); + Mockito.when(serverManager.getOnlineServers()) + .thenReturn(new HashMap()); + Mockito.when(serverManager.getLoad(SERVERNAME_A)).thenReturn(null); + Mockito.when(serverManager.getLoad(SERVERNAME_B)).thenReturn(null); + Mockito.when(serverManager.getLoad(SERVERNAME_C)).thenReturn(null); + + HMaster master = Mockito.mock(HMaster.class); + Mockito.when(master.getServerManager()).thenReturn(serverManager); + Mockito.when(master.getServerName()).thenReturn(ServerName.valueOf("master,1,1")); + Mockito.when(master.getConfiguration()).thenReturn(conf); + // TODO: Check if anything else is required +// Mockito.when(master.getCatalogTracker()).thenReturn(null); + Mockito.when(master.getServerManager()).thenReturn(serverManager); + FavoredNodeLoadBalancer balancer = (FavoredNodeLoadBalancer) LoadBalancerFactory + .getLoadBalancer(conf); + balancer.setConf(conf); + balancer.setMasterServices(master); + balancer.initialize(); + List regions = getRegionInfos(TableName.valueOf("testRR"), 5); + for (HRegionInfo hri : regions) { + balancer.updateFavoredNodesMap(hri, onlineServers); + } + //Pass only secondary and tertiary servers. + Map> roundRobinAssignment = balancer.roundRobinAssignment( + regions, Lists.newArrayList(SERVERNAME_B, SERVERNAME_C)); + assertNotNull(roundRobinAssignment); + List assigned = Lists.newArrayList(); + for (List hris : roundRobinAssignment.values()) { + assigned.addAll(hris); + } + assertEquals(5, assigned.size()); + } + + List getRegionInfos(TableName table, int count) { + HexStringSplit splitter = new HexStringSplit(); + byte[][] splitKeys = splitter.split(count); + int numRegions = splitKeys.length + 1; + List hRegionInfos = new ArrayList(numRegions); + byte[] startKey = null; + byte[] endKey = null; + for (int i = 0; i < numRegions; i++) { + endKey = (i == splitKeys.length) ? null : splitKeys[i]; + hRegionInfos.add(new HRegionInfo(table, startKey, endKey)); + startKey = endKey; + } + return hRegionInfos; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodesRepairChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodesRepairChore.java new file mode 100644 index 0000000..f7e38f6 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodesRepairChore.java @@ -0,0 +1,167 @@ +/** + * 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.balancer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CoordinatedStateManager; +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.HTableDescriptor; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; +import org.apache.hadoop.hbase.master.HMasterCommandLine.LocalHMaster; +import org.apache.hadoop.hbase.master.LoadBalancer; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.net.ScriptBasedMapping; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(MediumTests.class) +public class TestFavoredNodesRepairChore { + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final int SLAVES = 6; + private static HBaseAdmin admin; + private int REGION_NUM = 10; + protected static Configuration conf; + + @BeforeClass + public static void setUpOnce() throws Exception { + conf = TEST_UTIL.getConfiguration(); + conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, + FavoredStochasticBalancerTest.class, LoadBalancer.class); + conf.setBoolean(FavoredNodesPromoter.ALWAYS_ASSIGN_REGIONS, false); + //Don't let chore run, we will run manually when needed. + conf.setInt(FavoredNodesRepairChore.FAVORED_NODE_REPAIR_CHORE_FREQ, Integer.MAX_VALUE); + conf.set(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + ScriptBasedMapping.class.getName()); + conf.setInt("hbase.assignment.maximum.attempts", 3); + } + + @Before + public void startCluster() throws Exception { + TEST_UTIL.startMiniCluster(1, SLAVES, SLAVES, null, LocalHMaster.class, FNRegionServer.class); + TEST_UTIL.getDFSCluster().waitClusterUp(); + admin = TEST_UTIL.getHBaseAdmin(); + admin.setBalancerRunning(false, true); + } + + @After + public void stopCluster() throws Exception { + TEST_UTIL.cleanupTestDir(); + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testFavoredNodesInfoSync() throws Exception { + final TableName tableName = TableName.valueOf("testFavoredNodesInfoSync"); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + // TODO: Changed time to 60s from 10s. + TEST_UTIL.waitFor(60000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return admin.isTableAvailable(tableName); + } + }); + List regionsOfTable = admin.getTableRegions(tableName); + assertEquals(REGION_NUM, regionsOfTable.size()); + final HRegionInfo candidate = regionsOfTable.get(0); + FavoredNodesManager fnm = TEST_UTIL.getMiniHBaseCluster().getMaster().getFavoredNodesManager(); + List favNodes = Lists.newArrayList(fnm.getFavoredNodes(candidate)); + final ServerName currentRS = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() + .getRegionStates().getRegionServerOfRegion(candidate); + assertTrue("Current rs not part of favored nodes", + favNodes.remove(ServerName.valueOf(currentRS.getHostAndPort(), -1))); + List onlineServers = Lists.newArrayList(admin.getClusterStatus().getServers()); + assertTrue(onlineServers.remove(currentRS)); + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(onlineServers, + TEST_UTIL.getConfiguration()); + helper.initialize(); + favNodes.add(ServerName.valueOf(helper.generateMissingFavoredNode(favNodes).getHostAndPort(), + ServerName.NON_STARTCODE)); + assertEquals(3, favNodes.size()); + fnm.updateFavoredNodes(candidate, favNodes); + FavoredNodesRepairChore chore = new FavoredNodesRepairChore(TEST_UTIL.getHBaseCluster() + .getMaster()); + chore.syncFavoredNodesWithRegionServers(); + HRegionServer regionServer = getRegionServer(currentRS); + assertTrue(regionServer instanceof FNRegionServer); + List fnFromRS = ((FNRegionServer)regionServer).getFavoredNodes(candidate.getEncodedName()); + assertNotNull(fnFromRS); + assertEquals(favNodes, fnFromRS); + } + + HRegionServer getRegionServer(ServerName sn) { + for (RegionServerThread thread : TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) { + if (ServerName.isSameHostnameAndPort(thread.getRegionServer().getServerName(), sn)) { + return thread.getRegionServer(); + } + } + return null; + } + + static class FNRegionServer extends MiniHBaseClusterRegionServer { + Map> fnMap = new HashMap>(); + + public FNRegionServer(Configuration conf, CoordinatedStateManager cp) + throws IOException, InterruptedException { + super(conf, cp); + } + + List getFavoredNodes(String encodedRegionName) { + return fnMap.get(encodedRegionName); + } + + @Override + public void updateRegionFavoredNodesMapping(String encodedRegionName, + List favoredNodes) { + List fns = Lists.newArrayList(); + for (org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName sn : favoredNodes) { + fns.add(ProtobufUtil.toServerName(sn)); + } + fnMap.put(encodedRegionName, fns); + super.updateRegionFavoredNodesMapping(encodedRegionName, favoredNodes); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java new file mode 100644 index 0000000..a8c7c05 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java @@ -0,0 +1,451 @@ +/** + * 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.balancer; + +import static junit.framework.TestCase.assertNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Deque; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import com.google.common.collect.Sets; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ClusterStatus; +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.HTableDescriptor; +import org.apache.hadoop.hbase.master.RackManager; +import org.apache.hadoop.hbase.master.RegionStates; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.RegionLoad; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; +import org.apache.hadoop.hbase.master.LoadBalancer; +import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.net.ScriptBasedMapping; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; + +@Category(LargeTests.class) +public class TestFavoredStochasticBalancerPickers extends BalancerTestBase { + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Log LOG = LogFactory.getLog(TestFavoredStochasticBalancerPickers.class); + private static final int SLAVES = 6; + private static HBaseAdmin admin; + protected static Configuration conf; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + conf = TEST_UTIL.getConfiguration(); + // Enable the favored nodes based load balancer + conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, + FavoredStochasticBalancerTest.class, LoadBalancer.class); + conf.setFloat("hbase.min.locality.redistribute", 0.0f); + conf.setBoolean("hbase.redistribute.even.on.same.rack", true); + conf.set(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + ScriptBasedMapping.class.getName()); + conf.setInt("hbase.assignment.maximum.attempts", 3); + conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 30000); + conf.setInt("hbase.master.balancer.stochastic.moveCost", 0); + conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1200000); + conf.setBoolean("hbase.master.balancer.stochastic.execute.maxSteps", true); + //Don't let chore run. + conf.setInt(FavoredNodesRepairChore.FAVORED_NODE_REPAIR_CHORE_FREQ, Integer.MAX_VALUE); + } + + @Before + public void startCluster() throws Exception { + TEST_UTIL.startMiniCluster(SLAVES); + TEST_UTIL.getDFSCluster().waitClusterUp(); + TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(120*1000); + admin = TEST_UTIL.getHBaseAdmin(); + admin.setBalancerRunning(false, true); + } + + @After + public void stopCluster() throws Exception { + TEST_UTIL.cleanupTestDir(); + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testBasicBalance() throws Exception { + final int regions = 10; + TableName tableName = TableName.valueOf("testBasicBalance"); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), regions); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + admin.flush(tableName); + compactTable(TEST_UTIL, tableName); + + ServerName masterServerName = TEST_UTIL.getMiniHBaseCluster().getServerHoldingMeta(); + List masterRegions = admin.getOnlineRegions(masterServerName); + + RegionServerThread rs1 = TEST_UTIL.getHBaseCluster().startRegionServer(); + RegionServerThread rs2 = TEST_UTIL.getHBaseCluster().startRegionServer(); + // TODO: Increased timeing + TEST_UTIL.waitFor(10000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().size() == SLAVES + 2); + } + }); + // Now try to run balance, and verify no regions are moved to the 2 region servers recently + // started. + admin.setBalancerRunning(true, true); + assertTrue("Balancer did not run", admin.balancer()); + TEST_UTIL.waitFor(120000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseAdmin().getClusterStatus().getRegionsInTransition().size() == 0); + } + }); + List hris = admin.getOnlineRegions(rs1.getRegionServer().getServerName()); + for (HRegionInfo hri : hris) { + if (!masterRegions.contains(hri)) { + assertFalse(hri.getTable().equals(tableName)); + } + } + hris = admin.getOnlineRegions(rs2.getRegionServer().getServerName()); + for (HRegionInfo hri : hris) { + if (!masterRegions.contains(hri)) { + assertFalse(hri.getTable().equals(tableName)); + } + } + } + + @Test + public void testBalanceWhenServerDead() throws Exception { + final int regions = 3 * SLAVES; + TableName tableName = TableName.valueOf("testBalanceWhenServerDead"); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), regions); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + + ServerName masterServerName = TEST_UTIL.getMiniHBaseCluster().getServerHoldingMeta(); + admin.removeFavoredNode(masterServerName.getHostPort()); + runBalancer(); + + admin.flush(tableName); + compactTable(TEST_UTIL, tableName); + + RegionServerThread rs1 = TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().get(0); + TEST_UTIL.getHBaseCluster().killRegionServer(rs1.getRegionServer().getServerName()); + TEST_UTIL.waitFor(60000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + List regionServerThreads = TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads(); + return regionServerThreads.size() == (SLAVES - 1); + } + }); + TEST_UTIL.waitFor(120000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseAdmin().getClusterStatus().getRegionsInTransition().size() == 0); + } + }); + + runBalancer(); + + ClusterStatus status = admin.getClusterStatus(); + List serverLoad = Lists.newArrayList(); + for (ServerName sn : status.getServers()) { + if (!ServerName.isSameHostnameAndPort(sn, masterServerName)) { + serverLoad.add(new ServerAndLoad(sn, status.getLoad(sn).getLoad())); + } + } + assertClusterAsBalanced(serverLoad); + } + + @Test + public void testBalanceWhenServerRestarted() throws Exception { + final int regions = 3 * SLAVES; + final int numSystemTables = admin.listTableNamesByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR).length; + final TableName tableName = TableName.valueOf("testBalanceWhenServerDead"); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), regions); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + + ServerName masterServerName = TEST_UTIL.getMiniHBaseCluster().getServerHoldingMeta(); + admin.removeFavoredNode(masterServerName.getHostPort()); + + runBalancer(); + + admin.flush(tableName); + compactTable(TEST_UTIL, tableName); + RegionServerThread rs1 = TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().get(0); + ServerName tobeRestarted = rs1.getRegionServer().getServerName(); + TEST_UTIL.getHBaseCluster().killRegionServer(tobeRestarted); + TEST_UTIL.waitFor(10000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size() == SLAVES - 1); + } + }); + TEST_UTIL.waitFor(120000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseAdmin().getClusterStatus().getRegionsCount() >= (regions + numSystemTables)); + } + }); + + Configuration newConf = new Configuration(conf); + newConf.setInt(HConstants.REGIONSERVER_PORT, tobeRestarted.getPort()); + TEST_UTIL.getHBaseCluster().startRegionServer(newConf); + TEST_UTIL.waitFor(10000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size() == SLAVES); + } + }); + + runBalancer(); + + ClusterStatus status = admin.getClusterStatus(); + List serverLoad = Lists.newArrayList(); + for (ServerName sn : status.getServers()) { + if (!ServerName.isSameHostnameAndPort(sn, masterServerName)) { + serverLoad.add(new ServerAndLoad(sn, status.getLoad(sn).getLoad())); + } + } + assertClusterAsBalanced(serverLoad); + } + + private void runBalancer() throws Exception { + admin.setBalancerRunning(true, true); + int balancerAttempts = 10; + boolean balancerRan = false; + while (balancerAttempts > 0 && !balancerRan) { + balancerRan = admin.balancer(); + Thread.sleep(5 * 1000); + balancerAttempts--; + } + assertTrue("Balancer did not run", balancerRan); + TEST_UTIL.waitFor(120000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseAdmin().getClusterStatus().getRegionsInTransition().size() == 0); + } + }); + admin.setBalancerRunning(false, true); + } + + @Test + public void testPickers() throws Exception { + int regions = SLAVES * 3; + TableName tableName = TableName.valueOf("testPickers"); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), regions); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + admin.flush(tableName); + ServerName masterServerName = TEST_UTIL.getMiniHBaseCluster().getServerHoldingMeta(); + final ServerName mostLoadedServer = getRSWithMaxRegions(Lists.newArrayList(masterServerName)); + int numRegions = admin.getOnlineRegions(mostLoadedServer).size(); + assertNotNull(mostLoadedServer); + ServerName source = getRSWithMaxRegions(Lists.newArrayList(masterServerName, mostLoadedServer)); + assertNotNull(source); + int regionsToMove = admin.getOnlineRegions(source).size()/2; + List hris = admin.getOnlineRegions(source); + for (int i = 0; i < regionsToMove; i++) { + admin.move(hris.get(i).getEncodedNameAsBytes(), Bytes.toBytes(mostLoadedServer.getServerName())); + LOG.info("Moving region: " + hris.get(i).getRegionNameAsString() + " to " + mostLoadedServer); + } + final int finalRegions = numRegions + regionsToMove; + TEST_UTIL.waitFor(60000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseAdmin().getClusterStatus().getRegionsInTransition().size() == 0); + } + }); + TEST_UTIL.waitFor(60000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + int numRegions = TEST_UTIL.getHBaseAdmin().getOnlineRegions(mostLoadedServer).size(); + return (numRegions == finalRegions); + } + }); + RegionServerThread rs1 = TEST_UTIL.getHBaseCluster().startRegionServer(); + TEST_UTIL.waitFor(10000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseCluster().getRegionServerThreads().size() == SLAVES + 1); + } + }); + Map> serverAssignments = new HashMap>(); + ClusterStatus status = admin.getClusterStatus(); + for (ServerName sn : status.getServers()) { + if (!ServerName.isSameHostnameAndPort(sn, masterServerName)) { + serverAssignments.put(sn, admin.getOnlineRegions(sn)); + } + } + Map> loads = new HashMap>(); + RegionLocationFinder regionFinder = new RegionLocationFinder(); + regionFinder.setClusterStatus(admin.getClusterStatus()); + regionFinder.setConf(conf); + regionFinder.setServices(TEST_UTIL.getMiniHBaseCluster().getMaster()); + Cluster cluster = new Cluster(serverAssignments, loads, regionFinder, new RackManager(conf)); + FavoredStochasticBalancerTest balancer = (FavoredStochasticBalancerTest) TEST_UTIL + .getMiniHBaseCluster().getMaster().getLoadBalancer(); + FavoredNodesManager fnm = TEST_UTIL.getMiniHBaseCluster().getMaster().getFavoredNodesManager(); + cluster.sortServersByRegionCount(); + Integer[] servers = cluster.serverIndicesSortedByRegionCount; + LOG.info("Servers sorted by region count:" + Arrays.toString(servers)); + LOG.info("Cluster dump: " + cluster); + if (!mostLoadedServer.equals(cluster.servers[servers[servers.length -1]])) { + LOG.error("Most loaded server: " + mostLoadedServer + " does not match: " + + cluster.servers[servers[servers.length -1]]); + } + assertEquals(mostLoadedServer, cluster.servers[servers[servers.length -1]]); + FavoredStochasticBalancer.FavoredNodeLoadPicker loadPicker = balancer.new FavoredNodeLoadPicker(); + boolean userRegionPicked = false; + for (int i = 0; i < 100; i++) { + if (userRegionPicked) { + break; + } else { + Cluster.Action action = loadPicker.generate(cluster); + if (action.type == Cluster.Action.Type.MOVE_REGION) { + Cluster.MoveRegionAction moveRegionAction = (Cluster.MoveRegionAction) action; + HRegionInfo region = cluster.regions[moveRegionAction.region]; + assertNotEquals(-1, moveRegionAction.toServer); + ServerName destinationServer = cluster.servers[moveRegionAction.toServer]; + assertEquals(cluster.servers[moveRegionAction.fromServer], mostLoadedServer); + if (!region.getTable().isSystemTable()) { + List favNodes = fnm.getFavoredNodes(region); + assertTrue(favNodes.contains(ServerName.valueOf(destinationServer.getHostAndPort(), -1))); + userRegionPicked = true; + } + } + } + } + assertTrue("load picker did not pick expected regions in 100 iterations.", userRegionPicked); + } + + private ServerName getRSWithMaxRegions(ArrayList excludeNodes) throws IOException { + int maxRegions = 0; + ServerName maxLoadedServer = null; + + for (ServerName sn : admin.getClusterStatus().getServers()) { + if (admin.getOnlineRegions(sn).size() > maxRegions) { + if (excludeNodes == null || !doesMatchExcludeNodes(excludeNodes, sn)) { + maxRegions = admin.getOnlineRegions(sn).size(); + maxLoadedServer = sn; + } + } + } + return maxLoadedServer; + } + + private boolean doesMatchExcludeNodes(ArrayList excludeNodes, ServerName sn) { + for (ServerName excludeSN : excludeNodes) { + if (ServerName.isSameHostnameAndPort(sn, excludeSN)) { + return true; + } + } + return false; + } + + @Test + public void testMisplacedRegions() throws Exception { + int regions = 10; + TableName tableName = TableName.valueOf("testMisplacedRegions"); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), regions); + TEST_UTIL.waitTableAvailable(tableName); + final HRegionInfo misplacedRegion = admin.getTableRegions(tableName).get(0); + FavoredNodesManager fnm = TEST_UTIL.getHBaseCluster().getMaster().getFavoredNodesManager(); + List currentFN = fnm.getFavoredNodes(misplacedRegion); + assertNotNull(currentFN); + List serversForNewFN = Lists.newArrayList(); + for (ServerName sn : admin.getClusterStatus().getServers()) { + serversForNewFN.add(ServerName.valueOf(sn.getHostAndPort(), ServerName.NON_STARTCODE)); + } + for (ServerName sn : currentFN) { + serversForNewFN.remove(sn); + } + FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(serversForNewFN, conf); + helper.initialize(); + List newFavoredNodes = helper.generateFavoredNodes(misplacedRegion); + assertNotNull(newFavoredNodes); + assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, newFavoredNodes.size()); + fnm.updateFavoredNodes(misplacedRegion, newFavoredNodes); + + RegionStates regionStates = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() + .getRegionStates(); + final ServerName current = regionStates.getRegionServerOfRegion(misplacedRegion); + assertNull("Misplaced region is still hosted on favored node, not expected.", + FavoredNodesPlan.getFavoredServerPosition(fnm.getFavoredNodes(misplacedRegion), current)); + admin.setBalancerRunning(true, true); + assertTrue("Balancer did not run", admin.balancer()); + TEST_UTIL.waitFor(120000, 30000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + ServerName host = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() + .getRegionStates().getRegionServerOfRegion(misplacedRegion); + return !ServerName.isSameHostnameAndPort(host, current); + } + }); + for (HRegionInfo hri : admin.getTableRegions(tableName)) { + ServerName host = regionStates.getRegionServerOfRegion(hri); + assertNotNull("Region not on favored node.", + FavoredNodesPlan.getFavoredServerPosition(fnm.getFavoredNodes(hri), host)); + } + } + + void compactTable(HBaseTestingUtility util, TableName tableName) throws IOException { + for(RegionServerThread t : + util.getMiniHBaseCluster().getRegionServerThreads()) { + for(Region region : t.getRegionServer().getOnlineRegions(tableName)) { + region.compact(true); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java new file mode 100644 index 0000000..3155a6c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -0,0 +1,446 @@ +/** + * 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.balancer; + +import static org.junit.Assert.*; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ClusterStatus; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +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.HTableDescriptor; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.RegionStates; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.favored.FavoredNodesManager; +import org.apache.hadoop.hbase.master.LoadBalancer; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.net.ScriptBasedMapping; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + + +@Category(MediumTests.class) +public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final int SLAVES = 3; + private static HBaseAdmin admin; + private int REGION_NUM = 5; + private static boolean postSplit = false; + protected static Configuration conf; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + setUpOnce(); + } + + static void setUpOnce() throws Exception { + conf = TEST_UTIL.getConfiguration(); + // Enable the favored nodes based load balancer + conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, + FavoredStochasticBalancerTest.class, LoadBalancer.class); + conf.setFloat("hbase.min.locality.redistribute", 0.0f); + conf.setBoolean("hbase.redistribute.even.on.same.rack", true); + conf.set(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + ScriptBasedMapping.class.getName()); + conf.setInt("hbase.assignment.maximum.attempts", 3); + //Making it high value, tests should explicitly call catalog janitor if needed. + conf.setInt("hbase.catalogjanitor.interval", Integer.MAX_VALUE); + //Don't let chore run. + conf.setInt(FavoredNodesRepairChore.FAVORED_NODE_REPAIR_CHORE_FREQ, Integer.MAX_VALUE); + } + + @Before + public void startCluster() throws Exception { + TEST_UTIL.startMiniCluster(SLAVES); + TEST_UTIL.getDFSCluster().waitClusterUp(); + admin = TEST_UTIL.getHBaseAdmin(); + admin.setBalancerRunning(false, true); + } + + @After + public void stopCluster() throws Exception { + TEST_UTIL.cleanupTestDir(); + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testBasicRegionPlacement() throws Exception { + String tableName = "testBasicRegionPlacement"; + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + FavoredNodesManager fnm = TEST_UTIL.getHBaseCluster().getMaster().getFavoredNodesManager(); + List regionsOfTable = + TEST_UTIL.getHBaseAdmin().getTableRegions(TableName.valueOf(tableName)); + for (HRegionInfo rInfo : regionsOfTable) { + Set favNodes = Sets.newHashSet(fnm.getFavoredNodes(rInfo)); + assertNotNull(favNodes); + assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, favNodes.size()); + } + Map> replicaLoadMap = + fnm.getReplicaLoad(Lists.newArrayList(admin.getClusterStatus().getServers())); + assertTrue("Not all replica load collected.", + admin.getClusterStatus().getServers().size() == replicaLoadMap.size()); + for (Entry> entry : replicaLoadMap.entrySet()) { + assertTrue(entry.getValue().size() == 3); + assertTrue(entry.getValue().get(0) >= 0); + assertTrue(entry.getValue().get(1) >= 0); + assertTrue(entry.getValue().get(2) >= 0); + } + admin.disableTable(TableName.valueOf(tableName)); + admin.deleteTable(TableName.valueOf(tableName)); + for (HRegionInfo rInfo : regionsOfTable) { + List favNodes = fnm.getFavoredNodes(rInfo); + assertNull(favNodes); + } + replicaLoadMap = + fnm.getReplicaLoad(Lists.newArrayList(admin.getClusterStatus().getServers())); + assertTrue("replica load found " + replicaLoadMap.size() + " instead of 0.", + replicaLoadMap.size() == admin.getClusterStatus().getServers().size()); + } + + @Test + public void testRedistribute() throws Exception { + String tableName = "testRedistribute"; + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TEST_UTIL.getHBaseCluster().startRegionServer(); + TEST_UTIL.getHBaseCluster().startRegionServer(); + TEST_UTIL.waitFor(1000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseAdmin().getClusterStatus().getServers().size() == SLAVES + 2); + } + }); + ClusterStatus status = admin.getClusterStatus(); + assertTrue(TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size() == SLAVES + 2); + // Call redistribute + assertTrue(admin.redistributeFavoredNodes()); + assertTrue(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() + .getRegionStates().getRegionsInTransition().size() == 0); + // Check the current assignments remain as-is, but the all the region servers should have + // replicas + ClusterStatus newStatus = admin.getClusterStatus(); + for (ServerName sn : status.getServers()) { + assertEquals(status.getLoad(sn).getNumberOfRegions(), newStatus.getLoad(sn) + .getNumberOfRegions()); + } + Map> regionReplicaMap = + getReplicaMap(TableName.valueOf(tableName)); + int numOnlineServers = TEST_UTIL.getHBaseCluster().getRegionServerThreads().size(); + int minReplicas = REGION_NUM / numOnlineServers; + assertEquals(numOnlineServers, regionReplicaMap.keySet().size()); + checkMinReplicas(regionReplicaMap, minReplicas, TEST_UTIL.getHBaseAdmin().getClusterStatus() + .getServers()); + + ServerName serverToKill = + TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().get(0).getRegionServer() + .getServerName(); + TEST_UTIL.getHBaseCluster().killRegionServer(serverToKill); + TEST_UTIL.getHBaseCluster().waitForRegionServerToStop(serverToKill, -1); + TEST_UTIL.waitFor(1000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseAdmin().getClusterStatus().getServers().size() == 4); + } + }); + TEST_UTIL.waitFor(10000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (!TEST_UTIL.getHBaseCluster().getMaster().getServerManager() + .areDeadServersInProgress()); + } + }); + TEST_UTIL.waitFor(40000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates() + .getRegionsInTransition().size() == 0); + } + }); + List deadServersInUse = admin.checkFavoredNodes(); + assertNotNull(deadServersInUse); + assertEquals(1, deadServersInUse.size()); + assertTrue(ServerName.isSameHostnameAndPort(deadServersInUse.get(0), serverToKill)); + admin.removeFavoredNode(deadServersInUse.get(0).getHostPort()); + deadServersInUse = admin.checkFavoredNodes(); + assertNotNull(deadServersInUse); + assertEquals(0, deadServersInUse.size()); + assertTrue(admin.redistributeFavoredNodes()); + assertTrue(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() + .getRegionStates().getRegionsInTransition().size() == 0); + regionReplicaMap = getReplicaMap(TableName.valueOf(tableName)); + numOnlineServers = TEST_UTIL.getHBaseAdmin().getClusterStatus().getServersSize(); + minReplicas = REGION_NUM / numOnlineServers; + checkMinReplicas(regionReplicaMap, minReplicas, TEST_UTIL.getHBaseAdmin().getClusterStatus() + .getServers()); + assertTrue(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() + .getRegionStates().getRegionsInTransition().size() == 0); + } + + Map> getReplicaMap(TableName tableName) throws IOException { + List regionsOfTable = TEST_UTIL.getHBaseAdmin().getTableRegions(tableName); + FavoredNodesManager fnm = TEST_UTIL.getHBaseCluster().getMaster().getFavoredNodesManager(); + Map> regionReplicaMap = + new HashMap>(); + for (HRegionInfo rInfo : regionsOfTable) { + List favNodes = fnm.getFavoredNodes(rInfo); + assertNotNull(favNodes); + for (ServerName sn : favNodes) { + List replicas = regionReplicaMap.get(sn); + if (replicas == null) { + replicas = new ArrayList(); + } + replicas.add(rInfo); + regionReplicaMap.put(sn, replicas); + } + } + return regionReplicaMap; + } + + void checkMinReplicas(Map> replicaMap, int minReplicas, + Collection servers) { + assertEquals(servers.size(), replicaMap.size()); + for (ServerName sn : servers) { + assertTrue("Atleast min replica expected.", + replicaMap.get(ServerName.valueOf(sn.getHostAndPort(), ServerName.NON_STARTCODE)) + .size() >= minReplicas); + } + } + + @Test + public void testCompleteRedistribute() throws Exception { + String tableName = "testCompleteRedistribute"; + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM); + TEST_UTIL.waitTableAvailable(TableName.valueOf(tableName)); + admin.completeRedistributeFavoredNodes(); + TEST_UTIL.waitTableAvailable(TableName.valueOf(tableName)); + List tableRegions = admin.getTableRegions(TableName.valueOf(tableName)); + FavoredNodesManager fnm = TEST_UTIL.getHBaseCluster().getMaster().getFavoredNodesManager(); + RegionStates regionStates = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates(); + for (HRegionInfo hri : tableRegions) { + List favoredNodes = fnm.getFavoredNodes(hri); + assertNotNull(favoredNodes); + ServerName current = regionStates.getRegionServerOfRegion(hri); + assertNotNull("Region not hosted on favored nodes.", + FavoredNodesPlan.getFavoredServerPosition(favoredNodes, current)); + } + Map> regionReplicaMap = getReplicaMap(TableName.valueOf(tableName)); + int numOnlineServers = TEST_UTIL.getHBaseCluster().getRegionServerThreads().size(); + assertTrue(numOnlineServers == regionReplicaMap.keySet().size()); + List salList = new ArrayList(); + for (ServerName sn : regionReplicaMap.keySet()) { + salList.add(new ServerAndLoad(sn, regionReplicaMap.get(sn).size())); + } + assertClusterAsBalanced(salList); + } + + @Test + public void testRegionSplit() throws Exception { + final TableName tableName = TableName.valueOf("testRegionSplit"); + HTableDescriptor desc = new HTableDescriptor(tableName); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc); + Table table = admin.getConnection().getTable(tableName); + TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY); + List hris = admin.getTableRegions(tableName); + assertTrue(hris.size() == 1); + FavoredNodesManager fnm = TEST_UTIL.getHBaseCluster().getMaster().getFavoredNodesManager(); + Set parentFavNodes = Sets.newHashSet(fnm.getFavoredNodes(hris.get(0))); + assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, parentFavNodes.size()); + HRegion actualRegion = TEST_UTIL.getHBaseCluster().getRegions(tableName).get(0); + // install region co-processor to monitor splits + actualRegion.getCoprocessorHost().load(CustomObserver.class, + Coprocessor.PRIORITY_USER, TEST_UTIL.getConfiguration()); + admin.split(tableName, Bytes.toBytes("ggg")); + TEST_UTIL.waitFor(60000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return postSplit; + } + }); + // Do gets from both the daughter regions to see they are online. + Get splitRowKey = new Get(Bytes.toBytes("ggg")); + table.get(splitRowKey); + splitRowKey = new Get(Bytes.toBytes("abc")); + table.get(splitRowKey); + RegionStates regionStates = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager() + .getRegionStates(); + assertFalse( + "Parent region not offline after split, found in state + " + + regionStates.getRegionTransitionState(actualRegion.getRegionInfo().getEncodedName()), + regionStates.isRegionOnline(actualRegion.getRegionInfo())); + TEST_UTIL.waitTableAvailable(tableName); + for(RegionServerThread regionServerThread : TEST_UTIL.getMiniHBaseCluster() + .getLiveRegionServerThreads()) { + for(Region region : + regionServerThread.getRegionServer().getOnlineRegions(tableName)) { + region.compact(true); + } + } + final HRegionInfo parent = hris.get(0); + // The parents favored nodes would still be there, they will be cleaned up once catalog janitor runs. + assertNotNull(fnm.getFavoredNodes(parent)); + hris = admin.getTableRegions(tableName); + for (HRegionInfo hri : hris) { + assertNotNull(fnm.getFavoredNodes(hri)); + } + TEST_UTIL.waitFor(60000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return admin.runCatalogScan() > 0; + } + }); + assertNull(fnm.getFavoredNodes(parent)); + hris = admin.getTableRegions(tableName); + assertTrue(hris.size() == 2); + for (final HRegionInfo hri : hris) { + assertTrue("Favored nodes found null for region", + fnm.getFavoredNodes(hri) != null); + List favNodes = fnm.getFavoredNodes(hri); + assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, Sets.newHashSet(favNodes).size()); + int matchCount = 0; + for (ServerName server : favNodes) { + if (parentFavNodes.contains(server)) { + matchCount++; + } + } + assertTrue("Daughter region did not inherit 2 fns", matchCount >= 2); + ServerName sn = regionStates.getRegionServerOfRegion(hri); + final HRegionServer rs = getRegionServer(sn); + TEST_UTIL.waitFor(60000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (rs.getFavoredNodesForRegion(hri.getEncodedName()) != null); + } + }); + InetSocketAddress[] favoredSocketAddress = rs.getFavoredNodesForRegion(hri.getEncodedName()); + assertTrue(favoredSocketAddress.length == favNodes.size()); + assertTrue(favNodes.size() > 0); + int port = NetUtils.createSocketAddr( + conf.get(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT)) + .getPort(); + for (int j = 0; j < favoredSocketAddress.length; j++) { + InetSocketAddress addrFromRS = favoredSocketAddress[j]; + InetSocketAddress addrFromPlan = InetSocketAddress.createUnresolved(favNodes + .get(j).getHostname(), port); + + assertNotNull(addrFromRS); + assertNotNull(addrFromPlan); + assertTrue("Region server " + rs.getServerName().getHostAndPort() + " for region " + + hri.getRegionNameAsString() + " is " + addrFromRS + + " which is inconsistent with the plan " + addrFromPlan, + addrFromRS.equals(addrFromPlan)); + } + } + } + + /** + * Test for YHBASE-757. + * + * @throws Exception the exception + */ + @Test + public void testAssignmentWithNoFavNodes() throws Exception { + final String tableName = "testRegionWithNoFavNodes"; + HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); + admin.createTable(desc); + HRegionInfo hri = admin.getTableRegions(TableName.valueOf(tableName)).get(0); + LoadBalancer balancer = TEST_UTIL.getHBaseCluster().getMaster().getLoadBalancer(); + FavoredNodesManager fnm = TEST_UTIL.getHBaseCluster().getMaster().getFavoredNodesManager(); + fnm.deleteFavoredNodesForRegion(Lists.newArrayList(hri)); + assertNull("Favored nodes not found null after delete", fnm.getFavoredNodes(hri)); + ServerName desintination = balancer.randomAssignment(hri, Lists.newArrayList(admin + .getClusterStatus().getServers())); + assertNotNull(desintination); + List favoredNodes = fnm.getFavoredNodes(hri); + assertNotNull(favoredNodes); + boolean containsFN = false; + for (ServerName sn : favoredNodes) { + if (ServerName.isSameHostnameAndPort(desintination, sn)) { + containsFN = true; + } + } + assertTrue("Destination server does not belong to favored nodes.", containsFN); + } + + HRegionServer getRegionServer(ServerName sn) { + for ( int i= 0; i < SLAVES; i++) { + HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i); + if ( ServerName.isSameHostnameAndPort(server.getServerName(),sn)) { + return server; + } + } + return null; + } + + public static class CustomObserver extends BaseRegionObserver{ + @Override + public void start(CoprocessorEnvironment e) throws IOException { + postSplit = false; + } + + @Override + public void postCompleteSplit(ObserverContext ctx) + throws IOException { + postSplit = true; + } + + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java index 094687b..c31fbfa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java @@ -39,6 +39,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterStatus; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.ServerLoad; @@ -496,11 +497,12 @@ public class TestStochasticLoadBalancer extends BalancerTestBase { } @Ignore @Test (timeout = 800000) // Test is flakey. TODO: Fix! - public void testRegionReplicationOnMidClusterSameHosts() { + public void testRegionReplicationOnMidClusterSameHosts() throws HBaseIOException { conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L); conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f); loadBalancer.setConf(conf); + loadBalancer.initialize(); int numHosts = 100; int numRegions = 100 * 100; int replication = 3; // 3 replicas per region diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb index 67dde53..c403a13 100644 --- a/hbase-shell/src/main/ruby/hbase/admin.rb +++ b/hbase-shell/src/main/ruby/hbase/admin.rb @@ -869,6 +869,30 @@ module Hbase end #---------------------------------------------------------------------------------------------- + # Requests a cluster to redistribute favored nodes + # Returns true if redistribute ran. + def redistributeFavoredNodes() + @admin.redistributeFavoredNodes() + end + + #---------------------------------------------------------------------------------------------- + # Requests a cluster to perform complete redistribution of favored nodes + # Returns true if complete redistribute ran. + def completeRedistributeFavoredNodes() + @admin.completeRedistributeFavoredNodes() + end + + #---------------------------------------------------------------------------------------------- + # Scans all the favored nodes of regions to look for dead servers being used. + def checkFavoredNodes() + @admin.checkFavoredNodes().map { |s| s.getHostAndPort } + end + + #---------------------------------------------------------------------------------------------- + # Removes server from any favored nodes usage. This operation will not change region assignment. + def removeFavoredNode(server) + @admin.removeFavoredNode(server) + end # Enables/disables a region by name def online(region_name, on_off) # Open meta table diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index 02f8191..303e9d1 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -353,6 +353,10 @@ Shell.load_command_group( trace splitormerge_switch splitormerge_enabled + redistribute + complete_redistribute + check_favored_nodes + remove_favored_node ], # TODO remove older hlog_roll command :aliases => { diff --git a/hbase-shell/src/main/ruby/shell/commands/check_favored_nodes.rb b/hbase-shell/src/main/ruby/shell/commands/check_favored_nodes.rb new file mode 100644 index 0000000..1b1dcaa --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/check_favored_nodes.rb @@ -0,0 +1,35 @@ +# 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. +# +module Shell + module Commands + class CheckFavoredNodes < Command + def help + return <<-EOF +Scans all the regions to see if all the favored nodes used by the region are online +region servers. This command returns a list of dead servers being referenced. + hbase> check_favored_nodes +EOF + end + def command() + list = admin.checkFavoredNodes() + list.each do |server| + formatter.row([ server ]) + end + end + end + end +end diff --git a/hbase-shell/src/main/ruby/shell/commands/complete_redistribute.rb b/hbase-shell/src/main/ruby/shell/commands/complete_redistribute.rb new file mode 100644 index 0000000..1a462a6 --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/complete_redistribute.rb @@ -0,0 +1,44 @@ +# 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. +# + +module Shell + module Commands + class CompleteRedistribute < Command + def help + return <<-EOF +Trigger the balancer for completely redistributing favored nodes. All the favored nodes will be +redistributed.. Please note that this command will not move any regions from the region servers. +Examples: + hbase > complete_redistribute 'true' + hbase > complete_redistribute 'false' +EOF + end + + def command(shouldRun) + if shouldRun + format_simple_command do + formatter.row([admin.completeRedistributeFavoredNodes() ? "true": "false"]) + end + else + format_simple_command do + formatter.row(["false"]) + end + end + end + end + end +end diff --git a/hbase-shell/src/main/ruby/shell/commands/redistribute.rb b/hbase-shell/src/main/ruby/shell/commands/redistribute.rb new file mode 100644 index 0000000..25e07fb --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/redistribute.rb @@ -0,0 +1,46 @@ +# 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. +# + +module Shell + module Commands + class Redistribute < Command + def help + return <<-EOF +Trigger the balancer for redistributing favored nodes. Returns true if balancer ran and was able to +update the meta with the new favored node information. Please note that this balance action will not +move any regions from the region servers. +Examples: + + hbase > redistribute 'true' + hbase > redistribute 'false' +EOF + end + + def command(shouldRun) + if shouldRun + format_simple_command do + formatter.row([ admin.redistributeFavoredNodes()? "true": "false"]) + end + else + format_simple_command do + formatter.row(["false"]) + end + end + end + end + end +end diff --git a/hbase-shell/src/main/ruby/shell/commands/remove_favored_node.rb b/hbase-shell/src/main/ruby/shell/commands/remove_favored_node.rb new file mode 100644 index 0000000..0ee0659 --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/remove_favored_node.rb @@ -0,0 +1,34 @@ +# 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. +# +module Shell + module Commands + class RemoveFavoredNode < Command + def help + return <<-EOF +Removes the specified server (hostname:port) as favored node from regions. This operation does not +change any assignments, but alters favored node information in meta. The users have to make sure +that the specified server is not hosting any regions. + +hbase> remove_favored_node 'myhost:9999' +EOF + end + def command(server) + admin.removeFavoredNode(server) + end + end + end +end -- 2.10.1