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.client;
21  
22  import java.io.Closeable;
23  import java.io.IOException;
24  import java.io.InterruptedIOException;
25  import java.lang.reflect.Proxy;
26  import java.lang.reflect.UndeclaredThrowableException;
27  import java.net.SocketTimeoutException;
28  import java.util.ArrayList;
29  import java.util.Arrays;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.concurrent.atomic.AtomicInteger;
33  import java.util.concurrent.atomic.AtomicReference;
34  import java.util.regex.Pattern;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.hbase.Abortable;
40  import org.apache.hadoop.hbase.ClusterStatus;
41  import org.apache.hadoop.hbase.HBaseConfiguration;
42  import org.apache.hadoop.hbase.HColumnDescriptor;
43  import org.apache.hadoop.hbase.HConstants;
44  import org.apache.hadoop.hbase.HRegionInfo;
45  import org.apache.hadoop.hbase.HRegionLocation;
46  import org.apache.hadoop.hbase.HTableDescriptor;
47  import org.apache.hadoop.hbase.MasterNotRunningException;
48  import org.apache.hadoop.hbase.NotServingRegionException;
49  import org.apache.hadoop.hbase.RegionException;
50  import org.apache.hadoop.hbase.RemoteExceptionHandler;
51  import org.apache.hadoop.hbase.ServerName;
52  import org.apache.hadoop.hbase.TableExistsException;
53  import org.apache.hadoop.hbase.TableNotEnabledException;
54  import org.apache.hadoop.hbase.TableNotFoundException;
55  import org.apache.hadoop.hbase.UnknownRegionException;
56  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
57  import org.apache.hadoop.hbase.catalog.CatalogTracker;
58  import org.apache.hadoop.hbase.catalog.MetaReader;
59  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
60  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
61  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
62  import org.apache.hadoop.hbase.ipc.HMasterInterface;
63  import org.apache.hadoop.hbase.ipc.HRegionInterface;
64  import org.apache.hadoop.hbase.ipc.MasterExecRPCInvoker;
65  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
66  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest.CompactionState;
67  import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
68  import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
69  import org.apache.hadoop.hbase.snapshot.HSnapshotDescription;
70  import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
71  import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
72  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
73  import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
74  import org.apache.hadoop.hbase.util.Addressing;
75  import org.apache.hadoop.hbase.util.Bytes;
76  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
77  import org.apache.hadoop.hbase.util.Pair;
78  import org.apache.hadoop.ipc.RemoteException;
79  import org.apache.hadoop.util.StringUtils;
80  
81  import com.google.protobuf.ServiceException;
82  
83  /**
84   * Provides an interface to manage HBase database table metadata + general
85   * administrative functions.  Use HBaseAdmin to create, drop, list, enable and
86   * disable tables. Use it also to add and drop table column families.
87   *
88   * <p>See {@link HTable} to add, update, and delete data from an individual table.
89   * <p>Currently HBaseAdmin instances are not expected to be long-lived.  For
90   * example, an HBaseAdmin instance will not ride over a Master restart.
91   */
92  public class HBaseAdmin implements Abortable, Closeable {
93    private final Log LOG = LogFactory.getLog(this.getClass().getName());
94  //  private final HConnection connection;
95    private HConnection connection;
96    private volatile Configuration conf;
97    private final long pause;
98    private final int numRetries;
99    // Some operations can take a long time such as disable of big table.
100   // numRetries is for 'normal' stuff... Mutliply by this factor when
101   // want to wait a long time.
102   private final int retryLongerMultiplier;
103   private boolean aborted;
104 
105   private static volatile boolean synchronousBalanceSwitchSupported = true;
106   private final boolean cleanupConnectionOnClose; // close the connection in close()
107 
108   /**
109    * Constructor
110    *
111    * @param c Configuration object
112    * @throws MasterNotRunningException if the master is not running
113    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
114    */
115   public HBaseAdmin(Configuration c)
116   throws MasterNotRunningException, ZooKeeperConnectionException {
117     this.conf = HBaseConfiguration.create(c);
118     this.connection = HConnectionManager.getConnection(this.conf);
119     this.pause = this.conf.getLong("hbase.client.pause", 1000);
120     this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
121     this.retryLongerMultiplier = this.conf.getInt(
122         "hbase.client.retries.longer.multiplier", 10);
123     this.cleanupConnectionOnClose = true;
124 
125     int tries = 0;
126     while ( true ){
127       try {
128 
129         this.connection.getMaster();
130         return;
131 
132       } catch (MasterNotRunningException mnre) {
133         HConnectionManager.deleteStaleConnection(this.connection);
134         this.connection = HConnectionManager.getConnection(this.conf);
135       }
136 
137       tries++;
138       if (tries >= numRetries) {
139         // we should delete connection between client and zookeeper
140         HConnectionManager.deleteStaleConnection(this.connection);
141         throw new MasterNotRunningException("Retried " + numRetries + " times");
142       }
143 
144       try {
145         Thread.sleep(getPauseTime(tries));
146       } catch (InterruptedException e) {
147         Thread.currentThread().interrupt();
148         // we should delete connection between client and zookeeper
149         HConnectionManager.deleteStaleConnection(this.connection);
150         throw new MasterNotRunningException(
151           "Interrupted after "+tries+" tries");
152       }
153     }
154   }
155 
156  /**
157    * Constructor for externally managed HConnections.
158    * This constructor fails fast if the HMaster is not running.
159    * The HConnection can be re-used again in another attempt.
160    * This constructor fails fast.
161    *
162    * @param connection The HConnection instance to use
163    * @throws MasterNotRunningException if the master is not running
164    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
165    */
166   public HBaseAdmin(HConnection connection)
167       throws MasterNotRunningException, ZooKeeperConnectionException {
168     this.conf = connection.getConfiguration();
169     this.connection = connection;
170     this.cleanupConnectionOnClose = false;
171 
172     this.pause = this.conf.getLong("hbase.client.pause", 1000);
173     this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
174     this.retryLongerMultiplier = this.conf.getInt(
175         "hbase.client.retries.longer.multiplier", 10);
176 
177     this.connection.getMaster();
178   }
179 
180   /**
181    * @return A new CatalogTracker instance; call {@link #cleanupCatalogTracker(CatalogTracker)}
182    * to cleanup the returned catalog tracker.
183    * @throws ZooKeeperConnectionException
184    * @throws IOException
185    * @see #cleanupCatalogTracker(CatalogTracker)
186    */
187   private synchronized CatalogTracker getCatalogTracker()
188   throws ZooKeeperConnectionException, IOException {
189     CatalogTracker ct = null;
190     try {
191       ct = new CatalogTracker(this.conf);
192       ct.start();
193     } catch (InterruptedException e) {
194       // Let it out as an IOE for now until we redo all so tolerate IEs
195       Thread.currentThread().interrupt();
196       throw new IOException("Interrupted", e);
197     }
198     return ct;
199   }
200 
201   private void cleanupCatalogTracker(final CatalogTracker ct) {
202     ct.stop();
203   }
204 
205   @Override
206   public void abort(String why, Throwable e) {
207     // Currently does nothing but throw the passed message and exception
208     this.aborted = true;
209     throw new RuntimeException(why, e);
210   }
211 
212   @Override
213   public boolean isAborted(){
214     return this.aborted;
215   }
216 
217   /** @return HConnection used by this object. */
218   public HConnection getConnection() {
219     return connection;
220   }
221 
222   /**
223    * Get a connection to the currently set master.
224    * @return proxy connection to master server for this instance
225    * @throws MasterNotRunningException if the master is not running
226    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
227    * @deprecated  Master is an implementation detail for HBaseAdmin.
228    * Deprecated in HBase 0.94
229    */
230   @Deprecated
231   public HMasterInterface getMaster()
232   throws MasterNotRunningException, ZooKeeperConnectionException {
233     return this.connection.getMaster();
234   }
235 
236   /** @return - true if the master server is running
237    * @throws ZooKeeperConnectionException
238    * @throws MasterNotRunningException */
239   public boolean isMasterRunning()
240   throws MasterNotRunningException, ZooKeeperConnectionException {
241     return this.connection.isMasterRunning();
242   }
243 
244   /**
245    * @param tableName Table to check.
246    * @return True if table exists already.
247    * @throws IOException
248    */
249   public boolean tableExists(final String tableName)
250   throws IOException {
251     boolean b = false;
252     CatalogTracker ct = getCatalogTracker();
253     try {
254       b = MetaReader.tableExists(ct, tableName);
255     } finally {
256       cleanupCatalogTracker(ct);
257     }
258     return b;
259   }
260 
261   /**
262    * @param tableName Table to check.
263    * @return True if table exists already.
264    * @throws IOException
265    */
266   public boolean tableExists(final byte [] tableName)
267   throws IOException {
268     return tableExists(Bytes.toString(tableName));
269   }
270 
271   /**
272    * List all the userspace tables.  In other words, scan the META table.
273    *
274    * If we wanted this to be really fast, we could implement a special
275    * catalog table that just contains table names and their descriptors.
276    * Right now, it only exists as part of the META table's region info.
277    *
278    * @return - returns an array of HTableDescriptors
279    * @throws IOException if a remote or network exception occurs
280    */
281   public HTableDescriptor[] listTables() throws IOException {
282     return this.connection.listTables();
283   }
284 
285   /**
286    * List all the userspace tables matching the given pattern.
287    *
288    * @param pattern The compiled regular expression to match against
289    * @return - returns an array of HTableDescriptors
290    * @throws IOException if a remote or network exception occurs
291    * @see #listTables()
292    */
293   public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
294     List<HTableDescriptor> matched = new LinkedList<HTableDescriptor>();
295     HTableDescriptor[] tables = listTables();
296     for (HTableDescriptor table : tables) {
297       if (pattern.matcher(table.getNameAsString()).matches()) {
298         matched.add(table);
299       }
300     }
301     return matched.toArray(new HTableDescriptor[matched.size()]);
302   }
303 
304   /**
305    * List all the userspace tables matching the given regular expression.
306    *
307    * @param regex The regular expression to match against
308    * @return - returns an array of HTableDescriptors
309    * @throws IOException if a remote or network exception occurs
310    * @see #listTables(java.util.regex.Pattern)
311    */
312   public HTableDescriptor[] listTables(String regex) throws IOException {
313     return listTables(Pattern.compile(regex));
314   }
315 
316 
317   /**
318    * Method for getting the tableDescriptor
319    * @param tableName as a byte []
320    * @return the tableDescriptor
321    * @throws TableNotFoundException
322    * @throws IOException if a remote or network exception occurs
323    */
324   public HTableDescriptor getTableDescriptor(final byte [] tableName)
325   throws TableNotFoundException, IOException {
326     return this.connection.getHTableDescriptor(tableName);
327   }
328 
329   private long getPauseTime(int tries) {
330     int triesCount = tries;
331     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
332       triesCount = HConstants.RETRY_BACKOFF.length - 1;
333     }
334     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
335   }
336 
337   /**
338    * Creates a new table.
339    * Synchronous operation.
340    *
341    * @param desc table descriptor for table
342    *
343    * @throws IllegalArgumentException if the table name is reserved
344    * @throws MasterNotRunningException if master is not running
345    * @throws TableExistsException if table already exists (If concurrent
346    * threads, the table may have been created between test-for-existence
347    * and attempt-at-creation).
348    * @throws IOException if a remote or network exception occurs
349    */
350   public void createTable(HTableDescriptor desc)
351   throws IOException {
352     createTable(desc, null);
353   }
354 
355   /**
356    * Creates a new table with the specified number of regions.  The start key
357    * specified will become the end key of the first region of the table, and
358    * the end key specified will become the start key of the last region of the
359    * table (the first region has a null start key and the last region has a
360    * null end key).
361    *
362    * BigInteger math will be used to divide the key range specified into
363    * enough segments to make the required number of total regions.
364    *
365    * Synchronous operation.
366    *
367    * @param desc table descriptor for table
368    * @param startKey beginning of key range
369    * @param endKey end of key range
370    * @param numRegions the total number of regions to create
371    *
372    * @throws IllegalArgumentException if the table name is reserved
373    * @throws MasterNotRunningException if master is not running
374    * @throws TableExistsException if table already exists (If concurrent
375    * threads, the table may have been created between test-for-existence
376    * and attempt-at-creation).
377    * @throws IOException
378    */
379   public void createTable(HTableDescriptor desc, byte [] startKey,
380       byte [] endKey, int numRegions)
381   throws IOException {
382     HTableDescriptor.isLegalTableName(desc.getName());
383     if(numRegions < 3) {
384       throw new IllegalArgumentException("Must create at least three regions");
385     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
386       throw new IllegalArgumentException("Start key must be smaller than end key");
387     }
388     if (numRegions == 3) {
389       createTable(desc, new byte[][] { startKey, endKey });
390       return;
391     }
392     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
393     if(splitKeys == null || splitKeys.length != numRegions - 1) {
394       throw new IllegalArgumentException("Unable to split key range into enough regions");
395     }
396     createTable(desc, splitKeys);
397   }
398 
399   /**
400    * Creates a new table with an initial set of empty regions defined by the
401    * specified split keys.  The total number of regions created will be the
402    * number of split keys plus one. Synchronous operation.
403    * Note : Avoid passing empty split key.
404    *
405    * @param desc table descriptor for table
406    * @param splitKeys array of split keys for the initial regions of the table
407    *
408    * @throws IllegalArgumentException if the table name is reserved, if the split keys
409    * are repeated and if the split key has empty byte array.
410    * @throws MasterNotRunningException if master is not running
411    * @throws TableExistsException if table already exists (If concurrent
412    * threads, the table may have been created between test-for-existence
413    * and attempt-at-creation).
414    * @throws IOException
415    */
416   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
417   throws IOException {
418     HTableDescriptor.isLegalTableName(desc.getName());
419     try {
420       createTableAsync(desc, splitKeys);
421     } catch (SocketTimeoutException ste) {
422       LOG.warn("Creating " + desc.getNameAsString() + " took too long", ste);
423     }
424     int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
425     int prevRegCount = 0;
426     boolean doneWithMetaScan = false;
427     for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
428       ++tries) {
429       if (!doneWithMetaScan) {
430         // Wait for new table to come on-line
431         final AtomicInteger actualRegCount = new AtomicInteger(0);
432         MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
433           @Override
434           public boolean processRow(Result rowResult) throws IOException {
435             if (rowResult == null || rowResult.size() <= 0) {
436               return true;
437             }
438             HRegionInfo info = MetaReader.parseHRegionInfoFromCatalogResult(
439               rowResult, HConstants.REGIONINFO_QUALIFIER);
440             if (info == null) {
441               LOG.warn("No serialized HRegionInfo in " + rowResult);
442               return true;
443             }
444             if (!(Bytes.equals(info.getTableName(), desc.getName()))) {
445               return false;
446             }
447             String hostAndPort = null;
448             byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
449               HConstants.SERVER_QUALIFIER);
450             // Make sure that regions are assigned to server
451             if (value != null && value.length > 0) {
452               hostAndPort = Bytes.toString(value);
453             }
454             if (!(info.isOffline() || info.isSplit()) && hostAndPort != null) {
455               actualRegCount.incrementAndGet();
456             }
457             return true;
458           }
459         };
460         MetaScanner.metaScan(conf, connection, visitor, desc.getName());
461         if (actualRegCount.get() != numRegs) {
462           if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
463             throw new RegionOfflineException("Only " + actualRegCount.get() +
464               " of " + numRegs + " regions are online; retries exhausted.");
465           }
466           try { // Sleep
467             Thread.sleep(getPauseTime(tries));
468           } catch (InterruptedException e) {
469             throw new InterruptedIOException("Interrupted when opening" +
470               " regions; " + actualRegCount.get() + " of " + numRegs +
471               " regions processed so far");
472           }
473           if (actualRegCount.get() > prevRegCount) { // Making progress
474             prevRegCount = actualRegCount.get();
475             tries = -1;
476           }
477         } else {
478           doneWithMetaScan = true;
479           tries = -1;
480         }
481       } else if (isTableEnabled(desc.getName())) {
482         return;
483       } else {
484         try { // Sleep
485           Thread.sleep(getPauseTime(tries));
486         } catch (InterruptedException e) {
487           throw new InterruptedIOException("Interrupted when waiting" +
488             " for table to be enabled; meta scan was done");
489         }
490       }
491     }
492     throw new TableNotEnabledException(
493       "Retries exhausted while still waiting for table: "
494       + desc.getNameAsString() + " to be enabled");
495   }
496 
497   /**
498    * Creates a new table but does not block and wait for it to come online.
499    * Asynchronous operation.  To check if the table exists, use
500    * {@link: #isTableAvailable} -- it is not safe to create an HTable
501    * instance to this table before it is available.
502    * Note : Avoid passing empty split key.
503    * @param desc table descriptor for table
504    *
505    * @throws IllegalArgumentException Bad table name, if the split keys
506    * are repeated and if the split key has empty byte array.
507    * @throws MasterNotRunningException if master is not running
508    * @throws TableExistsException if table already exists (If concurrent
509    * threads, the table may have been created between test-for-existence
510    * and attempt-at-creation).
511    * @throws IOException
512    */
513   public void createTableAsync(HTableDescriptor desc, byte [][] splitKeys)
514   throws IOException {
515     HTableDescriptor.isLegalTableName(desc.getName());
516     if(splitKeys != null && splitKeys.length > 0) {
517       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
518       // Verify there are no duplicate split keys
519       byte [] lastKey = null;
520       for(byte [] splitKey : splitKeys) {
521         if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
522           throw new IllegalArgumentException(
523               "Empty split key must not be passed in the split keys.");
524         }
525         if(lastKey != null && Bytes.equals(splitKey, lastKey)) {
526           throw new IllegalArgumentException("All split keys must be unique, " +
527             "found duplicate: " + Bytes.toStringBinary(splitKey) +
528             ", " + Bytes.toStringBinary(lastKey));
529         }
530         lastKey = splitKey;
531       }
532     }
533     try {
534       getMaster().createTable(desc, splitKeys);
535     } catch (RemoteException e) {
536       throw e.unwrapRemoteException();
537     }
538   }
539 
540   /**
541    * Deletes a table.
542    * Synchronous operation.
543    *
544    * @param tableName name of table to delete
545    * @throws IOException if a remote or network exception occurs
546    */
547   public void deleteTable(final String tableName) throws IOException {
548     deleteTable(Bytes.toBytes(tableName));
549   }
550 
551   /**
552    * Deletes a table.
553    * Synchronous operation.
554    *
555    * @param tableName name of table to delete
556    * @throws IOException if a remote or network exception occurs
557    */
558   public void deleteTable(final byte [] tableName) throws IOException {
559     isMasterRunning();
560     HTableDescriptor.isLegalTableName(tableName);
561     HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
562     boolean tableExists = true;
563     try {
564       getMaster().deleteTable(tableName);
565     } catch (RemoteException e) {
566       throw RemoteExceptionHandler.decodeRemoteException(e);
567     }
568     // Wait until all regions deleted
569     HRegionInterface server =
570       connection.getHRegionConnection(firstMetaServer.getHostname(), firstMetaServer.getPort());
571     List<String> tableNameAsList = new ArrayList<String>(1);
572     tableNameAsList.add(Bytes.toString(tableName));
573     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
574       long scannerId = -1L;
575       try {
576 
577         Scan scan = MetaReader.getScanForTableName(tableName);
578         scan.addColumn(HConstants.CATALOG_FAMILY,
579             HConstants.REGIONINFO_QUALIFIER);
580         scannerId = server.openScanner(
581           firstMetaServer.getRegionInfo().getRegionName(), scan);
582         // Get a batch at a time.
583         Result values = server.next(scannerId);
584 
585         // let us wait until .META. table is updated and
586         // HMaster removes the table from its HTableDescriptors
587         if (values == null) {
588           tableExists = false;
589           HTableDescriptor[] htds = getMaster().getHTableDescriptors(tableNameAsList);
590           tableExists = (htds != null && htds.length > 0);
591           if (!tableExists) {
592             break;
593           }
594         }
595       } catch (IOException ex) {
596         if(tries == numRetries - 1) {           // no more tries left
597           if (ex instanceof RemoteException) {
598             ex = RemoteExceptionHandler.decodeRemoteException((RemoteException) ex);
599           }
600           throw ex;
601         }
602       } finally {
603         if (scannerId != -1L) {
604           try {
605             server.close(scannerId);
606           } catch (Exception ex) {
607             LOG.warn(ex);
608           }
609         }
610       }
611       try {
612         Thread.sleep(getPauseTime(tries));
613       } catch (InterruptedException e) {
614         // continue
615       }
616     }
617 
618     if (tableExists) {
619       throw new IOException("Retries exhausted, it took too long to wait"+
620         " for the table " + Bytes.toString(tableName) + " to be deleted.");
621     }
622     // Delete cached information to prevent clients from using old locations
623     this.connection.clearRegionCache(tableName);
624     LOG.info("Deleted " + Bytes.toString(tableName));
625   }
626 
627   /**
628    * Deletes tables matching the passed in pattern and wait on completion.
629    *
630    * Warning: Use this method carefully, there is no prompting and the effect is
631    * immediate. Consider using {@link #listTables(java.lang.String)} and
632    * {@link #deleteTable(byte[])}
633    *
634    * @param regex The regular expression to match table names against
635    * @return Table descriptors for tables that couldn't be deleted
636    * @throws IOException
637    * @see #deleteTables(java.util.regex.Pattern)
638    * @see #deleteTable(java.lang.String)
639    */
640   public HTableDescriptor[] deleteTables(String regex) throws IOException {
641     return deleteTables(Pattern.compile(regex));
642   }
643 
644   /**
645    * Delete tables matching the passed in pattern and wait on completion.
646    *
647    * Warning: Use this method carefully, there is no prompting and the effect is
648    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
649    * {@link #deleteTable(byte[])}
650    *
651    * @param pattern The pattern to match table names against
652    * @return Table descriptors for tables that couldn't be deleted
653    * @throws IOException
654    */
655   public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
656     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
657     for (HTableDescriptor table : listTables(pattern)) {
658       try {
659         deleteTable(table.getName());
660       } catch (IOException ex) {
661         LOG.info("Failed to delete table " + table.getNameAsString(), ex);
662         failed.add(table);
663       }
664     }
665     return failed.toArray(new HTableDescriptor[failed.size()]);
666   }
667 
668 
669   public void enableTable(final String tableName)
670   throws IOException {
671     enableTable(Bytes.toBytes(tableName));
672   }
673 
674   /**
675    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
676    * and {@link #isTableEnabled(byte[])} instead.
677    * The table has to be in disabled state for it to be enabled.
678    * @param tableName name of the table
679    * @throws IOException if a remote or network exception occurs
680    * There could be couple types of IOException
681    * TableNotFoundException means the table doesn't exist.
682    * TableNotDisabledException means the table isn't in disabled state.
683    * @see #isTableEnabled(byte[])
684    * @see #disableTable(byte[])
685    * @see #enableTableAsync(byte[])
686    */
687   public void enableTable(final byte [] tableName)
688   throws IOException {
689     enableTableAsync(tableName);
690 
691     // Wait until all regions are enabled
692     waitUntilTableIsEnabled(tableName);
693 
694     LOG.info("Enabled table " + Bytes.toString(tableName));
695   }
696 
697   /**
698    * Wait for the table to be enabled and available
699    * If enabling the table exceeds the retry period, an exception is thrown.
700    * @param tableName name of the table
701    * @throws IOException if a remote or network exception occurs or
702    *    table is not enabled after the retries period.
703    */
704   private void waitUntilTableIsEnabled(final byte[] tableName) throws IOException {
705     boolean enabled = false;
706     long start = EnvironmentEdgeManager.currentTimeMillis();
707     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
708       enabled = isTableEnabled(tableName) && isTableAvailable(tableName);
709       if (enabled) {
710         break;
711       }
712       long sleep = getPauseTime(tries);
713       if (LOG.isDebugEnabled()) {
714         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
715           "enabled in " + Bytes.toString(tableName));
716       }
717       try {
718         Thread.sleep(sleep);
719       } catch (InterruptedException e) {
720         Thread.currentThread().interrupt();
721         // Do this conversion rather than let it out because do not want to
722         // change the method signature.
723         throw new IOException("Interrupted", e);
724       }
725     }
726     if (!enabled) {
727       long msec = EnvironmentEdgeManager.currentTimeMillis() - start;
728       throw new IOException("Table '" + Bytes.toString(tableName) +
729         "' not yet enabled, after " + msec + "ms.");
730     }
731   }
732 
733   public void enableTableAsync(final String tableName)
734   throws IOException {
735     enableTableAsync(Bytes.toBytes(tableName));
736   }
737 
738   /**
739    * Brings a table on-line (enables it).  Method returns immediately though
740    * enable of table may take some time to complete, especially if the table
741    * is large (All regions are opened as part of enabling process).  Check
742    * {@link #isTableEnabled(byte[])} to learn when table is fully online.  If
743    * table is taking too long to online, check server logs.
744    * @param tableName
745    * @throws IOException
746    * @since 0.90.0
747    */
748   public void enableTableAsync(final byte [] tableName)
749   throws IOException {
750     HTableDescriptor.isLegalTableName(tableName);
751     isMasterRunning();
752     try {
753       getMaster().enableTable(tableName);
754     } catch (RemoteException e) {
755       throw e.unwrapRemoteException();
756     }
757     LOG.info("Started enable of " + Bytes.toString(tableName));
758   }
759 
760   /**
761    * Enable tables matching the passed in pattern and wait on completion.
762    *
763    * Warning: Use this method carefully, there is no prompting and the effect is
764    * immediate. Consider using {@link #listTables(java.lang.String)} and
765    * {@link #enableTable(byte[])}
766    *
767    * @param regex The regular expression to match table names against
768    * @throws IOException
769    * @see #enableTables(java.util.regex.Pattern)
770    * @see #enableTable(java.lang.String)
771    */
772   public HTableDescriptor[] enableTables(String regex) throws IOException {
773     return enableTables(Pattern.compile(regex));
774   }
775 
776   /**
777    * Enable tables matching the passed in pattern and wait on completion.
778    *
779    * Warning: Use this method carefully, there is no prompting and the effect is
780    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
781    * {@link #enableTable(byte[])}
782    *
783    * @param pattern The pattern to match table names against
784    * @throws IOException
785    */
786   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
787     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
788     for (HTableDescriptor table : listTables(pattern)) {
789       if (isTableDisabled(table.getName())) {
790         try {
791           enableTable(table.getName());
792         } catch (IOException ex) {
793           LOG.info("Failed to enable table " + table.getNameAsString(), ex);
794           failed.add(table);
795         }
796       }
797     }
798     return failed.toArray(new HTableDescriptor[failed.size()]);
799   }
800 
801   public void disableTableAsync(final String tableName) throws IOException {
802     disableTableAsync(Bytes.toBytes(tableName));
803   }
804 
805   /**
806    * Starts the disable of a table.  If it is being served, the master
807    * will tell the servers to stop serving it.  This method returns immediately.
808    * The disable of a table can take some time if the table is large (all
809    * regions are closed as part of table disable operation).
810    * Call {@link #isTableDisabled(byte[])} to check for when disable completes.
811    * If table is taking too long to online, check server logs.
812    * @param tableName name of table
813    * @throws IOException if a remote or network exception occurs
814    * @see #isTableDisabled(byte[])
815    * @see #isTableEnabled(byte[])
816    * @since 0.90.0
817    */
818   public void disableTableAsync(final byte [] tableName) throws IOException {
819     HTableDescriptor.isLegalTableName(tableName);
820     isMasterRunning();
821     try {
822       getMaster().disableTable(tableName);
823     } catch (RemoteException e) {
824       throw e.unwrapRemoteException();
825     }
826     LOG.info("Started disable of " + Bytes.toString(tableName));
827   }
828 
829   public void disableTable(final String tableName)
830   throws IOException {
831     disableTable(Bytes.toBytes(tableName));
832   }
833 
834   /**
835    * Disable table and wait on completion.  May timeout eventually.  Use
836    * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
837    * instead.
838    * The table has to be in enabled state for it to be disabled.
839    * @param tableName
840    * @throws IOException
841    * There could be couple types of IOException
842    * TableNotFoundException means the table doesn't exist.
843    * TableNotEnabledException means the table isn't in enabled state.
844    */
845   public void disableTable(final byte [] tableName)
846   throws IOException {
847     disableTableAsync(tableName);
848     // Wait until table is disabled
849     boolean disabled = false;
850     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
851       disabled = isTableDisabled(tableName);
852       if (disabled) {
853         break;
854       }
855       long sleep = getPauseTime(tries);
856       if (LOG.isDebugEnabled()) {
857         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
858           "disabled in " + Bytes.toString(tableName));
859       }
860       try {
861         Thread.sleep(sleep);
862       } catch (InterruptedException e) {
863         // Do this conversion rather than let it out because do not want to
864         // change the method signature.
865         Thread.currentThread().interrupt();
866         throw new IOException("Interrupted", e);
867       }
868     }
869     if (!disabled) {
870       throw new RegionException("Retries exhausted, it took too long to wait"+
871         " for the table " + Bytes.toString(tableName) + " to be disabled.");
872     }
873     LOG.info("Disabled " + Bytes.toString(tableName));
874   }
875 
876   /**
877    * Disable tables matching the passed in pattern and wait on completion.
878    *
879    * Warning: Use this method carefully, there is no prompting and the effect is
880    * immediate. Consider using {@link #listTables(java.lang.String)} and
881    * {@link #disableTable(byte[])}
882    *
883    * @param regex The regular expression to match table names against
884    * @return Table descriptors for tables that couldn't be disabled
885    * @throws IOException
886    * @see #disableTables(java.util.regex.Pattern)
887    * @see #disableTable(java.lang.String)
888    */
889   public HTableDescriptor[] disableTables(String regex) throws IOException {
890     return disableTables(Pattern.compile(regex));
891   }
892 
893   /**
894    * Disable tables matching the passed in pattern and wait on completion.
895    *
896    * Warning: Use this method carefully, there is no prompting and the effect is
897    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
898    * {@link #disableTable(byte[])}
899    *
900    * @param pattern The pattern to match table names against
901    * @return Table descriptors for tables that couldn't be disabled
902    * @throws IOException
903    */
904   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
905     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
906     for (HTableDescriptor table : listTables(pattern)) {
907       if (isTableEnabled(table.getName())) {
908         try {
909           disableTable(table.getName());
910         } catch (IOException ex) {
911           LOG.info("Failed to disable table " + table.getNameAsString(), ex);
912           failed.add(table);
913         }
914       }
915     }
916     return failed.toArray(new HTableDescriptor[failed.size()]);
917   }
918 
919   /**
920    * @param tableName name of table to check
921    * @return true if table is on-line
922    * @throws IOException if a remote or network exception occurs
923    */
924   public boolean isTableEnabled(String tableName) throws IOException {
925     return isTableEnabled(Bytes.toBytes(tableName));
926   }
927   /**
928    * @param tableName name of table to check
929    * @return true if table is on-line
930    * @throws IOException if a remote or network exception occurs
931    */
932   public boolean isTableEnabled(byte[] tableName) throws IOException {
933     if (!HTableDescriptor.isMetaTable(tableName)) {
934       HTableDescriptor.isLegalTableName(tableName);
935     }
936     if(!tableExists(tableName)){
937       throw new TableNotFoundException(Bytes.toString(tableName));
938     }
939     return connection.isTableEnabled(tableName);
940   }
941 
942   /**
943    * @param tableName name of table to check
944    * @return true if table is off-line
945    * @throws IOException if a remote or network exception occurs
946    */
947   public boolean isTableDisabled(final String tableName) throws IOException {
948     return isTableDisabled(Bytes.toBytes(tableName));
949   }
950 
951   /**
952    * @param tableName name of table to check
953    * @return true if table is off-line
954    * @throws IOException if a remote or network exception occurs
955    */
956   public boolean isTableDisabled(byte[] tableName) throws IOException {
957     if (!HTableDescriptor.isMetaTable(tableName)) {
958       HTableDescriptor.isLegalTableName(tableName);
959     }
960     return connection.isTableDisabled(tableName);
961   }
962 
963   /**
964    * @param tableName name of table to check
965    * @return true if all regions of the table are available
966    * @throws IOException if a remote or network exception occurs
967    */
968   public boolean isTableAvailable(byte[] tableName) throws IOException {
969     return connection.isTableAvailable(tableName);
970   }
971 
972   /**
973    * @param tableName name of table to check
974    * @return true if all regions of the table are available
975    * @throws IOException if a remote or network exception occurs
976    */
977   public boolean isTableAvailable(String tableName) throws IOException {
978     return connection.isTableAvailable(Bytes.toBytes(tableName));
979   }
980 
981   /**
982    * Get the status of alter command - indicates how many regions have received
983    * the updated schema Asynchronous operation.
984    *
985    * @param tableName
986    *          name of the table to get the status of
987    * @return Pair indicating the number of regions updated Pair.getFirst() is the
988    *         regions that are yet to be updated Pair.getSecond() is the total number
989    *         of regions of the table
990    * @throws IOException
991    *           if a remote or network exception occurs
992    */
993   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
994   throws IOException {
995     HTableDescriptor.isLegalTableName(tableName);
996     try {
997       return getMaster().getAlterStatus(tableName);
998     } catch (RemoteException e) {
999       throw RemoteExceptionHandler.decodeRemoteException(e);
1000     }
1001   }
1002 
1003   /**
1004    * Add a column to an existing table.
1005    * Asynchronous operation.
1006    *
1007    * @param tableName name of the table to add column to
1008    * @param column column descriptor of column to be added
1009    * @throws IOException if a remote or network exception occurs
1010    */
1011   public void addColumn(final String tableName, HColumnDescriptor column)
1012   throws IOException {
1013     addColumn(Bytes.toBytes(tableName), column);
1014   }
1015 
1016   /**
1017    * Add a column to an existing table.
1018    * Asynchronous operation.
1019    *
1020    * @param tableName name of the table to add column to
1021    * @param column column descriptor of column to be added
1022    * @throws IOException if a remote or network exception occurs
1023    */
1024   public void addColumn(final byte [] tableName, HColumnDescriptor column)
1025   throws IOException {
1026     HTableDescriptor.isLegalTableName(tableName);
1027     try {
1028       getMaster().addColumn(tableName, column);
1029     } catch (RemoteException e) {
1030       throw RemoteExceptionHandler.decodeRemoteException(e);
1031     }
1032   }
1033 
1034   /**
1035    * Delete a column from a table.
1036    * Asynchronous operation.
1037    *
1038    * @param tableName name of table
1039    * @param columnName name of column to be deleted
1040    * @throws IOException if a remote or network exception occurs
1041    */
1042   public void deleteColumn(final String tableName, final String columnName)
1043   throws IOException {
1044     deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName));
1045   }
1046 
1047   /**
1048    * Delete a column from a table.
1049    * Asynchronous operation.
1050    *
1051    * @param tableName name of table
1052    * @param columnName name of column to be deleted
1053    * @throws IOException if a remote or network exception occurs
1054    */
1055   public void deleteColumn(final byte [] tableName, final byte [] columnName)
1056   throws IOException {
1057     try {
1058       getMaster().deleteColumn(tableName, columnName);
1059     } catch (RemoteException e) {
1060       throw RemoteExceptionHandler.decodeRemoteException(e);
1061     }
1062   }
1063 
1064   /**
1065    * Modify an existing column family on a table.
1066    * Asynchronous operation.
1067    *
1068    * @param tableName name of table
1069    * @param descriptor new column descriptor to use
1070    * @throws IOException if a remote or network exception occurs
1071    */
1072   public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
1073   throws IOException {
1074     modifyColumn(Bytes.toBytes(tableName), descriptor);
1075   }
1076 
1077   /**
1078    * Modify an existing column family on a table.
1079    * Asynchronous operation.
1080    *
1081    * @param tableName name of table
1082    * @param descriptor new column descriptor to use
1083    * @throws IOException if a remote or network exception occurs
1084    */
1085   public void modifyColumn(final byte [] tableName, HColumnDescriptor descriptor)
1086   throws IOException {
1087     try {
1088       getMaster().modifyColumn(tableName, descriptor);
1089     } catch (RemoteException re) {
1090       // Convert RE exceptions in here; client shouldn't have to deal with them,
1091       // at least w/ the type of exceptions that come out of this method:
1092       // TableNotFoundException, etc.
1093       throw RemoteExceptionHandler.decodeRemoteException(re);
1094     }
1095   }
1096 
1097   /**
1098    * Close a region. For expert-admins.  Runs close on the regionserver.  The
1099    * master will not be informed of the close.
1100    * @param regionname region name to close
1101    * @param serverName If supplied, we'll use this location rather than
1102    * the one currently in <code>.META.</code>
1103    * @throws IOException if a remote or network exception occurs
1104    */
1105   public void closeRegion(final String regionname, final String serverName)
1106   throws IOException {
1107     closeRegion(Bytes.toBytes(regionname), serverName);
1108   }
1109 
1110   /**
1111    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1112    * master will not be informed of the close.
1113    * @param regionname region name to close
1114    * @param serverName The servername of the regionserver.  If passed null we
1115    * will use servername found in the .META. table. A server name
1116    * is made of host, port and startcode.  Here is an example:
1117    * <code> host187.example.com,60020,1289493121758</code>
1118    * @throws IOException if a remote or network exception occurs
1119    */
1120   public void closeRegion(final byte [] regionname, final String serverName)
1121   throws IOException {
1122     CatalogTracker ct = getCatalogTracker();
1123     try {
1124       if (serverName != null) {
1125         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1126         if (pair == null || pair.getFirst() == null) {
1127           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1128         } else {
1129           closeRegion(new ServerName(serverName), pair.getFirst());
1130         }
1131       } else {
1132         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1133         if (pair == null) {
1134           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1135         } else if (pair.getSecond() == null) {
1136           throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1137         } else {
1138           closeRegion(pair.getSecond(), pair.getFirst());
1139         }
1140       }
1141     } finally {
1142       cleanupCatalogTracker(ct);
1143     }
1144   }
1145 
1146   /**
1147    * For expert-admins. Runs close on the regionserver. Closes a region based on
1148    * the encoded region name. The region server name is mandatory. If the
1149    * servername is provided then based on the online regions in the specified
1150    * regionserver the specified region will be closed. The master will not be
1151    * informed of the close. Note that the regionname is the encoded regionname.
1152    *
1153    * @param encodedRegionName
1154    *          The encoded region name; i.e. the hash that makes up the region
1155    *          name suffix: e.g. if regionname is
1156    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
1157    *          , then the encoded region name is:
1158    *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
1159    * @param serverName
1160    *          The servername of the regionserver. A server name is made of host,
1161    *          port and startcode. This is mandatory. Here is an example:
1162    *          <code> host187.example.com,60020,1289493121758</code>
1163    * @return true if the region was closed, false if not.
1164    * @throws IOException
1165    *           if a remote or network exception occurs
1166    */
1167   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1168       final String serverName) throws IOException {
1169     byte[] encodedRegionNameInBytes = Bytes.toBytes(encodedRegionName);
1170     if (null == serverName || ("").equals(serverName.trim())) {
1171       throw new IllegalArgumentException(
1172           "The servername cannot be null or empty.");
1173     }
1174     ServerName sn = new ServerName(serverName);
1175     HRegionInterface rs = this.connection.getHRegionConnection(
1176         sn.getHostname(), sn.getPort());
1177     // Close the region without updating zk state.
1178     boolean isRegionClosed = rs.closeRegion(encodedRegionNameInBytes, false);
1179     if (false == isRegionClosed) {
1180       LOG.error("Not able to close the region " + encodedRegionName + ".");
1181     }
1182     return isRegionClosed;
1183   }
1184 
1185   /**
1186    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1187    * master will not be informed of the close.
1188    * @param sn
1189    * @param hri
1190    * @throws IOException
1191    */
1192   public void closeRegion(final ServerName sn, final HRegionInfo hri)
1193   throws IOException {
1194     HRegionInterface rs =
1195       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1196     // Close the region without updating zk state.
1197     rs.closeRegion(hri, false);
1198   }
1199 
1200   /**
1201    * Flush a table or an individual region.
1202    * Asynchronous operation.
1203    *
1204    * @param tableNameOrRegionName table or region to flush
1205    * @throws IOException if a remote or network exception occurs
1206    * @throws InterruptedException
1207    */
1208   public void flush(final String tableNameOrRegionName)
1209   throws IOException, InterruptedException {
1210     flush(Bytes.toBytes(tableNameOrRegionName));
1211   }
1212 
1213   /**
1214    * Flush a table or an individual region.
1215    * Asynchronous operation.
1216    *
1217    * @param tableNameOrRegionName table or region to flush
1218    * @throws IOException if a remote or network exception occurs
1219    * @throws InterruptedException
1220    */
1221   public void flush(final byte [] tableNameOrRegionName)
1222   throws IOException, InterruptedException {
1223     CatalogTracker ct = getCatalogTracker();
1224     try {
1225       Pair<HRegionInfo, ServerName> regionServerPair
1226         = getRegion(tableNameOrRegionName, ct);
1227       if (regionServerPair != null) {
1228         if (regionServerPair.getSecond() == null) {
1229           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1230         } else {
1231           flush(regionServerPair.getSecond(), regionServerPair.getFirst());
1232         }
1233       } else {
1234         final String tableName = tableNameString(tableNameOrRegionName, ct);
1235         List<Pair<HRegionInfo, ServerName>> pairs =
1236           MetaReader.getTableRegionsAndLocations(ct,
1237               tableName);
1238         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1239           if (pair.getFirst().isOffline()) continue;
1240           if (pair.getSecond() == null) continue;
1241           try {
1242             flush(pair.getSecond(), pair.getFirst());
1243           } catch (NotServingRegionException e) {
1244             if (LOG.isDebugEnabled()) {
1245               LOG.debug("Trying to flush " + pair.getFirst() + ": " +
1246                 StringUtils.stringifyException(e));
1247             }
1248           }
1249         }
1250       }
1251     } finally {
1252       cleanupCatalogTracker(ct);
1253     }
1254   }
1255 
1256   private void flush(final ServerName sn, final HRegionInfo hri)
1257   throws IOException {
1258     HRegionInterface rs =
1259       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1260     rs.flushRegion(hri);
1261   }
1262 
1263   /**
1264    * Compact a table or an individual region.
1265    * Asynchronous operation.
1266    *
1267    * @param tableNameOrRegionName table or region to compact
1268    * @throws IOException if a remote or network exception occurs
1269    * @throws InterruptedException
1270    */
1271   public void compact(final String tableNameOrRegionName)
1272   throws IOException, InterruptedException {
1273     compact(Bytes.toBytes(tableNameOrRegionName));
1274   }
1275 
1276   /**
1277    * Compact a table or an individual region.
1278    * Asynchronous operation.
1279    *
1280    * @param tableNameOrRegionName table or region to compact
1281    * @throws IOException if a remote or network exception occurs
1282    * @throws InterruptedException
1283    */
1284   public void compact(final byte [] tableNameOrRegionName)
1285   throws IOException, InterruptedException {
1286     compact(tableNameOrRegionName, null, false);
1287   }
1288 
1289   /**
1290    * Compact a column family within a table or region.
1291    * Asynchronous operation.
1292    *
1293    * @param tableOrRegionName table or region to compact
1294    * @param columnFamily column family within a table or region
1295    * @throws IOException if a remote or network exception occurs
1296    * @throws InterruptedException
1297    */
1298   public void compact(String tableOrRegionName, String columnFamily)
1299     throws IOException,  InterruptedException {
1300     compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
1301   }
1302 
1303   /**
1304    * Compact a column family within a table or region.
1305    * Asynchronous operation.
1306    *
1307    * @param tableNameOrRegionName table or region to compact
1308    * @param columnFamily column family within a table or region
1309    * @throws IOException if a remote or network exception occurs
1310    * @throws InterruptedException
1311    */
1312   public void compact(final byte [] tableNameOrRegionName, final byte[] columnFamily)
1313   throws IOException, InterruptedException {
1314     compact(tableNameOrRegionName, columnFamily, false);
1315   }
1316 
1317   /**
1318    * Major compact a table or an individual region.
1319    * Asynchronous operation.
1320    *
1321    * @param tableNameOrRegionName table or region to major compact
1322    * @throws IOException if a remote or network exception occurs
1323    * @throws InterruptedException
1324    */
1325   public void majorCompact(final String tableNameOrRegionName)
1326   throws IOException, InterruptedException {
1327     majorCompact(Bytes.toBytes(tableNameOrRegionName));
1328   }
1329 
1330   /**
1331    * Major compact a table or an individual region.
1332    * Asynchronous operation.
1333    *
1334    * @param tableNameOrRegionName table or region to major compact
1335    * @throws IOException if a remote or network exception occurs
1336    * @throws InterruptedException
1337    */
1338   public void majorCompact(final byte [] tableNameOrRegionName)
1339   throws IOException, InterruptedException {
1340     compact(tableNameOrRegionName, null, true);
1341   }
1342 
1343   /**
1344    * Major compact a column family within a table or region.
1345    * Asynchronous operation.
1346    *
1347    * @param tableNameOrRegionName table or region to major compact
1348    * @param columnFamily column family within a table or region
1349    * @throws IOException if a remote or network exception occurs
1350    * @throws InterruptedException
1351    */
1352   public void majorCompact(final String tableNameOrRegionName,
1353     final String columnFamily) throws IOException, InterruptedException {
1354     majorCompact(Bytes.toBytes(tableNameOrRegionName),
1355       Bytes.toBytes(columnFamily));
1356   }
1357 
1358   /**
1359    * Major compact a column family within a table or region.
1360    * Asynchronous operation.
1361    *
1362    * @param tableNameOrRegionName table or region to major compact
1363    * @param columnFamily column family within a table or region
1364    * @throws IOException if a remote or network exception occurs
1365    * @throws InterruptedException
1366    */
1367   public void majorCompact(final byte [] tableNameOrRegionName,
1368     final byte[] columnFamily) throws IOException, InterruptedException {
1369     compact(tableNameOrRegionName, columnFamily, true);
1370   }
1371 
1372   /**
1373    * Compact a table or an individual region.
1374    * Asynchronous operation.
1375    *
1376    * @param tableNameOrRegionName table or region to compact
1377    * @param columnFamily column family within a table or region
1378    * @param major True if we are to do a major compaction.
1379    * @throws IOException if a remote or network exception occurs
1380    * @throws InterruptedException
1381    */
1382   private void compact(final byte [] tableNameOrRegionName,
1383     final byte[] columnFamily, final boolean major)
1384   throws IOException, InterruptedException {
1385     CatalogTracker ct = getCatalogTracker();
1386     try {
1387       Pair<HRegionInfo, ServerName> regionServerPair
1388         = getRegion(tableNameOrRegionName, ct);
1389       if (regionServerPair != null) {
1390         if (regionServerPair.getSecond() == null) {
1391           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1392         } else {
1393           compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
1394         }
1395       } else {
1396         final String tableName = tableNameString(tableNameOrRegionName, ct);
1397         List<Pair<HRegionInfo, ServerName>> pairs =
1398           MetaReader.getTableRegionsAndLocations(ct,
1399               tableName);
1400         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1401           if (pair.getFirst().isOffline()) continue;
1402           if (pair.getSecond() == null) continue;
1403           try {
1404             compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
1405           } catch (NotServingRegionException e) {
1406             if (LOG.isDebugEnabled()) {
1407               LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1408                 pair.getFirst() + ": " +
1409                 StringUtils.stringifyException(e));
1410             }
1411           }
1412         }
1413       }
1414     } finally {
1415       cleanupCatalogTracker(ct);
1416     }
1417   }
1418 
1419   private void compact(final ServerName sn, final HRegionInfo hri,
1420       final boolean major, final byte [] family)
1421   throws IOException {
1422     HRegionInterface rs =
1423       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1424     if (family != null) {
1425       try {
1426         rs.compactRegion(hri, major, family);
1427       } catch (IOException ioe) {
1428         String notFoundMsg = "java.lang.NoSuchMethodException: org.apache.hadoop.hbase.ipc.HRegionInterface."
1429           + "compactRegion(org.apache.hadoop.hbase.HRegionInfo, boolean, [B)";
1430         if (ioe.getMessage().contains(notFoundMsg)) {
1431           throw new IOException("per-column family compaction not supported on this version "
1432             + "of the HBase server.  You may still compact at the table or region level by "
1433           	+ "omitting the column family name.  Alternatively, you can upgrade the HBase server");
1434         }
1435         throw ioe;
1436       }
1437     } else {
1438       rs.compactRegion(hri, major);
1439     }
1440   }
1441 
1442   /**
1443    * Move the region <code>r</code> to <code>dest</code>.
1444    * @param encodedRegionName The encoded region name; i.e. the hash that makes
1445    * up the region name suffix: e.g. if regionname is
1446    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
1447    * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
1448    * @param destServerName The servername of the destination regionserver.  If
1449    * passed the empty byte array we'll assign to a random server.  A server name
1450    * is made of host, port and startcode.  Here is an example:
1451    * <code> host187.example.com,60020,1289493121758</code>
1452    * @throws UnknownRegionException Thrown if we can't find a region named
1453    * <code>encodedRegionName</code>
1454    * @throws ZooKeeperConnectionException
1455    * @throws MasterNotRunningException
1456    */
1457   public void move(final byte [] encodedRegionName, final byte [] destServerName)
1458   throws UnknownRegionException, MasterNotRunningException, ZooKeeperConnectionException {
1459     getMaster().move(encodedRegionName, destServerName);
1460   }
1461 
1462   /**
1463    * @param regionName
1464    *          Region name to assign.
1465    * @throws MasterNotRunningException
1466    * @throws ZooKeeperConnectionException
1467    * @throws IOException
1468    */
1469   public void assign(final byte[] regionName) throws MasterNotRunningException,
1470       ZooKeeperConnectionException, IOException {
1471     getMaster().assign(regionName);
1472   }
1473 
1474   /**
1475    * Unassign a region from current hosting regionserver.  Region will then be
1476    * assigned to a regionserver chosen at random.  Region could be reassigned
1477    * back to the same server.  Use {@link #move(byte[], byte[])} if you want
1478    * to control the region movement.
1479    * @param regionName Region to unassign. Will clear any existing RegionPlan
1480    * if one found.
1481    * @param force If true, force unassign (Will remove region from
1482    * regions-in-transition too if present. If results in double assignment
1483    * use hbck -fix to resolve. To be used by experts).
1484    * @throws MasterNotRunningException
1485    * @throws ZooKeeperConnectionException
1486    * @throws IOException
1487    */
1488   public void unassign(final byte [] regionName, final boolean force)
1489   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
1490     getMaster().unassign(regionName, force);
1491   }
1492 
1493   /**
1494    * Turn the load balancer on or off.
1495    * @param b If true, enable balancer. If false, disable balancer.
1496    * @return Previous balancer value
1497    * @deprecated use setBalancerRunning(boolean, boolean) instead
1498    */
1499   @Deprecated
1500   public boolean balanceSwitch(final boolean b)
1501   throws MasterNotRunningException, ZooKeeperConnectionException {
1502     return getMaster().balanceSwitch(b);
1503   }
1504 
1505   /**
1506    * Turn the load balancer on or off.
1507    * @param on If true, enable balancer. If false, disable balancer.
1508    * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
1509    * @return Previous balancer value
1510    */
1511   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
1512   throws MasterNotRunningException, ZooKeeperConnectionException {
1513     if (synchronous && synchronousBalanceSwitchSupported) {
1514       try {
1515         return getMaster().synchronousBalanceSwitch(on);
1516       } catch (UndeclaredThrowableException ute) {
1517         String error = ute.getCause().getMessage();
1518         if (error != null && error.matches(
1519             "(?s).+NoSuchMethodException:.+synchronousBalanceSwitch.+")) {
1520           LOG.info("HMaster doesn't support synchronousBalanceSwitch");
1521           synchronousBalanceSwitchSupported = false;
1522         } else {
1523           throw ute;
1524         }
1525       }
1526     }
1527     return balanceSwitch(on);
1528   }
1529 
1530   /**
1531    * Invoke the balancer.  Will run the balancer and if regions to move, it will
1532    * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
1533    * logs.
1534    * @return True if balancer ran, false otherwise.
1535    */
1536   public boolean balancer()
1537   throws MasterNotRunningException, ZooKeeperConnectionException {
1538     return getMaster().balance();
1539   }
1540 
1541   /**
1542    * Split a table or an individual region.
1543    * Asynchronous operation.
1544    *
1545    * @param tableNameOrRegionName table or region to split
1546    * @throws IOException if a remote or network exception occurs
1547    * @throws InterruptedException
1548    */
1549   public void split(final String tableNameOrRegionName)
1550   throws IOException, InterruptedException {
1551     split(Bytes.toBytes(tableNameOrRegionName));
1552   }
1553 
1554   /**
1555    * Split a table or an individual region.  Implicitly finds an optimal split
1556    * point.  Asynchronous operation.
1557    *
1558    * @param tableNameOrRegionName table to region to split
1559    * @throws IOException if a remote or network exception occurs
1560    * @throws InterruptedException
1561    */
1562   public void split(final byte [] tableNameOrRegionName)
1563   throws IOException, InterruptedException {
1564     split(tableNameOrRegionName, null);
1565   }
1566 
1567   public void split(final String tableNameOrRegionName,
1568     final String splitPoint) throws IOException, InterruptedException {
1569     split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
1570   }
1571 
1572   /**
1573    * Split a table or an individual region.
1574    * Asynchronous operation.
1575    *
1576    * @param tableNameOrRegionName table to region to split
1577    * @param splitPoint the explicit position to split on
1578    * @throws IOException if a remote or network exception occurs
1579    * @throws InterruptedException interrupt exception occurred
1580    */
1581   public void split(final byte [] tableNameOrRegionName,
1582       final byte [] splitPoint) throws IOException, InterruptedException {
1583     CatalogTracker ct = getCatalogTracker();
1584     try {
1585       Pair<HRegionInfo, ServerName> regionServerPair
1586         = getRegion(tableNameOrRegionName, ct);
1587       if (regionServerPair != null) {
1588         if (regionServerPair.getSecond() == null) {
1589             throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1590         } else {
1591           split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
1592         }
1593       } else {
1594         final String tableName = tableNameString(tableNameOrRegionName, ct);
1595         List<Pair<HRegionInfo, ServerName>> pairs =
1596           MetaReader.getTableRegionsAndLocations(ct,
1597               tableName);
1598         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1599           // May not be a server for a particular row
1600           if (pair.getSecond() == null) continue;
1601           HRegionInfo r = pair.getFirst();
1602           // check for parents
1603           if (r.isSplitParent()) continue;
1604           // if a split point given, only split that particular region
1605           if (splitPoint != null && !r.containsRow(splitPoint)) continue;
1606           // call out to region server to do split now
1607           split(pair.getSecond(), pair.getFirst(), splitPoint);
1608         }
1609       }
1610     } finally {
1611       cleanupCatalogTracker(ct);
1612     }
1613   }
1614 
1615   private void split(final ServerName sn, final HRegionInfo hri,
1616       byte[] splitPoint) throws IOException {
1617     HRegionInterface rs =
1618       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1619     rs.splitRegion(hri, splitPoint);
1620   }
1621 
1622   /**
1623    * Modify an existing table, more IRB friendly version.
1624    * Asynchronous operation.  This means that it may be a while before your
1625    * schema change is updated across all of the table.
1626    *
1627    * @param tableName name of table.
1628    * @param htd modified description of the table
1629    * @throws IOException if a remote or network exception occurs
1630    */
1631   public void modifyTable(final byte [] tableName, HTableDescriptor htd)
1632   throws IOException {
1633     try {
1634       getMaster().modifyTable(tableName, htd);
1635     } catch (RemoteException re) {
1636       // Convert RE exceptions in here; client shouldn't have to deal with them,
1637       // at least w/ the type of exceptions that come out of this method:
1638       // TableNotFoundException, etc.
1639       throw RemoteExceptionHandler.decodeRemoteException(re);
1640     }
1641   }
1642 
1643   /**
1644    * @param tableNameOrRegionName Name of a table or name of a region.
1645    * @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
1646    * @return a pair of HRegionInfo and ServerName if <code>tableNameOrRegionName</code> is
1647    *  a verified region name (we call {@link  MetaReader#getRegion( CatalogTracker, byte[])}
1648    *  else null.
1649    * Throw an exception if <code>tableNameOrRegionName</code> is null.
1650    * @throws IOException
1651    */
1652   Pair<HRegionInfo, ServerName> getRegion(final byte[] tableNameOrRegionName,
1653       final CatalogTracker ct) throws IOException {
1654     if (tableNameOrRegionName == null) {
1655       throw new IllegalArgumentException("Pass a table name or region name");
1656     }
1657     Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, tableNameOrRegionName);
1658     if (pair == null) {
1659       final AtomicReference<Pair<HRegionInfo, ServerName>> result =
1660         new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
1661       final String encodedName = Bytes.toString(tableNameOrRegionName);
1662       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
1663         @Override
1664         public boolean processRow(Result data) throws IOException {
1665           if (data == null || data.size() <= 0) {
1666             return true;
1667           }
1668           HRegionInfo info = MetaReader.parseHRegionInfoFromCatalogResult(
1669             data, HConstants.REGIONINFO_QUALIFIER);
1670           if (info == null) {
1671             LOG.warn("No serialized HRegionInfo in " + data);
1672             return true;
1673           }
1674           if (!encodedName.equals(info.getEncodedName())) return true;
1675           ServerName sn = MetaReader.getServerNameFromCatalogResult(data);
1676           result.set(new Pair<HRegionInfo, ServerName>(info, sn));
1677           return false; // found the region, stop
1678         }
1679       };
1680 
1681       MetaScanner.metaScan(conf, connection, visitor, null);
1682       pair = result.get();
1683     }
1684     return pair;
1685   }
1686 
1687   /**
1688    * Convert the table name byte array into a table name string and check if table
1689    * exists or not.
1690    * @param tableNameBytes Name of a table.
1691    * @param ct A {@link #CatalogTracker} instance (caller of this method usually has one).
1692    * @return tableName in string form.
1693    * @throws IOException if a remote or network exception occurs.
1694    * @throws TableNotFoundException if table does not exist.
1695    */
1696   private String tableNameString(final byte[] tableNameBytes, CatalogTracker ct)
1697       throws IOException {
1698     String tableNameString = Bytes.toString(tableNameBytes);
1699     if (!MetaReader.tableExists(ct, tableNameString)) {
1700       throw new TableNotFoundException(tableNameString);
1701     }
1702     return tableNameString;
1703   }
1704 
1705   /**
1706    * Shuts down the HBase cluster
1707    * @throws IOException if a remote or network exception occurs
1708    */
1709   public synchronized void shutdown() throws IOException {
1710     isMasterRunning();
1711     try {
1712       getMaster().shutdown();
1713     } catch (RemoteException e) {
1714       throw RemoteExceptionHandler.decodeRemoteException(e);
1715     }
1716   }
1717 
1718   /**
1719    * Shuts down the current HBase master only.
1720    * Does not shutdown the cluster.
1721    * @see #shutdown()
1722    * @throws IOException if a remote or network exception occurs
1723    */
1724   public synchronized void stopMaster() throws IOException {
1725     isMasterRunning();
1726     try {
1727       getMaster().stopMaster();
1728     } catch (RemoteException e) {
1729       throw RemoteExceptionHandler.decodeRemoteException(e);
1730     }
1731   }
1732 
1733   /**
1734    * Stop the designated regionserver
1735    * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
1736    * <code>example.org:1234</code>
1737    * @throws IOException if a remote or network exception occurs
1738    */
1739   public synchronized void stopRegionServer(final String hostnamePort)
1740   throws IOException {
1741     String hostname = Addressing.parseHostname(hostnamePort);
1742     int port = Addressing.parsePort(hostnamePort);
1743     HRegionInterface rs =
1744       this.connection.getHRegionConnection(hostname, port);
1745     rs.stop("Called by admin client " + this.connection.toString());
1746   }
1747 
1748   /**
1749    * @return cluster status
1750    * @throws IOException if a remote or network exception occurs
1751    */
1752   public ClusterStatus getClusterStatus() throws IOException {
1753     return getMaster().getClusterStatus();
1754   }
1755 
1756   private HRegionLocation getFirstMetaServerForTable(final byte [] tableName)
1757   throws IOException {
1758     return connection.locateRegion(HConstants.META_TABLE_NAME,
1759       HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
1760   }
1761 
1762   /**
1763    * @return Configuration used by the instance.
1764    */
1765   public Configuration getConfiguration() {
1766     return this.conf;
1767   }
1768 
1769   /**
1770    * Check to see if HBase is running. Throw an exception if not.
1771    *
1772    * @param conf system configuration
1773    * @throws MasterNotRunningException if the master is not running
1774    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
1775    */
1776   public static void checkHBaseAvailable(Configuration conf)
1777   throws MasterNotRunningException, ZooKeeperConnectionException {
1778     Configuration copyOfConf = HBaseConfiguration.create(conf);
1779     copyOfConf.setInt("hbase.client.retries.number", 1);
1780     HBaseAdmin admin = new HBaseAdmin(copyOfConf);
1781     try {
1782       admin.close();
1783     } catch (IOException ioe) {
1784       admin.LOG.info("Failed to close connection", ioe);
1785     }
1786   }
1787 
1788   /**
1789    * get the regions of a given table.
1790    *
1791    * @param tableName the name of the table
1792    * @return Ordered list of {@link HRegionInfo}.
1793    * @throws IOException
1794    */
1795   public List<HRegionInfo> getTableRegions(final byte[] tableName)
1796   throws IOException {
1797     CatalogTracker ct = getCatalogTracker();
1798     List<HRegionInfo> Regions = null;
1799     try {
1800       Regions = MetaReader.getTableRegions(ct, tableName, true);
1801     } finally {
1802       cleanupCatalogTracker(ct);
1803     }
1804     return Regions;
1805   }
1806 
1807   public void close() throws IOException {
1808     if (cleanupConnectionOnClose && this.connection != null) {
1809       this.connection.close();
1810     }
1811   }
1812 
1813  /**
1814  * Get tableDescriptors
1815  * @param tableNames List of table names
1816  * @return HTD[] the tableDescriptor
1817  * @throws IOException if a remote or network exception occurs
1818  */
1819   public HTableDescriptor[] getTableDescriptors(List<String> tableNames)
1820   throws IOException {
1821     return this.connection.getHTableDescriptors(tableNames);
1822   }
1823 
1824   /**
1825    * Roll the log writer. That is, start writing log messages to a new file.
1826    *
1827    * @param serverName
1828    *          The servername of the regionserver. A server name is made of host,
1829    *          port and startcode. This is mandatory. Here is an example:
1830    *          <code> host187.example.com,60020,1289493121758</code>
1831    * @return If lots of logs, flush the returned regions so next time through
1832    * we can clean logs. Returns null if nothing to flush.  Names are actual
1833    * region names as returned by {@link HRegionInfo#getEncodedName()}
1834    * @throws IOException if a remote or network exception occurs
1835    * @throws FailedLogCloseException
1836    */
1837  public synchronized  byte[][] rollHLogWriter(String serverName)
1838       throws IOException, FailedLogCloseException {
1839     ServerName sn = new ServerName(serverName);
1840     HRegionInterface rs = this.connection.getHRegionConnection(
1841         sn.getHostname(), sn.getPort());
1842     return rs.rollHLogWriter();
1843   }
1844 
1845   public String[] getMasterCoprocessors() {
1846     try {
1847       return getClusterStatus().getMasterCoprocessors();
1848     } catch (IOException e) {
1849       LOG.error("Could not getClusterStatus()",e);
1850       return null;
1851     }
1852   }
1853 
1854   /**
1855    * Get the current compaction state of a table or region.
1856    * It could be in a major compaction, a minor compaction, both, or none.
1857    *
1858    * @param tableNameOrRegionName table or region to major compact
1859    * @throws IOException if a remote or network exception occurs
1860    * @throws InterruptedException
1861    * @return the current compaction state
1862    */
1863   public CompactionState getCompactionState(final String tableNameOrRegionName)
1864       throws IOException, InterruptedException {
1865     return getCompactionState(Bytes.toBytes(tableNameOrRegionName));
1866   }
1867 
1868   /**
1869    * Get the current compaction state of a table or region.
1870    * It could be in a major compaction, a minor compaction, both, or none.
1871    *
1872    * @param tableNameOrRegionName table or region to major compact
1873    * @throws IOException if a remote or network exception occurs
1874    * @throws InterruptedException
1875    * @return the current compaction state
1876    */
1877   public CompactionState getCompactionState(final byte [] tableNameOrRegionName)
1878       throws IOException, InterruptedException {
1879     CompactionState state = CompactionState.NONE;
1880     CatalogTracker ct = getCatalogTracker();
1881     try {
1882       Pair<HRegionInfo, ServerName> regionServerPair
1883         = getRegion(tableNameOrRegionName, ct);
1884       if (regionServerPair != null) {
1885         if (regionServerPair.getSecond() == null) {
1886           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1887         } else {
1888           ServerName sn = regionServerPair.getSecond();
1889           HRegionInterface rs =
1890             this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1891           return CompactionState.valueOf(
1892             rs.getCompactionState(regionServerPair.getFirst().getRegionName()));
1893         }
1894       } else {
1895         final String tableName = tableNameString(tableNameOrRegionName, ct);
1896         List<Pair<HRegionInfo, ServerName>> pairs =
1897           MetaReader.getTableRegionsAndLocations(ct, tableName);
1898         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1899           if (pair.getFirst().isOffline()) continue;
1900           if (pair.getSecond() == null) continue;
1901           try {
1902             ServerName sn = pair.getSecond();
1903             HRegionInterface rs =
1904               this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1905             switch (CompactionState.valueOf(
1906               rs.getCompactionState(pair.getFirst().getRegionName()))) {
1907             case MAJOR_AND_MINOR:
1908               return CompactionState.MAJOR_AND_MINOR;
1909             case MAJOR:
1910               if (state == CompactionState.MINOR) {
1911                 return CompactionState.MAJOR_AND_MINOR;
1912               }
1913               state = CompactionState.MAJOR;
1914               break;
1915             case MINOR:
1916               if (state == CompactionState.MAJOR) {
1917                 return CompactionState.MAJOR_AND_MINOR;
1918               }
1919               state = CompactionState.MINOR;
1920               break;
1921             case NONE:
1922               default: // nothing, continue
1923             }
1924           } catch (NotServingRegionException e) {
1925             if (LOG.isDebugEnabled()) {
1926               LOG.debug("Trying to get compaction state of " +
1927                 pair.getFirst() + ": " +
1928                 StringUtils.stringifyException(e));
1929             }
1930           }
1931         }
1932       }
1933     } finally {
1934       cleanupCatalogTracker(ct);
1935     }
1936     return state;
1937   }
1938 
1939   /**
1940    * Creates and returns a proxy to the CoprocessorProtocol instance running in the
1941    * master.
1942    *
1943    * @param protocol The class or interface defining the remote protocol
1944    * @return A CoprocessorProtocol instance
1945    */
1946   public <T extends CoprocessorProtocol> T coprocessorProxy(
1947       Class<T> protocol) {
1948     return (T) Proxy.newProxyInstance(this.getClass().getClassLoader(),
1949         new Class[]{protocol},
1950         new MasterExecRPCInvoker(conf,
1951             connection,
1952             protocol));
1953   }
1954 
1955 
1956   /**
1957    * Create a timestamp consistent snapshot for the given table.
1958    * <p>
1959    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
1960    * snapshot with the same name (even a different type or with different parameters) will fail with
1961    * a {@link SnapshotCreationException} indicating the duplicate naming.
1962    * <p>
1963    * Snapshot names follow the same naming constraints as tables in HBase. See
1964    * {@link HTableDescriptor#isLegalTableName(byte[])}.
1965    * @param snapshotName name of the snapshot to be created
1966    * @param tableName name of the table for which snapshot is created
1967    * @throws IOException if a remote or network exception occurs
1968    * @throws SnapshotCreationException if snapshot creation failed
1969    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
1970    */
1971   public void snapshot(final String snapshotName, final String tableName) throws IOException,
1972       SnapshotCreationException, IllegalArgumentException {
1973     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
1974   }
1975 
1976   /**
1977    * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
1978    * taken. If the table is disabled, an offline snapshot is taken.
1979    * <p>
1980    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
1981    * snapshot with the same name (even a different type or with different parameters) will fail with
1982    * a {@link SnapshotCreationException} indicating the duplicate naming.
1983    * <p>
1984    * Snapshot names follow the same naming constraints as tables in HBase. See
1985    * {@link HTableDescriptor#isLegalTableName(byte[])}.
1986    * @param snapshotName name of the snapshot to be created
1987    * @param tableName name of the table for which snapshot is created
1988    * @throws IOException if a remote or network exception occurs
1989    * @throws SnapshotCreationException if snapshot creation failed
1990    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
1991    */
1992   public void snapshot(final byte[] snapshotName, final byte[] tableName) throws IOException,
1993       SnapshotCreationException, IllegalArgumentException {
1994     snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
1995   }
1996 
1997   /**
1998    * Create typed snapshot of the table.
1999    * <p>
2000    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2001    * snapshot with the same name (even a different type or with different parameters) will fail with
2002    * a {@link SnapshotCreationException} indicating the duplicate naming.
2003    * <p>
2004    * Snapshot names follow the same naming constraints as tables in HBase. See
2005    * {@link HTableDescriptor#isLegalTableName(byte[])}.
2006    * <p>
2007    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
2008    * snapshots stored on the cluster
2009    * @param tableName name of the table to snapshot
2010    * @param type type of snapshot to take
2011    * @throws IOException we fail to reach the master
2012    * @throws SnapshotCreationException if snapshot creation failed
2013    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2014    */
2015   public void snapshot(final String snapshotName, final String tableName,
2016       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2017       IllegalArgumentException {
2018     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
2019     builder.setTable(tableName);
2020     builder.setName(snapshotName);
2021     builder.setType(type);
2022     snapshot(builder.build());
2023   }
2024 
2025   /**
2026    * Take a snapshot and wait for the server to complete that snapshot (blocking).
2027    * <p>
2028    * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
2029    * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
2030    * time for a single cluster).
2031    * <p>
2032    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2033    * snapshot with the same name (even a different type or with different parameters) will fail with
2034    * a {@link SnapshotCreationException} indicating the duplicate naming.
2035    * <p>
2036    * Snapshot names follow the same naming constraints as tables in HBase. See
2037    * {@link HTableDescriptor#isLegalTableName(byte[])}.
2038    * <p>
2039    * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
2040    * unless you are sure about the type of snapshot that you want to take.
2041    * @param snapshot snapshot to take
2042    * @throws IOException or we lose contact with the master.
2043    * @throws SnapshotCreationException if snapshot failed to be taken
2044    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2045    */
2046   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
2047       IllegalArgumentException {
2048     HSnapshotDescription snapshotWritable = new HSnapshotDescription(snapshot);
2049 
2050     try {
2051       // actually take the snapshot
2052       long max = takeSnapshotAsync(snapshot);
2053       long start = EnvironmentEdgeManager.currentTimeMillis();
2054       long maxPauseTime = max / this.numRetries;
2055       boolean done = false;
2056       int tries = 0;
2057       LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
2058           SnapshotDescriptionUtils.toString(snapshot) + "' to complete. (max " +
2059           maxPauseTime + " ms per retry)");
2060       while (tries == 0 || (EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done) {
2061         try {
2062           // sleep a backoff <= pauseTime amount
2063           long sleep = getPauseTime(tries++);
2064           sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2065           LOG.debug("(#" + tries + ") Sleeping: " + sleep +
2066             "ms while waiting for snapshot completion.");
2067           Thread.sleep(sleep);
2068 
2069         } catch (InterruptedException e) {
2070           LOG.debug("Interrupted while waiting for snapshot " + snapshot + " to complete");
2071           Thread.currentThread().interrupt();
2072         }
2073         LOG.debug("Getting current status of snapshot from master...");
2074         done = getMaster().isSnapshotDone(snapshotWritable);
2075       }
2076 
2077       if (!done) {
2078         throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
2079             + "' wasn't completed in expectedTime:" + max + " ms", snapshot);
2080       }
2081     } catch (RemoteException e) {
2082       throw RemoteExceptionHandler.decodeRemoteException(e);
2083     }
2084   }
2085 
2086   /**
2087    * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
2088    * <p>
2089    * Only a single snapshot should be taken at a time, or results may be undefined.
2090    * @param snapshot snapshot to take
2091    * @return the max time in millis to wait for the snapshot
2092    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
2093    * @throws SnapshotCreationException if snapshot creation failed
2094    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2095    */
2096   public long takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
2097       SnapshotCreationException {
2098     SnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
2099     HSnapshotDescription snapshotWritable = new HSnapshotDescription(snapshot);
2100     return getMaster().snapshot(snapshotWritable);
2101   }
2102 
2103   /**
2104    * Check the current state of the passed snapshot.
2105    * <p>
2106    * There are three possible states:
2107    * <ol>
2108    * <li>running - returns <tt>false</tt></li>
2109    * <li>finished - returns <tt>true</tt></li>
2110    * <li>finished with error - throws the exception that caused the snapshot to fail</li>
2111    * </ol>
2112    * <p>
2113    * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
2114    * run/started since the snapshot your are checking, you will recieve an
2115    * {@link UnknownSnapshotException}.
2116    * @param snapshot description of the snapshot to check
2117    * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
2118    * running
2119    * @throws IOException if we have a network issue
2120    * @throws HBaseSnapshotException if the snapshot failed
2121    * @throws UnknownSnapshotException if the requested snapshot is unknown
2122    */
2123   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
2124       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
2125     try {
2126       return getMaster().isSnapshotDone(new HSnapshotDescription(snapshot));
2127     } catch (RemoteException e) {
2128       throw RemoteExceptionHandler.decodeRemoteException(e);
2129     }
2130   }
2131 
2132   /**
2133    * Restore the specified snapshot on the original table. (The table must be disabled)
2134    * Before restoring the table, a new snapshot with the current table state is created.
2135    * In case of failure, the table will be rolled back to the its original state.
2136    *
2137    * @param snapshotName name of the snapshot to restore
2138    * @throws IOException if a remote or network exception occurs
2139    * @throws RestoreSnapshotException if snapshot failed to be restored
2140    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2141    */
2142   public void restoreSnapshot(final byte[] snapshotName)
2143       throws IOException, RestoreSnapshotException {
2144     restoreSnapshot(Bytes.toString(snapshotName));
2145   }
2146 
2147   /**
2148    * Restore the specified snapshot on the original table. (The table must be disabled)
2149    * Before restoring the table, a new snapshot with the current table state is created.
2150    * In case of failure, the table will be rolled back to its original state.
2151    *
2152    * @param snapshotName name of the snapshot to restore
2153    * @throws IOException if a remote or network exception occurs
2154    * @throws RestoreSnapshotException if snapshot failed to be restored
2155    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2156    */
2157   public void restoreSnapshot(final String snapshotName)
2158       throws IOException, RestoreSnapshotException {
2159     String rollbackSnapshot = snapshotName + "-" + EnvironmentEdgeManager.currentTimeMillis();
2160 
2161     String tableName = null;
2162     for (SnapshotDescription snapshotInfo: listSnapshots()) {
2163       if (snapshotInfo.getName().equals(snapshotName)) {
2164         tableName = snapshotInfo.getTable();
2165         break;
2166       }
2167     }
2168 
2169     if (tableName == null) {
2170       throw new RestoreSnapshotException(
2171         "Unable to find the table name for snapshot=" + snapshotName);
2172     }
2173 
2174     // Take a snapshot of the current state
2175     snapshot(rollbackSnapshot, tableName);
2176 
2177     // Restore snapshot
2178     try {
2179       internalRestoreSnapshot(snapshotName, tableName);
2180     } catch (IOException e) {
2181       // Try to rollback
2182       try {
2183         String msg = "Restore snapshot=" + snapshotName +
2184           " failed. Rollback to snapshot=" + rollbackSnapshot + " succeeded.";
2185         LOG.error(msg, e);
2186         internalRestoreSnapshot(rollbackSnapshot, tableName);
2187         throw new RestoreSnapshotException(msg, e);
2188       } catch (IOException ex) {
2189         String msg = "Failed to restore and rollback to snapshot=" + rollbackSnapshot;
2190         LOG.error(msg, ex);
2191         throw new RestoreSnapshotException(msg, ex);
2192       }
2193     }
2194   }
2195 
2196   /**
2197    * Create a new table by cloning the snapshot content.
2198    *
2199    * @param snapshotName name of the snapshot to be cloned
2200    * @param tableName name of the table where the snapshot will be restored
2201    * @throws IOException if a remote or network exception occurs
2202    * @throws TableExistsException if table to be created already exists
2203    * @throws RestoreSnapshotException if snapshot failed to be cloned
2204    * @throws IllegalArgumentException if the specified table has not a valid name
2205    */
2206   public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
2207       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2208     cloneSnapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
2209   }
2210 
2211   /**
2212    * Create a new table by cloning the snapshot content.
2213    *
2214    * @param snapshotName name of the snapshot to be cloned
2215    * @param tableName name of the table where the snapshot will be restored
2216    * @throws IOException if a remote or network exception occurs
2217    * @throws TableExistsException if table to be created already exists
2218    * @throws RestoreSnapshotException if snapshot failed to be cloned
2219    * @throws IllegalArgumentException if the specified table has not a valid name
2220    */
2221   public void cloneSnapshot(final String snapshotName, final String tableName)
2222       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2223     if (tableExists(tableName)) {
2224       throw new TableExistsException("Table '" + tableName + " already exists");
2225     }
2226     internalRestoreSnapshot(snapshotName, tableName);
2227     waitUntilTableIsEnabled(Bytes.toBytes(tableName));
2228   }
2229 
2230   /**
2231    * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
2232    * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
2233    * create an HTable instance to this table before it is available.
2234    * @param snapshot snapshot to restore
2235    * @param tableName table name to restore the snapshot on
2236    * @throws IOException if a remote or network exception occurs
2237    * @throws RestoreSnapshotException if snapshot failed to be restored
2238    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2239    */
2240   private void internalRestoreSnapshot(final String snapshotName, final String tableName)
2241       throws IOException, RestoreSnapshotException {
2242     HSnapshotDescription snapshot = new HSnapshotDescription(
2243       SnapshotDescription.newBuilder().setName(snapshotName).setTable(tableName).build());
2244 
2245     try {
2246       // actually restore the snapshot
2247       getMaster().restoreSnapshot(snapshot);
2248 
2249       final long maxPauseTime = 5000;
2250       boolean done = false;
2251       int tries = 0;
2252       while (!done) {
2253         try {
2254           // sleep a backoff <= pauseTime amount
2255           long sleep = getPauseTime(tries++);
2256           sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2257           LOG.debug(tries + ") Sleeping: " + sleep + " ms while we wait for snapshot restore to complete.");
2258           Thread.sleep(sleep);
2259         } catch (InterruptedException e) {
2260           LOG.debug("Interrupted while waiting for snapshot " + snapshot + " restore to complete");
2261           Thread.currentThread().interrupt();
2262         }
2263         LOG.debug("Getting current status of snapshot restore from master...");
2264         done = getMaster().isRestoreSnapshotDone(snapshot);
2265       }
2266       if (!done) {
2267         throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
2268       }
2269     } catch (RemoteException e) {
2270       throw RemoteExceptionHandler.decodeRemoteException(e);
2271     }
2272   }
2273 
2274   /**
2275    * List completed snapshots.
2276    * @return a list of snapshot descriptors for completed snapshots
2277    * @throws IOException if a network error occurs
2278    */
2279   public List<SnapshotDescription> listSnapshots() throws IOException {
2280     List<SnapshotDescription> snapshots = new LinkedList<SnapshotDescription>();
2281     try {
2282       for (HSnapshotDescription snapshot: getMaster().getCompletedSnapshots()) {
2283         snapshots.add(snapshot.getProto());
2284       }
2285     } catch (RemoteException e) {
2286       throw RemoteExceptionHandler.decodeRemoteException(e);
2287     }
2288     return snapshots;
2289   }
2290 
2291   /**
2292    * Delete an existing snapshot.
2293    * @param snapshotName name of the snapshot
2294    * @throws IOException if a remote or network exception occurs
2295    */
2296   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
2297     // make sure the snapshot is possibly valid
2298     HTableDescriptor.isLegalTableName(snapshotName);
2299     // do the delete
2300     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
2301       .setName(Bytes.toString(snapshotName)).build();
2302     try {
2303       getMaster().deleteSnapshot(new HSnapshotDescription(snapshot));
2304     } catch (RemoteException e) {
2305       throw RemoteExceptionHandler.decodeRemoteException(e);
2306     }
2307   }
2308 
2309   /**
2310    * Delete an existing snapshot.
2311    * @param snapshotName name of the snapshot
2312    * @throws IOException if a remote or network exception occurs
2313    */
2314   public void deleteSnapshot(final String snapshotName) throws IOException {
2315     deleteSnapshot(Bytes.toBytes(snapshotName));
2316   }
2317 }