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   * WITHOUTKey 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  
19  package org.apache.hadoop.hbase.master.procedure;
20  
21  import java.io.IOException;
22  import java.util.ArrayDeque;
23  import java.util.concurrent.locks.Condition;
24  import java.util.concurrent.locks.ReentrantLock;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.ServerName;
30  import org.apache.hadoop.hbase.TableExistsException;
31  import org.apache.hadoop.hbase.TableName;
32  import org.apache.hadoop.hbase.TableNotFoundException;
33  import org.apache.hadoop.hbase.classification.InterfaceAudience;
34  import org.apache.hadoop.hbase.classification.InterfaceStability;
35  import org.apache.hadoop.hbase.master.TableLockManager;
36  import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
37  import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
38  import org.apache.hadoop.hbase.procedure2.Procedure;
39  import org.apache.hadoop.hbase.procedure2.ProcedureRunnableSet;
40  
41  /**
42   * ProcedureRunnableSet for the Master Procedures.
43   * This RunnableSet tries to provide to the ProcedureExecutor procedures
44   * that can be executed without having to wait on a lock.
45   * Most of the master operations can be executed concurrently, if they
46   * are operating on different tables (e.g. two create table can be performed
47   * at the same, time assuming table A and table B) or against two different servers; say
48   * two servers that crashed at about the same time.
49   *
50   * <p>Each procedure should implement an interface providing information for this queue.
51   * for example table related procedures should implement TableProcedureInterface.
52   * each procedure will be pushed in its own queue, and based on the operation type
53   * we may take smarter decision. e.g. we can abort all the operations preceding
54   * a delete table, or similar.
55   */
56  @InterfaceAudience.Private
57  @InterfaceStability.Evolving
58  public class MasterProcedureScheduler implements ProcedureRunnableSet {
59    private static final Log LOG = LogFactory.getLog(MasterProcedureScheduler.class);
60  
61    private final TableLockManager lockManager;
62    private final ReentrantLock schedLock = new ReentrantLock();
63    private final Condition schedWaitCond = schedLock.newCondition();
64  
65    private final FairQueue<ServerName> serverRunQueue = new FairQueue<ServerName>();
66    private final FairQueue<TableName> tableRunQueue = new FairQueue<TableName>();
67    private int queueSize = 0;
68  
69    private final Object[] serverBuckets = new Object[128];
70    private Queue<String> namespaceMap = null;
71    private Queue<TableName> tableMap = null;
72  
73    private final int metaTablePriority;
74    private final int userTablePriority;
75    private final int sysTablePriority;
76  
77    // TODO: metrics
78    private long pollCalls = 0;
79    private long nullPollCalls = 0;
80  
81    public MasterProcedureScheduler(final Configuration conf, final TableLockManager lockManager) {
82      this.lockManager = lockManager;
83  
84      // TODO: should this be part of the HTD?
85      metaTablePriority = conf.getInt("hbase.master.procedure.queue.meta.table.priority", 3);
86      sysTablePriority = conf.getInt("hbase.master.procedure.queue.system.table.priority", 2);
87      userTablePriority = conf.getInt("hbase.master.procedure.queue.user.table.priority", 1);
88    }
89  
90    @Override
91    public void addFront(Procedure proc) {
92      doAdd(proc, true);
93    }
94  
95    @Override
96    public void addBack(Procedure proc) {
97      doAdd(proc, false);
98    }
99  
100   @Override
101   public void yield(final Procedure proc) {
102     doAdd(proc, isTableProcedure(proc));
103   }
104 
105   private void doAdd(final Procedure proc, final boolean addFront) {
106     schedLock.lock();
107     try {
108       if (isTableProcedure(proc)) {
109         doAdd(tableRunQueue, getTableQueue(getTableName(proc)), proc, addFront);
110       } else if (isServerProcedure(proc)) {
111         doAdd(serverRunQueue, getServerQueue(getServerName(proc)), proc, addFront);
112       } else {
113         // TODO: at the moment we only have Table and Server procedures
114         // if you are implementing a non-table/non-server procedure, you have two options: create
115         // a group for all the non-table/non-server procedures or try to find a key for your
116         // non-table/non-server procedures and implement something similar to the TableRunQueue.
117         throw new UnsupportedOperationException(
118           "RQs for non-table/non-server procedures are not implemented yet");
119       }
120       schedWaitCond.signal();
121     } finally {
122       schedLock.unlock();
123     }
124   }
125 
126   private <T extends Comparable<T>> void doAdd(final FairQueue<T> fairq,
127       final Queue<T> queue, final Procedure proc, final boolean addFront) {
128     queue.add(proc, addFront);
129     if (!(queue.isSuspended() || queue.hasExclusiveLock())) {
130       if (queue.size() == 1 && !IterableList.isLinked(queue)) {
131         fairq.add(queue);
132       }
133       queueSize++;
134     }
135   }
136 
137   @Override
138   public Procedure poll() {
139     return poll(-1);
140   }
141 
142   @edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
143   Procedure poll(long waitNsec) {
144     Procedure pollResult = null;
145     schedLock.lock();
146     try {
147       if (queueSize == 0) {
148         if (waitNsec < 0) {
149           schedWaitCond.await();
150         } else {
151           schedWaitCond.awaitNanos(waitNsec);
152         }
153         if (queueSize == 0) {
154           return null;
155         }
156       }
157 
158       // For now, let server handling have precedence over table handling; presumption is that it
159       // is more important handling crashed servers than it is running the
160       // enabling/disabling tables, etc.
161       pollResult = doPoll(serverRunQueue);
162       if (pollResult == null) {
163         pollResult = doPoll(tableRunQueue);
164       }
165 
166       // update metrics
167       pollCalls++;
168       nullPollCalls += (pollResult == null) ? 1 : 0;
169     } catch (InterruptedException e) {
170       Thread.currentThread().interrupt();
171     } finally {
172       schedLock.unlock();
173     }
174     return pollResult;
175   }
176 
177   private <T extends Comparable<T>> Procedure doPoll(final FairQueue<T> fairq) {
178     Queue<T> rq = fairq.poll();
179     if (rq == null || !rq.isAvailable()) {
180       return null;
181     }
182 
183     assert !rq.isSuspended() : "rq=" + rq + " is suspended";
184     Procedure pollResult = rq.poll();
185     this.queueSize--;
186     if (rq.isEmpty() || rq.requireExclusiveLock(pollResult)) {
187       removeFromRunQueue(fairq, rq);
188     }
189     return pollResult;
190   }
191 
192   @Override
193   public void clear() {
194     // NOTE: USED ONLY FOR TESTING
195     schedLock.lock();
196     try {
197       // Remove Servers
198       for (int i = 0; i < serverBuckets.length; ++i) {
199         clear((ServerQueue)serverBuckets[i], serverRunQueue);
200         serverBuckets[i] = null;
201       }
202 
203       // Remove Tables
204       clear(tableMap, tableRunQueue);
205       tableMap = null;
206 
207       assert queueSize == 0 : "expected queue size to be 0, got " + queueSize;
208     } finally {
209       schedLock.unlock();
210     }
211   }
212 
213   private <T extends Comparable<T>> void clear(Queue<T> treeMap, FairQueue<T> fairq) {
214     while (treeMap != null) {
215       Queue<T> node = AvlTree.getFirst(treeMap);
216       assert !node.isSuspended() : "can't clear suspended " + node.getKey();
217       treeMap = AvlTree.remove(treeMap, node.getKey());
218       removeFromRunQueue(fairq, node);
219     }
220   }
221 
222   @Override
223   public void signalAll() {
224     schedLock.lock();
225     try {
226       schedWaitCond.signalAll();
227     } finally {
228       schedLock.unlock();
229     }
230   }
231 
232   @Override
233   public int size() {
234     schedLock.lock();
235     try {
236       return queueSize;
237     } finally {
238       schedLock.unlock();
239     }
240   }
241 
242   @Override
243   public void completionCleanup(Procedure proc) {
244     if (proc instanceof TableProcedureInterface) {
245       TableProcedureInterface iProcTable = (TableProcedureInterface)proc;
246       boolean tableDeleted;
247       if (proc.hasException()) {
248         IOException procEx =  proc.getException().unwrapRemoteException();
249         if (iProcTable.getTableOperationType() == TableOperationType.CREATE) {
250           // create failed because the table already exist
251           tableDeleted = !(procEx instanceof TableExistsException);
252         } else {
253           // the operation failed because the table does not exist
254           tableDeleted = (procEx instanceof TableNotFoundException);
255         }
256       } else {
257         // the table was deleted
258         tableDeleted = (iProcTable.getTableOperationType() == TableOperationType.DELETE);
259       }
260       if (tableDeleted) {
261         markTableAsDeleted(iProcTable.getTableName());
262         return;
263       }
264     } else {
265       // No cleanup for ServerProcedureInterface types, yet.
266       return;
267     }
268   }
269 
270   private <T extends Comparable<T>> void addToRunQueue(FairQueue<T> fairq, Queue<T> queue) {
271     if (IterableList.isLinked(queue)) return;
272     if (!queue.isEmpty())  {
273       fairq.add(queue);
274       queueSize += queue.size();
275     }
276   }
277 
278   private <T extends Comparable<T>> void removeFromRunQueue(FairQueue<T> fairq, Queue<T> queue) {
279     if (!IterableList.isLinked(queue)) return;
280     fairq.remove(queue);
281     queueSize -= queue.size();
282   }
283 
284   // ============================================================================
285   //  TODO: Metrics
286   // ============================================================================
287   public long getPollCalls() {
288     return pollCalls;
289   }
290 
291   public long getNullPollCalls() {
292     return nullPollCalls;
293   }
294 
295   // ============================================================================
296   //  Event Helpers
297   // ============================================================================
298   public boolean waitEvent(ProcedureEvent event, Procedure procedure) {
299     return waitEvent(event, procedure, false);
300   }
301 
302   public boolean waitEvent(ProcedureEvent event, Procedure procedure, boolean suspendQueue) {
303     synchronized (event) {
304       if (event.isReady()) {
305         return false;
306       }
307 
308       // TODO: Suspend single procedure not implemented yet, fallback to suspending the queue
309       if (!suspendQueue) suspendQueue = true;
310 
311       if (isTableProcedure(procedure)) {
312         waitTableEvent(event, procedure, suspendQueue);
313       } else if (isServerProcedure(procedure)) {
314         waitServerEvent(event, procedure, suspendQueue);
315       } else {
316         // TODO: at the moment we only have Table and Server procedures
317         // if you are implementing a non-table/non-server procedure, you have two options: create
318         // a group for all the non-table/non-server procedures or try to find a key for your
319         // non-table/non-server procedures and implement something similar to the TableRunQueue.
320         throw new UnsupportedOperationException(
321           "RQs for non-table/non-server procedures are not implemented yet");
322       }
323     }
324     return true;
325   }
326 
327   private void waitTableEvent(ProcedureEvent event, Procedure procedure, boolean suspendQueue) {
328     final TableName tableName = getTableName(procedure);
329     final boolean isDebugEnabled = LOG.isDebugEnabled();
330 
331     schedLock.lock();
332     try {
333       TableQueue queue = getTableQueue(tableName);
334       if (queue.isSuspended()) return;
335 
336       // TODO: if !suspendQueue
337 
338       if (isDebugEnabled) {
339         LOG.debug("Suspend table queue " + tableName);
340       }
341       queue.setSuspended(true);
342       removeFromRunQueue(tableRunQueue, queue);
343       event.suspendTableQueue(queue);
344     } finally {
345       schedLock.unlock();
346     }
347   }
348 
349   private void waitServerEvent(ProcedureEvent event, Procedure procedure, boolean suspendQueue) {
350     final ServerName serverName = getServerName(procedure);
351     final boolean isDebugEnabled = LOG.isDebugEnabled();
352 
353     schedLock.lock();
354     try {
355       // TODO: This will change once we have the new AM
356       ServerQueue queue = getServerQueue(serverName);
357       if (queue.isSuspended()) return;
358 
359       // TODO: if !suspendQueue
360 
361       if (isDebugEnabled) {
362         LOG.debug("Suspend server queue " + serverName);
363       }
364       queue.setSuspended(true);
365       removeFromRunQueue(serverRunQueue, queue);
366       event.suspendServerQueue(queue);
367     } finally {
368       schedLock.unlock();
369     }
370   }
371 
372   public void suspend(ProcedureEvent event) {
373     final boolean isDebugEnabled = LOG.isDebugEnabled();
374     synchronized (event) {
375       event.setReady(false);
376       if (isDebugEnabled) {
377         LOG.debug("Suspend event " + event);
378       }
379     }
380   }
381 
382   public void wake(ProcedureEvent event) {
383     final boolean isDebugEnabled = LOG.isDebugEnabled();
384     synchronized (event) {
385       event.setReady(true);
386       if (isDebugEnabled) {
387         LOG.debug("Wake event " + event);
388       }
389 
390       schedLock.lock();
391       try {
392         while (event.hasWaitingTables()) {
393           Queue<TableName> queue = event.popWaitingTable();
394           addToRunQueue(tableRunQueue, queue);
395         }
396         // TODO: This will change once we have the new AM
397         while (event.hasWaitingServers()) {
398           Queue<ServerName> queue = event.popWaitingServer();
399           addToRunQueue(serverRunQueue, queue);
400         }
401 
402         if (queueSize > 1) {
403           schedWaitCond.signalAll();
404         } else if (queueSize > 0) {
405           schedWaitCond.signal();
406         }
407       } finally {
408         schedLock.unlock();
409       }
410     }
411   }
412 
413   public static class ProcedureEvent {
414     private final String description;
415 
416     private Queue<ServerName> waitingServers = null;
417     private Queue<TableName> waitingTables = null;
418     private boolean ready = false;
419 
420     public ProcedureEvent(String description) {
421       this.description = description;
422     }
423 
424     public synchronized boolean isReady() {
425       return ready;
426     }
427 
428     private synchronized void setReady(boolean isReady) {
429       this.ready = isReady;
430     }
431 
432     private void suspendTableQueue(Queue<TableName> queue) {
433       waitingTables = IterableList.append(waitingTables, queue);
434     }
435 
436     private void suspendServerQueue(Queue<ServerName> queue) {
437       waitingServers = IterableList.append(waitingServers, queue);
438     }
439 
440     private boolean hasWaitingTables() {
441       return waitingTables != null;
442     }
443 
444     private Queue<TableName> popWaitingTable() {
445       Queue<TableName> node = waitingTables;
446       waitingTables = IterableList.remove(waitingTables, node);
447       node.setSuspended(false);
448       return node;
449     }
450 
451     private boolean hasWaitingServers() {
452       return waitingServers != null;
453     }
454 
455     private Queue<ServerName> popWaitingServer() {
456       Queue<ServerName> node = waitingServers;
457       waitingServers = IterableList.remove(waitingServers, node);
458       node.setSuspended(false);
459       return node;
460     }
461 
462     @Override
463     public String toString() {
464       return String.format("ProcedureEvent(%s)", description);
465     }
466   }
467 
468   // ============================================================================
469   //  Table Queue Lookup Helpers
470   // ============================================================================
471   private TableQueue getTableQueueWithLock(TableName tableName) {
472     schedLock.lock();
473     try {
474       return getTableQueue(tableName);
475     } finally {
476       schedLock.unlock();
477     }
478   }
479 
480   private TableQueue getTableQueue(TableName tableName) {
481     Queue<TableName> node = AvlTree.get(tableMap, tableName);
482     if (node != null) return (TableQueue)node;
483 
484     NamespaceQueue nsQueue = getNamespaceQueue(tableName.getNamespaceAsString());
485     node = new TableQueue(tableName, nsQueue, getTablePriority(tableName));
486     tableMap = AvlTree.insert(tableMap, node);
487     return (TableQueue)node;
488   }
489 
490   private void removeTableQueue(TableName tableName) {
491     tableMap = AvlTree.remove(tableMap, tableName);
492   }
493 
494   private int getTablePriority(TableName tableName) {
495     if (tableName.equals(TableName.META_TABLE_NAME)) {
496       return metaTablePriority;
497     } else if (tableName.isSystemTable()) {
498       return sysTablePriority;
499     }
500     return userTablePriority;
501   }
502 
503   private static boolean isTableProcedure(Procedure proc) {
504     return proc instanceof TableProcedureInterface;
505   }
506 
507   private static TableName getTableName(Procedure proc) {
508     return ((TableProcedureInterface)proc).getTableName();
509   }
510 
511   // ============================================================================
512   //  Namespace Queue Lookup Helpers
513   // ============================================================================
514   private NamespaceQueue getNamespaceQueue(String namespace) {
515     Queue<String> node = AvlTree.get(namespaceMap, namespace);
516     if (node != null) return (NamespaceQueue)node;
517 
518     node = new NamespaceQueue(namespace);
519     namespaceMap = AvlTree.insert(namespaceMap, node);
520     return (NamespaceQueue)node;
521   }
522 
523   // ============================================================================
524   //  Server Queue Lookup Helpers
525   // ============================================================================
526   private ServerQueue getServerQueueWithLock(ServerName serverName) {
527     schedLock.lock();
528     try {
529       return getServerQueue(serverName);
530     } finally {
531       schedLock.unlock();
532     }
533   }
534 
535   private ServerQueue getServerQueue(ServerName serverName) {
536     int index = getBucketIndex(serverBuckets, serverName.hashCode());
537     Queue<ServerName> root = getTreeRoot(serverBuckets, index);
538     Queue<ServerName> node = AvlTree.get(root, serverName);
539     if (node != null) return (ServerQueue)node;
540 
541     node = new ServerQueue(serverName);
542     serverBuckets[index] = AvlTree.insert(root, node);
543     return (ServerQueue)node;
544   }
545 
546   private void removeServerQueue(ServerName serverName) {
547     int index = getBucketIndex(serverBuckets, serverName.hashCode());
548     serverBuckets[index] = AvlTree.remove((ServerQueue)serverBuckets[index], serverName);
549   }
550 
551   @SuppressWarnings("unchecked")
552   private static <T extends Comparable<T>> Queue<T> getTreeRoot(Object[] buckets, int index) {
553     return (Queue<T>) buckets[index];
554   }
555 
556   private static int getBucketIndex(Object[] buckets, int hashCode) {
557     return Math.abs(hashCode) % buckets.length;
558   }
559 
560   private static boolean isServerProcedure(Procedure proc) {
561     return proc instanceof ServerProcedureInterface;
562   }
563 
564   private static ServerName getServerName(Procedure proc) {
565     return ((ServerProcedureInterface)proc).getServerName();
566   }
567 
568   // ============================================================================
569   //  Table and Server Queue Implementation
570   // ============================================================================
571   public static class ServerQueue extends QueueImpl<ServerName> {
572     public ServerQueue(ServerName serverName) {
573       super(serverName);
574     }
575 
576     @Override
577     public boolean requireExclusiveLock(Procedure proc) {
578       ServerProcedureInterface spi = (ServerProcedureInterface)proc;
579       switch (spi.getServerOperationType()) {
580         case CRASH_HANDLER:
581           return true;
582         default:
583           break;
584       }
585       throw new UnsupportedOperationException("unexpected type " + spi.getServerOperationType());
586     }
587   }
588 
589   public static class TableQueue extends QueueImpl<TableName> {
590     private final NamespaceQueue namespaceQueue;
591 
592     private TableLock tableLock = null;
593 
594     public TableQueue(TableName tableName, NamespaceQueue namespaceQueue, int priority) {
595       super(tableName, priority);
596       this.namespaceQueue = namespaceQueue;
597     }
598 
599     public NamespaceQueue getNamespaceQueue() {
600       return namespaceQueue;
601     }
602 
603     @Override
604     public synchronized boolean isAvailable() {
605       return super.isAvailable() && !namespaceQueue.hasExclusiveLock();
606     }
607 
608     // TODO: We can abort pending/in-progress operation if the new call is
609     //       something like drop table. We can Override addBack(),
610     //       check the type and abort all the in-flight procedurs.
611     private boolean canAbortPendingOperations(Procedure proc) {
612       TableProcedureInterface tpi = (TableProcedureInterface)proc;
613       switch (tpi.getTableOperationType()) {
614         case DELETE:
615           return true;
616         default:
617           return false;
618       }
619     }
620 
621     @Override
622     public boolean requireExclusiveLock(Procedure proc) {
623       TableProcedureInterface tpi = (TableProcedureInterface)proc;
624       switch (tpi.getTableOperationType()) {
625         case CREATE:
626         case DELETE:
627         case DISABLE:
628         case ENABLE:
629           return true;
630         case EDIT:
631           // we allow concurrent edit on the NS table
632           return !tpi.getTableName().equals(TableName.NAMESPACE_TABLE_NAME);
633         case READ:
634           return false;
635         default:
636           break;
637       }
638       throw new UnsupportedOperationException("unexpected type " + tpi.getTableOperationType());
639     }
640 
641     private synchronized boolean tryZkSharedLock(final TableLockManager lockManager,
642         final String purpose) {
643       // Take zk-read-lock
644       TableName tableName = getKey();
645       tableLock = lockManager.readLock(tableName, purpose);
646       try {
647         tableLock.acquire();
648       } catch (IOException e) {
649         LOG.error("failed acquire read lock on " + tableName, e);
650         tableLock = null;
651         return false;
652       }
653       return true;
654     }
655 
656     private synchronized void releaseZkSharedLock(final TableLockManager lockManager) {
657       releaseTableLock(lockManager, isSingleSharedLock());
658     }
659 
660     private synchronized boolean tryZkExclusiveLock(final TableLockManager lockManager,
661         final String purpose) {
662       // Take zk-write-lock
663       TableName tableName = getKey();
664       tableLock = lockManager.writeLock(tableName, purpose);
665       try {
666         tableLock.acquire();
667       } catch (IOException e) {
668         LOG.error("failed acquire write lock on " + tableName, e);
669         tableLock = null;
670         return false;
671       }
672       return true;
673     }
674 
675     private synchronized void releaseZkExclusiveLock(final TableLockManager lockManager) {
676       releaseTableLock(lockManager, true);
677     }
678 
679     private void releaseTableLock(final TableLockManager lockManager, boolean reset) {
680       for (int i = 0; i < 3; ++i) {
681         try {
682           tableLock.release();
683           if (reset) {
684             tableLock = null;
685           }
686           break;
687         } catch (IOException e) {
688           LOG.warn("Could not release the table write-lock", e);
689         }
690       }
691     }
692   }
693 
694   /**
695    * the namespace is currently used just as a rwlock, not as a queue.
696    * because ns operation are not frequent enough. so we want to avoid
697    * having to move table queues around for suspend/resume.
698    */
699   private static class NamespaceQueue extends Queue<String> {
700     public NamespaceQueue(String namespace) {
701       super(namespace);
702     }
703 
704     @Override
705     public boolean requireExclusiveLock(Procedure proc) {
706       throw new UnsupportedOperationException();
707     }
708 
709     @Override
710     public void add(final Procedure proc, final boolean addToFront) {
711       throw new UnsupportedOperationException();
712     }
713 
714     @Override
715     public Procedure peek() {
716       throw new UnsupportedOperationException();
717     }
718 
719     @Override
720     public Procedure poll() {
721       throw new UnsupportedOperationException();
722     }
723 
724     @Override
725     public boolean isEmpty() {
726       throw new UnsupportedOperationException();
727     }
728 
729     @Override
730     public int size() {
731       throw new UnsupportedOperationException();
732     }
733   }
734 
735   // ============================================================================
736   //  Table Locking Helpers
737   // ============================================================================
738   /**
739    * Try to acquire the exclusive lock on the specified table.
740    * other operations in the table-queue will be executed after the lock is released.
741    * @param procedure the procedure trying to acquire the lock
742    * @param table Table to lock
743    * @return true if we were able to acquire the lock on the table, otherwise false.
744    */
745   public boolean tryAcquireTableExclusiveLock(final Procedure procedure, final TableName table) {
746     try {
747       schedLock.lock();
748       TableQueue queue = getTableQueue(table);
749       if (!queue.getNamespaceQueue().trySharedLock()) {
750         schedLock.unlock();
751         return false;
752       }
753 
754       if (!queue.tryExclusiveLock(procedure.getProcId())) {
755         queue.getNamespaceQueue().releaseSharedLock();
756         schedLock.unlock();
757         return false;
758       }
759 
760       removeFromRunQueue(tableRunQueue, queue);
761       schedLock.unlock();
762 
763       // Zk lock is expensive...
764       boolean hasXLock = queue.tryZkExclusiveLock(lockManager, procedure.toString());
765       if (!hasXLock) {
766         schedLock.lock();
767         queue.releaseExclusiveLock();
768         queue.getNamespaceQueue().releaseSharedLock();
769         addToRunQueue(tableRunQueue, queue);
770         schedLock.unlock();
771       }
772       return hasXLock;
773     } finally {
774       if(schedLock.isHeldByCurrentThread()){
775         schedLock.unlock();
776       }
777     }
778   }
779 
780   /**
781    * Release the exclusive lock taken with tryAcquireTableWrite()
782    * @param procedure the procedure releasing the lock
783    * @param table the name of the table that has the exclusive lock
784    */
785   public void releaseTableExclusiveLock(final Procedure procedure, final TableName table) {
786     try {
787       schedLock.lock();
788       TableQueue queue = getTableQueue(table);
789       schedLock.unlock();
790 
791       // Zk lock is expensive...
792       queue.releaseZkExclusiveLock(lockManager);
793 
794       schedLock.lock();
795       queue.releaseExclusiveLock();
796       queue.getNamespaceQueue().releaseSharedLock();
797       addToRunQueue(tableRunQueue, queue);
798       schedLock.unlock();
799     } finally {
800       if(schedLock.isHeldByCurrentThread()) {
801         schedLock.unlock();
802       }
803     }
804   }
805 
806   /**
807    * Try to acquire the shared lock on the specified table.
808    * other "read" operations in the table-queue may be executed concurrently,
809    * @param procedure the procedure trying to acquire the lock
810    * @param table Table to lock
811    * @return true if we were able to acquire the lock on the table, otherwise false.
812    */
813   public boolean tryAcquireTableSharedLock(final Procedure procedure, final TableName table) {
814     return tryAcquireTableQueueSharedLock(procedure, table) != null;
815   }
816 
817   private TableQueue tryAcquireTableQueueSharedLock(final Procedure procedure,
818       final TableName table) {
819     try {
820       schedLock.lock();
821       TableQueue queue = getTableQueue(table);
822       if (!queue.getNamespaceQueue().trySharedLock()) {
823         return null;
824       }
825 
826       if (!queue.trySharedLock()) {
827         queue.getNamespaceQueue().releaseSharedLock();
828         schedLock.unlock();
829         return null;
830       }
831 
832       schedLock.unlock();
833 
834       // Zk lock is expensive...
835       if (!queue.tryZkSharedLock(lockManager, procedure.toString())) {
836         schedLock.lock();
837         queue.releaseSharedLock();
838         queue.getNamespaceQueue().releaseSharedLock();
839         schedLock.unlock();
840         return null;
841       }
842       return queue;
843     } finally {
844       if(schedLock.isHeldByCurrentThread()) {
845         schedLock.unlock();
846       }
847     }
848   }
849 
850   /**
851    * Release the shared lock taken with tryAcquireTableRead()
852    * @param procedure the procedure releasing the lock
853    * @param table the name of the table that has the shared lock
854    */
855   public void releaseTableSharedLock(final Procedure procedure, final TableName table) {
856     final TableQueue queue = getTableQueueWithLock(table);
857 
858     // Zk lock is expensive...
859     queue.releaseZkSharedLock(lockManager);
860     try {
861       schedLock.lock();
862       queue.releaseSharedLock();
863       queue.getNamespaceQueue().releaseSharedLock();
864       schedLock.unlock();
865     } finally {
866       if(schedLock.isHeldByCurrentThread()) {
867         schedLock.unlock();
868       }
869     }
870   }
871 
872   /**
873    * Tries to remove the queue and the table-lock of the specified table.
874    * If there are new operations pending (e.g. a new create),
875    * the remove will not be performed.
876    * @param table the name of the table that should be marked as deleted
877    * @return true if deletion succeeded, false otherwise meaning that there are
878    *     other new operations pending for that table (e.g. a new create).
879    */
880   protected boolean markTableAsDeleted(final TableName table) {
881     final ReentrantLock l = schedLock;
882     l.lock();
883     try {
884       TableQueue queue = getTableQueue(table);
885       if (queue == null) return true;
886 
887       if (queue.isEmpty() && queue.tryExclusiveLock(0)) {
888         // remove the table from the run-queue and the map
889         if (IterableList.isLinked(queue)) {
890           tableRunQueue.remove(queue);
891         }
892 
893         // Remove the table lock
894         try {
895           lockManager.tableDeleted(table);
896         } catch (IOException e) {
897           LOG.warn("Received exception from TableLockManager.tableDeleted:", e); //not critical
898         }
899 
900         removeTableQueue(table);
901       } else {
902         // TODO: If there are no create, we can drop all the other ops
903         return false;
904       }
905     } finally {
906       l.unlock();
907     }
908     return true;
909   }
910 
911   // ============================================================================
912   //  Namespace Locking Helpers
913   // ============================================================================
914   /**
915    * Try to acquire the exclusive lock on the specified namespace.
916    * @see #releaseNamespaceExclusiveLock(Procedure,String)
917    * @param procedure the procedure trying to acquire the lock
918    * @param nsName Namespace to lock
919    * @return true if we were able to acquire the lock on the namespace, otherwise false.
920    */
921   public boolean tryAcquireNamespaceExclusiveLock(final Procedure procedure, final String nsName) {
922     schedLock.lock();
923     try {
924       TableQueue tableQueue = getTableQueue(TableName.NAMESPACE_TABLE_NAME);
925       if (!tableQueue.trySharedLock()) return false;
926 
927       NamespaceQueue nsQueue = getNamespaceQueue(nsName);
928       boolean hasLock = nsQueue.tryExclusiveLock(procedure.getProcId());
929       if (!hasLock) {
930         tableQueue.releaseSharedLock();
931       }
932       return hasLock;
933     } finally {
934       schedLock.unlock();
935     }
936   }
937 
938   /**
939    * Release the exclusive lock
940    * @see #tryAcquireNamespaceExclusiveLock(Procedure,String)
941    * @param procedure the procedure releasing the lock
942    * @param nsName the namespace that has the exclusive lock
943    */
944   public void releaseNamespaceExclusiveLock(final Procedure procedure, final String nsName) {
945     schedLock.lock();
946     try {
947       TableQueue tableQueue = getTableQueue(TableName.NAMESPACE_TABLE_NAME);
948       tableQueue.releaseSharedLock();
949 
950       NamespaceQueue queue = getNamespaceQueue(nsName);
951       queue.releaseExclusiveLock();
952     } finally {
953       schedLock.unlock();
954     }
955   }
956 
957   // ============================================================================
958   //  Server Locking Helpers
959   // ============================================================================
960   /**
961    * Try to acquire the exclusive lock on the specified server.
962    * @see #releaseServerExclusiveLock(Procedure,ServerName)
963    * @param procedure the procedure trying to acquire the lock
964    * @param serverName Server to lock
965    * @return true if we were able to acquire the lock on the server, otherwise false.
966    */
967   public boolean tryAcquireServerExclusiveLock(final Procedure procedure,
968       final ServerName serverName) {
969     schedLock.lock();
970     try {
971       ServerQueue queue = getServerQueue(serverName);
972       if (queue.tryExclusiveLock(procedure.getProcId())) {
973         removeFromRunQueue(serverRunQueue, queue);
974         return true;
975       }
976     } finally {
977       schedLock.unlock();
978     }
979     return false;
980   }
981 
982   /**
983    * Release the exclusive lock
984    * @see #tryAcquireServerExclusiveLock(Procedure,ServerName)
985    * @param procedure the procedure releasing the lock
986    * @param serverName the server that has the exclusive lock
987    */
988   public void releaseServerExclusiveLock(final Procedure procedure,
989       final ServerName serverName) {
990     schedLock.lock();
991     try {
992       ServerQueue queue = getServerQueue(serverName);
993       queue.releaseExclusiveLock();
994       addToRunQueue(serverRunQueue, queue);
995     } finally {
996       schedLock.unlock();
997     }
998   }
999 
1000   /**
1001    * Try to acquire the shared lock on the specified server.
1002    * @see #releaseServerSharedLock(Procedure,ServerName)
1003    * @param procedure the procedure releasing the lock
1004    * @param serverName Server to lock
1005    * @return true if we were able to acquire the lock on the server, otherwise false.
1006    */
1007   public boolean tryAcquireServerSharedLock(final Procedure procedure,
1008       final ServerName serverName) {
1009     return getServerQueueWithLock(serverName).trySharedLock();
1010   }
1011 
1012   /**
1013    * Release the shared lock taken
1014    * @see #tryAcquireServerSharedLock(Procedure,ServerName)
1015    * @param procedure the procedure releasing the lock
1016    * @param serverName the server that has the shared lock
1017    */
1018   public void releaseServerSharedLock(final Procedure procedure,
1019       final ServerName serverName) {
1020     getServerQueueWithLock(serverName).releaseSharedLock();
1021   }
1022 
1023   // ============================================================================
1024   //  Generic Helpers
1025   // ============================================================================
1026   private static interface QueueInterface {
1027     boolean isAvailable();
1028     boolean isEmpty();
1029     int size();
1030 
1031     void add(Procedure proc, boolean addFront);
1032     boolean requireExclusiveLock(Procedure proc);
1033     Procedure peek();
1034     Procedure poll();
1035 
1036     boolean isSuspended();
1037   }
1038 
1039   private static abstract class Queue<TKey extends Comparable<TKey>> implements QueueInterface {
1040     private Queue<TKey> avlRight = null;
1041     private Queue<TKey> avlLeft = null;
1042     private int avlHeight = 1;
1043 
1044     private Queue<TKey> iterNext = null;
1045     private Queue<TKey> iterPrev = null;
1046     private boolean suspended = false;
1047 
1048     private long exclusiveLockProcIdOwner = Long.MIN_VALUE;
1049     private int sharedLock = 0;
1050 
1051     private final TKey key;
1052     private final int priority;
1053 
1054     public Queue(TKey key) {
1055       this(key, 1);
1056     }
1057 
1058     public Queue(TKey key, int priority) {
1059       this.key = key;
1060       this.priority = priority;
1061     }
1062 
1063     protected TKey getKey() {
1064       return key;
1065     }
1066 
1067     protected int getPriority() {
1068       return priority;
1069     }
1070 
1071     /**
1072      * True if the queue is not in the run-queue and it is owned by an event.
1073      */
1074     @Override
1075     public boolean isSuspended() {
1076       return suspended;
1077     }
1078 
1079     protected boolean setSuspended(boolean isSuspended) {
1080       if (this.suspended == isSuspended) return false;
1081       this.suspended = isSuspended;
1082       return true;
1083     }
1084 
1085     // ======================================================================
1086     //  Read/Write Locking helpers
1087     // ======================================================================
1088     public synchronized boolean isLocked() {
1089       return hasExclusiveLock() || sharedLock > 0;
1090     }
1091 
1092     public synchronized boolean hasExclusiveLock() {
1093       return this.exclusiveLockProcIdOwner != Long.MIN_VALUE;
1094     }
1095 
1096     public synchronized boolean trySharedLock() {
1097       if (hasExclusiveLock()) return false;
1098       sharedLock++;
1099       return true;
1100     }
1101 
1102     public synchronized void releaseSharedLock() {
1103       sharedLock--;
1104     }
1105 
1106     protected synchronized boolean isSingleSharedLock() {
1107       return sharedLock == 1;
1108     }
1109 
1110     public synchronized boolean tryExclusiveLock(long procIdOwner) {
1111       assert procIdOwner != Long.MIN_VALUE;
1112       if (isLocked()) return false;
1113       exclusiveLockProcIdOwner = procIdOwner;
1114       return true;
1115     }
1116 
1117     public synchronized void releaseExclusiveLock() {
1118       exclusiveLockProcIdOwner = Long.MIN_VALUE;
1119     }
1120 
1121     // This should go away when we have the new AM and its events
1122     // and we move xlock to the lock-event-queue.
1123     @Override
1124     public synchronized boolean isAvailable() {
1125       return !hasExclusiveLock() && !isEmpty();
1126     }
1127 
1128     // ======================================================================
1129     //  Generic Helpers
1130     // ======================================================================
1131     public int compareKey(TKey cmpKey) {
1132       return key.compareTo(cmpKey);
1133     }
1134 
1135     public int compareTo(Queue<TKey> other) {
1136       return compareKey(other.key);
1137     }
1138 
1139     @Override
1140     public String toString() {
1141       return String.format("%s(%s)", getClass().getSimpleName(), key);
1142     }
1143   }
1144 
1145   // ======================================================================
1146   //  Helper Data Structures
1147   // ======================================================================
1148   private static abstract class QueueImpl<TKey extends Comparable<TKey>> extends Queue<TKey> {
1149     private final ArrayDeque<Procedure> runnables = new ArrayDeque<Procedure>();
1150 
1151     public QueueImpl(TKey key) {
1152       super(key);
1153     }
1154 
1155     public QueueImpl(TKey key, int priority) {
1156       super(key, priority);
1157     }
1158 
1159     @Override
1160     public void add(final Procedure proc, final boolean addToFront) {
1161       if (addToFront) {
1162         addFront(proc);
1163       } else {
1164         addBack(proc);
1165       }
1166     }
1167 
1168     protected void addFront(final Procedure proc) {
1169       runnables.addFirst(proc);
1170     }
1171 
1172     protected void addBack(final Procedure proc) {
1173       runnables.addLast(proc);
1174     }
1175 
1176     @Override
1177     public Procedure peek() {
1178       return runnables.peek();
1179     }
1180 
1181     @Override
1182     public Procedure poll() {
1183       return runnables.poll();
1184     }
1185 
1186     @Override
1187     public boolean isEmpty() {
1188       return runnables.isEmpty();
1189     }
1190 
1191     @Override
1192     public int size() {
1193       return runnables.size();
1194     }
1195   }
1196 
1197   private static class FairQueue<T extends Comparable<T>> {
1198     private final int quantum;
1199 
1200     private Queue<T> currentQueue = null;
1201     private Queue<T> queueHead = null;
1202     private int currentQuantum = 0;
1203 
1204     public FairQueue() {
1205       this(1);
1206     }
1207 
1208     public FairQueue(int quantum) {
1209       this.quantum = quantum;
1210     }
1211 
1212     public void add(Queue<T> queue) {
1213       queueHead = IterableList.append(queueHead, queue);
1214       if (currentQueue == null) setNextQueue(queueHead);
1215     }
1216 
1217     public void remove(Queue<T> queue) {
1218       Queue<T> nextQueue = queue.iterNext;
1219       queueHead = IterableList.remove(queueHead, queue);
1220       if (currentQueue == queue) {
1221         setNextQueue(queueHead != null ? nextQueue : null);
1222       }
1223     }
1224 
1225     public Queue<T> poll() {
1226       if (currentQuantum == 0) {
1227         if (!nextQueue()) {
1228           return null; // nothing here
1229         }
1230         currentQuantum = calculateQuantum(currentQueue) - 1;
1231       } else {
1232         currentQuantum--;
1233       }
1234 
1235       // This should go away when we have the new AM and its events
1236       if (!currentQueue.isAvailable()) {
1237         Queue<T> lastQueue = currentQueue;
1238         do {
1239           if (!nextQueue())
1240             return null;
1241         } while (currentQueue != lastQueue && !currentQueue.isAvailable());
1242 
1243         currentQuantum = calculateQuantum(currentQueue) - 1;
1244       }
1245       return currentQueue;
1246     }
1247 
1248     private boolean nextQueue() {
1249       if (currentQueue == null) return false;
1250       currentQueue = currentQueue.iterNext;
1251       return currentQueue != null;
1252     }
1253 
1254     private void setNextQueue(Queue<T> queue) {
1255       currentQueue = queue;
1256       if (queue != null) {
1257         currentQuantum = calculateQuantum(currentQueue);
1258       } else {
1259         currentQuantum = 0;
1260       }
1261     }
1262 
1263     private int calculateQuantum(final Queue queue) {
1264       return Math.max(1, queue.getPriority() * quantum); // TODO
1265     }
1266   }
1267 
1268   private static class AvlTree {
1269     public static <T extends Comparable<T>> Queue<T> get(Queue<T> root, T key) {
1270       while (root != null) {
1271         int cmp = root.compareKey(key);
1272         if (cmp > 0) {
1273           root = root.avlLeft;
1274         } else if (cmp < 0) {
1275           root = root.avlRight;
1276         } else {
1277           return root;
1278         }
1279       }
1280       return null;
1281     }
1282 
1283     public static <T extends Comparable<T>> Queue<T> getFirst(Queue<T> root) {
1284       if (root != null) {
1285         while (root.avlLeft != null) {
1286           root = root.avlLeft;
1287         }
1288       }
1289       return root;
1290     }
1291 
1292     public static <T extends Comparable<T>> Queue<T> getLast(Queue<T> root) {
1293       if (root != null) {
1294         while (root.avlRight != null) {
1295           root = root.avlRight;
1296         }
1297       }
1298       return root;
1299     }
1300 
1301     public static <T extends Comparable<T>> Queue<T> insert(Queue<T> root, Queue<T> node) {
1302       if (root == null) return node;
1303       if (node.compareTo(root) < 0) {
1304         root.avlLeft = insert(root.avlLeft, node);
1305       } else {
1306         root.avlRight = insert(root.avlRight, node);
1307       }
1308       return balance(root);
1309     }
1310 
1311     private static <T extends Comparable<T>> Queue<T> removeMin(Queue<T> p) {
1312       if (p.avlLeft == null)
1313         return p.avlRight;
1314       p.avlLeft = removeMin(p.avlLeft);
1315       return balance(p);
1316     }
1317 
1318     public static <T extends Comparable<T>> Queue<T> remove(Queue<T> root, T key) {
1319       if (root == null) return null;
1320 
1321       int cmp = root.compareKey(key);
1322       if (cmp == 0) {
1323         Queue<T> q = root.avlLeft;
1324         Queue<T> r = root.avlRight;
1325         if (r == null) return q;
1326         Queue<T> min = getFirst(r);
1327         min.avlRight = removeMin(r);
1328         min.avlLeft = q;
1329         return balance(min);
1330       } else if (cmp > 0) {
1331         root.avlLeft = remove(root.avlLeft, key);
1332       } else /* if (cmp < 0) */ {
1333         root.avlRight = remove(root.avlRight, key);
1334       }
1335       return balance(root);
1336     }
1337 
1338     private static <T extends Comparable<T>> Queue<T> balance(Queue<T> p) {
1339       fixHeight(p);
1340       int balance = balanceFactor(p);
1341       if (balance == 2) {
1342         if (balanceFactor(p.avlRight) < 0) {
1343           p.avlRight = rotateRight(p.avlRight);
1344         }
1345         return rotateLeft(p);
1346       } else if (balance == -2) {
1347         if (balanceFactor(p.avlLeft) > 0) {
1348           p.avlLeft = rotateLeft(p.avlLeft);
1349         }
1350         return rotateRight(p);
1351       }
1352       return p;
1353     }
1354 
1355     private static <T extends Comparable<T>> Queue<T> rotateRight(Queue<T> p) {
1356       Queue<T> q = p.avlLeft;
1357       p.avlLeft = q.avlRight;
1358       q.avlRight = p;
1359       fixHeight(p);
1360       fixHeight(q);
1361       return q;
1362     }
1363 
1364     private static <T extends Comparable<T>> Queue<T> rotateLeft(Queue<T> q) {
1365       Queue<T> p = q.avlRight;
1366       q.avlRight = p.avlLeft;
1367       p.avlLeft = q;
1368       fixHeight(q);
1369       fixHeight(p);
1370       return p;
1371     }
1372 
1373     private static <T extends Comparable<T>> void fixHeight(Queue<T> node) {
1374       int heightLeft = height(node.avlLeft);
1375       int heightRight = height(node.avlRight);
1376       node.avlHeight = 1 + Math.max(heightLeft, heightRight);
1377     }
1378 
1379     private static <T extends Comparable<T>> int height(Queue<T> node) {
1380       return node != null ? node.avlHeight : 0;
1381     }
1382 
1383     private static <T extends Comparable<T>> int balanceFactor(Queue<T> node) {
1384       return height(node.avlRight) - height(node.avlLeft);
1385     }
1386   }
1387 
1388   private static class IterableList {
1389     public static <T extends Comparable<T>> Queue<T> prepend(Queue<T> head, Queue<T> node) {
1390       assert !isLinked(node) : node + " is already linked";
1391       if (head != null) {
1392         Queue<T> tail = head.iterPrev;
1393         tail.iterNext = node;
1394         head.iterPrev = node;
1395         node.iterNext = head;
1396         node.iterPrev = tail;
1397       } else {
1398         node.iterNext = node;
1399         node.iterPrev = node;
1400       }
1401       return node;
1402     }
1403 
1404     public static <T extends Comparable<T>> Queue<T> append(Queue<T> head, Queue<T> node) {
1405       assert !isLinked(node) : node + " is already linked";
1406       if (head != null) {
1407         Queue<T> tail = head.iterPrev;
1408         tail.iterNext = node;
1409         node.iterNext = head;
1410         node.iterPrev = tail;
1411         head.iterPrev = node;
1412         return head;
1413       }
1414       node.iterNext = node;
1415       node.iterPrev = node;
1416       return node;
1417     }
1418 
1419     public static <T extends Comparable<T>> Queue<T> appendList(Queue<T> head, Queue<T> otherHead) {
1420       if (head == null) return otherHead;
1421       if (otherHead == null) return head;
1422 
1423       Queue<T> tail = head.iterPrev;
1424       Queue<T> otherTail = otherHead.iterPrev;
1425       tail.iterNext = otherHead;
1426       otherHead.iterPrev = tail;
1427       otherTail.iterNext = head;
1428       head.iterPrev = otherTail;
1429       return head;
1430     }
1431 
1432     private static <T extends Comparable<T>> Queue<T> remove(Queue<T> head, Queue<T> node) {
1433       assert isLinked(node) : node + " is not linked";
1434       if (node != node.iterNext) {
1435         node.iterPrev.iterNext = node.iterNext;
1436         node.iterNext.iterPrev = node.iterPrev;
1437         head = (head == node) ? node.iterNext : head;
1438       } else {
1439         head = null;
1440       }
1441       node.iterNext = null;
1442       node.iterPrev = null;
1443       return head;
1444     }
1445 
1446     private static <T extends Comparable<T>> boolean isLinked(Queue<T> node) {
1447       return node.iterPrev != null && node.iterNext != null;
1448     }
1449   }
1450 }