1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication;
20
21 import com.google.protobuf.InvalidProtocolBufferException;
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.hbase.Abortable;
25 import org.apache.hadoop.hbase.exceptions.DeserializationException;
26 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
27 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
28 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
29 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
30 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
31 import org.apache.zookeeper.KeeperException;
32
33 import java.io.IOException;
34 import java.util.concurrent.atomic.AtomicBoolean;
35
36
37
38
39
40 public class ReplicationStateImpl implements ReplicationStateInterface {
41
42 private ReplicationStateTracker stateTracker;
43 private final String stateZnode;
44 private final ZooKeeperWatcher zookeeper;
45 private final Abortable abortable;
46 private final AtomicBoolean replicating;
47
48 private static final Log LOG = LogFactory.getLog(ReplicationStateImpl.class);
49
50 public ReplicationStateImpl(final ZooKeeperWatcher zk, final String stateZnode,
51 final Abortable abortable, final AtomicBoolean replicating) {
52 this.zookeeper = zk;
53 this.stateZnode = stateZnode;
54 this.abortable = abortable;
55 this.replicating = replicating;
56
57
58 this.stateTracker = new ReplicationStateTracker(this.zookeeper, this.stateZnode,
59 this.abortable);
60 stateTracker.start();
61 readReplicationStateZnode();
62 }
63
64 public boolean getState() throws KeeperException {
65 return getReplication();
66 }
67
68 public void setState(boolean newState) throws KeeperException {
69 setReplicating(newState);
70 }
71
72 @Override
73 public void close() throws IOException {
74 if (stateTracker != null) stateTracker.stop();
75 }
76
77
78
79
80
81
82
83 private boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
84 ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
85 return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
86 }
87
88
89
90
91
92
93 private ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
94 throws DeserializationException {
95 ProtobufUtil.expectPBMagicPrefix(bytes);
96 int pblen = ProtobufUtil.lengthOfPBMagic();
97 ZooKeeperProtos.ReplicationState.Builder builder = ZooKeeperProtos.ReplicationState
98 .newBuilder();
99 ZooKeeperProtos.ReplicationState state;
100 try {
101 state = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
102 return state.getState();
103 } catch (InvalidProtocolBufferException e) {
104 throw new DeserializationException(e);
105 }
106 }
107
108
109
110
111
112 private void setReplicating(boolean newState) throws KeeperException {
113 ZKUtil.createWithParents(this.zookeeper, this.stateZnode);
114 byte[] stateBytes = (newState == true) ? ReplicationZookeeper.ENABLED_ZNODE_BYTES
115 : ReplicationZookeeper.DISABLED_ZNODE_BYTES;
116 ZKUtil.setData(this.zookeeper, this.stateZnode, stateBytes);
117 }
118
119
120
121
122
123
124
125 private boolean getReplication() throws KeeperException {
126 byte[] data = this.stateTracker.getData(false);
127 if (data == null || data.length == 0) {
128 setReplicating(true);
129 return true;
130 }
131 try {
132 return isStateEnabled(data);
133 } catch (DeserializationException e) {
134 throw ZKUtil.convert(e);
135 }
136 }
137
138
139
140
141 private void readReplicationStateZnode() {
142 try {
143 this.replicating.set(getReplication());
144 LOG.info("Replication is now " + (this.replicating.get() ? "started" : "stopped"));
145 } catch (KeeperException e) {
146 this.abortable.abort("Failed getting data on from " + this.stateZnode, e);
147 }
148 }
149
150
151
152
153 private class ReplicationStateTracker extends ZooKeeperNodeTracker {
154 public ReplicationStateTracker(ZooKeeperWatcher watcher, String stateZnode, Abortable abortable) {
155 super(watcher, stateZnode, abortable);
156 }
157
158 @Override
159 public synchronized void nodeDataChanged(String path) {
160 if (path.equals(node)) {
161 super.nodeDataChanged(path);
162 readReplicationStateZnode();
163 }
164 }
165 }
166 }