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     if(!tableExists(tableName)){
938       throw new TableNotFoundException(Bytes.toString(tableName));
939     }
940     return connection.isTableEnabled(tableName);
941   }
942 
943   /**
944    * @param tableName name of table to check
945    * @return true if table is off-line
946    * @throws IOException if a remote or network exception occurs
947    */
948   public boolean isTableDisabled(final String tableName) throws IOException {
949     return isTableDisabled(Bytes.toBytes(tableName));
950   }
951 
952   /**
953    * @param tableName name of table to check
954    * @return true if table is off-line
955    * @throws IOException if a remote or network exception occurs
956    */
957   public boolean isTableDisabled(byte[] tableName) throws IOException {
958     if (!HTableDescriptor.isMetaTable(tableName)) {
959       HTableDescriptor.isLegalTableName(tableName);
960     }
961     return connection.isTableDisabled(tableName);
962   }
963 
964   /**
965    * @param tableName name of table to check
966    * @return true if all regions of the table are available
967    * @throws IOException if a remote or network exception occurs
968    */
969   public boolean isTableAvailable(byte[] tableName) throws IOException {
970     return connection.isTableAvailable(tableName);
971   }
972 
973   /**
974    * @param tableName name of table to check
975    * @return true if all regions of the table are available
976    * @throws IOException if a remote or network exception occurs
977    */
978   public boolean isTableAvailable(String tableName) throws IOException {
979     return connection.isTableAvailable(Bytes.toBytes(tableName));
980   }
981 
982   /**
983    * Get the status of alter command - indicates how many regions have received
984    * the updated schema Asynchronous operation.
985    *
986    * @param tableName
987    *          name of the table to get the status of
988    * @return Pair indicating the number of regions updated Pair.getFirst() is the
989    *         regions that are yet to be updated Pair.getSecond() is the total number
990    *         of regions of the table
991    * @throws IOException
992    *           if a remote or network exception occurs
993    */
994   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
995   throws IOException {
996     HTableDescriptor.isLegalTableName(tableName);
997     try {
998       return getMaster().getAlterStatus(tableName);
999     } catch (RemoteException e) {
1000       throw RemoteExceptionHandler.decodeRemoteException(e);
1001     }
1002   }
1003 
1004   /**
1005    * Add a column to an existing table.
1006    * Asynchronous operation.
1007    *
1008    * @param tableName name of the table to add column to
1009    * @param column column descriptor of column to be added
1010    * @throws IOException if a remote or network exception occurs
1011    */
1012   public void addColumn(final String tableName, HColumnDescriptor column)
1013   throws IOException {
1014     addColumn(Bytes.toBytes(tableName), column);
1015   }
1016 
1017   /**
1018    * Add a column to an existing table.
1019    * Asynchronous operation.
1020    *
1021    * @param tableName name of the table to add column to
1022    * @param column column descriptor of column to be added
1023    * @throws IOException if a remote or network exception occurs
1024    */
1025   public void addColumn(final byte [] tableName, HColumnDescriptor column)
1026   throws IOException {
1027     HTableDescriptor.isLegalTableName(tableName);
1028     try {
1029       getMaster().addColumn(tableName, column);
1030     } catch (RemoteException e) {
1031       throw RemoteExceptionHandler.decodeRemoteException(e);
1032     }
1033   }
1034 
1035   /**
1036    * Delete a column from a table.
1037    * Asynchronous operation.
1038    *
1039    * @param tableName name of table
1040    * @param columnName name of column to be deleted
1041    * @throws IOException if a remote or network exception occurs
1042    */
1043   public void deleteColumn(final String tableName, final String columnName)
1044   throws IOException {
1045     deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName));
1046   }
1047 
1048   /**
1049    * Delete a column from a table.
1050    * Asynchronous operation.
1051    *
1052    * @param tableName name of table
1053    * @param columnName name of column to be deleted
1054    * @throws IOException if a remote or network exception occurs
1055    */
1056   public void deleteColumn(final byte [] tableName, final byte [] columnName)
1057   throws IOException {
1058     try {
1059       getMaster().deleteColumn(tableName, columnName);
1060     } catch (RemoteException e) {
1061       throw RemoteExceptionHandler.decodeRemoteException(e);
1062     }
1063   }
1064 
1065   /**
1066    * Modify an existing column family on a table.
1067    * Asynchronous operation.
1068    *
1069    * @param tableName name of table
1070    * @param descriptor new column descriptor to use
1071    * @throws IOException if a remote or network exception occurs
1072    */
1073   public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
1074   throws IOException {
1075     modifyColumn(Bytes.toBytes(tableName), descriptor);
1076   }
1077 
1078   /**
1079    * Modify an existing column family on a table.
1080    * Asynchronous operation.
1081    *
1082    * @param tableName name of table
1083    * @param descriptor new column descriptor to use
1084    * @throws IOException if a remote or network exception occurs
1085    */
1086   public void modifyColumn(final byte [] tableName, HColumnDescriptor descriptor)
1087   throws IOException {
1088     try {
1089       getMaster().modifyColumn(tableName, descriptor);
1090     } catch (RemoteException re) {
1091       // Convert RE exceptions in here; client shouldn't have to deal with them,
1092       // at least w/ the type of exceptions that come out of this method:
1093       // TableNotFoundException, etc.
1094       throw RemoteExceptionHandler.decodeRemoteException(re);
1095     }
1096   }
1097 
1098   /**
1099    * Close a region. For expert-admins.  Runs close on the regionserver.  The
1100    * master will not be informed of the close.
1101    * @param regionname region name to close
1102    * @param serverName If supplied, we'll use this location rather than
1103    * the one currently in <code>.META.</code>
1104    * @throws IOException if a remote or network exception occurs
1105    */
1106   public void closeRegion(final String regionname, final String serverName)
1107   throws IOException {
1108     closeRegion(Bytes.toBytes(regionname), serverName);
1109   }
1110 
1111   /**
1112    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1113    * master will not be informed of the close.
1114    * @param regionname region name to close
1115    * @param serverName The servername of the regionserver.  If passed null we
1116    * will use servername found in the .META. table. A server name
1117    * is made of host, port and startcode.  Here is an example:
1118    * <code> host187.example.com,60020,1289493121758</code>
1119    * @throws IOException if a remote or network exception occurs
1120    */
1121   public void closeRegion(final byte [] regionname, final String serverName)
1122   throws IOException {
1123     CatalogTracker ct = getCatalogTracker();
1124     try {
1125       if (serverName != null) {
1126         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1127         if (pair == null || pair.getFirst() == null) {
1128           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1129         } else {
1130           closeRegion(new ServerName(serverName), pair.getFirst());
1131         }
1132       } else {
1133         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1134         if (pair == null) {
1135           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1136         } else if (pair.getSecond() == null) {
1137           throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1138         } else {
1139           closeRegion(pair.getSecond(), pair.getFirst());
1140         }
1141       }
1142     } finally {
1143       cleanupCatalogTracker(ct);
1144     }
1145   }
1146 
1147   /**
1148    * For expert-admins. Runs close on the regionserver. Closes a region based on
1149    * the encoded region name. The region server name is mandatory. If the
1150    * servername is provided then based on the online regions in the specified
1151    * regionserver the specified region will be closed. The master will not be
1152    * informed of the close. Note that the regionname is the encoded regionname.
1153    *
1154    * @param encodedRegionName
1155    *          The encoded region name; i.e. the hash that makes up the region
1156    *          name suffix: e.g. if regionname is
1157    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
1158    *          , then the encoded region name is:
1159    *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
1160    * @param serverName
1161    *          The servername of the regionserver. A server name is made of host,
1162    *          port and startcode. This is mandatory. Here is an example:
1163    *          <code> host187.example.com,60020,1289493121758</code>
1164    * @return true if the region was closed, false if not.
1165    * @throws IOException
1166    *           if a remote or network exception occurs
1167    */
1168   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1169       final String serverName) throws IOException {
1170     byte[] encodedRegionNameInBytes = Bytes.toBytes(encodedRegionName);
1171     if (null == serverName || ("").equals(serverName.trim())) {
1172       throw new IllegalArgumentException(
1173           "The servername cannot be null or empty.");
1174     }
1175     ServerName sn = new ServerName(serverName);
1176     HRegionInterface rs = this.connection.getHRegionConnection(
1177         sn.getHostname(), sn.getPort());
1178     // Close the region without updating zk state.
1179     boolean isRegionClosed = rs.closeRegion(encodedRegionNameInBytes, false);
1180     if (false == isRegionClosed) {
1181       LOG.error("Not able to close the region " + encodedRegionName + ".");
1182     }
1183     return isRegionClosed;
1184   }
1185 
1186   /**
1187    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1188    * master will not be informed of the close.
1189    * @param sn
1190    * @param hri
1191    * @throws IOException
1192    */
1193   public void closeRegion(final ServerName sn, final HRegionInfo hri)
1194   throws IOException {
1195     HRegionInterface rs =
1196       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1197     // Close the region without updating zk state.
1198     rs.closeRegion(hri, false);
1199   }
1200 
1201   /**
1202    * Flush a table or an individual region.
1203    * Asynchronous operation.
1204    *
1205    * @param tableNameOrRegionName table or region to flush
1206    * @throws IOException if a remote or network exception occurs
1207    * @throws InterruptedException
1208    */
1209   public void flush(final String tableNameOrRegionName)
1210   throws IOException, InterruptedException {
1211     flush(Bytes.toBytes(tableNameOrRegionName));
1212   }
1213 
1214   /**
1215    * Flush a table or an individual region.
1216    * Asynchronous operation.
1217    *
1218    * @param tableNameOrRegionName table or region to flush
1219    * @throws IOException if a remote or network exception occurs
1220    * @throws InterruptedException
1221    */
1222   public void flush(final byte [] tableNameOrRegionName)
1223   throws IOException, InterruptedException {
1224     CatalogTracker ct = getCatalogTracker();
1225     try {
1226       Pair<HRegionInfo, ServerName> regionServerPair
1227         = getRegion(tableNameOrRegionName, ct);
1228       if (regionServerPair != null) {
1229         if (regionServerPair.getSecond() == null) {
1230           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1231         } else {
1232           flush(regionServerPair.getSecond(), regionServerPair.getFirst());
1233         }
1234       } else {
1235         final String tableName = tableNameString(tableNameOrRegionName, ct);
1236         List<Pair<HRegionInfo, ServerName>> pairs =
1237           MetaReader.getTableRegionsAndLocations(ct,
1238               tableName);
1239         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1240           if (pair.getFirst().isOffline()) continue;
1241           if (pair.getSecond() == null) continue;
1242           try {
1243             flush(pair.getSecond(), pair.getFirst());
1244           } catch (NotServingRegionException e) {
1245             if (LOG.isDebugEnabled()) {
1246               LOG.debug("Trying to flush " + pair.getFirst() + ": " +
1247                 StringUtils.stringifyException(e));
1248             }
1249           }
1250         }
1251       }
1252     } finally {
1253       cleanupCatalogTracker(ct);
1254     }
1255   }
1256 
1257   private void flush(final ServerName sn, final HRegionInfo hri)
1258   throws IOException {
1259     HRegionInterface rs =
1260       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1261     rs.flushRegion(hri);
1262   }
1263 
1264   /**
1265    * Compact a table or an individual region.
1266    * Asynchronous operation.
1267    *
1268    * @param tableNameOrRegionName table or region to compact
1269    * @throws IOException if a remote or network exception occurs
1270    * @throws InterruptedException
1271    */
1272   public void compact(final String tableNameOrRegionName)
1273   throws IOException, InterruptedException {
1274     compact(Bytes.toBytes(tableNameOrRegionName));
1275   }
1276 
1277   /**
1278    * Compact a table or an individual region.
1279    * Asynchronous operation.
1280    *
1281    * @param tableNameOrRegionName table or region to compact
1282    * @throws IOException if a remote or network exception occurs
1283    * @throws InterruptedException
1284    */
1285   public void compact(final byte [] tableNameOrRegionName)
1286   throws IOException, InterruptedException {
1287     compact(tableNameOrRegionName, null, false);
1288   }
1289 
1290   /**
1291    * Compact a column family within a table or region.
1292    * Asynchronous operation.
1293    *
1294    * @param tableOrRegionName table or region to compact
1295    * @param columnFamily column family within a table or region
1296    * @throws IOException if a remote or network exception occurs
1297    * @throws InterruptedException
1298    */
1299   public void compact(String tableOrRegionName, String columnFamily)
1300     throws IOException,  InterruptedException {
1301     compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
1302   }
1303 
1304   /**
1305    * Compact a column family within a table or region.
1306    * Asynchronous operation.
1307    *
1308    * @param tableNameOrRegionName table or region to compact
1309    * @param columnFamily column family within a table or region
1310    * @throws IOException if a remote or network exception occurs
1311    * @throws InterruptedException
1312    */
1313   public void compact(final byte [] tableNameOrRegionName, final byte[] columnFamily)
1314   throws IOException, InterruptedException {
1315     compact(tableNameOrRegionName, columnFamily, false);
1316   }
1317 
1318   /**
1319    * Major compact a table or an individual region.
1320    * Asynchronous operation.
1321    *
1322    * @param tableNameOrRegionName table or region to major compact
1323    * @throws IOException if a remote or network exception occurs
1324    * @throws InterruptedException
1325    */
1326   public void majorCompact(final String tableNameOrRegionName)
1327   throws IOException, InterruptedException {
1328     majorCompact(Bytes.toBytes(tableNameOrRegionName));
1329   }
1330 
1331   /**
1332    * Major compact a table or an individual region.
1333    * Asynchronous operation.
1334    *
1335    * @param tableNameOrRegionName table or region to major compact
1336    * @throws IOException if a remote or network exception occurs
1337    * @throws InterruptedException
1338    */
1339   public void majorCompact(final byte [] tableNameOrRegionName)
1340   throws IOException, InterruptedException {
1341     compact(tableNameOrRegionName, null, true);
1342   }
1343 
1344   /**
1345    * Major compact a column family within a table or region.
1346    * Asynchronous operation.
1347    *
1348    * @param tableNameOrRegionName table or region to major compact
1349    * @param columnFamily column family within a table or region
1350    * @throws IOException if a remote or network exception occurs
1351    * @throws InterruptedException
1352    */
1353   public void majorCompact(final String tableNameOrRegionName,
1354     final String columnFamily) throws IOException, InterruptedException {
1355     majorCompact(Bytes.toBytes(tableNameOrRegionName),
1356       Bytes.toBytes(columnFamily));
1357   }
1358 
1359   /**
1360    * Major compact a column family within a table or region.
1361    * Asynchronous operation.
1362    *
1363    * @param tableNameOrRegionName table or region to major compact
1364    * @param columnFamily column family within a table or region
1365    * @throws IOException if a remote or network exception occurs
1366    * @throws InterruptedException
1367    */
1368   public void majorCompact(final byte [] tableNameOrRegionName,
1369     final byte[] columnFamily) throws IOException, InterruptedException {
1370     compact(tableNameOrRegionName, columnFamily, true);
1371   }
1372 
1373   /**
1374    * Compact a table or an individual region.
1375    * Asynchronous operation.
1376    *
1377    * @param tableNameOrRegionName table or region to compact
1378    * @param columnFamily column family within a table or region
1379    * @param major True if we are to do a major compaction.
1380    * @throws IOException if a remote or network exception occurs
1381    * @throws InterruptedException
1382    */
1383   private void compact(final byte [] tableNameOrRegionName,
1384     final byte[] columnFamily, final boolean major)
1385   throws IOException, InterruptedException {
1386     CatalogTracker ct = getCatalogTracker();
1387     try {
1388       Pair<HRegionInfo, ServerName> regionServerPair
1389         = getRegion(tableNameOrRegionName, ct);
1390       if (regionServerPair != null) {
1391         if (regionServerPair.getSecond() == null) {
1392           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1393         } else {
1394           compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
1395         }
1396       } else {
1397         final String tableName = tableNameString(tableNameOrRegionName, ct);
1398         List<Pair<HRegionInfo, ServerName>> pairs =
1399           MetaReader.getTableRegionsAndLocations(ct,
1400               tableName);
1401         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1402           if (pair.getFirst().isOffline()) continue;
1403           if (pair.getSecond() == null) continue;
1404           try {
1405             compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
1406           } catch (NotServingRegionException e) {
1407             if (LOG.isDebugEnabled()) {
1408               LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1409                 pair.getFirst() + ": " +
1410                 StringUtils.stringifyException(e));
1411             }
1412           }
1413         }
1414       }
1415     } finally {
1416       cleanupCatalogTracker(ct);
1417     }
1418   }
1419 
1420   private void compact(final ServerName sn, final HRegionInfo hri,
1421       final boolean major, final byte [] family)
1422   throws IOException {
1423     HRegionInterface rs =
1424       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1425     if (family != null) {
1426       try {
1427         rs.compactRegion(hri, major, family);
1428       } catch (IOException ioe) {
1429         String notFoundMsg = "java.lang.NoSuchMethodException: org.apache.hadoop.hbase.ipc.HRegionInterface."
1430           + "compactRegion(org.apache.hadoop.hbase.HRegionInfo, boolean, [B)";
1431         if (ioe.getMessage().contains(notFoundMsg)) {
1432           throw new IOException("per-column family compaction not supported on this version "
1433             + "of the HBase server.  You may still compact at the table or region level by "
1434           	+ "omitting the column family name.  Alternatively, you can upgrade the HBase server");
1435         }
1436         throw ioe;
1437       }
1438     } else {
1439       rs.compactRegion(hri, major);
1440     }
1441   }
1442 
1443   /**
1444    * Move the region <code>r</code> to <code>dest</code>.
1445    * @param encodedRegionName The encoded region name; i.e. the hash that makes
1446    * up the region name suffix: e.g. if regionname is
1447    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
1448    * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
1449    * @param destServerName The servername of the destination regionserver.  If
1450    * passed the empty byte array we'll assign to a random server.  A server name
1451    * is made of host, port and startcode.  Here is an example:
1452    * <code> host187.example.com,60020,1289493121758</code>
1453    * @throws UnknownRegionException Thrown if we can't find a region named
1454    * <code>encodedRegionName</code>
1455    * @throws ZooKeeperConnectionException
1456    * @throws MasterNotRunningException
1457    */
1458   public void move(final byte [] encodedRegionName, final byte [] destServerName)
1459   throws UnknownRegionException, MasterNotRunningException, ZooKeeperConnectionException {
1460     getMaster().move(encodedRegionName, destServerName);
1461   }
1462 
1463   /**
1464    * @param regionName
1465    *          Region name to assign.
1466    * @throws MasterNotRunningException
1467    * @throws ZooKeeperConnectionException
1468    * @throws IOException
1469    */
1470   public void assign(final byte[] regionName) throws MasterNotRunningException,
1471       ZooKeeperConnectionException, IOException {
1472     getMaster().assign(regionName);
1473   }
1474 
1475   /**
1476    * Unassign a region from current hosting regionserver.  Region will then be
1477    * assigned to a regionserver chosen at random.  Region could be reassigned
1478    * back to the same server.  Use {@link #move(byte[], byte[])} if you want
1479    * to control the region movement.
1480    * @param regionName Region to unassign. Will clear any existing RegionPlan
1481    * if one found.
1482    * @param force If true, force unassign (Will remove region from
1483    * regions-in-transition too if present. If results in double assignment
1484    * use hbck -fix to resolve. To be used by experts).
1485    * @throws MasterNotRunningException
1486    * @throws ZooKeeperConnectionException
1487    * @throws IOException
1488    */
1489   public void unassign(final byte [] regionName, final boolean force)
1490   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
1491     getMaster().unassign(regionName, force);
1492   }
1493 
1494   /**
1495    * Turn the load balancer on or off.
1496    * @param b If true, enable balancer. If false, disable balancer.
1497    * @return Previous balancer value
1498    * @deprecated use setBalancerRunning(boolean, boolean) instead
1499    */
1500   @Deprecated
1501   public boolean balanceSwitch(final boolean b)
1502   throws MasterNotRunningException, ZooKeeperConnectionException {
1503     return getMaster().balanceSwitch(b);
1504   }
1505 
1506   /**
1507    * Turn the load balancer on or off.
1508    * @param on If true, enable balancer. If false, disable balancer.
1509    * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
1510    * @return Previous balancer value
1511    */
1512   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
1513   throws MasterNotRunningException, ZooKeeperConnectionException {
1514     if (synchronous && synchronousBalanceSwitchSupported) {
1515       try {
1516         return getMaster().synchronousBalanceSwitch(on);
1517       } catch (UndeclaredThrowableException ute) {
1518         String error = ute.getCause().getMessage();
1519         if (error != null && error.matches(
1520             "(?s).+NoSuchMethodException:.+synchronousBalanceSwitch.+")) {
1521           LOG.info("HMaster doesn't support synchronousBalanceSwitch");
1522           synchronousBalanceSwitchSupported = false;
1523         } else {
1524           throw ute;
1525         }
1526       }
1527     }
1528     return balanceSwitch(on);
1529   }
1530 
1531   /**
1532    * Invoke the balancer.  Will run the balancer and if regions to move, it will
1533    * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
1534    * logs.
1535    * @return True if balancer ran, false otherwise.
1536    */
1537   public boolean balancer()
1538   throws MasterNotRunningException, ZooKeeperConnectionException {
1539     return getMaster().balance();
1540   }
1541 
1542   /**
1543    * Split a table or an individual region.
1544    * Asynchronous operation.
1545    *
1546    * @param tableNameOrRegionName table or region to split
1547    * @throws IOException if a remote or network exception occurs
1548    * @throws InterruptedException
1549    */
1550   public void split(final String tableNameOrRegionName)
1551   throws IOException, InterruptedException {
1552     split(Bytes.toBytes(tableNameOrRegionName));
1553   }
1554 
1555   /**
1556    * Split a table or an individual region.  Implicitly finds an optimal split
1557    * point.  Asynchronous operation.
1558    *
1559    * @param tableNameOrRegionName table to region to split
1560    * @throws IOException if a remote or network exception occurs
1561    * @throws InterruptedException
1562    */
1563   public void split(final byte [] tableNameOrRegionName)
1564   throws IOException, InterruptedException {
1565     split(tableNameOrRegionName, null);
1566   }
1567 
1568   public void split(final String tableNameOrRegionName,
1569     final String splitPoint) throws IOException, InterruptedException {
1570     split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
1571   }
1572 
1573   /**
1574    * Split a table or an individual region.
1575    * Asynchronous operation.
1576    *
1577    * @param tableNameOrRegionName table to region to split
1578    * @param splitPoint the explicit position to split on
1579    * @throws IOException if a remote or network exception occurs
1580    * @throws InterruptedException interrupt exception occurred
1581    */
1582   public void split(final byte [] tableNameOrRegionName,
1583       final byte [] splitPoint) throws IOException, InterruptedException {
1584     CatalogTracker ct = getCatalogTracker();
1585     try {
1586       Pair<HRegionInfo, ServerName> regionServerPair
1587         = getRegion(tableNameOrRegionName, ct);
1588       if (regionServerPair != null) {
1589         if (regionServerPair.getSecond() == null) {
1590             throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1591         } else {
1592           split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
1593         }
1594       } else {
1595         final String tableName = tableNameString(tableNameOrRegionName, ct);
1596         List<Pair<HRegionInfo, ServerName>> pairs =
1597           MetaReader.getTableRegionsAndLocations(ct,
1598               tableName);
1599         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1600           // May not be a server for a particular row
1601           if (pair.getSecond() == null) continue;
1602           HRegionInfo r = pair.getFirst();
1603           // check for parents
1604           if (r.isSplitParent()) continue;
1605           // if a split point given, only split that particular region
1606           if (splitPoint != null && !r.containsRow(splitPoint)) continue;
1607           // call out to region server to do split now
1608           split(pair.getSecond(), pair.getFirst(), splitPoint);
1609         }
1610       }
1611     } finally {
1612       cleanupCatalogTracker(ct);
1613     }
1614   }
1615 
1616   private void split(final ServerName sn, final HRegionInfo hri,
1617       byte[] splitPoint) throws IOException {
1618     HRegionInterface rs =
1619       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1620     rs.splitRegion(hri, splitPoint);
1621   }
1622 
1623   /**
1624    * Modify an existing table, more IRB friendly version.
1625    * Asynchronous operation.  This means that it may be a while before your
1626    * schema change is updated across all of the table.
1627    *
1628    * @param tableName name of table.
1629    * @param htd modified description of the table
1630    * @throws IOException if a remote or network exception occurs
1631    */
1632   public void modifyTable(final byte [] tableName, HTableDescriptor htd)
1633   throws IOException {
1634     try {
1635       getMaster().modifyTable(tableName, htd);
1636     } catch (RemoteException re) {
1637       // Convert RE exceptions in here; client shouldn't have to deal with them,
1638       // at least w/ the type of exceptions that come out of this method:
1639       // TableNotFoundException, etc.
1640       throw RemoteExceptionHandler.decodeRemoteException(re);
1641     }
1642   }
1643 
1644   /**
1645    * @param tableNameOrRegionName Name of a table or name of a region.
1646    * @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
1647    * @return a pair of HRegionInfo and ServerName if <code>tableNameOrRegionName</code> is
1648    *  a verified region name (we call {@link  MetaReader#getRegion( CatalogTracker, byte[])}
1649    *  else null.
1650    * Throw an exception if <code>tableNameOrRegionName</code> is null.
1651    * @throws IOException
1652    */
1653   Pair<HRegionInfo, ServerName> getRegion(final byte[] tableNameOrRegionName,
1654       final CatalogTracker ct) throws IOException {
1655     if (tableNameOrRegionName == null) {
1656       throw new IllegalArgumentException("Pass a table name or region name");
1657     }
1658     Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, tableNameOrRegionName);
1659     if (pair == null) {
1660       final AtomicReference<Pair<HRegionInfo, ServerName>> result =
1661         new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
1662       final String encodedName = Bytes.toString(tableNameOrRegionName);
1663       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
1664         @Override
1665         public boolean processRow(Result data) throws IOException {
1666           if (data == null || data.size() <= 0) {
1667             return true;
1668           }
1669           HRegionInfo info = MetaReader.parseHRegionInfoFromCatalogResult(
1670             data, HConstants.REGIONINFO_QUALIFIER);
1671           if (info == null) {
1672             LOG.warn("No serialized HRegionInfo in " + data);
1673             return true;
1674           }
1675           if (!encodedName.equals(info.getEncodedName())) return true;
1676           ServerName sn = MetaReader.getServerNameFromCatalogResult(data);
1677           result.set(new Pair<HRegionInfo, ServerName>(info, sn));
1678           return false; // found the region, stop
1679         }
1680       };
1681 
1682       MetaScanner.metaScan(conf, visitor);
1683       pair = result.get();
1684     }
1685     return pair;
1686   }
1687 
1688   /**
1689    * Convert the table name byte array into a table name string and check if table
1690    * exists or not.
1691    * @param tableNameBytes Name of a table.
1692    * @param ct A {@link #CatalogTracker} instance (caller of this method usually has one).
1693    * @return tableName in string form.
1694    * @throws IOException if a remote or network exception occurs.
1695    * @throws TableNotFoundException if table does not exist.
1696    */
1697   private String tableNameString(final byte[] tableNameBytes, CatalogTracker ct)
1698       throws IOException {
1699     String tableNameString = Bytes.toString(tableNameBytes);
1700     if (!MetaReader.tableExists(ct, tableNameString)) {
1701       throw new TableNotFoundException(tableNameString);
1702     }
1703     return tableNameString;
1704   }
1705 
1706   /**
1707    * Shuts down the HBase cluster
1708    * @throws IOException if a remote or network exception occurs
1709    */
1710   public synchronized void shutdown() throws IOException {
1711     isMasterRunning();
1712     try {
1713       getMaster().shutdown();
1714     } catch (RemoteException e) {
1715       throw RemoteExceptionHandler.decodeRemoteException(e);
1716     }
1717   }
1718 
1719   /**
1720    * Shuts down the current HBase master only.
1721    * Does not shutdown the cluster.
1722    * @see #shutdown()
1723    * @throws IOException if a remote or network exception occurs
1724    */
1725   public synchronized void stopMaster() throws IOException {
1726     isMasterRunning();
1727     try {
1728       getMaster().stopMaster();
1729     } catch (RemoteException e) {
1730       throw RemoteExceptionHandler.decodeRemoteException(e);
1731     }
1732   }
1733 
1734   /**
1735    * Stop the designated regionserver
1736    * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
1737    * <code>example.org:1234</code>
1738    * @throws IOException if a remote or network exception occurs
1739    */
1740   public synchronized void stopRegionServer(final String hostnamePort)
1741   throws IOException {
1742     String hostname = Addressing.parseHostname(hostnamePort);
1743     int port = Addressing.parsePort(hostnamePort);
1744     HRegionInterface rs =
1745       this.connection.getHRegionConnection(hostname, port);
1746     rs.stop("Called by admin client " + this.connection.toString());
1747   }
1748 
1749   /**
1750    * @return cluster status
1751    * @throws IOException if a remote or network exception occurs
1752    */
1753   public ClusterStatus getClusterStatus() throws IOException {
1754     return getMaster().getClusterStatus();
1755   }
1756 
1757   private HRegionLocation getFirstMetaServerForTable(final byte [] tableName)
1758   throws IOException {
1759     return connection.locateRegion(HConstants.META_TABLE_NAME,
1760       HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
1761   }
1762 
1763   /**
1764    * @return Configuration used by the instance.
1765    */
1766   public Configuration getConfiguration() {
1767     return this.conf;
1768   }
1769 
1770   /**
1771    * Check to see if HBase is running. Throw an exception if not.
1772    *
1773    * @param conf system configuration
1774    * @throws MasterNotRunningException if the master is not running
1775    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
1776    */
1777   public static void checkHBaseAvailable(Configuration conf)
1778   throws MasterNotRunningException, ZooKeeperConnectionException {
1779     Configuration copyOfConf = HBaseConfiguration.create(conf);
1780     copyOfConf.setInt("hbase.client.retries.number", 1);
1781     HBaseAdmin admin = new HBaseAdmin(copyOfConf);
1782     try {
1783       admin.close();
1784     } catch (IOException ioe) {
1785       admin.LOG.info("Failed to close connection", ioe);
1786     }
1787   }
1788 
1789   /**
1790    * get the regions of a given table.
1791    *
1792    * @param tableName the name of the table
1793    * @return Ordered list of {@link HRegionInfo}.
1794    * @throws IOException
1795    */
1796   public List<HRegionInfo> getTableRegions(final byte[] tableName)
1797   throws IOException {
1798     CatalogTracker ct = getCatalogTracker();
1799     List<HRegionInfo> Regions = null;
1800     try {
1801       Regions = MetaReader.getTableRegions(ct, tableName, true);
1802     } finally {
1803       cleanupCatalogTracker(ct);
1804     }
1805     return Regions;
1806   }
1807 
1808   public void close() throws IOException {
1809     if (this.connection != null) {
1810       this.connection.close();
1811     }
1812   }
1813 
1814  /**
1815  * Get tableDescriptors
1816  * @param tableNames List of table names
1817  * @return HTD[] the tableDescriptor
1818  * @throws IOException if a remote or network exception occurs
1819  */
1820   public HTableDescriptor[] getTableDescriptors(List<String> tableNames)
1821   throws IOException {
1822     return this.connection.getHTableDescriptors(tableNames);
1823   }
1824 
1825   /**
1826    * Roll the log writer. That is, start writing log messages to a new file.
1827    *
1828    * @param serverName
1829    *          The servername of the regionserver. A server name is made of host,
1830    *          port and startcode. This is mandatory. Here is an example:
1831    *          <code> host187.example.com,60020,1289493121758</code>
1832    * @return If lots of logs, flush the returned regions so next time through
1833    * we can clean logs. Returns null if nothing to flush.  Names are actual
1834    * region names as returned by {@link HRegionInfo#getEncodedName()}
1835    * @throws IOException if a remote or network exception occurs
1836    * @throws FailedLogCloseException
1837    */
1838  public synchronized  byte[][] rollHLogWriter(String serverName)
1839       throws IOException, FailedLogCloseException {
1840     ServerName sn = new ServerName(serverName);
1841     HRegionInterface rs = this.connection.getHRegionConnection(
1842         sn.getHostname(), sn.getPort());
1843     return rs.rollHLogWriter();
1844   }
1845 
1846   public String[] getMasterCoprocessors() {
1847     try {
1848       return getClusterStatus().getMasterCoprocessors();
1849     } catch (IOException e) {
1850       LOG.error("Could not getClusterStatus()",e);
1851       return null;
1852     }
1853   }
1854 
1855   /**
1856    * Get the current compaction state of a table or region.
1857    * It could be in a major compaction, a minor compaction, both, or none.
1858    *
1859    * @param tableNameOrRegionName table or region to major compact
1860    * @throws IOException if a remote or network exception occurs
1861    * @throws InterruptedException
1862    * @return the current compaction state
1863    */
1864   public CompactionState getCompactionState(final String tableNameOrRegionName)
1865       throws IOException, InterruptedException {
1866     return getCompactionState(Bytes.toBytes(tableNameOrRegionName));
1867   }
1868 
1869   /**
1870    * Get the current compaction state of a table or region.
1871    * It could be in a major compaction, a minor compaction, both, or none.
1872    *
1873    * @param tableNameOrRegionName table or region to major compact
1874    * @throws IOException if a remote or network exception occurs
1875    * @throws InterruptedException
1876    * @return the current compaction state
1877    */
1878   public CompactionState getCompactionState(final byte [] tableNameOrRegionName)
1879       throws IOException, InterruptedException {
1880     CompactionState state = CompactionState.NONE;
1881     CatalogTracker ct = getCatalogTracker();
1882     try {
1883       Pair<HRegionInfo, ServerName> regionServerPair
1884         = getRegion(tableNameOrRegionName, ct);
1885       if (regionServerPair != null) {
1886         if (regionServerPair.getSecond() == null) {
1887           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1888         } else {
1889           ServerName sn = regionServerPair.getSecond();
1890           HRegionInterface rs =
1891             this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1892           return CompactionState.valueOf(
1893             rs.getCompactionState(regionServerPair.getFirst().getRegionName()));
1894         }
1895       } else {
1896         final String tableName = tableNameString(tableNameOrRegionName, ct);
1897         List<Pair<HRegionInfo, ServerName>> pairs =
1898           MetaReader.getTableRegionsAndLocations(ct, tableName);
1899         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1900           if (pair.getFirst().isOffline()) continue;
1901           if (pair.getSecond() == null) continue;
1902           try {
1903             ServerName sn = pair.getSecond();
1904             HRegionInterface rs =
1905               this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1906             switch (CompactionState.valueOf(
1907               rs.getCompactionState(pair.getFirst().getRegionName()))) {
1908             case MAJOR_AND_MINOR:
1909               return CompactionState.MAJOR_AND_MINOR;
1910             case MAJOR:
1911               if (state == CompactionState.MINOR) {
1912                 return CompactionState.MAJOR_AND_MINOR;
1913               }
1914               state = CompactionState.MAJOR;
1915               break;
1916             case MINOR:
1917               if (state == CompactionState.MAJOR) {
1918                 return CompactionState.MAJOR_AND_MINOR;
1919               }
1920               state = CompactionState.MINOR;
1921               break;
1922             case NONE:
1923               default: // nothing, continue
1924             }
1925           } catch (NotServingRegionException e) {
1926             if (LOG.isDebugEnabled()) {
1927               LOG.debug("Trying to get compaction state of " +
1928                 pair.getFirst() + ": " +
1929                 StringUtils.stringifyException(e));
1930             }
1931           }
1932         }
1933       }
1934     } finally {
1935       cleanupCatalogTracker(ct);
1936     }
1937     return state;
1938   }
1939 
1940   /**
1941    * Creates and returns a proxy to the CoprocessorProtocol instance running in the
1942    * master.
1943    *
1944    * @param protocol The class or interface defining the remote protocol
1945    * @return A CoprocessorProtocol instance
1946    */
1947   public <T extends CoprocessorProtocol> T coprocessorProxy(
1948       Class<T> protocol) {
1949     return (T) Proxy.newProxyInstance(this.getClass().getClassLoader(),
1950         new Class[]{protocol},
1951         new MasterExecRPCInvoker(conf,
1952             connection,
1953             protocol));
1954   }
1955 
1956 
1957   /**
1958    * Create a timestamp consistent snapshot for the given table.
1959    * <p>
1960    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
1961    * snapshot with the same name (even a different type or with different parameters) will fail with
1962    * a {@link SnapshotCreationException} indicating the duplicate naming.
1963    * <p>
1964    * Snapshot names follow the same naming constraints as tables in HBase. See
1965    * {@link HTableDescriptor#isLegalTableName(byte[])}.
1966    * @param snapshotName name of the snapshot to be created
1967    * @param tableName name of the table for which snapshot is created
1968    * @throws IOException if a remote or network exception occurs
1969    * @throws SnapshotCreationException if snapshot creation failed
1970    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
1971    */
1972   public void snapshot(final String snapshotName, final String tableName) throws IOException,
1973       SnapshotCreationException, IllegalArgumentException {
1974     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
1975   }
1976 
1977   /**
1978    * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
1979    * taken. If the table is disabled, an offline snapshot is taken.
1980    * <p>
1981    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
1982    * snapshot with the same name (even a different type or with different parameters) will fail with
1983    * a {@link SnapshotCreationException} indicating the duplicate naming.
1984    * <p>
1985    * Snapshot names follow the same naming constraints as tables in HBase. See
1986    * {@link HTableDescriptor#isLegalTableName(byte[])}.
1987    * @param snapshotName name of the snapshot to be created
1988    * @param tableName name of the table for which snapshot is created
1989    * @throws IOException if a remote or network exception occurs
1990    * @throws SnapshotCreationException if snapshot creation failed
1991    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
1992    */
1993   public void snapshot(final byte[] snapshotName, final byte[] tableName) throws IOException,
1994       SnapshotCreationException, IllegalArgumentException {
1995     snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
1996   }
1997 
1998   /**
1999    * Create typed snapshot of the table.
2000    * <p>
2001    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2002    * snapshot with the same name (even a different type or with different parameters) will fail with
2003    * a {@link SnapshotCreationException} indicating the duplicate naming.
2004    * <p>
2005    * Snapshot names follow the same naming constraints as tables in HBase. See
2006    * {@link HTableDescriptor#isLegalTableName(byte[])}.
2007    * <p>
2008    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
2009    * snapshots stored on the cluster
2010    * @param tableName name of the table to snapshot
2011    * @param type type of snapshot to take
2012    * @throws IOException we fail to reach the master
2013    * @throws SnapshotCreationException if snapshot creation failed
2014    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2015    */
2016   public void snapshot(final String snapshotName, final String tableName,
2017       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2018       IllegalArgumentException {
2019     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
2020     builder.setTable(tableName);
2021     builder.setName(snapshotName);
2022     builder.setType(type);
2023     snapshot(builder.build());
2024   }
2025 
2026   /**
2027    * Take a snapshot and wait for the server to complete that snapshot (blocking).
2028    * <p>
2029    * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
2030    * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
2031    * time for a single cluster).
2032    * <p>
2033    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2034    * snapshot with the same name (even a different type or with different parameters) will fail with
2035    * a {@link SnapshotCreationException} indicating the duplicate naming.
2036    * <p>
2037    * Snapshot names follow the same naming constraints as tables in HBase. See
2038    * {@link HTableDescriptor#isLegalTableName(byte[])}.
2039    * <p>
2040    * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
2041    * unless you are sure about the type of snapshot that you want to take.
2042    * @param snapshot snapshot to take
2043    * @throws IOException or we lose contact with the master.
2044    * @throws SnapshotCreationException if snapshot failed to be taken
2045    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2046    */
2047   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
2048       IllegalArgumentException {
2049     HSnapshotDescription snapshotWritable = new HSnapshotDescription(snapshot);
2050 
2051     try {
2052       // actually take the snapshot
2053       long max = takeSnapshotAsync(snapshot);
2054       long start = EnvironmentEdgeManager.currentTimeMillis();
2055       long maxPauseTime = max / this.numRetries;
2056       boolean done = false;
2057       int tries = 0;
2058       LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
2059           SnapshotDescriptionUtils.toString(snapshot) + "' to complete. (max " +
2060           maxPauseTime + " ms per retry)");
2061       while (tries == 0 || (EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done) {
2062         try {
2063           // sleep a backoff <= pauseTime amount
2064           long sleep = getPauseTime(tries++);
2065           sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2066           LOG.debug("(#" + tries + ") Sleeping: " + sleep +
2067             "ms while waiting for snapshot completion.");
2068           Thread.sleep(sleep);
2069 
2070         } catch (InterruptedException e) {
2071           LOG.debug("Interrupted while waiting for snapshot " + snapshot + " to complete");
2072           Thread.currentThread().interrupt();
2073         }
2074         LOG.debug("Getting current status of snapshot from master...");
2075         done = getMaster().isSnapshotDone(snapshotWritable);
2076       }
2077 
2078       if (!done) {
2079         throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
2080             + "' wasn't completed in expectedTime:" + max + " ms", snapshot);
2081       }
2082     } catch (RemoteException e) {
2083       throw RemoteExceptionHandler.decodeRemoteException(e);
2084     }
2085   }
2086 
2087   /**
2088    * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
2089    * <p>
2090    * Only a single snapshot should be taken at a time, or results may be undefined.
2091    * @param snapshot snapshot to take
2092    * @return the max time in millis to wait for the snapshot
2093    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
2094    * @throws SnapshotCreationException if snapshot creation failed
2095    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2096    */
2097   public long takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
2098       SnapshotCreationException {
2099     SnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
2100     HSnapshotDescription snapshotWritable = new HSnapshotDescription(snapshot);
2101     return getMaster().snapshot(snapshotWritable);
2102   }
2103 
2104   /**
2105    * Check the current state of the passed snapshot.
2106    * <p>
2107    * There are three possible states:
2108    * <ol>
2109    * <li>running - returns <tt>false</tt></li>
2110    * <li>finished - returns <tt>true</tt></li>
2111    * <li>finished with error - throws the exception that caused the snapshot to fail</li>
2112    * </ol>
2113    * <p>
2114    * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
2115    * run/started since the snapshot your are checking, you will recieve an
2116    * {@link UnknownSnapshotException}.
2117    * @param snapshot description of the snapshot to check
2118    * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
2119    * running
2120    * @throws IOException if we have a network issue
2121    * @throws HBaseSnapshotException if the snapshot failed
2122    * @throws UnknownSnapshotException if the requested snapshot is unknown
2123    */
2124   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
2125       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
2126     try {
2127       return getMaster().isSnapshotDone(new HSnapshotDescription(snapshot));
2128     } catch (RemoteException e) {
2129       throw RemoteExceptionHandler.decodeRemoteException(e);
2130     }
2131   }
2132 
2133   /**
2134    * Restore the specified snapshot on the original table. (The table must be disabled)
2135    * Before restoring the table, a new snapshot with the current table state is created.
2136    * In case of failure, the table will be rolled back to the its original state.
2137    *
2138    * @param snapshotName name of the snapshot to restore
2139    * @throws IOException if a remote or network exception occurs
2140    * @throws RestoreSnapshotException if snapshot failed to be restored
2141    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2142    */
2143   public void restoreSnapshot(final byte[] snapshotName)
2144       throws IOException, RestoreSnapshotException {
2145     restoreSnapshot(Bytes.toString(snapshotName));
2146   }
2147 
2148   /**
2149    * Restore the specified snapshot on the original table. (The table must be disabled)
2150    * Before restoring the table, a new snapshot with the current table state is created.
2151    * In case of failure, the table will be rolled back to its original state.
2152    *
2153    * @param snapshotName name of the snapshot to restore
2154    * @throws IOException if a remote or network exception occurs
2155    * @throws RestoreSnapshotException if snapshot failed to be restored
2156    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2157    */
2158   public void restoreSnapshot(final String snapshotName)
2159       throws IOException, RestoreSnapshotException {
2160     String rollbackSnapshot = snapshotName + "-" + EnvironmentEdgeManager.currentTimeMillis();
2161 
2162     String tableName = null;
2163     for (SnapshotDescription snapshotInfo: listSnapshots()) {
2164       if (snapshotInfo.getName().equals(snapshotName)) {
2165         tableName = snapshotInfo.getTable();
2166         break;
2167       }
2168     }
2169 
2170     if (tableName == null) {
2171       throw new RestoreSnapshotException(
2172         "Unable to find the table name for snapshot=" + snapshotName);
2173     }
2174 
2175     // Take a snapshot of the current state
2176     snapshot(rollbackSnapshot, tableName);
2177 
2178     // Restore snapshot
2179     try {
2180       internalRestoreSnapshot(snapshotName, tableName);
2181     } catch (IOException e) {
2182       // Try to rollback
2183       try {
2184         String msg = "Restore snapshot=" + snapshotName +
2185           " failed. Rollback to snapshot=" + rollbackSnapshot + " succeeded.";
2186         LOG.error(msg, e);
2187         internalRestoreSnapshot(rollbackSnapshot, tableName);
2188         throw new RestoreSnapshotException(msg, e);
2189       } catch (IOException ex) {
2190         String msg = "Failed to restore and rollback to snapshot=" + rollbackSnapshot;
2191         LOG.error(msg, ex);
2192         throw new RestoreSnapshotException(msg, ex);
2193       }
2194     }
2195   }
2196 
2197   /**
2198    * Create a new table by cloning the snapshot content.
2199    *
2200    * @param snapshotName name of the snapshot to be cloned
2201    * @param tableName name of the table where the snapshot will be restored
2202    * @throws IOException if a remote or network exception occurs
2203    * @throws TableExistsException if table to be created already exists
2204    * @throws RestoreSnapshotException if snapshot failed to be cloned
2205    * @throws IllegalArgumentException if the specified table has not a valid name
2206    */
2207   public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
2208       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2209     cloneSnapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
2210   }
2211 
2212   /**
2213    * Create a new table by cloning the snapshot content.
2214    *
2215    * @param snapshotName name of the snapshot to be cloned
2216    * @param tableName name of the table where the snapshot will be restored
2217    * @throws IOException if a remote or network exception occurs
2218    * @throws TableExistsException if table to be created already exists
2219    * @throws RestoreSnapshotException if snapshot failed to be cloned
2220    * @throws IllegalArgumentException if the specified table has not a valid name
2221    */
2222   public void cloneSnapshot(final String snapshotName, final String tableName)
2223       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2224     if (tableExists(tableName)) {
2225       throw new TableExistsException("Table '" + tableName + " already exists");
2226     }
2227     internalRestoreSnapshot(snapshotName, tableName);
2228     waitUntilTableIsEnabled(Bytes.toBytes(tableName));
2229   }
2230 
2231   /**
2232    * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
2233    * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
2234    * create an HTable instance to this table before it is available.
2235    * @param snapshot snapshot to restore
2236    * @param tableName table name to restore the snapshot on
2237    * @throws IOException if a remote or network exception occurs
2238    * @throws RestoreSnapshotException if snapshot failed to be restored
2239    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2240    */
2241   private void internalRestoreSnapshot(final String snapshotName, final String tableName)
2242       throws IOException, RestoreSnapshotException {
2243     HSnapshotDescription snapshot = new HSnapshotDescription(
2244       SnapshotDescription.newBuilder().setName(snapshotName).setTable(tableName).build());
2245 
2246     try {
2247       // actually restore the snapshot
2248       getMaster().restoreSnapshot(snapshot);
2249 
2250       final long maxPauseTime = 5000;
2251       boolean done = false;
2252       int tries = 0;
2253       while (!done) {
2254         try {
2255           // sleep a backoff <= pauseTime amount
2256           long sleep = getPauseTime(tries++);
2257           sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2258           LOG.debug(tries + ") Sleeping: " + sleep + " ms while we wait for snapshot restore to complete.");
2259           Thread.sleep(sleep);
2260         } catch (InterruptedException e) {
2261           LOG.debug("Interrupted while waiting for snapshot " + snapshot + " restore to complete");
2262           Thread.currentThread().interrupt();
2263         }
2264         LOG.debug("Getting current status of snapshot restore from master...");
2265         done = getMaster().isRestoreSnapshotDone(snapshot);
2266       }
2267       if (!done) {
2268         throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
2269       }
2270     } catch (RemoteException e) {
2271       throw RemoteExceptionHandler.decodeRemoteException(e);
2272     }
2273   }
2274 
2275   /**
2276    * List completed snapshots.
2277    * @return a list of snapshot descriptors for completed snapshots
2278    * @throws IOException if a network error occurs
2279    */
2280   public List<SnapshotDescription> listSnapshots() throws IOException {
2281     List<SnapshotDescription> snapshots = new LinkedList<SnapshotDescription>();
2282     try {
2283       for (HSnapshotDescription snapshot: getMaster().getCompletedSnapshots()) {
2284         snapshots.add(snapshot.getProto());
2285       }
2286     } catch (RemoteException e) {
2287       throw RemoteExceptionHandler.decodeRemoteException(e);
2288     }
2289     return snapshots;
2290   }
2291 
2292   /**
2293    * Delete an existing snapshot.
2294    * @param snapshotName name of the snapshot
2295    * @throws IOException if a remote or network exception occurs
2296    */
2297   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
2298     // make sure the snapshot is possibly valid
2299     HTableDescriptor.isLegalTableName(snapshotName);
2300     // do the delete
2301     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
2302       .setName(Bytes.toString(snapshotName)).build();
2303     try {
2304       getMaster().deleteSnapshot(new HSnapshotDescription(snapshot));
2305     } catch (RemoteException e) {
2306       throw RemoteExceptionHandler.decodeRemoteException(e);
2307     }
2308   }
2309 
2310   /**
2311    * Delete an existing snapshot.
2312    * @param snapshotName name of the snapshot
2313    * @throws IOException if a remote or network exception occurs
2314    */
2315   public void deleteSnapshot(final String snapshotName) throws IOException {
2316     deleteSnapshot(Bytes.toBytes(snapshotName));
2317   }
2318 }