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 org.apache.hadoop.hbase.Abortable; 022import org.apache.hadoop.hbase.exceptions.DeserializationException; 023import org.apache.hadoop.hbase.util.Bytes; 024import org.apache.yetus.audience.InterfaceAudience; 025import org.apache.zookeeper.KeeperException; 026import org.slf4j.Logger; 027import org.slf4j.LoggerFactory; 028 029import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 030import org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos; 031 032/** 033 * Tracks the load balancer state up in ZK 034 */ 035@InterfaceAudience.Private 036public class LoadBalancerTracker extends ZKNodeTracker { 037 private static final Logger LOG = LoggerFactory.getLogger(LoadBalancerTracker.class); 038 039 public LoadBalancerTracker(ZKWatcher watcher, Abortable abortable) { 040 super(watcher, watcher.getZNodePaths().balancerZNode, abortable); 041 } 042 043 /** 044 * Return true if the balance switch is on, false otherwise 045 */ 046 public boolean isBalancerOn() { 047 byte[] upData = super.getData(false); 048 try { 049 // if data in ZK is null, use default of on. 050 return upData == null || parseFrom(upData).getBalancerOn(); 051 } catch (DeserializationException dex) { 052 LOG.error("ZK state for LoadBalancer could not be parsed {}", Bytes.toStringBinary(upData)); 053 // return false to be safe. 054 return false; 055 } 056 } 057 058 /** 059 * Set the balancer on/off. 060 * @param balancerOn true if the balancher should be on, false otherwise 061 * @throws KeeperException if a ZooKeeper operation fails 062 */ 063 public void setBalancerOn(boolean balancerOn) throws KeeperException { 064 byte[] upData = toByteArray(balancerOn); 065 066 try { 067 ZKUtil.setData(watcher, watcher.getZNodePaths().balancerZNode, upData); 068 } catch (KeeperException.NoNodeException nne) { 069 ZKUtil.createAndWatch(watcher, watcher.getZNodePaths().balancerZNode, upData); 070 } 071 super.nodeDataChanged(watcher.getZNodePaths().balancerZNode); 072 } 073 074 private byte[] toByteArray(boolean isBalancerOn) { 075 LoadBalancerProtos.LoadBalancerState.Builder builder = 076 LoadBalancerProtos.LoadBalancerState.newBuilder(); 077 builder.setBalancerOn(isBalancerOn); 078 return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); 079 } 080 081 private LoadBalancerProtos.LoadBalancerState parseFrom(byte[] pbBytes) 082 throws DeserializationException { 083 ProtobufUtil.expectPBMagicPrefix(pbBytes); 084 LoadBalancerProtos.LoadBalancerState.Builder builder = 085 LoadBalancerProtos.LoadBalancerState.newBuilder(); 086 try { 087 int magicLen = ProtobufUtil.lengthOfPBMagic(); 088 ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen); 089 } catch (IOException e) { 090 throw new DeserializationException(e); 091 } 092 return builder.build(); 093 } 094}