View Javadoc

1   /*
2    * Copyright 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.master;
21  
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.util.List;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.classification.InterfaceAudience;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.InterProcessLock;
31  import org.apache.hadoop.hbase.InterProcessLock.MetadataHandler;
32  import org.apache.hadoop.hbase.InterProcessReadWriteLock;
33  import org.apache.hadoop.hbase.ServerName;
34  import org.apache.hadoop.hbase.exceptions.LockTimeoutException;
35  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
36  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
39  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
40  import org.apache.hadoop.hbase.zookeeper.lock.ZKInterProcessReadWriteLock;
41  import org.apache.zookeeper.KeeperException;
42  
43  import com.google.protobuf.ByteString;
44  import com.google.protobuf.InvalidProtocolBufferException;
45  
46  /**
47   * A manager for distributed table level locks.
48   */
49  @InterfaceAudience.Private
50  public abstract class TableLockManager {
51  
52    private static final Log LOG = LogFactory.getLog(TableLockManager.class);
53  
54    /** Configuration key for enabling table-level locks for schema changes */
55    public static final String TABLE_LOCK_ENABLE =
56      "hbase.table.lock.enable";
57  
58    /** by default we should enable table-level locks for schema changes */
59    private static final boolean DEFAULT_TABLE_LOCK_ENABLE = true;
60  
61    /** Configuration key for time out for trying to acquire table locks */
62    protected static final String TABLE_WRITE_LOCK_TIMEOUT_MS =
63      "hbase.table.write.lock.timeout.ms";
64  
65    /** Configuration key for time out for trying to acquire table locks */
66    protected static final String TABLE_READ_LOCK_TIMEOUT_MS =
67      "hbase.table.read.lock.timeout.ms";
68  
69    protected static final int DEFAULT_TABLE_WRITE_LOCK_TIMEOUT_MS =
70      600 * 1000; //10 min default
71  
72    protected static final int DEFAULT_TABLE_READ_LOCK_TIMEOUT_MS =
73      600 * 1000; //10 min default
74  
75    /**
76     * A distributed lock for a table.
77     */
78    @InterfaceAudience.Private
79    public static interface TableLock {
80      /**
81       * Acquire the lock, with the configured lock timeout.
82       * @throws LockTimeoutException If unable to acquire a lock within a specified
83       * time period (if any)
84       * @throws IOException If unrecoverable error occurs
85       */
86      public void acquire() throws IOException;
87  
88      /**
89       * Release the lock already held.
90       * @throws IOException If there is an unrecoverable error releasing the lock
91       */
92      public void release() throws IOException;
93    }
94  
95    /**
96     * Returns a TableLock for locking the table for exclusive access
97     * @param tableName Table to lock
98     * @param purpose Human readable reason for locking the table
99     * @return A new TableLock object for acquiring a write lock
100    */
101   public abstract TableLock writeLock(byte[] tableName, String purpose);
102 
103   /**
104    * Returns a TableLock for locking the table for shared access among read-lock holders
105    * @param tableName Table to lock
106    * @param purpose Human readable reason for locking the table
107    * @return A new TableLock object for acquiring a read lock
108    */
109   public abstract TableLock readLock(byte[] tableName, String purpose);
110 
111   /**
112    * Force releases all table write locks and lock attempts even if this thread does
113    * not own the lock. The behavior of the lock holders still thinking that they
114    * have the lock is undefined. This should be used carefully and only when
115    * we can ensure that all write-lock holders have died. For example if only
116    * the master can hold write locks, then we can reap it's locks when the backup
117    * master starts.
118    */
119   public abstract void reapAllTableWriteLocks() throws IOException;
120 
121   /**
122    * Called after a table has been deleted, and after the table lock is  released.
123    * TableLockManager should do cleanup for the table state.
124    * @param tableName name of the table
125    * @throws IOException If there is an unrecoverable error releasing the lock
126    */
127   public abstract void tableDeleted(byte[] tableName)
128       throws IOException;
129 
130   /**
131    * Creates and returns a TableLockManager according to the configuration
132    */
133   public static TableLockManager createTableLockManager(Configuration conf,
134       ZooKeeperWatcher zkWatcher, ServerName serverName) {
135     // Initialize table level lock manager for schema changes, if enabled.
136     if (conf.getBoolean(TABLE_LOCK_ENABLE,
137         DEFAULT_TABLE_LOCK_ENABLE)) {
138       int writeLockTimeoutMs = conf.getInt(TABLE_WRITE_LOCK_TIMEOUT_MS,
139           DEFAULT_TABLE_WRITE_LOCK_TIMEOUT_MS);
140       int readLockTimeoutMs = conf.getInt(TABLE_READ_LOCK_TIMEOUT_MS,
141           DEFAULT_TABLE_READ_LOCK_TIMEOUT_MS);
142       return new ZKTableLockManager(zkWatcher, serverName, writeLockTimeoutMs, readLockTimeoutMs);
143     }
144 
145     return new NullTableLockManager();
146   }
147 
148   /**
149    * A null implementation
150    */
151   @InterfaceAudience.Private
152   public static class NullTableLockManager extends TableLockManager {
153     static class NullTableLock implements TableLock {
154       @Override
155       public void acquire() throws IOException {
156       }
157       @Override
158       public void release() throws IOException {
159       }
160     }
161     @Override
162     public TableLock writeLock(byte[] tableName, String purpose) {
163       return new NullTableLock();
164     }
165     @Override
166     public TableLock readLock(byte[] tableName, String purpose) {
167       return new NullTableLock();
168     }
169     @Override
170     public void reapAllTableWriteLocks() throws IOException {
171     }
172     @Override
173     public void tableDeleted(byte[] tableName) throws IOException {
174     }
175   }
176 
177   /**
178    * ZooKeeper based TableLockManager
179    */
180   @InterfaceAudience.Private
181   private static class ZKTableLockManager extends TableLockManager {
182 
183     private static final MetadataHandler METADATA_HANDLER = new MetadataHandler() {
184       @Override
185       public void handleMetadata(byte[] ownerMetadata) {
186         if (!LOG.isDebugEnabled()) {
187           return;
188         }
189         ZooKeeperProtos.TableLock data = fromBytes(ownerMetadata);
190         if (data == null) {
191           return;
192         }
193         LOG.debug("Table is locked by: " +
194             String.format("[tableName=%s, lockOwner=%s, threadId=%s, " +
195                 "purpose=%s, isShared=%s]", Bytes.toString(data.getTableName().toByteArray()),
196                 ProtobufUtil.toServerName(data.getLockOwner()), data.getThreadId(),
197                 data.getPurpose(), data.getIsShared()));
198       }
199     };
200 
201     private static class TableLockImpl implements TableLock {
202       long lockTimeoutMs;
203       byte[] tableName;
204       String tableNameStr;
205       InterProcessLock lock;
206       boolean isShared;
207       ZooKeeperWatcher zkWatcher;
208       ServerName serverName;
209       String purpose;
210 
211       public TableLockImpl(byte[] tableName, ZooKeeperWatcher zkWatcher,
212           ServerName serverName, long lockTimeoutMs, boolean isShared, String purpose) {
213         this.tableName = tableName;
214         tableNameStr = Bytes.toString(tableName);
215         this.zkWatcher = zkWatcher;
216         this.serverName = serverName;
217         this.lockTimeoutMs = lockTimeoutMs;
218         this.isShared = isShared;
219         this.purpose = purpose;
220       }
221 
222       @Override
223       public void acquire() throws IOException {
224         if (LOG.isDebugEnabled()) {
225           LOG.debug("Attempt to acquire table " + (isShared ? "read" : "write")
226               + " lock on :" + tableNameStr + " for:" + purpose);
227         }
228 
229         lock = createTableLock();
230         try {
231           if (lockTimeoutMs == -1) {
232             // Wait indefinitely
233             lock.acquire();
234           } else {
235             if (!lock.tryAcquire(lockTimeoutMs)) {
236               throw new LockTimeoutException("Timed out acquiring " +
237                 (isShared ? "read" : "write") + "lock for table:" + tableNameStr +
238                 "for:" + purpose + " after " + lockTimeoutMs + " ms.");
239             }
240           }
241         } catch (InterruptedException e) {
242           LOG.warn("Interrupted acquiring a lock for " + tableNameStr, e);
243           Thread.currentThread().interrupt();
244           throw new InterruptedIOException("Interrupted acquiring a lock");
245         }
246         LOG.debug("Acquired table " + (isShared ? "read" : "write")
247             + " lock on :" + tableNameStr + " for:" + purpose);
248       }
249 
250       @Override
251       public void release() throws IOException {
252         if (LOG.isDebugEnabled()) {
253           LOG.debug("Attempt to release table " + (isShared ? "read" : "write")
254               + " lock on :" + tableNameStr);
255         }
256         if (lock == null) {
257           throw new IllegalStateException("Table " + tableNameStr +
258             " is not locked!");
259         }
260 
261         try {
262           lock.release();
263         } catch (InterruptedException e) {
264           LOG.warn("Interrupted while releasing a lock for " + tableNameStr);
265           Thread.currentThread().interrupt();
266           throw new InterruptedIOException();
267         }
268         if (LOG.isDebugEnabled()) {
269           LOG.debug("Released table lock on :" + tableNameStr);
270         }
271       }
272 
273       private InterProcessLock createTableLock() {
274         String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, tableNameStr);
275 
276         ZooKeeperProtos.TableLock data = ZooKeeperProtos.TableLock.newBuilder()
277           .setTableName(ByteString.copyFrom(tableName))
278           .setLockOwner(ProtobufUtil.toServerName(serverName))
279           .setThreadId(Thread.currentThread().getId())
280           .setPurpose(purpose)
281           .setIsShared(isShared).build();
282         byte[] lockMetadata = toBytes(data);
283 
284         InterProcessReadWriteLock lock = new ZKInterProcessReadWriteLock(zkWatcher, tableLockZNode,
285           METADATA_HANDLER);
286         return isShared ? lock.readLock(lockMetadata) : lock.writeLock(lockMetadata);
287       }
288     }
289 
290     private static byte[] toBytes(ZooKeeperProtos.TableLock data) {
291       return ProtobufUtil.prependPBMagic(data.toByteArray());
292     }
293 
294     private static ZooKeeperProtos.TableLock fromBytes(byte[] bytes) {
295       int pblen = ProtobufUtil.lengthOfPBMagic();
296       if (bytes == null || bytes.length < pblen) {
297         return null;
298       }
299       try {
300         ZooKeeperProtos.TableLock data = ZooKeeperProtos.TableLock.newBuilder().mergeFrom(
301             bytes, pblen, bytes.length - pblen).build();
302         return data;
303       } catch (InvalidProtocolBufferException ex) {
304         LOG.warn("Exception in deserialization", ex);
305       }
306       return null;
307     }
308 
309     private final ServerName serverName;
310     private final ZooKeeperWatcher zkWatcher;
311     private final long writeLockTimeoutMs;
312     private final long readLockTimeoutMs;
313 
314     /**
315      * Initialize a new manager for table-level locks.
316      * @param zkWatcher
317      * @param serverName Address of the server responsible for acquiring and
318      * releasing the table-level locks
319      * @param writeLockTimeoutMs Timeout (in milliseconds) for acquiring a write lock for a
320      * given table, or -1 for no timeout
321      * @param readLockTimeoutMs Timeout (in milliseconds) for acquiring a read lock for a
322      * given table, or -1 for no timeout
323      */
324     public ZKTableLockManager(ZooKeeperWatcher zkWatcher,
325       ServerName serverName, long writeLockTimeoutMs, long readLockTimeoutMs) {
326       this.zkWatcher = zkWatcher;
327       this.serverName = serverName;
328       this.writeLockTimeoutMs = writeLockTimeoutMs;
329       this.readLockTimeoutMs = readLockTimeoutMs;
330     }
331 
332     @Override
333     public TableLock writeLock(byte[] tableName, String purpose) {
334       return new TableLockImpl(tableName, zkWatcher,
335           serverName, writeLockTimeoutMs, false, purpose);
336     }
337 
338     public TableLock readLock(byte[] tableName, String purpose) {
339       return new TableLockImpl(tableName, zkWatcher,
340           serverName, readLockTimeoutMs, true, purpose);
341     }
342 
343     @Override
344     public void reapAllTableWriteLocks() throws IOException {
345       //get the table names
346       try {
347         List<String> tableNames;
348         try {
349           tableNames = ZKUtil.listChildrenNoWatch(zkWatcher, zkWatcher.tableLockZNode);
350         } catch (KeeperException e) {
351           LOG.error("Unexpected ZooKeeper error when listing children", e);
352           throw new IOException("Unexpected ZooKeeper exception", e);
353         }
354 
355         for (String tableName : tableNames) {
356           String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, tableName);
357           ZKInterProcessReadWriteLock lock = new ZKInterProcessReadWriteLock(
358               zkWatcher, tableLockZNode, null);
359           lock.writeLock(null).reapAllLocks();
360         }
361       } catch (IOException ex) {
362         throw ex;
363       } catch (Exception ex) {
364         LOG.warn("Caught exception while reaping table write locks", ex);
365       }
366     }
367 
368     @Override
369     public void tableDeleted(byte[] tableName) throws IOException {
370       //table write lock from DeleteHandler is already released, just delete the parent znode
371       String tableNameStr = Bytes.toString(tableName);
372       String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, tableNameStr);
373       try {
374         ZKUtil.deleteNode(zkWatcher, tableLockZNode);
375       } catch (KeeperException ex) {
376         if (ex.code() == KeeperException.Code.NOTEMPTY) {
377           //we might get this in rare occasions where a CREATE table or some other table operation
378           //is waiting to acquire the lock. In this case, parent znode won't be deleted.
379           LOG.warn("Could not delete the znode for table locks because NOTEMPTY: "
380               + tableLockZNode);
381           return;
382         }
383         throw new IOException(ex);
384       }
385     }
386   }
387 }