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.getZNodePaths().switchZNode) < 0) {
053        ZKUtil.createAndFailSilent(watcher, watcher.getZNodePaths().switchZNode);
054      }
055    } catch (KeeperException e) {
056      throw new RuntimeException(e);
057    }
058    splitZnode = ZNodePaths.joinZNode(watcher.getZNodePaths().switchZNode,
059      conf.get("zookeeper.znode.switch.split", "split"));
060    mergeZnode = ZNodePaths.joinZNode(watcher.getZNodePaths().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}