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