View Javadoc

1   /**
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.zookeeper;
21  
22  import java.io.IOException;
23  import java.lang.management.ManagementFactory;
24  import java.util.ArrayList;
25  import java.util.List;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.util.Bytes;
30  import org.apache.hadoop.hbase.util.RetryCounter;
31  import org.apache.hadoop.hbase.util.RetryCounterFactory;
32  import org.apache.zookeeper.AsyncCallback;
33  import org.apache.zookeeper.CreateMode;
34  import org.apache.zookeeper.KeeperException;
35  import org.apache.zookeeper.Watcher;
36  import org.apache.zookeeper.ZooKeeper;
37  import org.apache.zookeeper.ZooKeeper.States;
38  import org.apache.zookeeper.data.ACL;
39  import org.apache.zookeeper.data.Stat;
40  
41  /**
42   * A zookeeper that can handle 'recoverable' errors.
43   * To handle recoverable errors, developers need to realize that there are two 
44   * classes of requests: idempotent and non-idempotent requests. Read requests 
45   * and unconditional sets and deletes are examples of idempotent requests, they 
46   * can be reissued with the same results. 
47   * (Although, the delete may throw a NoNodeException on reissue its effect on 
48   * the ZooKeeper state is the same.) Non-idempotent requests need special 
49   * handling, application and library writers need to keep in mind that they may 
50   * need to encode information in the data or name of znodes to detect 
51   * retries. A simple example is a create that uses a sequence flag. 
52   * If a process issues a create("/x-", ..., SEQUENCE) and gets a connection 
53   * loss exception, that process will reissue another 
54   * create("/x-", ..., SEQUENCE) and get back x-111. When the process does a 
55   * getChildren("/"), it sees x-1,x-30,x-109,x-110,x-111, now it could be 
56   * that x-109 was the result of the previous create, so the process actually 
57   * owns both x-109 and x-111. An easy way around this is to use "x-process id-" 
58   * when doing the create. If the process is using an id of 352, before reissuing
59   * the create it will do a getChildren("/") and see "x-222-1", "x-542-30", 
60   * "x-352-109", x-333-110". The process will know that the original create 
61   * succeeded an the znode it created is "x-352-109".
62   * @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
63   */
64  public class RecoverableZooKeeper {
65    private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
66    // the actual ZooKeeper client instance
67    private ZooKeeper zk;
68    private final RetryCounterFactory retryCounterFactory;
69    // An identifier of this process in the cluster
70    private final String identifier;
71    private final byte[] id;
72    private int retryIntervalMillis;
73  
74    private static final int ID_OFFSET =  Bytes.SIZEOF_INT;
75    // the magic number is to be backward compatible
76    private static final byte MAGIC =(byte) 0XFF;
77    private static final int MAGIC_OFFSET = Bytes.SIZEOF_BYTE;
78  
79    public RecoverableZooKeeper(String quorumServers, int seesionTimeout,
80        Watcher watcher, int maxRetries, int retryIntervalMillis) 
81    throws IOException {
82      this.zk = new ZooKeeper(quorumServers, seesionTimeout, watcher);
83      this.retryCounterFactory =
84        new RetryCounterFactory(maxRetries, retryIntervalMillis);
85      this.retryIntervalMillis = retryIntervalMillis;
86  
87      // the identifier = processID@hostName
88      this.identifier = ManagementFactory.getRuntimeMXBean().getName();
89      LOG.info("The identifier of this process is " + identifier);
90      this.id = Bytes.toBytes(identifier);
91    }
92  
93    /**
94     * delete is an idempotent operation. Retry before throw out exception.
95     * This function will not throw out NoNodeException if the path is not existed
96     * @param path
97     * @param version
98     * @throws InterruptedException
99     * @throws KeeperException
100    */
101   public void delete(String path, int version)
102   throws InterruptedException, KeeperException {
103     RetryCounter retryCounter = retryCounterFactory.create();
104     boolean isRetry = false; // False for first attempt, true for all retries.
105     while (true) {
106       try {
107         zk.delete(path, version);
108         return;
109       } catch (KeeperException e) {
110         switch (e.code()) {
111           case NONODE:
112             if (isRetry) {
113               LOG.info("Node " + path + " already deleted. Assuming that a " +
114                   "previous attempt succeeded.");
115               return;
116             }
117             LOG.warn("Node " + path + " already deleted, and this is not a " +
118                      "retry");
119             throw e;
120 
121           case CONNECTIONLOSS:
122           case OPERATIONTIMEOUT:
123             LOG.warn("Possibly transient ZooKeeper exception: " + e);
124             if (!retryCounter.shouldRetry()) {
125               LOG.error("ZooKeeper delete failed after "
126                 + retryCounter.getMaxRetries() + " retries");
127               throw e;
128             }
129             break;
130 
131           default:
132             throw e;
133         }
134       }
135       retryCounter.sleepUntilNextRetry();
136       retryCounter.useRetry();
137       isRetry = true;
138     }
139   }
140 
141   /**
142    * exists is an idempotent operation. Retry before throw out exception
143    * @param path
144    * @param watcher
145    * @return A Stat instance
146    * @throws KeeperException
147    * @throws InterruptedException
148    */
149   public Stat exists(String path, Watcher watcher)
150   throws KeeperException, InterruptedException {
151     RetryCounter retryCounter = retryCounterFactory.create();
152     while (true) {
153       try {
154         return zk.exists(path, watcher);
155       } catch (KeeperException e) {
156         switch (e.code()) {
157           case CONNECTIONLOSS:
158           case OPERATIONTIMEOUT:
159             LOG.warn("Possibly transient ZooKeeper exception: " + e);
160             if (!retryCounter.shouldRetry()) {
161               LOG.error("ZooKeeper exists failed after "
162                 + retryCounter.getMaxRetries() + " retries");
163               throw e;
164             }
165             break;
166 
167           default:
168             throw e;
169         }
170       }
171       retryCounter.sleepUntilNextRetry();
172       retryCounter.useRetry();
173     }
174   }
175 
176   /**
177    * exists is an idempotent operation. Retry before throw out exception
178    * @param path
179    * @param watch
180    * @return A Stat instance
181    * @throws KeeperException
182    * @throws InterruptedException
183    */
184   public Stat exists(String path, boolean watch)
185   throws KeeperException, InterruptedException {
186     RetryCounter retryCounter = retryCounterFactory.create();
187     while (true) {
188       try {
189         return zk.exists(path, watch);
190       } catch (KeeperException e) {
191         switch (e.code()) {
192           case CONNECTIONLOSS:
193           case OPERATIONTIMEOUT:
194             LOG.warn("Possibly transient ZooKeeper exception: " + e);
195             if (!retryCounter.shouldRetry()) {
196               LOG.error("ZooKeeper exists failed after "
197                 + retryCounter.getMaxRetries() + " retries");
198               throw e;
199             }
200             break;
201 
202           default:
203             throw e;
204         }
205       }
206       retryCounter.sleepUntilNextRetry();
207       retryCounter.useRetry();
208     }
209   }
210 
211   /**
212    * getChildren is an idempotent operation. Retry before throw out exception
213    * @param path
214    * @param watcher
215    * @return List of children znodes
216    * @throws KeeperException
217    * @throws InterruptedException
218    */
219   public List<String> getChildren(String path, Watcher watcher)
220     throws KeeperException, InterruptedException {
221     RetryCounter retryCounter = retryCounterFactory.create();
222     while (true) {
223       try {
224         return zk.getChildren(path, watcher);
225       } catch (KeeperException e) {
226         switch (e.code()) {
227           case CONNECTIONLOSS:
228           case OPERATIONTIMEOUT:
229             LOG.warn("Possibly transient ZooKeeper exception: " + e);
230             if (!retryCounter.shouldRetry()) {
231               LOG.error("ZooKeeper getChildren failed after "
232                 + retryCounter.getMaxRetries() + " retries");
233               throw e;
234             }
235             break;
236 
237           default:
238             throw e;
239         }
240       }
241       retryCounter.sleepUntilNextRetry();
242       retryCounter.useRetry();
243     }
244   }
245 
246   /**
247    * getChildren is an idempotent operation. Retry before throw out exception
248    * @param path
249    * @param watch
250    * @return List of children znodes
251    * @throws KeeperException
252    * @throws InterruptedException
253    */
254   public List<String> getChildren(String path, boolean watch)
255   throws KeeperException, InterruptedException {
256     RetryCounter retryCounter = retryCounterFactory.create();
257     while (true) {
258       try {
259         return zk.getChildren(path, watch);
260       } catch (KeeperException e) {
261         switch (e.code()) {
262           case CONNECTIONLOSS:
263           case OPERATIONTIMEOUT:
264             LOG.warn("Possibly transient ZooKeeper exception: " + e);
265             if (!retryCounter.shouldRetry()) {
266               LOG.error("ZooKeeper getChildren failed after "
267                 + retryCounter.getMaxRetries() + " retries");
268               throw e;
269             }
270             break;
271 
272           default:
273             throw e;
274         }
275       }
276       retryCounter.sleepUntilNextRetry();
277       retryCounter.useRetry();
278     }
279   }
280 
281   /**
282    * getData is an idempotent operation. Retry before throw out exception
283    * @param path
284    * @param watcher
285    * @param stat
286    * @return Data
287    * @throws KeeperException
288    * @throws InterruptedException
289    */
290   public byte[] getData(String path, Watcher watcher, Stat stat)
291   throws KeeperException, InterruptedException {
292     RetryCounter retryCounter = retryCounterFactory.create();
293     while (true) {
294       try {
295         byte[] revData = zk.getData(path, watcher, stat);       
296         return this.removeMetaData(revData);
297       } catch (KeeperException e) {
298         switch (e.code()) {
299           case CONNECTIONLOSS:
300           case OPERATIONTIMEOUT:
301             LOG.warn("Possibly transient ZooKeeper exception: " + e);
302             if (!retryCounter.shouldRetry()) {
303               LOG.error("ZooKeeper getData failed after "
304                 + retryCounter.getMaxRetries() + " retries");
305               throw e;
306             }
307             break;
308 
309           default:
310             throw e;
311         }
312       }
313       retryCounter.sleepUntilNextRetry();
314       retryCounter.useRetry();
315     }
316   }
317 
318   /**
319    * getData is an idemnpotent operation. Retry before throw out exception
320    * @param path
321    * @param watch
322    * @param stat
323    * @return Data
324    * @throws KeeperException
325    * @throws InterruptedException
326    */
327   public byte[] getData(String path, boolean watch, Stat stat)
328   throws KeeperException, InterruptedException {
329     RetryCounter retryCounter = retryCounterFactory.create();
330     while (true) {
331       try {
332         byte[] revData = zk.getData(path, watch, stat);
333         return this.removeMetaData(revData);
334       } catch (KeeperException e) {
335         switch (e.code()) {
336           case CONNECTIONLOSS:
337           case OPERATIONTIMEOUT:
338             LOG.warn("Possibly transient ZooKeeper exception: " + e);
339             if (!retryCounter.shouldRetry()) {
340               LOG.error("ZooKeeper getData failed after "
341                 + retryCounter.getMaxRetries() + " retries");
342               throw e;
343             }
344             break;
345 
346           default:
347             throw e;
348         }
349       }
350       retryCounter.sleepUntilNextRetry();
351       retryCounter.useRetry();
352     }
353   }
354 
355   /**
356    * setData is NOT an idempotent operation. Retry may cause BadVersion Exception
357    * Adding an identifier field into the data to check whether 
358    * badversion is caused by the result of previous correctly setData
359    * @param path
360    * @param data
361    * @param version
362    * @return Stat instance
363    * @throws KeeperException
364    * @throws InterruptedException
365    */
366   public Stat setData(String path, byte[] data, int version)
367   throws KeeperException, InterruptedException {
368     RetryCounter retryCounter = retryCounterFactory.create();
369     byte[] newData = appendMetaData(data);
370     while (true) {
371       try {
372         return zk.setData(path, newData, version);
373       } catch (KeeperException e) {
374         switch (e.code()) {
375           case CONNECTIONLOSS:
376           case OPERATIONTIMEOUT:
377             LOG.warn("Possibly transient ZooKeeper exception: " + e);
378             if (!retryCounter.shouldRetry()) {
379               LOG.error("ZooKeeper setData failed after "
380                 + retryCounter.getMaxRetries() + " retries");
381               throw e;
382             }
383             break;
384           case BADVERSION:
385             // try to verify whether the previous setData success or not
386             try{
387               Stat stat = new Stat();
388               byte[] revData = zk.getData(path, false, stat);
389               int idLength = Bytes.toInt(revData, ID_OFFSET);
390               int dataLength = revData.length-ID_OFFSET-idLength;
391               int dataOffset = ID_OFFSET+idLength;
392               
393               if(Bytes.compareTo(revData, ID_OFFSET, id.length, 
394                   revData, dataOffset, dataLength) == 0) {
395                 // the bad version is caused by previous successful setData
396                 return stat;
397               }
398             } catch(KeeperException keeperException){
399               // the ZK is not reliable at this moment. just throw out exception
400               throw keeperException;
401             }            
402           
403           // throw out other exceptions and verified bad version exceptions
404           default:
405             throw e;
406         }
407       }
408       retryCounter.sleepUntilNextRetry();
409       retryCounter.useRetry();
410     }
411   }
412 
413   /**
414    * <p>
415    * NONSEQUENTIAL create is idempotent operation. 
416    * Retry before throw out exceptions.
417    * But this function will not throw out the NodeExist exception back to the
418    * application.
419    * </p>
420    * <p>
421    * But SEQUENTIAL is NOT idempotent operation. It is necessary to add 
422    * identifier to the path to verify, whether the previous one is successful 
423    * or not.
424    * </p>
425    * 
426    * @param path
427    * @param data
428    * @param acl
429    * @param createMode
430    * @return Path
431    * @throws KeeperException
432    * @throws InterruptedException
433    */
434   public String create(String path, byte[] data, List<ACL> acl,
435       CreateMode createMode)
436   throws KeeperException, InterruptedException {
437     byte[] newData = appendMetaData(data);
438     switch (createMode) {
439       case EPHEMERAL:
440       case PERSISTENT:
441         return createNonSequential(path, newData, acl, createMode);
442 
443       case EPHEMERAL_SEQUENTIAL:
444       case PERSISTENT_SEQUENTIAL:
445         return createSequential(path, newData, acl, createMode);
446 
447       default:
448         throw new IllegalArgumentException("Unrecognized CreateMode: " + 
449             createMode);
450     }
451   }
452 
453   private String createNonSequential(String path, byte[] data, List<ACL> acl, 
454       CreateMode createMode) throws KeeperException, InterruptedException {
455     RetryCounter retryCounter = retryCounterFactory.create();
456     boolean isRetry = false; // False for first attempt, true for all retries.
457     while (true) {
458       try {
459         return zk.create(path, data, acl, createMode);
460       } catch (KeeperException e) {
461         switch (e.code()) {
462           case NODEEXISTS:
463             if (isRetry) {
464               // If the connection was lost, there is still a possibility that
465               // we have successfully created the node at our previous attempt,
466               // so we read the node and compare. 
467               byte[] currentData = zk.getData(path, false, null);
468               if (currentData != null &&
469                   Bytes.compareTo(currentData, data) == 0) { 
470                 // We successfully created a non-sequential node
471                 return path;
472               }
473               LOG.error("Node " + path + " already exists with " + 
474                   Bytes.toStringBinary(currentData) + ", could not write " +
475                   Bytes.toStringBinary(data));
476               throw e;
477             }
478             LOG.error("Node " + path + " already exists and this is not a " +
479                 "retry");
480             throw e;
481 
482           case CONNECTIONLOSS:
483           case OPERATIONTIMEOUT:
484             LOG.warn("Possibly transient ZooKeeper exception: " + e);
485             if (!retryCounter.shouldRetry()) {
486               LOG.error("ZooKeeper create failed after "
487                 + retryCounter.getMaxRetries() + " retries");
488               throw e;
489             }
490             break;
491 
492           default:
493             throw e;
494         }
495       }
496       retryCounter.sleepUntilNextRetry();
497       retryCounter.useRetry();
498       isRetry = true;
499     }
500   }
501   
502   private String createSequential(String path, byte[] data, 
503       List<ACL> acl, CreateMode createMode)
504   throws KeeperException, InterruptedException {
505     RetryCounter retryCounter = retryCounterFactory.create();
506     boolean first = true;
507     String newPath = path+this.identifier;
508     while (true) {
509       try {
510         if (!first) {
511           // Check if we succeeded on a previous attempt
512           String previousResult = findPreviousSequentialNode(newPath);
513           if (previousResult != null) {
514             return previousResult;
515           }
516         }
517         first = false;
518         return zk.create(newPath, data, acl, createMode);
519       } catch (KeeperException e) {
520         switch (e.code()) {
521           case CONNECTIONLOSS:
522           case OPERATIONTIMEOUT:
523             LOG.warn("Possibly transient ZooKeeper exception: " + e);
524             if (!retryCounter.shouldRetry()) {
525               LOG.error("ZooKeeper create failed after "
526                 + retryCounter.getMaxRetries() + " retries");
527               throw e;
528             }
529             break;
530 
531           default:
532             throw e;
533         }
534       }
535       retryCounter.sleepUntilNextRetry();
536       retryCounter.useRetry();
537     }
538   }
539 
540   private String findPreviousSequentialNode(String path)
541     throws KeeperException, InterruptedException {
542     int lastSlashIdx = path.lastIndexOf('/');
543     assert(lastSlashIdx != -1);
544     String parent = path.substring(0, lastSlashIdx);
545     String nodePrefix = path.substring(lastSlashIdx+1);
546 
547     List<String> nodes = zk.getChildren(parent, false);
548     List<String> matching = filterByPrefix(nodes, nodePrefix);
549     for (String node : matching) {
550       String nodePath = parent + "/" + node;
551       Stat stat = zk.exists(nodePath, false);
552       if (stat != null) {
553         return nodePath;
554       }
555     }
556     return null;
557   }
558   
559   public byte[] removeMetaData(byte[] data) {
560     if(data == null || data.length == 0) {
561       return data;
562     }
563     // check the magic data; to be backward compatible
564     byte magic = data[0];
565     if(magic != MAGIC) {
566       return data;
567     }
568     
569     int idLength = Bytes.toInt(data, MAGIC_OFFSET);
570     int dataLength = data.length-MAGIC_OFFSET-ID_OFFSET-idLength;
571     int dataOffset = MAGIC_OFFSET+ID_OFFSET+idLength;
572 
573     byte[] newData = new byte[dataLength];
574     System.arraycopy(data, dataOffset, newData, 0, dataLength);
575     
576     return newData;
577     
578   }
579   
580   private byte[] appendMetaData(byte[] data) {
581     if(data == null || data.length == 0){
582       return data;
583     }
584     
585     byte[] newData = new byte[MAGIC_OFFSET+ID_OFFSET+id.length+data.length];
586     int pos = 0;
587     pos = Bytes.putByte(newData, pos, MAGIC);
588     pos = Bytes.putInt(newData, pos, id.length);
589     pos = Bytes.putBytes(newData, pos, id, 0, id.length);
590     pos = Bytes.putBytes(newData, pos, data, 0, data.length);
591 
592     return newData;
593   }
594 
595   public long getSessionId() {
596     return zk.getSessionId();
597   }
598 
599   public void close() throws InterruptedException {
600     zk.close();
601   }
602 
603   public States getState() {
604     return zk.getState();
605   }
606 
607   public ZooKeeper getZooKeeper() {
608     return zk;
609   }
610 
611   public byte[] getSessionPasswd() {
612     return zk.getSessionPasswd();
613   }
614 
615   public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) {
616     this.zk.sync(path, null, null);
617   }
618 
619   /**
620    * Filters the given node list by the given prefixes.
621    * This method is all-inclusive--if any element in the node list starts
622    * with any of the given prefixes, then it is included in the result.
623    *
624    * @param nodes the nodes to filter
625    * @param prefixes the prefixes to include in the result
626    * @return list of every element that starts with one of the prefixes
627    */
628   private static List<String> filterByPrefix(List<String> nodes, 
629       String... prefixes) {
630     List<String> lockChildren = new ArrayList<String>();
631     for (String child : nodes){
632       for (String prefix : prefixes){
633         if (child.startsWith(prefix)){
634           lockChildren.add(child);
635           break;
636         }
637       }
638     }
639     return lockChildren;
640   }
641 }