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