001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.master;
020
021import edu.umd.cs.findbugs.annotations.NonNull;
022import java.io.IOException;
023import java.util.List;
024import java.util.Map;
025import org.apache.hadoop.conf.Configurable;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.hbase.ClusterMetrics;
028import org.apache.hadoop.hbase.HBaseIOException;
029import org.apache.hadoop.hbase.ServerName;
030import org.apache.hadoop.hbase.Stoppable;
031import org.apache.hadoop.hbase.TableName;
032import org.apache.hadoop.hbase.client.RegionInfo;
033import org.apache.hadoop.hbase.conf.ConfigurationObserver;
034import org.apache.yetus.audience.InterfaceAudience;
035
036/**
037 * Makes decisions about the placement and movement of Regions across
038 * RegionServers.
039 *
040 * <p>Cluster-wide load balancing will occur only when there are no regions in
041 * transition and according to a fixed period of a time using {@link #balanceCluster(Map)}.
042 *
043 * <p>On cluster startup, bulk assignment can be used to determine
044 * locations for all Regions in a cluster.
045 *
046 * <p>This class produces plans for the
047 * {@link org.apache.hadoop.hbase.master.assignment.AssignmentManager}
048 * to execute.
049 */
050@InterfaceAudience.Private
051public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObserver {
052  /**
053   * Master can carry regions as of hbase-2.0.0.
054   * By default, it carries no tables.
055   * TODO: Add any | system as flags to indicate what it can do.
056   *
057   * @deprecated since 2.4.0, will be removed in 3.0.0.
058   * @see <a href="https://issues.apache.org/jira/browse/HBASE-15549">HBASE-15549</a>
059   */
060  @Deprecated
061  String TABLES_ON_MASTER = "hbase.balancer.tablesOnMaster";
062
063  /**
064   * Master carries system tables.
065   *
066   * @deprecated since 2.4.0, will be removed in 3.0.0.
067   * @see <a href="https://issues.apache.org/jira/browse/HBASE-15549">HBASE-15549</a>
068   */
069  @Deprecated
070  String SYSTEM_TABLES_ON_MASTER = "hbase.balancer.tablesOnMaster.systemTablesOnly";
071
072  // Used to signal to the caller that the region(s) cannot be assigned
073  // We deliberately use 'localhost' so the operation will fail fast
074  ServerName BOGUS_SERVER_NAME = ServerName.valueOf("localhost,1,1");
075
076  /**
077   * Set the current cluster status.  This allows a LoadBalancer to map host name to a server
078   * @param st
079   */
080  void setClusterMetrics(ClusterMetrics st);
081
082  /**
083   * Set the master service.
084   * @param masterServices
085   */
086  void setMasterServices(MasterServices masterServices);
087
088  /**
089   * Perform the major balance operation for cluster, will invoke {@link #balanceTable} to do
090   * actual balance. Normally not need override this method, except SimpleLoadBalancer and
091   * RSGroupBasedLoadBalancer.
092   * @param loadOfAllTable region load of servers for all table
093   * @return a list of regions to be moved, including source and destination, or null if cluster is
094   *         already balanced
095   */
096  List<RegionPlan> balanceCluster(Map<TableName,
097      Map<ServerName, List<RegionInfo>>> loadOfAllTable) throws IOException;
098
099  /**
100   * Perform the major balance operation for table, all class implement of {@link LoadBalancer}
101   * should override this method
102   * @param tableName the table to be balanced
103   * @param loadOfOneTable region load of servers for the specific one table
104   * @return List of plans
105   */
106  List<RegionPlan> balanceTable(TableName tableName,
107      Map<ServerName, List<RegionInfo>> loadOfOneTable);
108  /**
109   * Perform a Round Robin assignment of regions.
110   * @param regions
111   * @param servers
112   * @return Map of servername to regioninfos
113   */
114  @NonNull
115  Map<ServerName, List<RegionInfo>> roundRobinAssignment(List<RegionInfo> regions,
116      List<ServerName> servers) throws HBaseIOException;
117
118  /**
119   * Assign regions to the previously hosting region server
120   * @param regions
121   * @param servers
122   * @return List of plans
123   */
124  @NonNull
125  Map<ServerName, List<RegionInfo>> retainAssignment(Map<RegionInfo, ServerName> regions,
126      List<ServerName> servers) throws HBaseIOException;
127
128  /**
129   * Get a random region server from the list
130   * @param regionInfo Region for which this selection is being done.
131   * @param servers
132   * @return Servername
133   */
134  ServerName randomAssignment(
135    RegionInfo regionInfo, List<ServerName> servers
136  ) throws HBaseIOException;
137
138  /**
139   * Initialize the load balancer. Must be called after setters.
140   * @throws HBaseIOException
141   */
142  void initialize() throws HBaseIOException;
143
144  /**
145   * Marks the region as online at balancer.
146   * @param regionInfo
147   * @param sn
148   */
149  void regionOnline(RegionInfo regionInfo, ServerName sn);
150
151  /**
152   * Marks the region as offline at balancer.
153   * @param regionInfo
154   */
155  void regionOffline(RegionInfo regionInfo);
156
157  /*
158   * Notification that config has changed
159   * @param conf
160   */
161  @Override
162  void onConfigurationChange(Configuration conf);
163
164  /**
165   * If balancer needs to do initialization after Master has started up, lets do that here.
166   */
167  void postMasterStartupInitialize();
168
169  /*Updates balancer status tag reported to JMX*/
170  void updateBalancerStatus(boolean status);
171
172  /**
173   * @return true if Master carries regions
174   * @deprecated since 2.4.0, will be removed in 3.0.0.
175   * @see <a href="https://issues.apache.org/jira/browse/HBASE-15549">HBASE-15549</a>
176   */
177  @Deprecated
178  static boolean isTablesOnMaster(Configuration conf) {
179    return conf.getBoolean(TABLES_ON_MASTER, false);
180  }
181
182  /**
183   * @deprecated since 2.4.0, will be removed in 3.0.0.
184   * @see <a href="https://issues.apache.org/jira/browse/HBASE-15549">HBASE-15549</a>
185   */
186  @Deprecated
187  static boolean isSystemTablesOnlyOnMaster(Configuration conf) {
188    return conf.getBoolean(SYSTEM_TABLES_ON_MASTER, false);
189  }
190
191  /**
192   * @deprecated since 2.4.0, will be removed in 3.0.0.
193   * @see <a href="https://issues.apache.org/jira/browse/HBASE-15549">HBASE-15549</a>
194   */
195  @Deprecated
196  static boolean isMasterCanHostUserRegions(Configuration conf) {
197    return isTablesOnMaster(conf) && !isSystemTablesOnlyOnMaster(conf);
198  }
199}