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 org.apache.hadoop.hbase.Abortable;
024import org.apache.hadoop.hbase.HConstants;
025import org.apache.hadoop.hbase.ServerName;
026import org.apache.hadoop.hbase.exceptions.DeserializationException;
027import org.apache.yetus.audience.InterfaceAudience;
028import org.apache.zookeeper.KeeperException;
029import org.apache.zookeeper.data.Stat;
030
031import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
032import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
033import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
034import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
035
036/**
037 * Manages the location of the current active Master for the RegionServer.
038 * <p>
039 * Listens for ZooKeeper events related to the master address. The node
040 * <code>/master</code> will contain the address of the current master.
041 * This listener is interested in
042 * <code>NodeDeleted</code> and <code>NodeCreated</code> events on
043 * <code>/master</code>.
044 * <p>
045 * Utilizes {@link ZKNodeTracker} for zk interactions.
046 * <p>
047 * You can get the current master via {@link #getMasterAddress()} or via
048 * {@link #getMasterAddress(ZKWatcher)} if you do not have a running
049 * instance of this Tracker in your context.
050 * <p>
051 * This class also includes utility for interacting with the master znode, for
052 * writing and reading the znode content.
053 */
054@InterfaceAudience.Private
055public class MasterAddressTracker extends ZKNodeTracker {
056  /**
057   * Construct a master address listener with the specified
058   * <code>zookeeper</code> reference.
059   * <p>
060   * This constructor does not trigger any actions, you must call methods
061   * explicitly.  Normally you will just want to execute {@link #start()} to
062   * begin tracking of the master address.
063   *
064   * @param watcher zk reference and watcher
065   * @param abortable abortable in case of fatal error
066   */
067  public MasterAddressTracker(ZKWatcher watcher, Abortable abortable) {
068    super(watcher, watcher.getZNodePaths().masterAddressZNode, abortable);
069  }
070
071  /**
072   * Get the address of the current master if one is available.  Returns null
073   * if no current master.
074   * @return Server name or null if timed out.
075   */
076  public ServerName getMasterAddress() {
077    return getMasterAddress(false);
078  }
079
080  /**
081   * Get the info port of the current master of one is available.
082   * Return 0 if no current master or zookeeper is unavailable
083   * @return info port or 0 if timed out
084   */
085  public int getMasterInfoPort() {
086    try {
087      final ZooKeeperProtos.Master master = parse(this.getData(false));
088      if (master == null) {
089        return 0;
090      }
091      return master.getInfoPort();
092    } catch (DeserializationException e) {
093      LOG.warn("Failed parse master zk node data", e);
094      return 0;
095    }
096  }
097  /**
098   * Get the info port of the backup master if it is available.
099   * Return 0 if no backup master or zookeeper is unavailable
100   * @param sn server name of backup master
101   * @return info port or 0 if timed out or exceptions
102   */
103  public int getBackupMasterInfoPort(final ServerName sn) {
104    String backupZNode = ZNodePaths.joinZNode(watcher.getZNodePaths().backupMasterAddressesZNode,
105      sn.toString());
106    try {
107      byte[] data = ZKUtil.getData(watcher, backupZNode);
108      final ZooKeeperProtos.Master backup = parse(data);
109      if (backup == null) {
110        return 0;
111      }
112      return backup.getInfoPort();
113    } catch (Exception e) {
114      LOG.warn("Failed to get backup master: " + sn + "'s info port.", e);
115      return 0;
116    }
117  }
118
119  /**
120   * Get the address of the current master if one is available.  Returns null
121   * if no current master. If refresh is set, try to load the data from ZK again,
122   * otherwise, cached data will be used.
123   *
124   * @param refresh whether to refresh the data by calling ZK directly.
125   * @return Server name or null if timed out.
126   */
127  public ServerName getMasterAddress(final boolean refresh) {
128    try {
129      return ProtobufUtil.parseServerNameFrom(super.getData(refresh));
130    } catch (DeserializationException e) {
131      LOG.warn("Failed parse", e);
132      return null;
133    }
134  }
135
136  /**
137   * Get master address.
138   * Use this instead of {@link #getMasterAddress()} if you do not have an
139   * instance of this tracker in your context.
140   * @param zkw ZKWatcher to use
141   * @return ServerName stored in the the master address znode or null if no
142   *         znode present.
143   * @throws KeeperException if a ZooKeeper operation fails
144   * @throws IOException if the address of the ZooKeeper master cannot be retrieved
145   */
146  public static ServerName getMasterAddress(final ZKWatcher zkw)
147          throws KeeperException, IOException {
148    byte [] data;
149    try {
150      data = ZKUtil.getData(zkw, zkw.getZNodePaths().masterAddressZNode);
151    } catch (InterruptedException e) {
152      throw new InterruptedIOException();
153    }
154    // TODO javadoc claims we return null in this case. :/
155    if (data == null){
156      throw new IOException("Can't get master address from ZooKeeper; znode data == null");
157    }
158    try {
159      return ProtobufUtil.parseServerNameFrom(data);
160    } catch (DeserializationException e) {
161      KeeperException ke = new KeeperException.DataInconsistencyException();
162      ke.initCause(e);
163      throw ke;
164    }
165  }
166
167  /**
168   * Get master info port.
169   * Use this instead of {@link #getMasterInfoPort()} if you do not have an
170   * instance of this tracker in your context.
171   * @param zkw ZKWatcher to use
172   * @return master info port in the the master address znode or null if no
173   *         znode present.
174   *         // TODO can't return null for 'int' return type. non-static verison returns 0
175   * @throws KeeperException if a ZooKeeper operation fails
176   * @throws IOException if the address of the ZooKeeper master cannot be retrieved
177   */
178  public static int getMasterInfoPort(final ZKWatcher zkw) throws KeeperException, IOException {
179    byte[] data;
180    try {
181      data = ZKUtil.getData(zkw, zkw.getZNodePaths().masterAddressZNode);
182    } catch (InterruptedException e) {
183      throw new InterruptedIOException();
184    }
185    // TODO javadoc claims we return null in this case. :/
186    if (data == null) {
187      throw new IOException("Can't get master address from ZooKeeper; znode data == null");
188    }
189    try {
190      return parse(data).getInfoPort();
191    } catch (DeserializationException e) {
192      KeeperException ke = new KeeperException.DataInconsistencyException();
193      ke.initCause(e);
194      throw ke;
195    }
196  }
197
198  /**
199   * Set master address into the <code>master</code> znode or into the backup
200   * subdirectory of backup masters; switch off the passed in <code>znode</code>
201   * path.
202   * @param zkw The ZKWatcher to use.
203   * @param znode Where to create the znode; could be at the top level or it
204   *              could be under backup masters
205   * @param master ServerName of the current master must not be null.
206   * @return true if node created, false if not; a watch is set in both cases
207   * @throws KeeperException if a ZooKeeper operation fails
208   */
209  public static boolean setMasterAddress(final ZKWatcher zkw,
210      final String znode, final ServerName master, int infoPort)
211    throws KeeperException {
212    return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort));
213  }
214
215  /**
216   * Check if there is a master available.
217   * @return true if there is a master set, false if not.
218   */
219  public boolean hasMaster() {
220    return super.getData(false) != null;
221  }
222
223  /**
224   * @param sn must not be null
225   * @return Content of the master znode as a serialized pb with the pb
226   *         magic as prefix.
227   */
228  static byte[] toByteArray(final ServerName sn, int infoPort) {
229    ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
230    HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
231    snbuilder.setHostName(sn.getHostname());
232    snbuilder.setPort(sn.getPort());
233    snbuilder.setStartCode(sn.getStartcode());
234    mbuilder.setMaster(snbuilder.build());
235    mbuilder.setRpcVersion(HConstants.RPC_CURRENT_VERSION);
236    mbuilder.setInfoPort(infoPort);
237    return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray());
238  }
239
240  /**
241   * @param data zookeeper data. may be null
242   * @return pb object of master, null if no active master
243   * @throws DeserializationException if the parsing fails
244   */
245  public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException {
246    if (data == null) {
247      return null;
248    }
249    int prefixLen = ProtobufUtil.lengthOfPBMagic();
250    try {
251      return ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
252    } catch (InvalidProtocolBufferException e) {
253      throw new DeserializationException(e);
254    }
255  }
256  /**
257   * delete the master znode if its content is same as the parameter
258   * @param zkw must not be null
259   * @param content must not be null
260   */
261  public static boolean deleteIfEquals(ZKWatcher zkw, final String content) {
262    if (content == null){
263      throw new IllegalArgumentException("Content must not be null");
264    }
265
266    try {
267      Stat stat = new Stat();
268      byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.getZNodePaths().masterAddressZNode, stat);
269      ServerName sn = ProtobufUtil.parseServerNameFrom(data);
270      if (sn != null && content.equals(sn.toString())) {
271        return (ZKUtil.deleteNode(zkw, zkw.getZNodePaths().masterAddressZNode, stat.getVersion()));
272      }
273    } catch (KeeperException e) {
274      LOG.warn("Can't get or delete the master znode", e);
275    } catch (DeserializationException e) {
276      LOG.warn("Can't get or delete the master znode", e);
277    }
278
279    return false;
280  }
281}