View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.zookeeper;
19  
20  import java.io.IOException;
21  
22  import org.apache.hadoop.conf.Configuration;
23  import org.apache.hadoop.hbase.Abortable;
24  import org.apache.hadoop.hbase.classification.InterfaceAudience;
25  import org.apache.hadoop.hbase.client.Admin;
26  import org.apache.hadoop.hbase.exceptions.DeserializationException;
27  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
28  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SwitchState;
29  import org.apache.hadoop.hbase.util.Bytes;
30  import org.apache.zookeeper.KeeperException;
31  
32  
33  /**
34   * Tracks the switch of split and merge states in ZK
35   *
36   */
37  @InterfaceAudience.Private
38  public class SplitOrMergeTracker {
39  
40    private String splitZnode;
41    private String mergeZnode;
42  
43    private SwitchStateTracker splitStateTracker;
44    private SwitchStateTracker mergeStateTracker;
45  
46    public SplitOrMergeTracker(ZooKeeperWatcher watcher, Configuration conf,
47                               Abortable abortable) {
48      try {
49        if (ZKUtil.checkExists(watcher, watcher.getSwitchZNode()) < 0) {
50          ZKUtil.createAndFailSilent(watcher, watcher.getSwitchZNode());
51        }
52      } catch (KeeperException e) {
53        throw new RuntimeException(e);
54      }
55      splitZnode = ZKUtil.joinZNode(watcher.getSwitchZNode(),
56        conf.get("zookeeper.znode.switch.split", "split"));
57      mergeZnode = ZKUtil.joinZNode(watcher.getSwitchZNode(),
58        conf.get("zookeeper.znode.switch.merge", "merge"));
59      splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable);
60      mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable);
61    }
62  
63    public void start() {
64      splitStateTracker.start();
65      mergeStateTracker.start();
66    }
67  
68    public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) {
69      switch (switchType) {
70        case SPLIT:
71          return splitStateTracker.isSwitchEnabled();
72        case MERGE:
73          return mergeStateTracker.isSwitchEnabled();
74        default:
75          break;
76      }
77      return false;
78    }
79  
80    public void setSplitOrMergeEnabled(boolean enabled, Admin.MasterSwitchType switchType)
81      throws KeeperException {
82      switch (switchType) {
83        case SPLIT:
84          splitStateTracker.setSwitchEnabled(enabled);
85          break;
86        case MERGE:
87          mergeStateTracker.setSwitchEnabled(enabled);
88          break;
89        default:
90          break;
91      }
92    }
93  
94    private static class SwitchStateTracker extends ZooKeeperNodeTracker {
95  
96      public SwitchStateTracker(ZooKeeperWatcher watcher, String node, Abortable abortable) {
97        super(watcher, node, abortable);
98      }
99  
100     /**
101      * Return true if the switch is on, false otherwise
102      */
103     public boolean isSwitchEnabled() {
104       byte [] upData = super.getData(false);
105       try {
106         // if data in ZK is null, use default of on.
107         return upData == null || parseFrom(upData).getEnabled();
108       } catch (DeserializationException dex) {
109         LOG.error("ZK state for LoadBalancer could not be parsed " + Bytes.toStringBinary(upData));
110         // return false to be safe.
111         return false;
112       }
113     }
114 
115     /**
116      * Set the switch on/off
117      * @param enabled switch enabled or not?
118      * @throws KeeperException keepException will be thrown out
119      */
120     public void setSwitchEnabled(boolean enabled) throws KeeperException {
121       byte [] upData = toByteArray(enabled);
122       try {
123         ZKUtil.setData(watcher, node, upData);
124       } catch(KeeperException.NoNodeException nne) {
125         ZKUtil.createAndWatch(watcher, node, upData);
126       }
127       super.nodeDataChanged(node);
128     }
129 
130     private byte [] toByteArray(boolean enabled) {
131       SwitchState.Builder builder = SwitchState.newBuilder();
132       builder.setEnabled(enabled);
133       return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
134     }
135 
136     private SwitchState parseFrom(byte [] bytes)
137       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 }