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