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.util.UUID; 021import org.apache.hadoop.hbase.Abortable; 022import org.apache.hadoop.hbase.ClusterId; 023import org.apache.hadoop.hbase.exceptions.DeserializationException; 024import org.apache.yetus.audience.InterfaceAudience; 025import org.apache.zookeeper.KeeperException; 026 027/** 028 * Publishes and synchronizes a unique identifier specific to a given HBase cluster. The stored 029 * identifier is read from the file system by the active master on startup, and is subsequently 030 * available to all watchers (including clients). 031 */ 032@InterfaceAudience.Private 033public class ZKClusterId { 034 private final ZKWatcher watcher; 035 private final Abortable abortable; 036 private String id; 037 038 public ZKClusterId(ZKWatcher watcher, Abortable abortable) { 039 this.watcher = watcher; 040 this.abortable = abortable; 041 } 042 043 public boolean hasId() { 044 return getId() != null; 045 } 046 047 public String getId() { 048 try { 049 if (id == null) { 050 id = readClusterIdZNode(watcher); 051 } 052 } catch (KeeperException ke) { 053 abortable.abort("Unexpected exception from ZooKeeper reading cluster ID", ke); 054 } 055 return id; 056 } 057 058 public static String readClusterIdZNode(ZKWatcher watcher) throws KeeperException { 059 if (ZKUtil.checkExists(watcher, watcher.getZNodePaths().clusterIdZNode) != -1) { 060 byte[] data; 061 try { 062 data = ZKUtil.getData(watcher, watcher.getZNodePaths().clusterIdZNode); 063 } catch (InterruptedException e) { 064 Thread.currentThread().interrupt(); 065 return null; 066 } 067 if (data != null) { 068 try { 069 return ClusterId.parseFrom(data).toString(); 070 } catch (DeserializationException e) { 071 throw ZKUtil.convert(e); 072 } 073 } 074 } 075 return null; 076 } 077 078 public static void setClusterId(ZKWatcher watcher, ClusterId id) throws KeeperException { 079 ZKUtil.createSetData(watcher, watcher.getZNodePaths().clusterIdZNode, id.toByteArray()); 080 } 081 082 /** 083 * Get the UUID for the provided ZK watcher. Doesn't handle any ZK exceptions 084 * @param zkw watcher connected to an ensemble 085 * @return the UUID read from zookeeper 086 * @throws KeeperException if a ZooKeeper operation fails 087 */ 088 public static UUID getUUIDForCluster(ZKWatcher zkw) throws KeeperException { 089 String uuid = readClusterIdZNode(zkw); 090 return uuid == null ? null : UUID.fromString(uuid); 091 } 092}