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