View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import java.io.IOException;
23  import java.net.InetAddress;
24  import java.util.ArrayList;
25  import java.util.Collections;
26  import java.util.HashMap;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Set;
30  import java.util.concurrent.ConcurrentHashMap;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.hbase.ClockOutOfSyncException;
36  import org.apache.hadoop.hbase.HRegionInfo;
37  import org.apache.hadoop.hbase.HServerAddress;
38  import org.apache.hadoop.hbase.HServerLoad;
39  import org.apache.hadoop.hbase.PleaseHoldException;
40  import org.apache.hadoop.hbase.Server;
41  import org.apache.hadoop.hbase.ServerName;
42  import org.apache.hadoop.hbase.YouAreDeadException;
43  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
44  import org.apache.hadoop.hbase.client.HConnection;
45  import org.apache.hadoop.hbase.client.HConnectionManager;
46  import org.apache.hadoop.hbase.client.RetriesExhaustedException;
47  import org.apache.hadoop.hbase.ipc.HRegionInterface;
48  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
49  import org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler;
50  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
51  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
52  
53  /**
54   * The ServerManager class manages info about region servers.
55   * <p>
56   * Maintains lists of online and dead servers.  Processes the startups,
57   * shutdowns, and deaths of region servers.
58   * <p>
59   * Servers are distinguished in two different ways.  A given server has a
60   * location, specified by hostname and port, and of which there can only be one
61   * online at any given time.  A server instance is specified by the location
62   * (hostname and port) as well as the startcode (timestamp from when the server
63   * was started).  This is used to differentiate a restarted instance of a given
64   * server from the original instance.
65   */
66  public class ServerManager {
67    private static final Log LOG = LogFactory.getLog(ServerManager.class);
68  
69    // Set if we are to shutdown the cluster.
70    private volatile boolean clusterShutdown = false;
71  
72    /** Map of registered servers to their current load */
73    private final Map<ServerName, HServerLoad> onlineServers =
74      new ConcurrentHashMap<ServerName, HServerLoad>();
75  
76    // TODO: This is strange to have two maps but HSI above is used on both sides
77    /**
78     * Map from full server-instance name to the RPC connection for this server.
79     */
80    private final Map<ServerName, HRegionInterface> serverConnections =
81      new HashMap<ServerName, HRegionInterface>();
82  
83    /**
84     * List of region servers <ServerName> that should not get any more new
85     * regions.
86     */
87    private final ArrayList<ServerName> drainingServers =
88      new ArrayList<ServerName>();
89  
90    private final Server master;
91    private final MasterServices services;
92    private final HConnection connection;
93  
94    private final DeadServer deadservers;
95  
96    private final long maxSkew;
97  
98    /**
99     * Constructor.
100    * @param master
101    * @param services
102    * @throws ZooKeeperConnectionException
103    */
104   public ServerManager(final Server master, final MasterServices services)
105       throws ZooKeeperConnectionException {
106     this(master, services, true);
107   }
108 
109   ServerManager(final Server master, final MasterServices services,
110       final boolean connect) throws ZooKeeperConnectionException {
111     this.master = master;
112     this.services = services;
113     Configuration c = master.getConfiguration();
114     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
115     this.deadservers = new DeadServer();
116     this.connection = connect ? HConnectionManager.getConnection(c) : null;
117   }
118 
119   /**
120    * Let the server manager know a new regionserver has come online
121    * @param ia The remote address
122    * @param port The remote port
123    * @param serverStartcode
124    * @param serverCurrentTime The current time of the region server in ms
125    * @return The ServerName we know this server as.
126    * @throws IOException
127    */
128   ServerName regionServerStartup(final InetAddress ia, final int port,
129     final long serverStartcode, long serverCurrentTime)
130   throws IOException {
131     // Test for case where we get a region startup message from a regionserver
132     // that has been quickly restarted but whose znode expiration handler has
133     // not yet run, or from a server whose fail we are currently processing.
134     // Test its host+port combo is present in serverAddresstoServerInfo.  If it
135     // is, reject the server and trigger its expiration. The next time it comes
136     // in, it should have been removed from serverAddressToServerInfo and queued
137     // for processing by ProcessServerShutdown.
138     ServerName sn = new ServerName(ia.getHostName(), port, serverStartcode);
139     checkClockSkew(sn, serverCurrentTime);
140     checkIsDead(sn, "STARTUP");
141     checkAlreadySameHostPort(sn);
142     recordNewServer(sn, HServerLoad.EMPTY_HSERVERLOAD);
143     return sn;
144   }
145 
146   void regionServerReport(ServerName sn, HServerLoad hsl)
147   throws YouAreDeadException, PleaseHoldException {
148     checkIsDead(sn, "REPORT");
149     if (!this.onlineServers.containsKey(sn)) {
150       // Already have this host+port combo and its just different start code?
151       checkAlreadySameHostPort(sn);
152       // Just let the server in. Presume master joining a running cluster.
153       // recordNewServer is what happens at the end of reportServerStartup.
154       // The only thing we are skipping is passing back to the regionserver
155       // the ServerName to use. Here we presume a master has already done
156       // that so we'll press on with whatever it gave us for ServerName.
157       recordNewServer(sn, hsl);
158     } else {
159       this.onlineServers.put(sn, hsl);
160     }
161   }
162 
163   /**
164    * Test to see if we have a server of same host and port already.
165    * @param serverName
166    * @throws PleaseHoldException
167    */
168   void checkAlreadySameHostPort(final ServerName serverName)
169   throws PleaseHoldException {
170     ServerName existingServer =
171       ServerName.findServerWithSameHostnamePort(getOnlineServersList(), serverName);
172     if (existingServer != null) {
173       String message = "Server serverName=" + serverName +
174         " rejected; we already have " + existingServer.toString() +
175         " registered with same hostname and port";
176       LOG.info(message);
177       if (existingServer.getStartcode() < serverName.getStartcode()) {
178         LOG.info("Triggering server recovery; existingServer " +
179           existingServer + " looks stale, new server:" + serverName);
180         expireServer(existingServer);
181       }
182       throw new PleaseHoldException(message);
183     }
184   }
185 
186   /**
187    * Checks if the clock skew between the server and the master. If the clock
188    * skew is too much it will throw an Exception.
189    * @param serverName Incoming servers's name
190    * @param serverCurrentTime
191    * @throws ClockOutOfSyncException
192    */
193   private void checkClockSkew(final ServerName serverName,
194       final long serverCurrentTime)
195   throws ClockOutOfSyncException {
196     long skew = System.currentTimeMillis() - serverCurrentTime;
197     if (skew > maxSkew) {
198       String message = "Server " + serverName + " has been " +
199         "rejected; Reported time is too far out of sync with master.  " +
200         "Time difference of " + skew + "ms > max allowed of " + maxSkew + "ms";
201       LOG.warn(message);
202       throw new ClockOutOfSyncException(message);
203     }
204   }
205 
206   /**
207    * If this server is on the dead list, reject it with a YouAreDeadException.
208    * If it was dead but came back with a new start code, remove the old entry
209    * from the dead list.
210    * @param serverName
211    * @param what START or REPORT
212    * @throws YouAreDeadException
213    */
214   private void checkIsDead(final ServerName serverName, final String what)
215       throws YouAreDeadException {
216     if (this.deadservers.isDeadServer(serverName)) {
217       // host name, port and start code all match with existing one of the
218       // dead servers. So, this server must be dead.
219       String message = "Server " + what + " rejected; currently processing " +
220           serverName + " as dead server";
221       LOG.debug(message);
222       throw new YouAreDeadException(message);
223     }
224 
225     if (this.deadservers.cleanPreviousInstance(serverName)) {
226       // This server has now become alive after we marked it as dead.
227       // We removed it's previous entry from the dead list to reflect it.
228       LOG.debug(what + ":" + " Server " + serverName + " came back up," +
229           " removed it from the dead servers list");
230     }
231   }
232 
233   /**
234    * Adds the onlineServers list.
235    * @param hsl
236    * @param serverName The remote servers name.
237    */
238   void recordNewServer(final ServerName serverName, final  HServerLoad hsl) {
239     LOG.info("Registering server=" + serverName);
240     this.onlineServers.put(serverName, hsl);
241     this.serverConnections.remove(serverName);
242   }
243 
244   /**
245    * @param serverName
246    * @return HServerLoad if serverName is known else null
247    */
248   public HServerLoad getLoad(final ServerName serverName) {
249     return this.onlineServers.get(serverName);
250   }
251 
252   /**
253    * @param address
254    * @return HServerLoad if serverName is known else null
255    * @deprecated Use {@link #getLoad(HServerAddress)}
256    */
257   public HServerLoad getLoad(final HServerAddress address) {
258     ServerName sn = new ServerName(address.toString(), ServerName.NON_STARTCODE);
259     ServerName actual =
260       ServerName.findServerWithSameHostnamePort(this.getOnlineServersList(), sn);
261     return actual == null? null: getLoad(actual);
262   }
263 
264   /**
265    * Compute the average load across all region servers.
266    * Currently, this uses a very naive computation - just uses the number of
267    * regions being served, ignoring stats about number of requests.
268    * @return the average load
269    */
270   public double getAverageLoad() {
271     int totalLoad = 0;
272     int numServers = 0;
273     double averageLoad = 0.0;
274     for (HServerLoad hsl: this.onlineServers.values()) {
275         numServers++;
276         totalLoad += hsl.getNumberOfRegions();
277     }
278     averageLoad = (double)totalLoad / (double)numServers;
279     return averageLoad;
280   }
281 
282   /** @return the count of active regionservers */
283   int countOfRegionServers() {
284     // Presumes onlineServers is a concurrent map
285     return this.onlineServers.size();
286   }
287 
288   /**
289    * @return Read-only map of servers to serverinfo
290    */
291   public Map<ServerName, HServerLoad> getOnlineServers() {
292     // Presumption is that iterating the returned Map is OK.
293     synchronized (this.onlineServers) {
294       return Collections.unmodifiableMap(this.onlineServers);
295     }
296   }
297 
298   public Set<ServerName> getDeadServers() {
299     return this.deadservers.clone();
300   }
301 
302   /**
303    * Checks if any dead servers are currently in progress.
304    * @return true if any RS are being processed as dead, false if not
305    */
306   public boolean areDeadServersInProgress() {
307     return this.deadservers.areDeadServersInProgress();
308   }
309 
310   void letRegionServersShutdown() {
311     synchronized (onlineServers) {
312       while (!onlineServers.isEmpty()) {
313         StringBuilder sb = new StringBuilder();
314         for (ServerName key: this.onlineServers.keySet()) {
315           if (sb.length() > 0) {
316             sb.append(", ");
317           }
318           sb.append(key);
319         }
320         LOG.info("Waiting on regionserver(s) to go down " + sb.toString());
321         try {
322           this.onlineServers.wait(1000);
323         } catch (InterruptedException e) {
324           // continue
325         }
326       }
327     }
328   }
329 
330   /*
331    * Expire the passed server.  Add it to list of deadservers and queue a
332    * shutdown processing.
333    */
334   public synchronized void expireServer(final ServerName serverName) {
335     if (!this.onlineServers.containsKey(serverName)) {
336       LOG.warn("Received expiration of " + serverName +
337         " but server is not currently online");
338       return;
339     }
340     if (this.deadservers.contains(serverName)) {
341       // TODO: Can this happen?  It shouldn't be online in this case?
342       LOG.warn("Received expiration of " + serverName +
343           " but server shutdown is already in progress");
344       return;
345     }
346     // Remove the server from the known servers lists and update load info BUT
347     // add to deadservers first; do this so it'll show in dead servers list if
348     // not in online servers list.
349     this.deadservers.add(serverName);
350     this.onlineServers.remove(serverName);
351     this.serverConnections.remove(serverName);
352     // If cluster is going down, yes, servers are going to be expiring; don't
353     // process as a dead server
354     if (this.clusterShutdown) {
355       LOG.info("Cluster shutdown set; " + serverName +
356         " expired; onlineServers=" + this.onlineServers.size());
357       if (this.onlineServers.isEmpty()) {
358         master.stop("Cluster shutdown set; onlineServer=0");
359       }
360       return;
361     }
362 
363     boolean carryingRoot = services.getAssignmentManager().isCarryingRoot(serverName);
364     boolean carryingMeta = services.getAssignmentManager().isCarryingMeta(serverName);
365     if (carryingRoot || carryingMeta) {
366       this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
367         this.services, this.deadservers, serverName, carryingRoot, carryingMeta));
368     } else {
369       this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
370         this.services, this.deadservers, serverName, true));
371     }
372     LOG.debug("Added=" + serverName +
373       " to dead servers, submitted shutdown handler to be executed, root=" +
374         carryingRoot + ", meta=" + carryingMeta);
375   }
376 
377   /*
378    * Remove the server from the drain list.
379    */
380   public boolean removeServerFromDrainList(final ServerName sn) {
381     // Warn if the server (sn) is not online.  ServerName is of the form:
382     // <hostname> , <port> , <startcode>
383 
384     if (!this.isServerOnline(sn)) {
385       LOG.warn("Server " + sn + " is not currently online. " +
386                "Removing from draining list anyway, as requested.");
387     }
388     // Remove the server from the draining servers lists.
389     return this.drainingServers.remove(sn);
390   }
391 
392   /*
393    * Add the server to the drain list.
394    */
395   public boolean addServerToDrainList(final ServerName sn) {
396     // Warn if the server (sn) is not online.  ServerName is of the form:
397     // <hostname> , <port> , <startcode>
398 
399     if (!this.isServerOnline(sn)) {
400       LOG.warn("Server " + sn + " is not currently online. " +
401                "Ignoring request to add it to draining list.");
402       return false;
403     }
404     // Add the server to the draining servers lists, if it's not already in
405     // it.
406     if (this.drainingServers.contains(sn)) {
407       LOG.warn("Server " + sn + " is already in the draining server list." +
408                "Ignoring request to add it again.");
409       return false;
410     }
411     return this.drainingServers.add(sn);
412   }
413 
414   // RPC methods to region servers
415 
416   /**
417    * Sends an OPEN RPC to the specified server to open the specified region.
418    * <p>
419    * Open should not fail but can if server just crashed.
420    * <p>
421    * @param server server to open a region
422    * @param region region to open
423    * @param versionOfOfflineNode that needs to be present in the offline node
424    * when RS tries to change the state from OFFLINE to other states.
425    */
426   public RegionOpeningState sendRegionOpen(final ServerName server,
427       HRegionInfo region, int versionOfOfflineNode)
428   throws IOException {
429     HRegionInterface hri = getServerConnection(server);
430     if (hri == null) {
431       LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
432         " failed because no RPC connection found to this server");
433       return RegionOpeningState.FAILED_OPENING;
434     }
435     return (versionOfOfflineNode == -1) ? hri.openRegion(region) : hri
436         .openRegion(region, versionOfOfflineNode);
437   }
438 
439   /**
440    * Sends an OPEN RPC to the specified server to open the specified region.
441    * <p>
442    * Open should not fail but can if server just crashed.
443    * <p>
444    * @param server server to open a region
445    * @param regions regions to open
446    */
447   public void sendRegionOpen(ServerName server, List<HRegionInfo> regions)
448   throws IOException {
449     HRegionInterface hri = getServerConnection(server);
450     if (hri == null) {
451       LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
452         " failed because no RPC connection found to this server");
453       return;
454     }
455     hri.openRegions(regions);
456   }
457 
458   /**
459    * Sends an CLOSE RPC to the specified server to close the specified region.
460    * <p>
461    * A region server could reject the close request because it either does not
462    * have the specified region or the region is being split.
463    * @param server server to open a region
464    * @param region region to open
465    * @return true if server acknowledged close, false if not
466    * @throws IOException
467    */
468   public boolean sendRegionClose(ServerName server, HRegionInfo region)
469   throws IOException {
470     if (server == null) throw new NullPointerException("Passed server is null");
471     HRegionInterface hri = getServerConnection(server);
472     if (hri == null) {
473       throw new IOException("Attempting to send CLOSE RPC to server " +
474         server.toString() + " for region " +
475         region.getRegionNameAsString() +
476         " failed because no RPC connection found to this server");
477     }
478     return hri.closeRegion(region);
479   }
480 
481   /**
482    * @param sn
483    * @return
484    * @throws IOException
485    * @throws RetriesExhaustedException wrapping a ConnectException if failed
486    * putting up proxy.
487    */
488   private HRegionInterface getServerConnection(final ServerName sn)
489   throws IOException {
490     HRegionInterface hri = this.serverConnections.get(sn.toString());
491     if (hri == null) {
492       LOG.debug("New connection to " + sn.toString());
493       hri = this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
494       this.serverConnections.put(sn, hri);
495     }
496     return hri;
497   }
498 
499   /**
500    * Waits for the regionservers to report in.
501    * @throws InterruptedException
502    */
503   public void waitForRegionServers(MonitoredTask status)
504   throws InterruptedException {
505     long interval = this.master.getConfiguration().
506       getLong("hbase.master.wait.on.regionservers.interval", 3000);
507     // So, number of regionservers > 0 and its been n since last check in, break,
508     // else just stall here
509     int count = 0;
510     for (int oldcount = countOfRegionServers(); !this.master.isStopped();) {
511       Thread.sleep(interval);
512       count = countOfRegionServers();
513       if (count == oldcount && count > 0) break;
514 
515       String msg;
516       if (count == 0) {
517         msg = "Waiting on regionserver(s) to checkin";
518       } else {
519         msg = "Waiting on regionserver(s) count to settle; currently=" + count;
520       }
521       LOG.info(msg);
522       status.setStatus(msg);
523       oldcount = count;
524     }
525   }
526 
527   /**
528    * @return A copy of the internal list of online servers.
529    */
530   public List<ServerName> getOnlineServersList() {
531     // TODO: optimize the load balancer call so we don't need to make a new list
532     // TODO: FIX. THIS IS POPULAR CALL.
533     return new ArrayList<ServerName>(this.onlineServers.keySet());
534   }
535 
536   /**
537    * @return A copy of the internal list of draining servers.
538    */
539   public List<ServerName> getDrainingServersList() {
540     return new ArrayList<ServerName>(this.drainingServers);
541   }
542 
543   public boolean isServerOnline(ServerName serverName) {
544     return onlineServers.containsKey(serverName);
545   }
546 
547   public void shutdownCluster() {
548     this.clusterShutdown = true;
549     this.master.stop("Cluster shutdown requested");
550   }
551 
552   public boolean isClusterShutdown() {
553     return this.clusterShutdown;
554   }
555 
556   /**
557    * Stop the ServerManager.  Currently closes the connection to the master.
558    */
559   public void stop() {
560     if (connection != null) {
561       try {
562         connection.close();
563       } catch (IOException e) {
564         LOG.error("Attempt to close connection to master failed", e);
565       }
566     }
567   }
568 }