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