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; 020import java.io.IOException; 021import java.io.InterruptedIOException; 022import java.util.List; 023import java.util.Optional; 024import java.util.concurrent.atomic.AtomicBoolean; 025import org.apache.hadoop.hbase.Server; 026import org.apache.hadoop.hbase.ServerName; 027import org.apache.hadoop.hbase.ZNodeClearer; 028import org.apache.hadoop.hbase.exceptions.DeserializationException; 029import org.apache.hadoop.hbase.monitoring.MonitoredTask; 030import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; 031import org.apache.hadoop.hbase.zookeeper.ZKListener; 032import org.apache.hadoop.hbase.zookeeper.ZKUtil; 033import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 034import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 035import org.apache.yetus.audience.InterfaceAudience; 036import org.apache.zookeeper.KeeperException; 037import org.slf4j.Logger; 038import org.slf4j.LoggerFactory; 039import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; 040import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 041 042/** 043 * Handles everything on master-side related to master election. Keeps track of 044 * currently active master and registered backup masters. 045 * 046 * <p>Listens and responds to ZooKeeper notifications on the master znodes, 047 * both <code>nodeCreated</code> and <code>nodeDeleted</code>. 048 * 049 * <p>Contains blocking methods which will hold up backup masters, waiting 050 * for the active master to fail. 051 * 052 * <p>This class is instantiated in the HMaster constructor and the method 053 * #blockUntilBecomingActiveMaster() is called to wait until becoming 054 * the active master of the cluster. 055 */ 056@InterfaceAudience.Private 057public class ActiveMasterManager extends ZKListener { 058 private static final Logger LOG = LoggerFactory.getLogger(ActiveMasterManager.class); 059 060 final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false); 061 final AtomicBoolean clusterShutDown = new AtomicBoolean(false); 062 063 // This server's information. Package-private for child implementations. 064 int infoPort; 065 final ServerName sn; 066 final Server master; 067 068 // Active master's server name. Invalidated anytime active master changes (based on ZK 069 // notifications) and lazily fetched on-demand. 070 // ServerName is immutable, so we don't need heavy synchronization around it. 071 volatile ServerName activeMasterServerName; 072 // Registered backup masters. List is kept up to date based on ZK change notifications to 073 // backup znode. 074 private volatile ImmutableList<ServerName> backupMasters; 075 076 /** 077 * @param watcher ZK watcher 078 * @param sn ServerName 079 * @param master In an instance of a Master. 080 */ 081 ActiveMasterManager(ZKWatcher watcher, ServerName sn, Server master) 082 throws InterruptedIOException { 083 super(watcher); 084 watcher.registerListener(this); 085 this.sn = sn; 086 this.master = master; 087 updateBackupMasters(); 088 } 089 090 // will be set after jetty server is started 091 public void setInfoPort(int infoPort) { 092 this.infoPort = infoPort; 093 } 094 095 @Override 096 public void nodeCreated(String path) { 097 handle(path); 098 } 099 100 @Override 101 public void nodeChildrenChanged(String path) { 102 if (path.equals(watcher.getZNodePaths().backupMasterAddressesZNode)) { 103 try { 104 updateBackupMasters(); 105 } catch (InterruptedIOException ioe) { 106 LOG.error("Error updating backup masters", ioe); 107 } 108 } 109 } 110 111 @Override 112 public void nodeDeleted(String path) { 113 // We need to keep track of the cluster's shutdown status while 114 // we wait on the current master. We consider that, if the cluster 115 // was already in a "shutdown" state when we started, that this master 116 // is part of a new cluster that was started shortly after the old cluster 117 // shut down, so that state is now irrelevant. This means that the shutdown 118 // state must be set while we wait on the active master in order 119 // to shutdown this master. See HBASE-8519. 120 if(path.equals(watcher.getZNodePaths().clusterStateZNode) && !master.isStopped()) { 121 clusterShutDown.set(true); 122 } 123 handle(path); 124 } 125 126 void handle(final String path) { 127 if (path.equals(watcher.getZNodePaths().masterAddressZNode) && !master.isStopped()) { 128 handleMasterNodeChange(); 129 } 130 } 131 132 private void updateBackupMasters() throws InterruptedIOException { 133 backupMasters = 134 ImmutableList.copyOf(MasterAddressTracker.getBackupMastersAndRenewWatch(watcher)); 135 } 136 137 /** 138 * Fetches the active master's ServerName from zookeeper. 139 */ 140 private void fetchAndSetActiveMasterServerName() { 141 LOG.debug("Attempting to fetch active master sn from zk"); 142 try { 143 activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher); 144 } catch (IOException | KeeperException e) { 145 // Log and ignore for now and re-fetch later if needed. 146 LOG.error("Error fetching active master information", e); 147 } 148 } 149 150 public Optional<ServerName> getActiveMasterServerName() { 151 if (!clusterHasActiveMaster.get()) { 152 return Optional.empty(); 153 } 154 if (activeMasterServerName == null) { 155 fetchAndSetActiveMasterServerName(); 156 } 157 // It could still be null, but return whatever we have. 158 return Optional.ofNullable(activeMasterServerName); 159 } 160 161 /** 162 * Handle a change in the master node. Doesn't matter whether this was called 163 * from a nodeCreated or nodeDeleted event because there are no guarantees 164 * that the current state of the master node matches the event at the time of 165 * our next ZK request. 166 * 167 * <p>Uses the watchAndCheckExists method which watches the master address node 168 * regardless of whether it exists or not. If it does exist (there is an 169 * active master), it returns true. Otherwise it returns false. 170 * 171 * <p>A watcher is set which guarantees that this method will get called again if 172 * there is another change in the master node. 173 */ 174 private void handleMasterNodeChange() { 175 // Watch the node and check if it exists. 176 try { 177 synchronized(clusterHasActiveMaster) { 178 if (ZKUtil.watchAndCheckExists(watcher, watcher.getZNodePaths().masterAddressZNode)) { 179 // A master node exists, there is an active master 180 LOG.trace("A master is now available"); 181 clusterHasActiveMaster.set(true); 182 } else { 183 // Node is no longer there, cluster does not have an active master 184 LOG.debug("No master available. Notifying waiting threads"); 185 clusterHasActiveMaster.set(false); 186 // Notify any thread waiting to become the active master 187 clusterHasActiveMaster.notifyAll(); 188 } 189 // Reset the active master sn. Will be re-fetched later if needed. 190 // We don't want to make a synchronous RPC under a monitor. 191 activeMasterServerName = null; 192 } 193 } catch (KeeperException ke) { 194 master.abort("Received an unexpected KeeperException, aborting", ke); 195 } 196 } 197 198 /** 199 * Block until becoming the active master. 200 * 201 * Method blocks until there is not another active master and our attempt 202 * to become the new active master is successful. 203 * 204 * This also makes sure that we are watching the master znode so will be 205 * notified if another master dies. 206 * @param checkInterval the interval to check if the master is stopped 207 * @param startupStatus the monitor status to track the progress 208 * @return True if no issue becoming active master else false if another 209 * master was running or if some other problem (zookeeper, stop flag has been 210 * set on this Master) 211 */ 212 boolean blockUntilBecomingActiveMaster( 213 int checkInterval, MonitoredTask startupStatus) { 214 String backupZNode = ZNodePaths.joinZNode( 215 this.watcher.getZNodePaths().backupMasterAddressesZNode, this.sn.toString()); 216 while (!(master.isAborted() || master.isStopped())) { 217 startupStatus.setStatus("Trying to register in ZK as active master"); 218 // Try to become the active master, watch if there is another master. 219 // Write out our ServerName as versioned bytes. 220 try { 221 if (MasterAddressTracker.setMasterAddress(this.watcher, 222 this.watcher.getZNodePaths().masterAddressZNode, this.sn, infoPort)) { 223 224 // If we were a backup master before, delete our ZNode from the backup 225 // master directory since we are the active now) 226 if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) { 227 LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory"); 228 ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode); 229 } 230 // Save the znode in a file, this will allow to check if we crash in the launch scripts 231 ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString()); 232 233 // We are the master, return 234 startupStatus.setStatus("Successfully registered as active master."); 235 this.clusterHasActiveMaster.set(true); 236 activeMasterServerName = sn; 237 LOG.info("Registered as active master=" + this.sn); 238 return true; 239 } 240 241 // Invalidate the active master name so that subsequent requests do not get any stale 242 // master information. Will be re-fetched if needed. 243 activeMasterServerName = null; 244 // There is another active master running elsewhere or this is a restart 245 // and the master ephemeral node has not expired yet. 246 this.clusterHasActiveMaster.set(true); 247 248 String msg; 249 byte[] bytes = 250 ZKUtil.getDataAndWatch(this.watcher, this.watcher.getZNodePaths().masterAddressZNode); 251 if (bytes == null) { 252 msg = ("A master was detected, but went down before its address " + 253 "could be read. Attempting to become the next active master"); 254 } else { 255 ServerName currentMaster; 256 try { 257 currentMaster = ProtobufUtil.parseServerNameFrom(bytes); 258 } catch (DeserializationException e) { 259 LOG.warn("Failed parse", e); 260 // Hopefully next time around we won't fail the parse. Dangerous. 261 continue; 262 } 263 if (ServerName.isSameAddress(currentMaster, this.sn)) { 264 msg = ("Current master has this master's address, " + 265 currentMaster + "; master was restarted? Deleting node."); 266 // Hurry along the expiration of the znode. 267 ZKUtil.deleteNode(this.watcher, this.watcher.getZNodePaths().masterAddressZNode); 268 269 // We may have failed to delete the znode at the previous step, but 270 // we delete the file anyway: a second attempt to delete the znode is likely to fail 271 // again. 272 ZNodeClearer.deleteMyEphemeralNodeOnDisk(); 273 } else { 274 msg = "Another master is the active master, " + currentMaster + 275 "; waiting to become the next active master"; 276 } 277 } 278 LOG.info(msg); 279 startupStatus.setStatus(msg); 280 } catch (KeeperException ke) { 281 master.abort("Received an unexpected KeeperException, aborting", ke); 282 return false; 283 } 284 synchronized (this.clusterHasActiveMaster) { 285 while (clusterHasActiveMaster.get() && !master.isStopped()) { 286 try { 287 clusterHasActiveMaster.wait(checkInterval); 288 } catch (InterruptedException e) { 289 // We expect to be interrupted when a master dies, 290 // will fall out if so 291 LOG.debug("Interrupted waiting for master to die", e); 292 } 293 } 294 if (clusterShutDown.get()) { 295 this.master.stop( 296 "Cluster went down before this master became active"); 297 } 298 } 299 } 300 return false; 301 } 302 303 /** 304 * @return True if cluster has an active master. 305 */ 306 boolean hasActiveMaster() { 307 try { 308 if (ZKUtil.checkExists(watcher, watcher.getZNodePaths().masterAddressZNode) >= 0) { 309 return true; 310 } 311 } 312 catch (KeeperException ke) { 313 LOG.info("Received an unexpected KeeperException when checking " + 314 "isActiveMaster : "+ ke); 315 } 316 return false; 317 } 318 319 public void stop() { 320 try { 321 synchronized (clusterHasActiveMaster) { 322 // Master is already stopped, wake up the manager 323 // thread so that it can shutdown soon. 324 clusterHasActiveMaster.notifyAll(); 325 } 326 // If our address is in ZK, delete it on our way out 327 ServerName activeMaster = null; 328 try { 329 activeMaster = MasterAddressTracker.getMasterAddress(this.watcher); 330 } catch (IOException e) { 331 LOG.warn("Failed get of master address: " + e.toString()); 332 } 333 if (activeMaster != null && activeMaster.equals(this.sn)) { 334 ZKUtil.deleteNode(watcher, watcher.getZNodePaths().masterAddressZNode); 335 // We may have failed to delete the znode at the previous step, but 336 // we delete the file anyway: a second attempt to delete the znode is likely to fail again. 337 ZNodeClearer.deleteMyEphemeralNodeOnDisk(); 338 } 339 } catch (KeeperException e) { 340 LOG.debug(this.watcher.prefix("Failed delete of our master address node; " + 341 e.getMessage())); 342 } 343 } 344 345 /** 346 * @return list of registered backup masters. 347 */ 348 public List<ServerName> getBackupMasters() { 349 return backupMasters; 350 } 351}