View Javadoc

1   /**
2    * Copyright 2010 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 org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.hbase.Abortable;
25  import org.apache.zookeeper.KeeperException;
26  
27  /**
28   * Tracks the availability and value of a single ZooKeeper node.
29   *
30   * <p>Utilizes the {@link ZooKeeperListener} interface to get the necessary
31   * ZooKeeper events related to the node.
32   *
33   * <p>This is the base class used by trackers in both the Master and
34   * RegionServers.
35   */
36  public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
37    
38    static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);
39    /** Path of node being tracked */
40    protected final String node;
41  
42    /** Data of the node being tracked */
43    private byte [] data;
44  
45    /** Used to abort if a fatal error occurs */
46    protected final Abortable abortable;
47  
48    private boolean stopped = false;
49  
50    /**
51     * Constructs a new ZK node tracker.
52     *
53     * <p>After construction, use {@link #start} to kick off tracking.
54     *
55     * @param watcher
56     * @param node
57     * @param abortable
58     */
59    public ZooKeeperNodeTracker(ZooKeeperWatcher watcher, String node,
60        Abortable abortable) {
61      super(watcher);
62      this.node = node;
63      this.abortable = abortable;
64      this.data = null;
65    }
66  
67    /**
68     * Starts the tracking of the node in ZooKeeper.
69     *
70     * <p>Use {@link #blockUntilAvailable()} to block until the node is available
71     * or {@link #getData(boolean)} to get the data of the node if it is available.
72     */
73    public synchronized void start() {
74      this.watcher.registerListener(this);
75      try {
76        if(ZKUtil.watchAndCheckExists(watcher, node)) {
77          byte [] data = ZKUtil.getDataAndWatch(watcher, node);
78          if(data != null) {
79            this.data = data;
80          } else {
81            // It existed but now does not, try again to ensure a watch is set
82            LOG.debug("Try starting again because there is no data from " + node);
83            start();
84          }
85        }
86      } catch (KeeperException e) {
87        abortable.abort("Unexpected exception during initialization, aborting", e);
88      }
89    }
90  
91    public synchronized void stop() {
92      this.stopped = true;
93      notifyAll();
94    }
95  
96    /**
97     * Gets the data of the node, blocking until the node is available.
98     *
99     * @return data of the node
100    * @throws InterruptedException if the waiting thread is interrupted
101    */
102   public synchronized byte [] blockUntilAvailable()
103   throws InterruptedException {
104     return blockUntilAvailable(0, false);
105   }
106 
107   /**
108    * Gets the data of the node, blocking until the node is available or the
109    * specified timeout has elapsed.
110    *
111    * @param timeout maximum time to wait for the node data to be available,
112    * n milliseconds.  Pass 0 for no timeout.
113    * @return data of the node
114    * @throws InterruptedException if the waiting thread is interrupted
115    */
116   public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh)
117   throws InterruptedException {
118     if (timeout < 0) throw new IllegalArgumentException();
119     boolean notimeout = timeout == 0;
120     long startTime = System.currentTimeMillis();
121     long remaining = timeout;
122     if (refresh) {
123       try {
124         this.data = ZKUtil.getDataAndWatch(watcher, node);
125       } catch(KeeperException e) {
126         abortable.abort("Unexpected exception handling blockUntilAvailable", e);
127       }
128     }
129     while (!this.stopped && (notimeout || remaining > 0) && this.data == null) {
130       if (notimeout) {
131         wait();
132         continue;
133       }
134       wait(remaining);
135       remaining = timeout - (System.currentTimeMillis() - startTime);
136     }
137     return this.data;
138   }
139 
140   /**
141    * Gets the data of the node.
142    *
143    * <p>If the node is currently available, the most up-to-date known version of
144    * the data is returned.  If the node is not currently available, null is
145    * returned.
146    * @param refresh whether to refresh the data by calling ZK directly.
147    * @return data of the node, null if unavailable
148    */
149   public synchronized byte [] getData(boolean refresh) {
150     if (refresh) {
151       try {
152         this.data = ZKUtil.getDataAndWatch(watcher, node);
153       } catch(KeeperException e) {
154         abortable.abort("Unexpected exception handling getData", e);
155       }
156     }
157     return this.data;
158   }
159 
160   public String getNode() {
161     return this.node;
162   }
163 
164   @Override
165   public synchronized void nodeCreated(String path) {
166     if (!path.equals(node)) return;
167     try {
168       byte [] data = ZKUtil.getDataAndWatch(watcher, node);
169       if (data != null) {
170         this.data = data;
171         notifyAll();
172       } else {
173         nodeDeleted(path);
174       }
175     } catch(KeeperException e) {
176       abortable.abort("Unexpected exception handling nodeCreated event", e);
177     }
178   }
179 
180   @Override
181   public synchronized void nodeDeleted(String path) {
182     if(path.equals(node)) {
183       try {
184         if(ZKUtil.watchAndCheckExists(watcher, node)) {
185           nodeCreated(path);
186         } else {
187           this.data = null;
188         }
189       } catch(KeeperException e) {
190         abortable.abort("Unexpected exception handling nodeDeleted event", e);
191       }
192     }
193   }
194 
195   @Override
196   public synchronized void nodeDataChanged(String path) {
197     if(path.equals(node)) {
198       nodeCreated(path);
199     }
200   }
201   
202   /**
203    * Checks if the baseznode set as per the property 'zookeeper.znode.parent'
204    * exists.
205    * @return true if baseznode exists.
206    *         false if doesnot exists.
207    */
208   public boolean checkIfBaseNodeAvailable() {
209     try {
210       if (ZKUtil.checkExists(watcher, watcher.baseZNode) == -1) {
211         return false;
212       }
213     } catch (KeeperException e) {
214       abortable
215           .abort(
216               "Exception while checking if basenode exists.",
217               e);
218     }
219     return true;
220   }
221 }