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