1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.zookeeper;
20
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.hbase.classification.InterfaceAudience;
24 import org.apache.hadoop.hbase.Abortable;
25 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
26 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
27 import org.apache.zookeeper.KeeperException;
28
29
30
31
32
33
34
35
36 @InterfaceAudience.Private
37 public class ClusterStatusTracker extends ZooKeeperNodeTracker {
38 private static final Log LOG = LogFactory.getLog(ClusterStatusTracker.class);
39
40
41
42
43
44
45
46
47
48 public ClusterStatusTracker(ZooKeeperWatcher watcher, Abortable abortable) {
49 super(watcher, watcher.clusterStateZNode, abortable);
50 }
51
52
53
54
55
56
57 public boolean isClusterUp() {
58 return super.getData(false) != null;
59 }
60
61
62
63
64
65 public void setClusterUp()
66 throws KeeperException {
67 byte [] upData = toByteArray();
68 try {
69 ZKUtil.createAndWatch(watcher, watcher.clusterStateZNode, upData);
70 } catch(KeeperException.NodeExistsException nee) {
71 ZKUtil.setData(watcher, watcher.clusterStateZNode, upData);
72 }
73 }
74
75
76
77
78
79 public void setClusterDown()
80 throws KeeperException {
81 try {
82 ZKUtil.deleteNode(watcher, watcher.clusterStateZNode);
83 } catch(KeeperException.NoNodeException nne) {
84 LOG.warn("Attempted to set cluster as down but already down, cluster " +
85 "state node (" + watcher.clusterStateZNode + ") not found");
86 }
87 }
88
89
90
91
92
93 static byte [] toByteArray() {
94 ZooKeeperProtos.ClusterUp.Builder builder =
95 ZooKeeperProtos.ClusterUp.newBuilder();
96 builder.setStartDate(new java.util.Date().toString());
97 return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
98 }
99 }