1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.procedure;
19
20 import java.io.IOException;
21 import java.util.Arrays;
22 import java.util.List;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.classification.InterfaceAudience;
27 import org.apache.hadoop.classification.InterfaceStability;
28 import org.apache.hadoop.hbase.errorhandling.ForeignException;
29 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
30 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
31 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
32 import org.apache.zookeeper.KeeperException;
33
34 import com.google.protobuf.InvalidProtocolBufferException;
35
36
37
38
39 @InterfaceAudience.Public
40 @InterfaceStability.Evolving
41 public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
42 public static final Log LOG = LogFactory.getLog(ZKProcedureCoordinatorRpcs.class);
43 private ZKProcedureUtil zkProc = null;
44 protected ProcedureCoordinator coordinator = null;
45
46 ZooKeeperWatcher watcher;
47 String procedureType;
48 String coordName;
49
50
51
52
53
54
55
56
57 public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher,
58 String procedureClass, String coordName) throws KeeperException {
59 this.watcher = watcher;
60 this.procedureType = procedureClass;
61 this.coordName = coordName;
62 }
63
64
65
66
67
68
69
70
71
72
73
74 @Override
75 final public void sendGlobalBarrierAcquire(Procedure proc, byte[] info, List<String> nodeNames)
76 throws IOException, IllegalArgumentException {
77 String procName = proc.getName();
78
79 String abortNode = zkProc.getAbortZNode(procName);
80 try {
81
82 if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), abortNode)) {
83 abort(abortNode);
84 }
85
86
87 } catch (KeeperException e) {
88 LOG.error("Failed to watch abort", e);
89 throw new IOException("Failed while watching abort node:" + abortNode, e);
90 }
91
92
93 String acquire = zkProc.getAcquiredBarrierNode(procName);
94 LOG.debug("Creating acquire znode:" + acquire);
95 try {
96
97 byte[] data = ProtobufUtil.prependPBMagic(info);
98 ZKUtil.createWithParents(zkProc.getWatcher(), acquire, data);
99
100 for (String node : nodeNames) {
101 String znode = ZKUtil.joinZNode(acquire, node);
102 LOG.debug("Watching for acquire node:" + znode);
103 if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
104 coordinator.memberAcquiredBarrier(procName, node);
105 }
106 }
107 } catch (KeeperException e) {
108 throw new IOException("Failed while creating acquire node:" + acquire, e);
109 }
110 }
111
112 @Override
113 public void sendGlobalBarrierReached(Procedure proc, List<String> nodeNames) throws IOException {
114 String procName = proc.getName();
115 String reachedNode = zkProc.getReachedBarrierNode(procName);
116 LOG.debug("Creating reached barrier zk node:" + reachedNode);
117 try {
118
119 ZKUtil.createWithParents(zkProc.getWatcher(), reachedNode);
120
121 for (String node : nodeNames) {
122 String znode = ZKUtil.joinZNode(reachedNode, node);
123 if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
124 coordinator.memberFinishedBarrier(procName, node);
125 }
126 }
127 } catch (KeeperException e) {
128 throw new IOException("Failed while creating reached node:" + reachedNode, e);
129 }
130 }
131
132
133
134
135
136 @Override
137 final public void resetMembers(Procedure proc) throws IOException {
138 String procName = proc.getName();
139 boolean stillGettingNotifications = false;
140 do {
141 try {
142 LOG.debug("Attempting to clean out zk node for op:" + procName);
143 zkProc.clearZNodes(procName);
144 stillGettingNotifications = false;
145 } catch (KeeperException.NotEmptyException e) {
146
147
148 stillGettingNotifications = true;
149 } catch (KeeperException e) {
150 throw new IOException("Failed to complete reset procedure " + procName, e);
151 }
152 } while (stillGettingNotifications);
153 }
154
155
156
157
158
159 final public boolean start(final ProcedureCoordinator coordinator) {
160 if (this.coordinator != null) {
161 throw new IllegalStateException(
162 "ZKProcedureCoordinator already started and already has listener installed");
163 }
164 this.coordinator = coordinator;
165
166 try {
167 this.zkProc = new ZKProcedureUtil(watcher, procedureType, coordName) {
168 @Override
169 public void nodeCreated(String path) {
170 if (!isInProcedurePath(path)) return;
171 LOG.debug("Node created: " + path);
172 logZKTree(this.baseZNode);
173 if (isAcquiredPathNode(path)) {
174
175 coordinator.memberAcquiredBarrier(ZKUtil.getNodeName(ZKUtil.getParent(path)),
176 ZKUtil.getNodeName(path));
177 } else if (isReachedPathNode(path)) {
178
179
180
181 coordinator.memberFinishedBarrier(ZKUtil.getNodeName(ZKUtil.getParent(path)),
182 ZKUtil.getNodeName(path));
183 } else if (isAbortPathNode(path)) {
184 abort(path);
185 }
186 }
187 };
188 zkProc.clearChildZNodes();
189 } catch (KeeperException e) {
190 LOG.error("Unable to start the ZK-based Procedure Coordinator rpcs.", e);
191 return false;
192 }
193
194 LOG.debug("Starting the controller for procedure member:" + zkProc.getMemberName());
195 return true;
196 }
197
198
199
200
201
202
203
204 @Override
205 final public void sendAbortToMembers(Procedure proc, ForeignException ee) {
206 String procName = proc.getName();
207 LOG.debug("Aborting procedure '" + procName + "' in zk");
208 String procAbortNode = zkProc.getAbortZNode(procName);
209 try {
210 LOG.debug("Creating abort znode:" + procAbortNode);
211 String source = (ee.getSource() == null) ? coordName : ee.getSource();
212 byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee));
213
214 ZKUtil.createAndFailSilent(zkProc.getWatcher(), procAbortNode, errorInfo);
215 LOG.debug("Finished creating abort node:" + procAbortNode);
216 } catch (KeeperException e) {
217
218
219 zkProc.logZKTree(zkProc.baseZNode);
220 coordinator.rpcConnectionFailure("Failed to post zk node:" + procAbortNode
221 + " to abort procedure '" + procName + "'", new IOException(e));
222 }
223 }
224
225
226
227
228
229 protected void abort(String abortNode) {
230 String procName = ZKUtil.getNodeName(abortNode);
231 ForeignException ee = null;
232 try {
233 byte[] data = ZKUtil.getData(zkProc.getWatcher(), abortNode);
234 if (!ProtobufUtil.isPBMagicPrefix(data)) {
235 LOG.warn("Got an error notification for op:" + abortNode
236 + " but we can't read the information. Killing the procedure.");
237
238 ee = new ForeignException(coordName, "Data in abort node is illegally formatted. ignoring content.");
239 } else {
240
241 data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
242 ee = ForeignException.deserialize(data);
243 }
244 } catch (InvalidProtocolBufferException e) {
245 LOG.warn("Got an error notification for op:" + abortNode
246 + " but we can't read the information. Killing the procedure.");
247
248 ee = new ForeignException(coordName, e);
249 } catch (KeeperException e) {
250 coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
251 + zkProc.getAbortZnode(), new IOException(e));
252 }
253 coordinator.abortProcedure(procName, ee);
254 }
255
256 @Override
257 final public void close() throws IOException {
258 zkProc.close();
259 }
260
261
262
263
264 final ZKProcedureUtil getZkProcedureUtil() {
265 return zkProc;
266 }
267 }