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