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