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
10 * http://www.apache.org/licenses/LICENSE-2.0
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.
18 package org
.apache
.hadoop
.hbase
.master
;
20 import java
.io
.IOException
;
22 import org
.apache
.hadoop
.conf
.Configuration
;
23 import org
.apache
.hadoop
.hbase
.Abortable
;
24 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKNodeTracker
;
25 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
;
26 import org
.apache
.hadoop
.hbase
.zookeeper
.ZNodePaths
;
27 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKWatcher
;
28 import org
.apache
.hadoop
.hbase
.client
.MasterSwitchType
;
29 import org
.apache
.hadoop
.hbase
.exceptions
.DeserializationException
;
30 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.ProtobufUtil
;
31 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ZooKeeperProtos
.SwitchState
;
32 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
33 import org
.apache
.yetus
.audience
.InterfaceAudience
;
34 import org
.apache
.zookeeper
.KeeperException
;
38 * Tracks the switch of split and merge states in ZK
40 @InterfaceAudience.Private
41 public class SplitOrMergeTracker
{
43 private String splitZnode
;
44 private String mergeZnode
;
46 private SwitchStateTracker splitStateTracker
;
47 private SwitchStateTracker mergeStateTracker
;
49 public SplitOrMergeTracker(ZKWatcher watcher
, Configuration conf
,
50 Abortable abortable
) {
52 if (ZKUtil
.checkExists(watcher
, watcher
.getZNodePaths().switchZNode
) < 0) {
53 ZKUtil
.createAndFailSilent(watcher
, watcher
.getZNodePaths().switchZNode
);
55 } catch (KeeperException e
) {
56 throw new RuntimeException(e
);
58 splitZnode
= ZNodePaths
.joinZNode(watcher
.getZNodePaths().switchZNode
,
59 conf
.get("zookeeper.znode.switch.split", "split"));
60 mergeZnode
= ZNodePaths
.joinZNode(watcher
.getZNodePaths().switchZNode
,
61 conf
.get("zookeeper.znode.switch.merge", "merge"));
62 splitStateTracker
= new SwitchStateTracker(watcher
, splitZnode
, abortable
);
63 mergeStateTracker
= new SwitchStateTracker(watcher
, mergeZnode
, abortable
);
67 splitStateTracker
.start();
68 mergeStateTracker
.start();
71 public boolean isSplitOrMergeEnabled(MasterSwitchType switchType
) {
74 return splitStateTracker
.isSwitchEnabled();
76 return mergeStateTracker
.isSwitchEnabled();
83 public void setSplitOrMergeEnabled(boolean enabled
, MasterSwitchType switchType
)
84 throws KeeperException
{
87 splitStateTracker
.setSwitchEnabled(enabled
);
90 mergeStateTracker
.setSwitchEnabled(enabled
);
97 private static class SwitchStateTracker
extends ZKNodeTracker
{
99 public SwitchStateTracker(ZKWatcher watcher
, String node
, Abortable abortable
) {
100 super(watcher
, node
, abortable
);
104 * Return true if the switch is on, false otherwise
106 public boolean isSwitchEnabled() {
107 byte [] upData
= super.getData(false);
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.
119 * Set the switch on/off
120 * @param enabled switch enabled or not?
121 * @throws KeeperException keepException will be thrown out
123 public void setSwitchEnabled(boolean enabled
) throws KeeperException
{
124 byte [] upData
= toByteArray(enabled
);
126 ZKUtil
.setData(watcher
, node
, upData
);
127 } catch(KeeperException
.NoNodeException nne
) {
128 ZKUtil
.createAndWatch(watcher
, node
, upData
);
130 super.nodeDataChanged(node
);
133 private byte [] toByteArray(boolean enabled
) {
134 SwitchState
.Builder builder
= SwitchState
.newBuilder();
135 builder
.setEnabled(enabled
);
136 return ProtobufUtil
.prependPBMagic(builder
.build().toByteArray());
139 private SwitchState
parseFrom(byte [] bytes
)
140 throws DeserializationException
{
141 ProtobufUtil
.expectPBMagicPrefix(bytes
);
142 SwitchState
.Builder builder
= SwitchState
.newBuilder();
144 int magicLen
= ProtobufUtil
.lengthOfPBMagic();
145 ProtobufUtil
.mergeFrom(builder
, bytes
, magicLen
, bytes
.length
- magicLen
);
146 } catch (IOException e
) {
147 throw new DeserializationException(e
);
149 return builder
.build();