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 org.apache.hadoop.hbase.Abortable; 021import org.apache.yetus.audience.InterfaceAudience; 022import org.apache.zookeeper.KeeperException; 023import org.slf4j.Logger; 024import org.slf4j.LoggerFactory; 025 026import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 027import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; 028 029/** 030 * Tracker on cluster settings up in zookeeper. This is not related to 031 * {@link org.apache.hadoop.hbase.ClusterStatus}. That class is a data structure that holds snapshot 032 * of current view on cluster. This class is about tracking cluster attributes up in zookeeper. 033 */ 034@InterfaceAudience.Private 035public class ClusterStatusTracker extends ZKNodeTracker { 036 private static final Logger LOG = LoggerFactory.getLogger(ClusterStatusTracker.class); 037 038 /** 039 * Creates a cluster status tracker. 040 * <p> 041 * After construction, use {@link #start} to kick off tracking. 042 * @param watcher reference to the {@link ZKWatcher} which also contains configuration and 043 * constants 044 * @param abortable used to abort if a fatal error occurs 045 */ 046 public ClusterStatusTracker(ZKWatcher watcher, Abortable abortable) { 047 super(watcher, watcher.getZNodePaths().clusterStateZNode, abortable); 048 } 049 050 /** 051 * Checks if cluster is up. 052 * @return true if the cluster up ('shutdown' is its name up in zk) znode exists with data, false 053 * if not 054 */ 055 public boolean isClusterUp() { 056 return super.getData(false) != null; 057 } 058 059 /** 060 * Sets the cluster as up. 061 * @throws KeeperException unexpected zk exception 062 */ 063 public void setClusterUp() throws KeeperException { 064 byte[] upData = toByteArray(); 065 try { 066 ZKUtil.createAndWatch(watcher, watcher.getZNodePaths().clusterStateZNode, upData); 067 } catch (KeeperException.NodeExistsException nee) { 068 ZKUtil.setData(watcher, watcher.getZNodePaths().clusterStateZNode, upData); 069 } 070 } 071 072 /** 073 * Sets the cluster as down by deleting the znode. 074 * @throws KeeperException unexpected zk exception 075 */ 076 public void setClusterDown() throws KeeperException { 077 try { 078 ZKUtil.deleteNode(watcher, watcher.getZNodePaths().clusterStateZNode); 079 } catch (KeeperException.NoNodeException nne) { 080 LOG.warn("Attempted to set cluster as down but already down, cluster " + "state node (" 081 + watcher.getZNodePaths().clusterStateZNode + ") not found"); 082 } 083 } 084 085 /** Returns Content of the clusterup znode as a serialized pb with the pb magic as prefix. */ 086 static byte[] toByteArray() { 087 ZooKeeperProtos.ClusterUp.Builder builder = ZooKeeperProtos.ClusterUp.newBuilder(); 088 builder.setStartDate(new java.util.Date().toString()); 089 return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); 090 } 091}