View Javadoc

1   /**
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
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
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.
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.util.Bytes;
31  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
32  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
33  import org.apache.zookeeper.KeeperException;
34  
35  import com.google.protobuf.InvalidProtocolBufferException;
36  
37  /**
38   * ZooKeeper based controller for a procedure member.
39   * <p>
40   * There can only be one {@link ZKProcedureMemberRpcs} per procedure type per member,
41   * since each procedure type is bound to a single set of znodes. You can have multiple
42   * {@link ZKProcedureMemberRpcs} on the same server, each serving a different member
43   * name, but each individual rpcs is still bound to a single member name (and since they are
44   * used to determine global progress, its important to not get this wrong).
45   * <p>
46   * To make this slightly more confusing, you can run multiple, concurrent procedures at the same
47   * time (as long as they have different types), from the same controller, but the same node name
48   * must be used for each procedure (though there is no conflict between the two procedure as long
49   * as they have distinct names).
50   * <p>
51   * There is no real error recovery with this mechanism currently -- if any the coordinator fails,
52   * its re-initialization will delete the znodes and require all in progress subprocedures to start
53   * anew.
54   */
55  @InterfaceAudience.Public
56  @InterfaceStability.Evolving
57  public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
58  
59    private static final Log LOG = LogFactory.getLog(ZKProcedureMemberRpcs.class);
60    private final String memberName;
61  
62    protected ProcedureMember member;
63    private ZKProcedureUtil zkController;
64  
65    /**
66     * Must call {@link #start(ProcedureMember)} before this can be used.
67     * @param watcher {@link ZooKeeperWatcher} to be owned by <tt>this</tt>. Closed via
68     *          {@link #close()}.
69     * @param procType name of the znode describing the procedure type
70     * @param memberName name of the member to join the procedure
71     * @throws KeeperException if we can't reach zookeeper
72     */
73    public ZKProcedureMemberRpcs(ZooKeeperWatcher watcher,
74        String procType, String memberName) throws KeeperException {
75      this.zkController = new ZKProcedureUtil(watcher, procType, memberName) {
76        @Override
77        public void nodeCreated(String path) {
78          if (!isInProcedurePath(path)) {
79            return;
80          }
81  
82          LOG.info("Received created event:" + path);
83          // if it is a simple start/end/abort then we just rewatch the node
84          if (isAcquiredNode(path)) {
85            waitForNewProcedures();
86            return;
87          } else if (isAbortNode(path)) {
88            watchForAbortedProcedures();
89            return;
90          }
91          String parent = ZKUtil.getParent(path);
92          // if its the end barrier, the procedure can be completed
93          if (isReachedNode(parent)) {
94            receivedReachedGlobalBarrier(path);
95            return;
96          } else if (isAbortNode(parent)) {
97            abort(path);
98            return;
99          } else if (isAcquiredNode(parent)) {
100           startNewSubprocedure(path);
101         } else {
102           LOG.debug("Ignoring created notification for node:" + path);
103         }
104       }
105 
106       @Override
107       public void nodeChildrenChanged(String path) {
108         if (path.equals(this.acquiredZnode)) {
109           LOG.info("Received procedure start children changed event: " + path);
110           waitForNewProcedures();
111         } else if (path.equals(this.abortZnode)) {
112           LOG.info("Received procedure abort children changed event: " + path);
113           watchForAbortedProcedures();
114         }
115       }
116     };
117     this.memberName = memberName;
118   }
119 
120   public ZKProcedureUtil getZkController() {
121     return zkController;
122   }
123 
124   @Override
125   public String getMemberName() {
126     return memberName;
127   }
128 
129   /**
130    * Pass along the procedure global barrier notification to any listeners
131    * @param path full znode path that cause the notification
132    */
133   private void receivedReachedGlobalBarrier(String path) {
134     LOG.debug("Recieved reached global barrier:" + path);
135     String procName = ZKUtil.getNodeName(path);
136     this.member.receivedReachedGlobalBarrier(procName);
137   }
138 
139   private void watchForAbortedProcedures() {
140     LOG.debug("Checking for aborted procedures on node: '" + zkController.getAbortZnode() + "'");
141     try {
142       // this is the list of the currently aborted procedues
143       for (String node : ZKUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(),
144         zkController.getAbortZnode())) {
145         String abortNode = ZKUtil.joinZNode(zkController.getAbortZnode(), node);
146         abort(abortNode);
147       }
148     } catch (KeeperException e) {
149       member.controllerConnectionFailure("Failed to list children for abort node:"
150           + zkController.getAbortZnode(), new IOException(e));
151     }
152   }
153 
154   private void waitForNewProcedures() {
155     // watch for new procedues that we need to start subprocedures for
156     LOG.debug("Looking for new procedures under znode:'" + zkController.getAcquiredBarrier() + "'");
157     List<String> runningProcedures = null;
158     try {
159       runningProcedures = ZKUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(),
160         zkController.getAcquiredBarrier());
161       if (runningProcedures == null) {
162         LOG.debug("No running procedures.");
163         return;
164       }
165     } catch (KeeperException e) {
166       member.controllerConnectionFailure("General failure when watching for new procedures",
167         new IOException(e));
168     }
169     if (runningProcedures == null) {
170       LOG.debug("No running procedures.");
171       return;
172     }
173     for (String procName : runningProcedures) {
174       // then read in the procedure information
175       String path = ZKUtil.joinZNode(zkController.getAcquiredBarrier(), procName);
176       startNewSubprocedure(path);
177     }
178   }
179 
180   /**
181    * Kick off a new sub-procedure on the listener with the data stored in the passed znode.
182    * <p>
183    * Will attempt to create the same procedure multiple times if an procedure znode with the same
184    * name is created. It is left up the coordinator to ensure this doesn't occur.
185    * @param path full path to the znode for the procedure to start
186    */
187   private synchronized void startNewSubprocedure(String path) {
188     LOG.debug("Found procedure znode: " + path);
189     String opName = ZKUtil.getNodeName(path);
190     // start watching for an abort notification for the procedure
191     String abortZNode = zkController.getAbortZNode(opName);
192     try {
193       if (ZKUtil.watchAndCheckExists(zkController.getWatcher(), abortZNode)) {
194         LOG.debug("Not starting:" + opName + " because we already have an abort notification.");
195         return;
196       }
197     } catch (KeeperException e) {
198       member.controllerConnectionFailure("Failed to get the abort znode (" + abortZNode
199           + ") for procedure :" + opName, new IOException(e));
200       return;
201     }
202 
203     // get the data for the procedure
204     Subprocedure subproc = null;
205     try {
206       byte[] data = ZKUtil.getData(zkController.getWatcher(), path);
207       LOG.debug("start proc data length is " + data.length);
208       if (!ProtobufUtil.isPBMagicPrefix(data)) {
209         String msg = "Data in for starting procuedure " + opName +
210           " is illegally formatted (no pb magic). " +
211           "Killing the procedure: " + Bytes.toString(data);
212         LOG.error(msg);
213         throw new IllegalArgumentException(msg);
214       }
215       data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
216       LOG.debug("Found data for znode:" + path);
217       subproc = member.createSubprocedure(opName, data);
218       member.submitSubprocedure(subproc);
219     } catch (IllegalArgumentException iae ) {
220       LOG.error("Illegal argument exception", iae);
221       sendMemberAborted(subproc, new ForeignException(getMemberName(), iae));
222     } catch (IllegalStateException ise) {
223       LOG.error("Illegal state exception ", ise);
224       sendMemberAborted(subproc, new ForeignException(getMemberName(), ise));
225     } catch (KeeperException e) {
226       member.controllerConnectionFailure("Failed to get data for new procedure:" + opName,
227         new IOException(e));
228     }
229   }
230 
231   /**
232    * This attempts to create an acquired state znode for the procedure (snapshot name).
233    *
234    * It then looks for the reached znode to trigger in-barrier execution.  If not present we
235    * have a watcher, if present then trigger the in-barrier action.
236    */
237   @Override
238   public void sendMemberAcquired(Subprocedure sub) throws IOException {
239     String procName = sub.getName();
240     try {
241       LOG.debug("Member: '" + memberName + "' joining acquired barrier for procedure (" + procName
242           + ") in zk");
243       String acquiredZNode = ZKUtil.joinZNode(ZKProcedureUtil.getAcquireBarrierNode(
244         zkController, procName), memberName);
245       ZKUtil.createAndFailSilent(zkController.getWatcher(), acquiredZNode);
246 
247       // watch for the complete node for this snapshot
248       String reachedBarrier = zkController.getReachedBarrierNode(procName);
249       LOG.debug("Watch for global barrier reached:" + reachedBarrier);
250       if (ZKUtil.watchAndCheckExists(zkController.getWatcher(), reachedBarrier)) {
251         receivedReachedGlobalBarrier(reachedBarrier);
252       }
253     } catch (KeeperException e) {
254       member.controllerConnectionFailure("Failed to acquire barrier for procedure: "
255           + procName + " and member: " + memberName, new IOException(e));
256     }
257   }
258 
259   /**
260    * This acts as the ack for a completed snapshot
261    */
262   @Override
263   public void sendMemberCompleted(Subprocedure sub) throws IOException {
264     String procName = sub.getName();
265     LOG.debug("Marking procedure  '" + procName + "' completed for member '" + memberName
266         + "' in zk");
267     String joinPath = ZKUtil.joinZNode(zkController.getReachedBarrierNode(procName), memberName);
268     try {
269       ZKUtil.createAndFailSilent(zkController.getWatcher(), joinPath);
270     } catch (KeeperException e) {
271       member.controllerConnectionFailure("Failed to post zk node:" + joinPath
272           + " to join procedure barrier.", new IOException(e));
273     }
274   }
275 
276   /**
277    * This should be called by the member and should write a serialized root cause exception as
278    * to the abort znode.
279    */
280   @Override
281   public void sendMemberAborted(Subprocedure sub, ForeignException ee) {
282     if (sub == null) {
283       LOG.error("Failed due to null subprocedure", ee);
284       return;
285     }
286     String procName = sub.getName();
287     LOG.debug("Aborting procedure (" + procName + ") in zk");
288     String procAbortZNode = zkController.getAbortZNode(procName);
289     try {
290       String source = (ee.getSource() == null) ? memberName: ee.getSource();
291       byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee));
292       ZKUtil.createAndFailSilent(zkController.getWatcher(), procAbortZNode, errorInfo);
293       LOG.debug("Finished creating abort znode:" + procAbortZNode);
294     } catch (KeeperException e) {
295       // possible that we get this error for the procedure if we already reset the zk state, but in
296       // that case we should still get an error for that procedure anyways
297       zkController.logZKTree(zkController.getBaseZnode());
298       member.controllerConnectionFailure("Failed to post zk node:" + procAbortZNode
299           + " to abort procedure", new IOException(e));
300     }
301   }
302 
303   /**
304    * Pass along the found abort notification to the listener
305    * @param abortZNode full znode path to the failed procedure information
306    */
307   protected void abort(String abortZNode) {
308     LOG.debug("Aborting procedure member for znode " + abortZNode);
309     String opName = ZKUtil.getNodeName(abortZNode);
310     try {
311       byte[] data = ZKUtil.getData(zkController.getWatcher(), abortZNode);
312 
313       // figure out the data we need to pass
314       ForeignException ee;
315       try {
316         if (!ProtobufUtil.isPBMagicPrefix(data)) {
317           String msg = "Illegally formatted data in abort node for proc " + opName
318               + ".  Killing the procedure.";
319           LOG.error(msg);
320           // we got a remote exception, but we can't describe it so just return exn from here
321           ee = new ForeignException(getMemberName(), new IllegalArgumentException(msg));
322         } else {
323           data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
324           ee = ForeignException.deserialize(data);
325         }
326       } catch (InvalidProtocolBufferException e) {
327         LOG.warn("Got an error notification for op:" + opName
328             + " but we can't read the information. Killing the procedure.");
329         // we got a remote exception, but we can't describe it so just return exn from here
330         ee = new ForeignException(getMemberName(), e);
331       }
332 
333       this.member.receiveAbortProcedure(opName, ee);
334     } catch (KeeperException e) {
335       member.controllerConnectionFailure("Failed to get data for abort znode:" + abortZNode
336           + zkController.getAbortZnode(), new IOException(e));
337     }
338   }
339 
340   public void start(ProcedureMember listener) {
341     LOG.debug("Starting procedure member '" + this.memberName + "'");
342     this.member = listener;
343     watchForAbortedProcedures();
344     waitForNewProcedures();
345   }
346 
347   @Override
348   public void close() throws IOException {
349     zkController.close();
350   }
351 
352 }