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.master; 019 020import java.io.IOException; 021 022import org.apache.hadoop.conf.Configuration; 023import org.apache.hadoop.hbase.Abortable; 024import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker; 025import org.apache.hadoop.hbase.zookeeper.ZKUtil; 026import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 027import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 028import org.apache.hadoop.hbase.client.MasterSwitchType; 029import org.apache.hadoop.hbase.exceptions.DeserializationException; 030import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 031import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SwitchState; 032import org.apache.hadoop.hbase.util.Bytes; 033import org.apache.yetus.audience.InterfaceAudience; 034import org.apache.zookeeper.KeeperException; 035 036 037/** 038 * Tracks the switch of split and merge states in ZK 039 */ 040@InterfaceAudience.Private 041public class SplitOrMergeTracker { 042 043 private String splitZnode; 044 private String mergeZnode; 045 046 private SwitchStateTracker splitStateTracker; 047 private SwitchStateTracker mergeStateTracker; 048 049 public SplitOrMergeTracker(ZKWatcher watcher, Configuration conf, 050 Abortable abortable) { 051 try { 052 if (ZKUtil.checkExists(watcher, watcher.znodePaths.switchZNode) < 0) { 053 ZKUtil.createAndFailSilent(watcher, watcher.znodePaths.switchZNode); 054 } 055 } catch (KeeperException e) { 056 throw new RuntimeException(e); 057 } 058 splitZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode, 059 conf.get("zookeeper.znode.switch.split", "split")); 060 mergeZnode = ZNodePaths.joinZNode(watcher.znodePaths.switchZNode, 061 conf.get("zookeeper.znode.switch.merge", "merge")); 062 splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable); 063 mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable); 064 } 065 066 public void start() { 067 splitStateTracker.start(); 068 mergeStateTracker.start(); 069 } 070 071 public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) { 072 switch (switchType) { 073 case SPLIT: 074 return splitStateTracker.isSwitchEnabled(); 075 case MERGE: 076 return mergeStateTracker.isSwitchEnabled(); 077 default: 078 break; 079 } 080 return false; 081 } 082 083 public void setSplitOrMergeEnabled(boolean enabled, MasterSwitchType switchType) 084 throws KeeperException { 085 switch (switchType) { 086 case SPLIT: 087 splitStateTracker.setSwitchEnabled(enabled); 088 break; 089 case MERGE: 090 mergeStateTracker.setSwitchEnabled(enabled); 091 break; 092 default: 093 break; 094 } 095 } 096 097 private static class SwitchStateTracker extends ZKNodeTracker { 098 099 public SwitchStateTracker(ZKWatcher watcher, String node, Abortable abortable) { 100 super(watcher, node, abortable); 101 } 102 103 /** 104 * Return true if the switch is on, false otherwise 105 */ 106 public boolean isSwitchEnabled() { 107 byte [] upData = super.getData(false); 108 try { 109 // if data in ZK is null, use default of on. 110 return upData == null || parseFrom(upData).getEnabled(); 111 } catch (DeserializationException dex) { 112 LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData)); 113 // return false to be safe. 114 return false; 115 } 116 } 117 118 /** 119 * Set the switch on/off 120 * @param enabled switch enabled or not? 121 * @throws KeeperException keepException will be thrown out 122 */ 123 public void setSwitchEnabled(boolean enabled) throws KeeperException { 124 byte [] upData = toByteArray(enabled); 125 try { 126 ZKUtil.setData(watcher, node, upData); 127 } catch(KeeperException.NoNodeException nne) { 128 ZKUtil.createAndWatch(watcher, node, upData); 129 } 130 super.nodeDataChanged(node); 131 } 132 133 private byte [] toByteArray(boolean enabled) { 134 SwitchState.Builder builder = SwitchState.newBuilder(); 135 builder.setEnabled(enabled); 136 return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); 137 } 138 139 private SwitchState parseFrom(byte [] bytes) 140 throws DeserializationException { 141 ProtobufUtil.expectPBMagicPrefix(bytes); 142 SwitchState.Builder builder = SwitchState.newBuilder(); 143 try { 144 int magicLen = ProtobufUtil.lengthOfPBMagic(); 145 ProtobufUtil.mergeFrom(builder, bytes, magicLen, bytes.length - magicLen); 146 } catch (IOException e) { 147 throw new DeserializationException(e); 148 } 149 return builder.build(); 150 } 151 } 152}