001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.zookeeper; 019 020import java.io.IOException; 021import java.io.InterruptedIOException; 022 023import java.util.ArrayList; 024import java.util.Collections; 025import java.util.Comparator; 026import java.util.List; 027import org.apache.hadoop.hbase.Abortable; 028import org.apache.hadoop.hbase.HConstants; 029import org.apache.hadoop.hbase.ServerName; 030import org.apache.hadoop.hbase.exceptions.DeserializationException; 031import org.apache.yetus.audience.InterfaceAudience; 032import org.apache.zookeeper.KeeperException; 033import org.apache.zookeeper.data.Stat; 034 035import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 036import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 037import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; 038import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; 039 040/** 041 * Manages the location of the current active Master for the RegionServer. 042 * <p> 043 * Listens for ZooKeeper events related to the master address. The node 044 * <code>/master</code> will contain the address of the current master. 045 * This listener is interested in 046 * <code>NodeDeleted</code> and <code>NodeCreated</code> events on 047 * <code>/master</code>. 048 * <p> 049 * Utilizes {@link ZKNodeTracker} for zk interactions. 050 * <p> 051 * You can get the current master via {@link #getMasterAddress()} or via 052 * {@link #getMasterAddress(ZKWatcher)} if you do not have a running 053 * instance of this Tracker in your context. 054 * <p> 055 * This class also includes utility for interacting with the master znode, for 056 * writing and reading the znode content. 057 */ 058@InterfaceAudience.Private 059public class MasterAddressTracker extends ZKNodeTracker { 060 /** 061 * Construct a master address listener with the specified 062 * <code>zookeeper</code> reference. 063 * <p> 064 * This constructor does not trigger any actions, you must call methods 065 * explicitly. Normally you will just want to execute {@link #start()} to 066 * begin tracking of the master address. 067 * 068 * @param watcher zk reference and watcher 069 * @param abortable abortable in case of fatal error 070 */ 071 public MasterAddressTracker(ZKWatcher watcher, Abortable abortable) { 072 super(watcher, watcher.getZNodePaths().masterAddressZNode, abortable); 073 } 074 075 /** 076 * Get the address of the current master if one is available. Returns null 077 * if no current master. 078 * @return Server name or null if timed out. 079 */ 080 public ServerName getMasterAddress() { 081 return getMasterAddress(false); 082 } 083 084 /** 085 * Get the info port of the current master of one is available. 086 * Return 0 if no current master or zookeeper is unavailable 087 * @return info port or 0 if timed out 088 */ 089 public int getMasterInfoPort() { 090 try { 091 final ZooKeeperProtos.Master master = parse(this.getData(false)); 092 if (master == null) { 093 return 0; 094 } 095 return master.getInfoPort(); 096 } catch (DeserializationException e) { 097 LOG.warn("Failed parse master zk node data", e); 098 return 0; 099 } 100 } 101 /** 102 * Get the info port of the backup master if it is available. 103 * Return 0 if no backup master or zookeeper is unavailable 104 * @param sn server name of backup master 105 * @return info port or 0 if timed out or exceptions 106 */ 107 public int getBackupMasterInfoPort(final ServerName sn) { 108 String backupZNode = ZNodePaths.joinZNode(watcher.getZNodePaths().backupMasterAddressesZNode, 109 sn.toString()); 110 try { 111 byte[] data = ZKUtil.getData(watcher, backupZNode); 112 final ZooKeeperProtos.Master backup = parse(data); 113 if (backup == null) { 114 return 0; 115 } 116 return backup.getInfoPort(); 117 } catch (Exception e) { 118 LOG.warn("Failed to get backup master: " + sn + "'s info port.", e); 119 return 0; 120 } 121 } 122 123 /** 124 * Get the address of the current master if one is available. Returns null 125 * if no current master. If refresh is set, try to load the data from ZK again, 126 * otherwise, cached data will be used. 127 * 128 * @param refresh whether to refresh the data by calling ZK directly. 129 * @return Server name or null if timed out. 130 */ 131 public ServerName getMasterAddress(final boolean refresh) { 132 try { 133 return ProtobufUtil.parseServerNameFrom(super.getData(refresh)); 134 } catch (DeserializationException e) { 135 LOG.warn("Failed parse", e); 136 return null; 137 } 138 } 139 140 /** 141 * Get master address. 142 * Use this instead of {@link #getMasterAddress()} if you do not have an 143 * instance of this tracker in your context. 144 * @param zkw ZKWatcher to use 145 * @return ServerName stored in the the master address znode or null if no 146 * znode present. 147 * @throws KeeperException if a ZooKeeper operation fails 148 * @throws IOException if the address of the ZooKeeper master cannot be retrieved 149 */ 150 public static ServerName getMasterAddress(final ZKWatcher zkw) 151 throws KeeperException, IOException { 152 byte [] data; 153 try { 154 data = ZKUtil.getData(zkw, zkw.getZNodePaths().masterAddressZNode); 155 } catch (InterruptedException e) { 156 throw new InterruptedIOException(); 157 } 158 // TODO javadoc claims we return null in this case. :/ 159 if (data == null){ 160 throw new IOException("Can't get master address from ZooKeeper; znode data == null"); 161 } 162 try { 163 return ProtobufUtil.parseServerNameFrom(data); 164 } catch (DeserializationException e) { 165 KeeperException ke = new KeeperException.DataInconsistencyException(); 166 ke.initCause(e); 167 throw ke; 168 } 169 } 170 171 /** 172 * Get master info port. 173 * Use this instead of {@link #getMasterInfoPort()} if you do not have an 174 * instance of this tracker in your context. 175 * @param zkw ZKWatcher to use 176 * @return master info port in the the master address znode or null if no 177 * znode present. 178 * // TODO can't return null for 'int' return type. non-static verison returns 0 179 * @throws KeeperException if a ZooKeeper operation fails 180 * @throws IOException if the address of the ZooKeeper master cannot be retrieved 181 */ 182 public static int getMasterInfoPort(final ZKWatcher zkw) throws KeeperException, IOException { 183 byte[] data; 184 try { 185 data = ZKUtil.getData(zkw, zkw.getZNodePaths().masterAddressZNode); 186 } catch (InterruptedException e) { 187 throw new InterruptedIOException(); 188 } 189 // TODO javadoc claims we return null in this case. :/ 190 if (data == null) { 191 throw new IOException("Can't get master address from ZooKeeper; znode data == null"); 192 } 193 try { 194 return parse(data).getInfoPort(); 195 } catch (DeserializationException e) { 196 KeeperException ke = new KeeperException.DataInconsistencyException(); 197 ke.initCause(e); 198 throw ke; 199 } 200 } 201 202 /** 203 * Set master address into the <code>master</code> znode or into the backup 204 * subdirectory of backup masters; switch off the passed in <code>znode</code> 205 * path. 206 * @param zkw The ZKWatcher to use. 207 * @param znode Where to create the znode; could be at the top level or it 208 * could be under backup masters 209 * @param master ServerName of the current master must not be null. 210 * @return true if node created, false if not; a watch is set in both cases 211 * @throws KeeperException if a ZooKeeper operation fails 212 */ 213 public static boolean setMasterAddress(final ZKWatcher zkw, 214 final String znode, final ServerName master, int infoPort) 215 throws KeeperException { 216 return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort)); 217 } 218 219 /** 220 * Check if there is a master available. 221 * @return true if there is a master set, false if not. 222 */ 223 public boolean hasMaster() { 224 return super.getData(false) != null; 225 } 226 227 /** 228 * @param sn must not be null 229 * @return Content of the master znode as a serialized pb with the pb 230 * magic as prefix. 231 */ 232 static byte[] toByteArray(final ServerName sn, int infoPort) { 233 ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder(); 234 HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder(); 235 snbuilder.setHostName(sn.getHostname()); 236 snbuilder.setPort(sn.getPort()); 237 snbuilder.setStartCode(sn.getStartcode()); 238 mbuilder.setMaster(snbuilder.build()); 239 mbuilder.setRpcVersion(HConstants.RPC_CURRENT_VERSION); 240 mbuilder.setInfoPort(infoPort); 241 return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray()); 242 } 243 244 /** 245 * @param data zookeeper data. may be null 246 * @return pb object of master, null if no active master 247 * @throws DeserializationException if the parsing fails 248 */ 249 public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException { 250 if (data == null) { 251 return null; 252 } 253 int prefixLen = ProtobufUtil.lengthOfPBMagic(); 254 try { 255 return ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen); 256 } catch (InvalidProtocolBufferException e) { 257 throw new DeserializationException(e); 258 } 259 } 260 /** 261 * delete the master znode if its content is same as the parameter 262 * @param zkw must not be null 263 * @param content must not be null 264 */ 265 public static boolean deleteIfEquals(ZKWatcher zkw, final String content) { 266 if (content == null){ 267 throw new IllegalArgumentException("Content must not be null"); 268 } 269 270 try { 271 Stat stat = new Stat(); 272 byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.getZNodePaths().masterAddressZNode, stat); 273 ServerName sn = ProtobufUtil.parseServerNameFrom(data); 274 if (sn != null && content.equals(sn.toString())) { 275 return (ZKUtil.deleteNode(zkw, zkw.getZNodePaths().masterAddressZNode, stat.getVersion())); 276 } 277 } catch (KeeperException e) { 278 LOG.warn("Can't get or delete the master znode", e); 279 } catch (DeserializationException e) { 280 LOG.warn("Can't get or delete the master znode", e); 281 } 282 283 return false; 284 } 285 286 public List<ServerName> getBackupMasters() throws InterruptedIOException { 287 return getBackupMastersAndRenewWatch(watcher); 288 } 289 290 /** 291 * Retrieves the list of registered backup masters and renews a watch on the znode for children 292 * updates. 293 * @param zkw Zookeeper watcher to use 294 * @return List of backup masters. 295 * @throws InterruptedIOException if there is any issue fetching the required data from Zookeeper. 296 */ 297 public static List<ServerName> getBackupMastersAndRenewWatch( 298 ZKWatcher zkw) throws InterruptedIOException { 299 // Build Set of backup masters from ZK nodes 300 List<String> backupMasterStrings = null; 301 try { 302 backupMasterStrings = ZKUtil.listChildrenAndWatchForNewChildren(zkw, 303 zkw.getZNodePaths().backupMasterAddressesZNode); 304 } catch (KeeperException e) { 305 LOG.warn(zkw.prefix("Unable to list backup servers"), e); 306 } 307 308 List<ServerName> backupMasters = Collections.emptyList(); 309 if (backupMasterStrings != null && !backupMasterStrings.isEmpty()) { 310 backupMasters = new ArrayList<>(backupMasterStrings.size()); 311 for (String s: backupMasterStrings) { 312 try { 313 byte [] bytes; 314 try { 315 bytes = ZKUtil.getData(zkw, ZNodePaths.joinZNode( 316 zkw.getZNodePaths().backupMasterAddressesZNode, s)); 317 } catch (InterruptedException e) { 318 throw new InterruptedIOException(); 319 } 320 if (bytes != null) { 321 ServerName sn; 322 try { 323 sn = ProtobufUtil.parseServerNameFrom(bytes); 324 } catch (DeserializationException e) { 325 LOG.warn("Failed parse, skipping registering backup server", e); 326 continue; 327 } 328 backupMasters.add(sn); 329 } 330 } catch (KeeperException e) { 331 LOG.warn(zkw.prefix("Unable to get information about " + 332 "backup servers"), e); 333 } 334 } 335 backupMasters.sort(Comparator.comparing(ServerName::getServerName)); 336 } 337 return backupMasters; 338 } 339}