View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.client;
20  
21  import java.io.Closeable;
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.net.SocketTimeoutException;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.LinkedList;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.Map.Entry;
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.classification.InterfaceAudience;
38  import org.apache.hadoop.classification.InterfaceStability;
39  import org.apache.hadoop.conf.Configuration;
40  import org.apache.hadoop.hbase.Abortable;
41  import org.apache.hadoop.hbase.ClusterStatus;
42  import org.apache.hadoop.hbase.HBaseConfiguration;
43  import org.apache.hadoop.hbase.HBaseIOException;
44  import org.apache.hadoop.hbase.HColumnDescriptor;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.HRegionInfo;
47  import org.apache.hadoop.hbase.HRegionLocation;
48  import org.apache.hadoop.hbase.HTableDescriptor;
49  import org.apache.hadoop.hbase.MasterNotRunningException;
50  import org.apache.hadoop.hbase.NamespaceDescriptor;
51  import org.apache.hadoop.hbase.NotServingRegionException;
52  import org.apache.hadoop.hbase.RegionException;
53  import org.apache.hadoop.hbase.ServerName;
54  import org.apache.hadoop.hbase.TableExistsException;
55  import org.apache.hadoop.hbase.TableName;
56  import org.apache.hadoop.hbase.TableNotDisabledException;
57  import org.apache.hadoop.hbase.TableNotEnabledException;
58  import org.apache.hadoop.hbase.TableNotFoundException;
59  import org.apache.hadoop.hbase.UnknownRegionException;
60  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
61  import org.apache.hadoop.hbase.catalog.CatalogTracker;
62  import org.apache.hadoop.hbase.catalog.MetaReader;
63  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
64  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
65  import org.apache.hadoop.hbase.exceptions.DeserializationException;
66  import org.apache.hadoop.hbase.exceptions.MergeRegionException;
67  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
68  import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
69  import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
70  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
71  import org.apache.hadoop.hbase.protobuf.RequestConverter;
72  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
73  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
74  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
75  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
76  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
77  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
78  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
79  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
80  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
81  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
82  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
83  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
84  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
85  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
86  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
87  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
88  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
89  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
90  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
91  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
92  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
93  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
94  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
95  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
96  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
97  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
98  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
99  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
100 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
101 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
102 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
103 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
104 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
105 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
106 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
107 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
108 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
109 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
110 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
111 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
112 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
113 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
114 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
115 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
116 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
117 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
118 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
119 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
121 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
122 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
123 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
124 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
125 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
126 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
127 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
128 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
129 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
130 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
131 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
132 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
133 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
134 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
135 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
136 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
137 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
138 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
139 import org.apache.hadoop.hbase.util.Addressing;
140 import org.apache.hadoop.hbase.util.Bytes;
141 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
142 import org.apache.hadoop.hbase.util.Pair;
143 import org.apache.hadoop.ipc.RemoteException;
144 import org.apache.hadoop.util.StringUtils;
145 import org.apache.zookeeper.KeeperException;
146 
147 import com.google.protobuf.ByteString;
148 import com.google.protobuf.ServiceException;
149 
150 /**
151  * Provides an interface to manage HBase database table metadata + general
152  * administrative functions.  Use HBaseAdmin to create, drop, list, enable and
153  * disable tables. Use it also to add and drop table column families.
154  *
155  * <p>See {@link HTable} to add, update, and delete data from an individual table.
156  * <p>Currently HBaseAdmin instances are not expected to be long-lived.  For
157  * example, an HBaseAdmin instance will not ride over a Master restart.
158  */
159 @InterfaceAudience.Public
160 @InterfaceStability.Evolving
161 public class HBaseAdmin implements Abortable, Closeable {
162   private static final Log LOG = LogFactory.getLog(HBaseAdmin.class);
163 
164   // We use the implementation class rather then the interface because we
165   //  need the package protected functions to get the connection to master
166   private ClusterConnection connection;
167 
168   private volatile Configuration conf;
169   private final long pause;
170   private final int numRetries;
171   // Some operations can take a long time such as disable of big table.
172   // numRetries is for 'normal' stuff... Multiply by this factor when
173   // want to wait a long time.
174   private final int retryLongerMultiplier;
175   private boolean aborted;
176   private boolean cleanupConnectionOnClose = false; // close the connection in close()
177   private boolean closed = false;
178   private int operationTimeout;
179 
180   private RpcRetryingCallerFactory rpcCallerFactory;
181 
182   /**
183    * Constructor.
184    * See {@link #HBaseAdmin(HConnection connection)}
185    *
186    * @param c Configuration object. Copied internally.
187    */
188   public HBaseAdmin(Configuration c)
189   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
190     // Will not leak connections, as the new implementation of the constructor
191     // does not throw exceptions anymore.
192     this(ConnectionManager.getConnectionInternal(new Configuration(c)));
193     this.cleanupConnectionOnClose = true;
194   }
195 
196   public int getOperationTimeout() {
197     return operationTimeout;
198   }
199 
200 
201   /**
202    * Constructor for externally managed HConnections.
203    * The connection to master will be created when required by admin functions.
204    *
205    * @param connection The HConnection instance to use
206    * @throws MasterNotRunningException, ZooKeeperConnectionException are not
207    *  thrown anymore but kept into the interface for backward api compatibility
208    * @deprecated Do not use this internal ctor.
209    */
210   @Deprecated
211   public HBaseAdmin(HConnection connection)
212       throws MasterNotRunningException, ZooKeeperConnectionException {
213     this((ClusterConnection)connection);
214   }
215 
216   HBaseAdmin(ClusterConnection connection) {
217     this.conf = connection.getConfiguration();
218     this.connection = connection;
219 
220     this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
221         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
222     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
223         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
224     this.retryLongerMultiplier = this.conf.getInt(
225         "hbase.client.retries.longer.multiplier", 10);
226     this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
227         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
228 
229     this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
230   }
231 
232   /**
233    * @return A new CatalogTracker instance; call {@link #cleanupCatalogTracker(CatalogTracker)}
234    * to cleanup the returned catalog tracker.
235    * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
236    * @throws IOException
237    * @see #cleanupCatalogTracker(CatalogTracker)
238    */
239   private synchronized CatalogTracker getCatalogTracker()
240   throws ZooKeeperConnectionException, IOException {
241     CatalogTracker ct = null;
242     try {
243       ct = new CatalogTracker(this.conf);
244       ct.start();
245     } catch (InterruptedException e) {
246       // Let it out as an IOE for now until we redo all so tolerate IEs
247       throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
248     }
249     return ct;
250   }
251 
252   private void cleanupCatalogTracker(final CatalogTracker ct) {
253     ct.stop();
254   }
255 
256   @Override
257   public void abort(String why, Throwable e) {
258     // Currently does nothing but throw the passed message and exception
259     this.aborted = true;
260     throw new RuntimeException(why, e);
261   }
262 
263   @Override
264   public boolean isAborted(){
265     return this.aborted;
266   }
267 
268   /** @return HConnection used by this object. */
269   public HConnection getConnection() {
270     return connection;
271   }
272 
273   /** @return - true if the master server is running. Throws an exception
274    *  otherwise.
275    * @throws ZooKeeperConnectionException
276    * @throws MasterNotRunningException
277    */
278   public boolean isMasterRunning()
279   throws MasterNotRunningException, ZooKeeperConnectionException {
280     return connection.isMasterRunning();
281   }
282 
283   /**
284    * @param tableName Table to check.
285    * @return True if table exists already.
286    * @throws IOException
287    */
288   public boolean tableExists(final TableName tableName)
289   throws IOException {
290     boolean b = false;
291     CatalogTracker ct = getCatalogTracker();
292     try {
293       b = MetaReader.tableExists(ct, tableName);
294     } finally {
295       cleanupCatalogTracker(ct);
296     }
297     return b;
298   }
299 
300   public boolean tableExists(final byte[] tableName)
301   throws IOException {
302     return tableExists(TableName.valueOf(tableName));
303   }
304 
305   public boolean tableExists(final String tableName)
306   throws IOException {
307     return tableExists(TableName.valueOf(tableName));
308   }
309 
310   /**
311    * List all the userspace tables.  In other words, scan the hbase:meta table.
312    *
313    * If we wanted this to be really fast, we could implement a special
314    * catalog table that just contains table names and their descriptors.
315    * Right now, it only exists as part of the hbase:meta table's region info.
316    *
317    * @return - returns an array of HTableDescriptors
318    * @throws IOException if a remote or network exception occurs
319    */
320   public HTableDescriptor[] listTables() throws IOException {
321     return this.connection.listTables();
322   }
323 
324   /**
325    * List all the userspace tables matching the given pattern.
326    *
327    * @param pattern The compiled regular expression to match against
328    * @return - returns an array of HTableDescriptors
329    * @throws IOException if a remote or network exception occurs
330    * @see #listTables()
331    */
332   public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
333     List<HTableDescriptor> matched = new LinkedList<HTableDescriptor>();
334     HTableDescriptor[] tables = listTables();
335     for (HTableDescriptor table : tables) {
336       if (pattern.matcher(table.getTableName().getNameAsString()).matches()) {
337         matched.add(table);
338       }
339     }
340     return matched.toArray(new HTableDescriptor[matched.size()]);
341   }
342 
343   /**
344    * List all the userspace tables matching the given regular expression.
345    *
346    * @param regex The regular expression to match against
347    * @return - returns an array of HTableDescriptors
348    * @throws IOException if a remote or network exception occurs
349    * @see #listTables(java.util.regex.Pattern)
350    */
351   public HTableDescriptor[] listTables(String regex) throws IOException {
352     return listTables(Pattern.compile(regex));
353   }
354 
355   /**
356    * List all of the names of userspace tables.
357    * @return String[] table names
358    * @throws IOException if a remote or network exception occurs
359    */
360   @Deprecated
361   public String[] getTableNames() throws IOException {
362     return this.connection.getTableNames();
363   }
364 
365   /**
366    * List all of the names of userspace tables matching the given regular expression.
367    * @param pattern The regular expression to match against
368    * @return String[] table names
369    * @throws IOException if a remote or network exception occurs
370    */
371   @Deprecated
372   public String[] getTableNames(Pattern pattern) throws IOException {
373     List<String> matched = new ArrayList<String>();
374     for (String name: this.connection.getTableNames()) {
375       if (pattern.matcher(name).matches()) {
376         matched.add(name);
377       }
378     }
379     return matched.toArray(new String[matched.size()]);
380   }
381 
382   /**
383    * List all of the names of userspace tables matching the given regular expression.
384    * @param regex The regular expression to match against
385    * @return String[] table names
386    * @throws IOException if a remote or network exception occurs
387    */
388   @Deprecated
389   public String[] getTableNames(String regex) throws IOException {
390     return getTableNames(Pattern.compile(regex));
391   }
392 
393   /**
394    * List all of the names of userspace tables.
395    * @return TableName[] table names
396    * @throws IOException if a remote or network exception occurs
397    */
398   public TableName[] listTableNames() throws IOException {
399     return this.connection.listTableNames();
400   }
401 
402   /**
403    * Method for getting the tableDescriptor
404    * @param tableName as a byte []
405    * @return the tableDescriptor
406    * @throws TableNotFoundException
407    * @throws IOException if a remote or network exception occurs
408    */
409   public HTableDescriptor getTableDescriptor(final TableName tableName)
410   throws TableNotFoundException, IOException {
411     return this.connection.getHTableDescriptor(tableName);
412   }
413 
414   public HTableDescriptor getTableDescriptor(final byte[] tableName)
415   throws TableNotFoundException, IOException {
416     return getTableDescriptor(TableName.valueOf(tableName));
417   }
418 
419   private long getPauseTime(int tries) {
420     int triesCount = tries;
421     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
422       triesCount = HConstants.RETRY_BACKOFF.length - 1;
423     }
424     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
425   }
426 
427   /**
428    * Creates a new table.
429    * Synchronous operation.
430    *
431    * @param desc table descriptor for table
432    *
433    * @throws IllegalArgumentException if the table name is reserved
434    * @throws MasterNotRunningException if master is not running
435    * @throws TableExistsException if table already exists (If concurrent
436    * threads, the table may have been created between test-for-existence
437    * and attempt-at-creation).
438    * @throws IOException if a remote or network exception occurs
439    */
440   public void createTable(HTableDescriptor desc)
441   throws IOException {
442     createTable(desc, null);
443   }
444 
445   /**
446    * Creates a new table with the specified number of regions.  The start key
447    * specified will become the end key of the first region of the table, and
448    * the end key specified will become the start key of the last region of the
449    * table (the first region has a null start key and the last region has a
450    * null end key).
451    *
452    * BigInteger math will be used to divide the key range specified into
453    * enough segments to make the required number of total regions.
454    *
455    * Synchronous operation.
456    *
457    * @param desc table descriptor for table
458    * @param startKey beginning of key range
459    * @param endKey end of key range
460    * @param numRegions the total number of regions to create
461    *
462    * @throws IllegalArgumentException if the table name is reserved
463    * @throws MasterNotRunningException if master is not running
464    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
465    * threads, the table may have been created between test-for-existence
466    * and attempt-at-creation).
467    * @throws IOException
468    */
469   public void createTable(HTableDescriptor desc, byte [] startKey,
470       byte [] endKey, int numRegions)
471   throws IOException {
472     if(numRegions < 3) {
473       throw new IllegalArgumentException("Must create at least three regions");
474     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
475       throw new IllegalArgumentException("Start key must be smaller than end key");
476     }
477     if (numRegions == 3) {
478       createTable(desc, new byte[][]{startKey, endKey});
479       return;
480     }
481     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
482     if(splitKeys == null || splitKeys.length != numRegions - 1) {
483       throw new IllegalArgumentException("Unable to split key range into enough regions");
484     }
485     createTable(desc, splitKeys);
486   }
487 
488   /**
489    * Creates a new table with an initial set of empty regions defined by the
490    * specified split keys.  The total number of regions created will be the
491    * number of split keys plus one. Synchronous operation.
492    * Note : Avoid passing empty split key.
493    *
494    * @param desc table descriptor for table
495    * @param splitKeys array of split keys for the initial regions of the table
496    *
497    * @throws IllegalArgumentException if the table name is reserved, if the split keys
498    * are repeated and if the split key has empty byte array.
499    * @throws MasterNotRunningException if master is not running
500    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
501    * threads, the table may have been created between test-for-existence
502    * and attempt-at-creation).
503    * @throws IOException
504    */
505   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
506   throws IOException {
507     try {
508       createTableAsync(desc, splitKeys);
509     } catch (SocketTimeoutException ste) {
510       LOG.warn("Creating " + desc.getTableName() + " took too long", ste);
511     }
512     int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
513     int prevRegCount = 0;
514     boolean doneWithMetaScan = false;
515     for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
516       ++tries) {
517       if (!doneWithMetaScan) {
518         // Wait for new table to come on-line
519         final AtomicInteger actualRegCount = new AtomicInteger(0);
520         MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
521           @Override
522           public boolean processRow(Result rowResult) throws IOException {
523             HRegionInfo info = HRegionInfo.getHRegionInfo(rowResult);
524             if (info == null) {
525               LOG.warn("No serialized HRegionInfo in " + rowResult);
526               return true;
527             }
528             if (!info.getTable().equals(desc.getTableName())) {
529               return false;
530             }
531             ServerName serverName = HRegionInfo.getServerName(rowResult);
532             // Make sure that regions are assigned to server
533             if (!(info.isOffline() || info.isSplit()) && serverName != null
534                 && serverName.getHostAndPort() != null) {
535               actualRegCount.incrementAndGet();
536             }
537             return true;
538           }
539         };
540         MetaScanner.metaScan(conf, connection, visitor, desc.getTableName());
541         if (actualRegCount.get() < numRegs) {
542           if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
543             throw new RegionOfflineException("Only " + actualRegCount.get() +
544               " of " + numRegs + " regions are online; retries exhausted.");
545           }
546           try { // Sleep
547             Thread.sleep(getPauseTime(tries));
548           } catch (InterruptedException e) {
549             throw new InterruptedIOException("Interrupted when opening" +
550               " regions; " + actualRegCount.get() + " of " + numRegs +
551               " regions processed so far");
552           }
553           if (actualRegCount.get() > prevRegCount) { // Making progress
554             prevRegCount = actualRegCount.get();
555             tries = -1;
556           }
557         } else {
558           doneWithMetaScan = true;
559           tries = -1;
560         }
561       } else if (isTableEnabled(desc.getTableName())) {
562         return;
563       } else {
564         try { // Sleep
565           Thread.sleep(getPauseTime(tries));
566         } catch (InterruptedException e) {
567           throw new InterruptedIOException("Interrupted when waiting" +
568             " for table to be enabled; meta scan was done");
569         }
570       }
571     }
572     throw new TableNotEnabledException(
573       "Retries exhausted while still waiting for table: "
574       + desc.getTableName() + " to be enabled");
575   }
576 
577   /**
578    * Creates a new table but does not block and wait for it to come online.
579    * Asynchronous operation.  To check if the table exists, use
580    * {@link #isTableAvailable} -- it is not safe to create an HTable
581    * instance to this table before it is available.
582    * Note : Avoid passing empty split key.
583    * @param desc table descriptor for table
584    *
585    * @throws IllegalArgumentException Bad table name, if the split keys
586    * are repeated and if the split key has empty byte array.
587    * @throws MasterNotRunningException if master is not running
588    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
589    * threads, the table may have been created between test-for-existence
590    * and attempt-at-creation).
591    * @throws IOException
592    */
593   public void createTableAsync(
594     final HTableDescriptor desc, final byte [][] splitKeys)
595   throws IOException {
596     if(desc.getTableName() == null) {
597       throw new IllegalArgumentException("TableName cannot be null");
598     }
599     if(splitKeys != null && splitKeys.length > 0) {
600       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
601       // Verify there are no duplicate split keys
602       byte [] lastKey = null;
603       for(byte [] splitKey : splitKeys) {
604         if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
605           throw new IllegalArgumentException(
606               "Empty split key must not be passed in the split keys.");
607         }
608         if(lastKey != null && Bytes.equals(splitKey, lastKey)) {
609           throw new IllegalArgumentException("All split keys must be unique, " +
610             "found duplicate: " + Bytes.toStringBinary(splitKey) +
611             ", " + Bytes.toStringBinary(lastKey));
612         }
613         lastKey = splitKey;
614       }
615     }
616 
617     executeCallable(new MasterCallable<Void>(getConnection()) {
618       @Override
619       public Void call(int callTimeout) throws ServiceException {
620         CreateTableRequest request = RequestConverter.buildCreateTableRequest(desc, splitKeys);
621         master.createTable(null, request);
622         return null;
623       }
624     });
625   }
626 
627   public void deleteTable(final String tableName) throws IOException {
628     deleteTable(TableName.valueOf(tableName));
629   }
630 
631   public void deleteTable(final byte[] tableName) throws IOException {
632     deleteTable(TableName.valueOf(tableName));
633   }
634 
635   /**
636    * Deletes a table.
637    * Synchronous operation.
638    *
639    * @param tableName name of table to delete
640    * @throws IOException if a remote or network exception occurs
641    */
642   public void deleteTable(final TableName tableName) throws IOException {
643     HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
644     boolean tableExists = true;
645 
646     executeCallable(new MasterCallable<Void>(getConnection()) {
647       @Override
648       public Void call(int callTimeout) throws ServiceException {
649         DeleteTableRequest req = RequestConverter.buildDeleteTableRequest(tableName);
650         master.deleteTable(null,req);
651         return null;
652       }
653     });
654 
655     // Wait until all regions deleted
656     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
657       try {
658 
659         Scan scan = MetaReader.getScanForTableName(tableName);
660         scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
661         ScanRequest request = RequestConverter.buildScanRequest(
662           firstMetaServer.getRegionInfo().getRegionName(), scan, 1, true);
663         Result[] values = null;
664         // Get a batch at a time.
665         ClientService.BlockingInterface server = connection.getClient(firstMetaServer
666             .getServerName());
667         PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
668         try {
669           controller.setPriority(tableName);
670           ScanResponse response = server.scan(controller, request);
671           values = ResponseConverter.getResults(controller.cellScanner(), response);
672         } catch (ServiceException se) {
673           throw ProtobufUtil.getRemoteException(se);
674         }
675 
676         // let us wait until hbase:meta table is updated and
677         // HMaster removes the table from its HTableDescriptors
678         if (values == null || values.length == 0) {
679           tableExists = false;
680           GetTableDescriptorsResponse htds;
681           MasterKeepAliveConnection master = connection.getKeepAliveMasterService();
682           try {
683             GetTableDescriptorsRequest req =
684               RequestConverter.buildGetTableDescriptorsRequest(tableName);
685             htds = master.getTableDescriptors(null, req);
686           } catch (ServiceException se) {
687             throw ProtobufUtil.getRemoteException(se);
688           } finally {
689             master.close();
690           }
691           tableExists = !htds.getTableSchemaList().isEmpty();
692           if (!tableExists) {
693             break;
694           }
695         }
696       } catch (IOException ex) {
697         if(tries == numRetries - 1) {           // no more tries left
698           if (ex instanceof RemoteException) {
699             throw ((RemoteException) ex).unwrapRemoteException();
700           } else {
701             throw ex;
702           }
703         }
704       }
705       try {
706         Thread.sleep(getPauseTime(tries));
707       } catch (InterruptedException e) {
708         throw new InterruptedIOException("Interrupted when waiting" +
709             " for table to be deleted");
710       }
711     }
712 
713     if (tableExists) {
714       throw new IOException("Retries exhausted, it took too long to wait"+
715         " for the table " + tableName + " to be deleted.");
716     }
717     // Delete cached information to prevent clients from using old locations
718     this.connection.clearRegionCache(tableName);
719     LOG.info("Deleted " + tableName);
720   }
721 
722   /**
723    * Deletes tables matching the passed in pattern and wait on completion.
724    *
725    * Warning: Use this method carefully, there is no prompting and the effect is
726    * immediate. Consider using {@link #listTables(java.lang.String)} and
727    * {@link #deleteTable(byte[])}
728    *
729    * @param regex The regular expression to match table names against
730    * @return Table descriptors for tables that couldn't be deleted
731    * @throws IOException
732    * @see #deleteTables(java.util.regex.Pattern)
733    * @see #deleteTable(java.lang.String)
734    */
735   public HTableDescriptor[] deleteTables(String regex) throws IOException {
736     return deleteTables(Pattern.compile(regex));
737   }
738 
739   /**
740    * Delete tables matching the passed in pattern and wait on completion.
741    *
742    * Warning: Use this method carefully, there is no prompting and the effect is
743    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
744    * {@link #deleteTable(byte[])}
745    *
746    * @param pattern The pattern to match table names against
747    * @return Table descriptors for tables that couldn't be deleted
748    * @throws IOException
749    */
750   public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
751     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
752     for (HTableDescriptor table : listTables(pattern)) {
753       try {
754         deleteTable(table.getTableName());
755       } catch (IOException ex) {
756         LOG.info("Failed to delete table " + table.getTableName(), ex);
757         failed.add(table);
758       }
759     }
760     return failed.toArray(new HTableDescriptor[failed.size()]);
761   }
762 
763 
764   /**
765    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
766    * and {@link #isTableEnabled(byte[])} instead.
767    * The table has to be in disabled state for it to be enabled.
768    * @param tableName name of the table
769    * @throws IOException if a remote or network exception occurs
770    * There could be couple types of IOException
771    * TableNotFoundException means the table doesn't exist.
772    * TableNotDisabledException means the table isn't in disabled state.
773    * @see #isTableEnabled(byte[])
774    * @see #disableTable(byte[])
775    * @see #enableTableAsync(byte[])
776    */
777   public void enableTable(final TableName tableName)
778   throws IOException {
779     enableTableAsync(tableName);
780 
781     // Wait until all regions are enabled
782     waitUntilTableIsEnabled(tableName);
783 
784     LOG.info("Enabled table " + tableName);
785   }
786 
787   public void enableTable(final byte[] tableName)
788   throws IOException {
789     enableTable(TableName.valueOf(tableName));
790   }
791 
792   public void enableTable(final String tableName)
793   throws IOException {
794     enableTable(TableName.valueOf(tableName));
795   }
796 
797   /**
798    * Wait for the table to be enabled and available
799    * If enabling the table exceeds the retry period, an exception is thrown.
800    * @param tableName name of the table
801    * @throws IOException if a remote or network exception occurs or
802    *    table is not enabled after the retries period.
803    */
804   private void waitUntilTableIsEnabled(final TableName tableName) throws IOException {
805     boolean enabled = false;
806     long start = EnvironmentEdgeManager.currentTimeMillis();
807     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
808       try {
809         enabled = isTableEnabled(tableName);
810       } catch (TableNotFoundException tnfe) {
811         // wait for table to be created
812         enabled = false;
813       }
814       enabled = enabled && isTableAvailable(tableName);
815       if (enabled) {
816         break;
817       }
818       long sleep = getPauseTime(tries);
819       if (LOG.isDebugEnabled()) {
820         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
821           "enabled in " + tableName);
822       }
823       try {
824         Thread.sleep(sleep);
825       } catch (InterruptedException e) {
826         // Do this conversion rather than let it out because do not want to
827         // change the method signature.
828         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
829       }
830     }
831     if (!enabled) {
832       long msec = EnvironmentEdgeManager.currentTimeMillis() - start;
833       throw new IOException("Table '" + tableName +
834         "' not yet enabled, after " + msec + "ms.");
835     }
836   }
837 
838   /**
839    * Brings a table on-line (enables it).  Method returns immediately though
840    * enable of table may take some time to complete, especially if the table
841    * is large (All regions are opened as part of enabling process).  Check
842    * {@link #isTableEnabled(byte[])} to learn when table is fully online.  If
843    * table is taking too long to online, check server logs.
844    * @param tableName
845    * @throws IOException
846    * @since 0.90.0
847    */
848   public void enableTableAsync(final TableName tableName)
849   throws IOException {
850     TableName.isLegalFullyQualifiedTableName(tableName.getName());
851     executeCallable(new MasterCallable<Void>(getConnection()) {
852       @Override
853       public Void call(int callTimeout) throws ServiceException {
854         LOG.info("Started enable of " + tableName);
855         EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
856         master.enableTable(null,req);
857         return null;
858       }
859     });
860   }
861 
862   public void enableTableAsync(final byte[] tableName)
863   throws IOException {
864     enableTable(TableName.valueOf(tableName));
865   }
866 
867   public void enableTableAsync(final String tableName)
868   throws IOException {
869     enableTableAsync(TableName.valueOf(tableName));
870   }
871 
872   /**
873    * Enable tables matching the passed in pattern and wait on completion.
874    *
875    * Warning: Use this method carefully, there is no prompting and the effect is
876    * immediate. Consider using {@link #listTables(java.lang.String)} and
877    * {@link #enableTable(byte[])}
878    *
879    * @param regex The regular expression to match table names against
880    * @throws IOException
881    * @see #enableTables(java.util.regex.Pattern)
882    * @see #enableTable(java.lang.String)
883    */
884   public HTableDescriptor[] enableTables(String regex) throws IOException {
885     return enableTables(Pattern.compile(regex));
886   }
887 
888   /**
889    * Enable tables matching the passed in pattern and wait on completion.
890    *
891    * Warning: Use this method carefully, there is no prompting and the effect is
892    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
893    * {@link #enableTable(byte[])}
894    *
895    * @param pattern The pattern to match table names against
896    * @throws IOException
897    */
898   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
899     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
900     for (HTableDescriptor table : listTables(pattern)) {
901       if (isTableDisabled(table.getTableName())) {
902         try {
903           enableTable(table.getTableName());
904         } catch (IOException ex) {
905           LOG.info("Failed to enable table " + table.getTableName(), ex);
906           failed.add(table);
907         }
908       }
909     }
910     return failed.toArray(new HTableDescriptor[failed.size()]);
911   }
912 
913   /**
914    * Starts the disable of a table.  If it is being served, the master
915    * will tell the servers to stop serving it.  This method returns immediately.
916    * The disable of a table can take some time if the table is large (all
917    * regions are closed as part of table disable operation).
918    * Call {@link #isTableDisabled(byte[])} to check for when disable completes.
919    * If table is taking too long to online, check server logs.
920    * @param tableName name of table
921    * @throws IOException if a remote or network exception occurs
922    * @see #isTableDisabled(byte[])
923    * @see #isTableEnabled(byte[])
924    * @since 0.90.0
925    */
926   public void disableTableAsync(final TableName tableName) throws IOException {
927     TableName.isLegalFullyQualifiedTableName(tableName.getName());
928     executeCallable(new MasterCallable<Void>(getConnection()) {
929       @Override
930       public Void call(int callTimeout) throws ServiceException {
931         LOG.info("Started disable of " + tableName);
932         DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
933         master.disableTable(null,req);
934         return null;
935       }
936     });
937   }
938 
939   public void disableTableAsync(final byte[] tableName) throws IOException {
940     disableTableAsync(TableName.valueOf(tableName));
941   }
942 
943   public void disableTableAsync(final String tableName) throws IOException {
944     disableTableAsync(TableName.valueOf(tableName));
945   }
946 
947   /**
948    * Disable table and wait on completion.  May timeout eventually.  Use
949    * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
950    * instead.
951    * The table has to be in enabled state for it to be disabled.
952    * @param tableName
953    * @throws IOException
954    * There could be couple types of IOException
955    * TableNotFoundException means the table doesn't exist.
956    * TableNotEnabledException means the table isn't in enabled state.
957    */
958   public void disableTable(final TableName tableName)
959   throws IOException {
960     disableTableAsync(tableName);
961     // Wait until table is disabled
962     boolean disabled = false;
963     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
964       disabled = isTableDisabled(tableName);
965       if (disabled) {
966         break;
967       }
968       long sleep = getPauseTime(tries);
969       if (LOG.isDebugEnabled()) {
970         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
971           "disabled in " + tableName);
972       }
973       try {
974         Thread.sleep(sleep);
975       } catch (InterruptedException e) {
976         // Do this conversion rather than let it out because do not want to
977         // change the method signature.
978         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
979       }
980     }
981     if (!disabled) {
982       throw new RegionException("Retries exhausted, it took too long to wait"+
983         " for the table " + tableName + " to be disabled.");
984     }
985     LOG.info("Disabled " + tableName);
986   }
987 
988   public void disableTable(final byte[] tableName)
989   throws IOException {
990     disableTable(TableName.valueOf(tableName));
991   }
992 
993   public void disableTable(final String tableName)
994   throws IOException {
995     disableTable(TableName.valueOf(tableName));
996   }
997 
998   /**
999    * Disable tables matching the passed in pattern and wait on completion.
1000    *
1001    * Warning: Use this method carefully, there is no prompting and the effect is
1002    * immediate. Consider using {@link #listTables(java.lang.String)} and
1003    * {@link #disableTable(byte[])}
1004    *
1005    * @param regex The regular expression to match table names against
1006    * @return Table descriptors for tables that couldn't be disabled
1007    * @throws IOException
1008    * @see #disableTables(java.util.regex.Pattern)
1009    * @see #disableTable(java.lang.String)
1010    */
1011   public HTableDescriptor[] disableTables(String regex) throws IOException {
1012     return disableTables(Pattern.compile(regex));
1013   }
1014 
1015   /**
1016    * Disable tables matching the passed in pattern and wait on completion.
1017    *
1018    * Warning: Use this method carefully, there is no prompting and the effect is
1019    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
1020    * {@link #disableTable(byte[])}
1021    *
1022    * @param pattern The pattern to match table names against
1023    * @return Table descriptors for tables that couldn't be disabled
1024    * @throws IOException
1025    */
1026   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
1027     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
1028     for (HTableDescriptor table : listTables(pattern)) {
1029       if (isTableEnabled(table.getTableName())) {
1030         try {
1031           disableTable(table.getTableName());
1032         } catch (IOException ex) {
1033           LOG.info("Failed to disable table " + table.getTableName(), ex);
1034           failed.add(table);
1035         }
1036       }
1037     }
1038     return failed.toArray(new HTableDescriptor[failed.size()]);
1039   }
1040 
1041   /*
1042    * Checks whether table exists. If not, throws TableNotFoundException
1043    * @param tableName
1044    */
1045   private void checkTableExistence(TableName tableName) throws IOException {
1046     if (!tableExists(tableName)) {
1047       throw new TableNotFoundException(tableName);
1048     }
1049   }
1050 
1051   /**
1052    * @param tableName name of table to check
1053    * @return true if table is on-line
1054    * @throws IOException if a remote or network exception occurs
1055    */
1056   public boolean isTableEnabled(TableName tableName) throws IOException {
1057     checkTableExistence(tableName);
1058     return connection.isTableEnabled(tableName);
1059   }
1060 
1061   public boolean isTableEnabled(byte[] tableName) throws IOException {
1062     return isTableEnabled(TableName.valueOf(tableName));
1063   }
1064 
1065   public boolean isTableEnabled(String tableName) throws IOException {
1066     return isTableEnabled(TableName.valueOf(tableName));
1067   }
1068 
1069 
1070 
1071   /**
1072    * @param tableName name of table to check
1073    * @return true if table is off-line
1074    * @throws IOException if a remote or network exception occurs
1075    */
1076   public boolean isTableDisabled(TableName tableName) throws IOException {
1077     checkTableExistence(tableName);
1078     return connection.isTableDisabled(tableName);
1079   }
1080 
1081   public boolean isTableDisabled(byte[] tableName) throws IOException {
1082     return isTableDisabled(TableName.valueOf(tableName));
1083   }
1084 
1085   public boolean isTableDisabled(String tableName) throws IOException {
1086     return isTableDisabled(TableName.valueOf(tableName));
1087   }
1088 
1089   /**
1090    * @param tableName name of table to check
1091    * @return true if all regions of the table are available
1092    * @throws IOException if a remote or network exception occurs
1093    */
1094   public boolean isTableAvailable(TableName tableName) throws IOException {
1095     return connection.isTableAvailable(tableName);
1096   }
1097 
1098   public boolean isTableAvailable(byte[] tableName) throws IOException {
1099     return isTableAvailable(TableName.valueOf(tableName));
1100   }
1101 
1102   public boolean isTableAvailable(String tableName) throws IOException {
1103     return isTableAvailable(TableName.valueOf(tableName));
1104   }
1105 
1106   /**
1107    * Use this api to check if the table has been created with the specified number of
1108    * splitkeys which was used while creating the given table.
1109    * Note : If this api is used after a table's region gets splitted, the api may return
1110    * false.
1111    * @param tableName
1112    *          name of table to check
1113    * @param splitKeys
1114    *          keys to check if the table has been created with all split keys
1115    * @throws IOException
1116    *           if a remote or network excpetion occurs
1117    */
1118   public boolean isTableAvailable(TableName tableName,
1119                                   byte[][] splitKeys) throws IOException {
1120     return connection.isTableAvailable(tableName, splitKeys);
1121   }
1122 
1123   public boolean isTableAvailable(byte[] tableName,
1124                                   byte[][] splitKeys) throws IOException {
1125     return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1126   }
1127 
1128   public boolean isTableAvailable(String tableName,
1129                                   byte[][] splitKeys) throws IOException {
1130     return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1131   }
1132 
1133   /**
1134    * Get the status of alter command - indicates how many regions have received
1135    * the updated schema Asynchronous operation.
1136    *
1137    * @param tableName TableName instance
1138    * @return Pair indicating the number of regions updated Pair.getFirst() is the
1139    *         regions that are yet to be updated Pair.getSecond() is the total number
1140    *         of regions of the table
1141    * @throws IOException
1142    *           if a remote or network exception occurs
1143    */
1144   public Pair<Integer, Integer> getAlterStatus(final TableName tableName)
1145   throws IOException {
1146     return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
1147       @Override
1148       public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
1149         GetSchemaAlterStatusRequest req = RequestConverter
1150             .buildGetSchemaAlterStatusRequest(tableName);
1151         GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req);
1152         Pair<Integer, Integer> pair = new Pair<Integer, Integer>(Integer.valueOf(ret
1153             .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
1154         return pair;
1155       }
1156     });
1157   }
1158 
1159   /**
1160    * Get the status of alter command - indicates how many regions have received
1161    * the updated schema Asynchronous operation.
1162    *
1163    * @param tableName
1164    *          name of the table to get the status of
1165    * @return Pair indicating the number of regions updated Pair.getFirst() is the
1166    *         regions that are yet to be updated Pair.getSecond() is the total number
1167    *         of regions of the table
1168    * @throws IOException
1169    *           if a remote or network exception occurs
1170    */
1171   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
1172    throws IOException {
1173     return getAlterStatus(TableName.valueOf(tableName));
1174   }
1175 
1176   /**
1177    * Add a column to an existing table.
1178    * Asynchronous operation.
1179    *
1180    * @param tableName name of the table to add column to
1181    * @param column column descriptor of column to be added
1182    * @throws IOException if a remote or network exception occurs
1183    */
1184   public void addColumn(final byte[] tableName, HColumnDescriptor column)
1185   throws IOException {
1186     addColumn(TableName.valueOf(tableName), column);
1187   }
1188 
1189 
1190   /**
1191    * Add a column to an existing table.
1192    * Asynchronous operation.
1193    *
1194    * @param tableName name of the table to add column to
1195    * @param column column descriptor of column to be added
1196    * @throws IOException if a remote or network exception occurs
1197    */
1198   public void addColumn(final String tableName, HColumnDescriptor column)
1199   throws IOException {
1200     addColumn(TableName.valueOf(tableName), column);
1201   }
1202 
1203   /**
1204    * Add a column to an existing table.
1205    * Asynchronous operation.
1206    *
1207    * @param tableName name of the table to add column to
1208    * @param column column descriptor of column to be added
1209    * @throws IOException if a remote or network exception occurs
1210    */
1211   public void addColumn(final TableName tableName, final HColumnDescriptor column)
1212   throws IOException {
1213     executeCallable(new MasterCallable<Void>(getConnection()) {
1214       @Override
1215       public Void call(int callTimeout) throws ServiceException {
1216         AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, column);
1217         master.addColumn(null,req);
1218         return null;
1219       }
1220     });
1221   }
1222 
1223   /**
1224    * Delete a column from a table.
1225    * Asynchronous operation.
1226    *
1227    * @param tableName name of table
1228    * @param columnName name of column to be deleted
1229    * @throws IOException if a remote or network exception occurs
1230    */
1231   public void deleteColumn(final byte[] tableName, final String columnName)
1232   throws IOException {
1233     deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
1234   }
1235 
1236   /**
1237    * Delete a column from a table.
1238    * Asynchronous operation.
1239    *
1240    * @param tableName name of table
1241    * @param columnName name of column to be deleted
1242    * @throws IOException if a remote or network exception occurs
1243    */
1244   public void deleteColumn(final String tableName, final String columnName)
1245   throws IOException {
1246     deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
1247   }
1248 
1249   /**
1250    * Delete a column from a table.
1251    * Asynchronous operation.
1252    *
1253    * @param tableName name of table
1254    * @param columnName name of column to be deleted
1255    * @throws IOException if a remote or network exception occurs
1256    */
1257   public void deleteColumn(final TableName tableName, final byte [] columnName)
1258   throws IOException {
1259     executeCallable(new MasterCallable<Void>(getConnection()) {
1260       @Override
1261       public Void call(int callTimeout) throws ServiceException {
1262         DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(tableName, columnName);
1263         master.deleteColumn(null,req);
1264         return null;
1265       }
1266     });
1267   }
1268 
1269   /**
1270    * Modify an existing column family on a table.
1271    * Asynchronous operation.
1272    *
1273    * @param tableName name of table
1274    * @param descriptor new column descriptor to use
1275    * @throws IOException if a remote or network exception occurs
1276    */
1277   public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
1278   throws IOException {
1279     modifyColumn(TableName.valueOf(tableName), descriptor);
1280   }
1281 
1282   /**
1283    * Modify an existing column family on a table.
1284    * Asynchronous operation.
1285    *
1286    * @param tableName name of table
1287    * @param descriptor new column descriptor to use
1288    * @throws IOException if a remote or network exception occurs
1289    */
1290   public void modifyColumn(final byte[] tableName, HColumnDescriptor descriptor)
1291   throws IOException {
1292     modifyColumn(TableName.valueOf(tableName), descriptor);
1293   }
1294 
1295 
1296 
1297   /**
1298    * Modify an existing column family on a table.
1299    * Asynchronous operation.
1300    *
1301    * @param tableName name of table
1302    * @param descriptor new column descriptor to use
1303    * @throws IOException if a remote or network exception occurs
1304    */
1305   public void modifyColumn(final TableName tableName, final HColumnDescriptor descriptor)
1306   throws IOException {
1307     executeCallable(new MasterCallable<Void>(getConnection()) {
1308       @Override
1309       public Void call(int callTimeout) throws ServiceException {
1310         ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(tableName, descriptor);
1311         master.modifyColumn(null,req);
1312         return null;
1313       }
1314     });
1315   }
1316 
1317   /**
1318    * Close a region. For expert-admins.  Runs close on the regionserver.  The
1319    * master will not be informed of the close.
1320    * @param regionname region name to close
1321    * @param serverName If supplied, we'll use this location rather than
1322    * the one currently in <code>hbase:meta</code>
1323    * @throws IOException if a remote or network exception occurs
1324    */
1325   public void closeRegion(final String regionname, final String serverName)
1326   throws IOException {
1327     closeRegion(Bytes.toBytes(regionname), serverName);
1328   }
1329 
1330   /**
1331    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1332    * master will not be informed of the close.
1333    * @param regionname region name to close
1334    * @param serverName The servername of the regionserver.  If passed null we
1335    * will use servername found in the hbase:meta table. A server name
1336    * is made of host, port and startcode.  Here is an example:
1337    * <code> host187.example.com,60020,1289493121758</code>
1338    * @throws IOException if a remote or network exception occurs
1339    */
1340   public void closeRegion(final byte [] regionname, final String serverName)
1341   throws IOException {
1342     CatalogTracker ct = getCatalogTracker();
1343     try {
1344       if (serverName != null) {
1345         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1346         if (pair == null || pair.getFirst() == null) {
1347           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1348         } else {
1349           closeRegion(ServerName.valueOf(serverName), pair.getFirst());
1350         }
1351       } else {
1352         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1353         if (pair == null) {
1354           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1355         } else if (pair.getSecond() == null) {
1356           throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1357         } else {
1358           closeRegion(pair.getSecond(), pair.getFirst());
1359         }
1360       }
1361     } finally {
1362       cleanupCatalogTracker(ct);
1363     }
1364   }
1365 
1366   /**
1367    * For expert-admins. Runs close on the regionserver. Closes a region based on
1368    * the encoded region name. The region server name is mandatory. If the
1369    * servername is provided then based on the online regions in the specified
1370    * regionserver the specified region will be closed. The master will not be
1371    * informed of the close. Note that the regionname is the encoded regionname.
1372    *
1373    * @param encodedRegionName
1374    *          The encoded region name; i.e. the hash that makes up the region
1375    *          name suffix: e.g. if regionname is
1376    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
1377    *          , then the encoded region name is:
1378    *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
1379    * @param serverName
1380    *          The servername of the regionserver. A server name is made of host,
1381    *          port and startcode. This is mandatory. Here is an example:
1382    *          <code> host187.example.com,60020,1289493121758</code>
1383    * @return true if the region was closed, false if not.
1384    * @throws IOException
1385    *           if a remote or network exception occurs
1386    */
1387   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1388       final String serverName) throws IOException {
1389     if (null == serverName || ("").equals(serverName.trim())) {
1390       throw new IllegalArgumentException(
1391           "The servername cannot be null or empty.");
1392     }
1393     ServerName sn = ServerName.valueOf(serverName);
1394     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1395     // Close the region without updating zk state.
1396     CloseRegionRequest request =
1397       RequestConverter.buildCloseRegionRequest(sn, encodedRegionName, false);
1398     try {
1399       CloseRegionResponse response = admin.closeRegion(null, request);
1400       boolean isRegionClosed = response.getClosed();
1401       if (false == isRegionClosed) {
1402         LOG.error("Not able to close the region " + encodedRegionName + ".");
1403       }
1404       return isRegionClosed;
1405     } catch (ServiceException se) {
1406       throw ProtobufUtil.getRemoteException(se);
1407     }
1408   }
1409 
1410   /**
1411    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1412    * master will not be informed of the close.
1413    * @param sn
1414    * @param hri
1415    * @throws IOException
1416    */
1417   public void closeRegion(final ServerName sn, final HRegionInfo hri)
1418   throws IOException {
1419     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1420     // Close the region without updating zk state.
1421     ProtobufUtil.closeRegion(admin, sn, hri.getRegionName(), false);
1422   }
1423 
1424   /**
1425    * Get all the online regions on a region server.
1426    */
1427   public List<HRegionInfo> getOnlineRegions(
1428       final ServerName sn) throws IOException {
1429     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1430     return ProtobufUtil.getOnlineRegions(admin);
1431   }
1432 
1433   /**
1434    * Flush a table or an individual region.
1435    * Synchronous operation.
1436    *
1437    * @param tableNameOrRegionName table or region to flush
1438    * @throws IOException if a remote or network exception occurs
1439    * @throws InterruptedException
1440    */
1441   public void flush(final String tableNameOrRegionName)
1442   throws IOException, InterruptedException {
1443     flush(Bytes.toBytes(tableNameOrRegionName));
1444   }
1445 
1446   /**
1447    * Flush a table or an individual region.
1448    * Synchronous operation.
1449    *
1450    * @param tableNameOrRegionName table or region to flush
1451    * @throws IOException if a remote or network exception occurs
1452    * @throws InterruptedException
1453    */
1454   public void flush(final byte[] tableNameOrRegionName)
1455   throws IOException, InterruptedException {
1456     CatalogTracker ct = getCatalogTracker();
1457     try {
1458       Pair<HRegionInfo, ServerName> regionServerPair
1459         = getRegion(tableNameOrRegionName, ct);
1460       if (regionServerPair != null) {
1461         if (regionServerPair.getSecond() == null) {
1462           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1463         } else {
1464           flush(regionServerPair.getSecond(), regionServerPair.getFirst());
1465         }
1466       } else {
1467         final TableName tableName = checkTableExists(
1468             TableName.valueOf(tableNameOrRegionName), ct);
1469         List<Pair<HRegionInfo, ServerName>> pairs =
1470           MetaReader.getTableRegionsAndLocations(ct,
1471               tableName);
1472         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1473           if (pair.getFirst().isOffline()) continue;
1474           if (pair.getSecond() == null) continue;
1475           try {
1476             flush(pair.getSecond(), pair.getFirst());
1477           } catch (NotServingRegionException e) {
1478             if (LOG.isDebugEnabled()) {
1479               LOG.debug("Trying to flush " + pair.getFirst() + ": " +
1480                 StringUtils.stringifyException(e));
1481             }
1482           }
1483         }
1484       }
1485     } finally {
1486       cleanupCatalogTracker(ct);
1487     }
1488   }
1489 
1490   private void flush(final ServerName sn, final HRegionInfo hri)
1491   throws IOException {
1492     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1493     FlushRegionRequest request =
1494       RequestConverter.buildFlushRegionRequest(hri.getRegionName());
1495     try {
1496       admin.flushRegion(null, request);
1497     } catch (ServiceException se) {
1498       throw ProtobufUtil.getRemoteException(se);
1499     }
1500   }
1501 
1502   /**
1503    * Compact a table or an individual region.
1504    * Asynchronous operation.
1505    *
1506    * @param tableNameOrRegionName table or region to compact
1507    * @throws IOException if a remote or network exception occurs
1508    * @throws InterruptedException
1509    */
1510   public void compact(final String tableNameOrRegionName)
1511   throws IOException, InterruptedException {
1512     compact(Bytes.toBytes(tableNameOrRegionName));
1513   }
1514 
1515   /**
1516    * Compact a table or an individual region.
1517    * Asynchronous operation.
1518    *
1519    * @param tableNameOrRegionName table or region to compact
1520    * @throws IOException if a remote or network exception occurs
1521    * @throws InterruptedException
1522    */
1523   public void compact(final byte[] tableNameOrRegionName)
1524   throws IOException, InterruptedException {
1525     compact(tableNameOrRegionName, null, false);
1526   }
1527 
1528   /**
1529    * Compact a column family within a table or region.
1530    * Asynchronous operation.
1531    *
1532    * @param tableOrRegionName table or region to compact
1533    * @param columnFamily column family within a table or region
1534    * @throws IOException if a remote or network exception occurs
1535    * @throws InterruptedException
1536    */
1537   public void compact(String tableOrRegionName, String columnFamily)
1538     throws IOException,  InterruptedException {
1539     compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
1540   }
1541 
1542   /**
1543    * Compact a column family within a table or region.
1544    * Asynchronous operation.
1545    *
1546    * @param tableNameOrRegionName table or region to compact
1547    * @param columnFamily column family within a table or region
1548    * @throws IOException if a remote or network exception occurs
1549    * @throws InterruptedException
1550    */
1551   public void compact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
1552   throws IOException, InterruptedException {
1553     compact(tableNameOrRegionName, columnFamily, false);
1554   }
1555 
1556   /**
1557    * Major compact a table or an individual region.
1558    * Asynchronous operation.
1559    *
1560    * @param tableNameOrRegionName table or region to major compact
1561    * @throws IOException if a remote or network exception occurs
1562    * @throws InterruptedException
1563    */
1564   public void majorCompact(final String tableNameOrRegionName)
1565   throws IOException, InterruptedException {
1566     majorCompact(Bytes.toBytes(tableNameOrRegionName));
1567   }
1568 
1569   /**
1570    * Major compact a table or an individual region.
1571    * Asynchronous operation.
1572    *
1573    * @param tableNameOrRegionName table or region to major compact
1574    * @throws IOException if a remote or network exception occurs
1575    * @throws InterruptedException
1576    */
1577   public void majorCompact(final byte[] tableNameOrRegionName)
1578   throws IOException, InterruptedException {
1579     compact(tableNameOrRegionName, null, true);
1580   }
1581 
1582   /**
1583    * Major compact a column family within a table or region.
1584    * Asynchronous operation.
1585    *
1586    * @param tableNameOrRegionName table or region to major compact
1587    * @param columnFamily column family within a table or region
1588    * @throws IOException if a remote or network exception occurs
1589    * @throws InterruptedException
1590    */
1591   public void majorCompact(final String tableNameOrRegionName,
1592     final String columnFamily) throws IOException, InterruptedException {
1593     majorCompact(Bytes.toBytes(tableNameOrRegionName),
1594       Bytes.toBytes(columnFamily));
1595   }
1596 
1597   /**
1598    * Major compact a column family within a table or region.
1599    * Asynchronous operation.
1600    *
1601    * @param tableNameOrRegionName table or region to major compact
1602    * @param columnFamily column family within a table or region
1603    * @throws IOException if a remote or network exception occurs
1604    * @throws InterruptedException
1605    */
1606   public void majorCompact(final byte[] tableNameOrRegionName,
1607     final byte[] columnFamily) throws IOException, InterruptedException {
1608     compact(tableNameOrRegionName, columnFamily, true);
1609   }
1610 
1611   /**
1612    * Compact a table or an individual region.
1613    * Asynchronous operation.
1614    *
1615    * @param tableNameOrRegionName table or region to compact
1616    * @param columnFamily column family within a table or region
1617    * @param major True if we are to do a major compaction.
1618    * @throws IOException if a remote or network exception occurs
1619    * @throws InterruptedException
1620    */
1621   private void compact(final byte[] tableNameOrRegionName,
1622     final byte[] columnFamily,final boolean major)
1623   throws IOException, InterruptedException {
1624     CatalogTracker ct = getCatalogTracker();
1625     try {
1626       Pair<HRegionInfo, ServerName> regionServerPair
1627         = getRegion(tableNameOrRegionName, ct);
1628       if (regionServerPair != null) {
1629         if (regionServerPair.getSecond() == null) {
1630           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1631         } else {
1632           compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
1633         }
1634       } else {
1635         final TableName tableName =
1636             checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
1637         List<Pair<HRegionInfo, ServerName>> pairs =
1638           MetaReader.getTableRegionsAndLocations(ct,
1639               tableName);
1640         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1641           if (pair.getFirst().isOffline()) continue;
1642           if (pair.getSecond() == null) continue;
1643           try {
1644             compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
1645           } catch (NotServingRegionException e) {
1646             if (LOG.isDebugEnabled()) {
1647               LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1648                 pair.getFirst() + ": " +
1649                 StringUtils.stringifyException(e));
1650             }
1651           }
1652         }
1653       }
1654     } finally {
1655       cleanupCatalogTracker(ct);
1656     }
1657   }
1658 
1659   private void compact(final ServerName sn, final HRegionInfo hri,
1660       final boolean major, final byte [] family)
1661   throws IOException {
1662     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1663     CompactRegionRequest request =
1664       RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
1665     try {
1666       admin.compactRegion(null, request);
1667     } catch (ServiceException se) {
1668       throw ProtobufUtil.getRemoteException(se);
1669     }
1670   }
1671 
1672   /**
1673    * Move the region <code>r</code> to <code>dest</code>.
1674    * @param encodedRegionName The encoded region name; i.e. the hash that makes
1675    * up the region name suffix: e.g. if regionname is
1676    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
1677    * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
1678    * @param destServerName The servername of the destination regionserver.  If
1679    * passed the empty byte array we'll assign to a random server.  A server name
1680    * is made of host, port and startcode.  Here is an example:
1681    * <code> host187.example.com,60020,1289493121758</code>
1682    * @throws UnknownRegionException Thrown if we can't find a region named
1683    * <code>encodedRegionName</code>
1684    * @throws ZooKeeperConnectionException
1685    * @throws MasterNotRunningException
1686    */
1687   public void move(final byte [] encodedRegionName, final byte [] destServerName)
1688   throws HBaseIOException, MasterNotRunningException, ZooKeeperConnectionException {
1689     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1690     try {
1691       MoveRegionRequest request =
1692         RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
1693       stub.moveRegion(null,request);
1694     } catch (ServiceException se) {
1695       IOException ioe = ProtobufUtil.getRemoteException(se);
1696       if (ioe instanceof HBaseIOException) {
1697         throw (HBaseIOException)ioe;
1698       }
1699       LOG.error("Unexpected exception: " + se + " from calling HMaster.moveRegion");
1700     } catch (DeserializationException de) {
1701       LOG.error("Could not parse destination server name: " + de);
1702     } finally {
1703       stub.close();
1704     }
1705   }
1706 
1707   /**
1708    * @param regionName
1709    *          Region name to assign.
1710    * @throws MasterNotRunningException
1711    * @throws ZooKeeperConnectionException
1712    * @throws IOException
1713    */
1714   public void assign(final byte[] regionName) throws MasterNotRunningException,
1715       ZooKeeperConnectionException, IOException {
1716     final byte[] toBeAssigned = getRegionName(regionName);
1717     executeCallable(new MasterCallable<Void>(getConnection()) {
1718       @Override
1719       public Void call(int callTimeout) throws ServiceException {
1720         AssignRegionRequest request =
1721           RequestConverter.buildAssignRegionRequest(toBeAssigned);
1722         master.assignRegion(null,request);
1723         return null;
1724       }
1725     });
1726   }
1727 
1728   /**
1729    * Unassign a region from current hosting regionserver.  Region will then be
1730    * assigned to a regionserver chosen at random.  Region could be reassigned
1731    * back to the same server.  Use {@link #move(byte[], byte[])} if you want
1732    * to control the region movement.
1733    * @param regionName Region to unassign. Will clear any existing RegionPlan
1734    * if one found.
1735    * @param force If true, force unassign (Will remove region from
1736    * regions-in-transition too if present. If results in double assignment
1737    * use hbck -fix to resolve. To be used by experts).
1738    * @throws MasterNotRunningException
1739    * @throws ZooKeeperConnectionException
1740    * @throws IOException
1741    */
1742   public void unassign(final byte [] regionName, final boolean force)
1743   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
1744     final byte[] toBeUnassigned = getRegionName(regionName);
1745     executeCallable(new MasterCallable<Void>(getConnection()) {
1746       @Override
1747       public Void call(int callTimeout) throws ServiceException {
1748         UnassignRegionRequest request =
1749           RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
1750         master.unassignRegion(null,request);
1751         return null;
1752       }
1753     });
1754   }
1755 
1756   /**
1757    * Offline specified region from master's in-memory state. It will not attempt to reassign the
1758    * region as in unassign. This API can be used when a region not served by any region server and
1759    * still online as per Master's in memory state. If this API is incorrectly used on active region
1760    * then master will loose track of that region.
1761    *
1762    * This is a special method that should be used by experts or hbck.
1763    *
1764    * @param regionName
1765    *          Region to offline.
1766    * @throws IOException
1767    */
1768   public void offline(final byte [] regionName)
1769   throws IOException {
1770     MasterKeepAliveConnection master = connection.getKeepAliveMasterService();
1771     try {
1772       master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
1773     } catch (ServiceException se) {
1774       throw ProtobufUtil.getRemoteException(se);
1775     } finally {
1776       master.close();
1777     }
1778   }
1779 
1780   /**
1781    * Turn the load balancer on or off.
1782    * @param on If true, enable balancer. If false, disable balancer.
1783    * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
1784    * @return Previous balancer value
1785    */
1786   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
1787   throws MasterNotRunningException, ZooKeeperConnectionException {
1788     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1789     try {
1790       SetBalancerRunningRequest req =
1791         RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
1792       return stub.setBalancerRunning(null, req).getPrevBalanceValue();
1793     } catch (ServiceException se) {
1794       IOException ioe = ProtobufUtil.getRemoteException(se);
1795       if (ioe instanceof MasterNotRunningException) {
1796         throw (MasterNotRunningException)ioe;
1797       }
1798       if (ioe instanceof ZooKeeperConnectionException) {
1799         throw (ZooKeeperConnectionException)ioe;
1800       }
1801 
1802       // Throwing MasterNotRunningException even though not really valid in order to not
1803       // break interface by adding additional exception type.
1804       throw new MasterNotRunningException("Unexpected exception when calling balanceSwitch",se);
1805     } finally {
1806       stub.close();
1807     }
1808   }
1809 
1810   /**
1811    * Invoke the balancer.  Will run the balancer and if regions to move, it will
1812    * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
1813    * logs.
1814    * @return True if balancer ran, false otherwise.
1815    */
1816   public boolean balancer()
1817   throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException {
1818     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1819     try {
1820       return stub.balance(null,RequestConverter.buildBalanceRequest()).getBalancerRan();
1821     } finally {
1822       stub.close();
1823     }
1824   }
1825 
1826   /**
1827    * Enable/Disable the catalog janitor
1828    * @param enable if true enables the catalog janitor
1829    * @return the previous state
1830    * @throws ServiceException
1831    * @throws MasterNotRunningException
1832    */
1833   public boolean enableCatalogJanitor(boolean enable)
1834       throws ServiceException, MasterNotRunningException {
1835     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1836     try {
1837       return stub.enableCatalogJanitor(null,
1838           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
1839     } finally {
1840       stub.close();
1841     }
1842   }
1843 
1844   /**
1845    * Ask for a scan of the catalog table
1846    * @return the number of entries cleaned
1847    * @throws ServiceException
1848    * @throws MasterNotRunningException
1849    */
1850   public int runCatalogScan() throws ServiceException, MasterNotRunningException {
1851     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1852     try {
1853       return stub.runCatalogScan(null,
1854           RequestConverter.buildCatalogScanRequest()).getScanResult();
1855     } finally {
1856       stub.close();
1857     }
1858   }
1859 
1860   /**
1861    * Query on the catalog janitor state (Enabled/Disabled?)
1862    * @throws ServiceException
1863    * @throws org.apache.hadoop.hbase.MasterNotRunningException
1864    */
1865   public boolean isCatalogJanitorEnabled() throws ServiceException, MasterNotRunningException {
1866     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1867     try {
1868       return stub.isCatalogJanitorEnabled(null,
1869           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
1870     } finally {
1871       stub.close();
1872     }
1873   }
1874 
1875   /**
1876    * Merge two regions. Asynchronous operation.
1877    * @param encodedNameOfRegionA encoded name of region a
1878    * @param encodedNameOfRegionB encoded name of region b
1879    * @param forcible true if do a compulsory merge, otherwise we will only merge
1880    *          two adjacent regions
1881    * @throws IOException
1882    */
1883   public void mergeRegions(final byte[] encodedNameOfRegionA,
1884       final byte[] encodedNameOfRegionB, final boolean forcible)
1885       throws IOException {
1886     MasterKeepAliveConnection master = connection
1887         .getKeepAliveMasterService();
1888     try {
1889       DispatchMergingRegionsRequest request = RequestConverter
1890           .buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
1891               encodedNameOfRegionB, forcible);
1892       master.dispatchMergingRegions(null, request);
1893     } catch (ServiceException se) {
1894       IOException ioe = ProtobufUtil.getRemoteException(se);
1895       if (ioe instanceof UnknownRegionException) {
1896         throw (UnknownRegionException) ioe;
1897       }
1898       if (ioe instanceof MergeRegionException) {
1899         throw (MergeRegionException) ioe;
1900       }
1901       LOG.error("Unexpected exception: " + se
1902           + " from calling HMaster.dispatchMergingRegions");
1903     } catch (DeserializationException de) {
1904       LOG.error("Could not parse destination server name: " + de);
1905     } finally {
1906       master.close();
1907     }
1908   }
1909 
1910   /**
1911    * Split a table or an individual region.
1912    * Asynchronous operation.
1913    *
1914    * @param tableNameOrRegionName table or region to split
1915    * @throws IOException if a remote or network exception occurs
1916    * @throws InterruptedException
1917    */
1918   public void split(final String tableNameOrRegionName)
1919   throws IOException, InterruptedException {
1920     split(Bytes.toBytes(tableNameOrRegionName));
1921   }
1922 
1923   /**
1924    * Split a table or an individual region.  Implicitly finds an optimal split
1925    * point.  Asynchronous operation.
1926    *
1927    * @param tableNameOrRegionName table to region to split
1928    * @throws IOException if a remote or network exception occurs
1929    * @throws InterruptedException
1930    */
1931   public void split(final byte[] tableNameOrRegionName)
1932   throws IOException, InterruptedException {
1933     split(tableNameOrRegionName, null);
1934   }
1935 
1936   public void split(final String tableNameOrRegionName,
1937     final String splitPoint) throws IOException, InterruptedException {
1938     split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
1939   }
1940 
1941   /**
1942    * Split a table or an individual region.
1943    * Asynchronous operation.
1944    *
1945    * @param tableNameOrRegionName table to region to split
1946    * @param splitPoint the explicit position to split on
1947    * @throws IOException if a remote or network exception occurs
1948    * @throws InterruptedException interrupt exception occurred
1949    */
1950   public void split(final byte[] tableNameOrRegionName,
1951       final byte [] splitPoint) throws IOException, InterruptedException {
1952     CatalogTracker ct = getCatalogTracker();
1953     try {
1954       Pair<HRegionInfo, ServerName> regionServerPair
1955         = getRegion(tableNameOrRegionName, ct);
1956       if (regionServerPair != null) {
1957         if (regionServerPair.getSecond() == null) {
1958             throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1959         } else {
1960           split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
1961         }
1962       } else {
1963         final TableName tableName =
1964             checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
1965         List<Pair<HRegionInfo, ServerName>> pairs =
1966           MetaReader.getTableRegionsAndLocations(ct,
1967               tableName);
1968         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1969           // May not be a server for a particular row
1970           if (pair.getSecond() == null) continue;
1971           HRegionInfo r = pair.getFirst();
1972           // check for parents
1973           if (r.isSplitParent()) continue;
1974           // if a split point given, only split that particular region
1975           if (splitPoint != null && !r.containsRow(splitPoint)) continue;
1976           // call out to region server to do split now
1977           split(pair.getSecond(), pair.getFirst(), splitPoint);
1978         }
1979       }
1980     } finally {
1981       cleanupCatalogTracker(ct);
1982     }
1983   }
1984 
1985   private void split(final ServerName sn, final HRegionInfo hri,
1986       byte[] splitPoint) throws IOException {
1987     if (hri.getStartKey() != null && splitPoint != null &&
1988          Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
1989        throw new IOException("should not give a splitkey which equals to startkey!");
1990     }
1991     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1992     ProtobufUtil.split(admin, hri, splitPoint);
1993   }
1994 
1995   /**
1996    * Modify an existing table, more IRB friendly version.
1997    * Asynchronous operation.  This means that it may be a while before your
1998    * schema change is updated across all of the table.
1999    *
2000    * @param tableName name of table.
2001    * @param htd modified description of the table
2002    * @throws IOException if a remote or network exception occurs
2003    */
2004   public void modifyTable(final TableName tableName, final HTableDescriptor htd)
2005   throws IOException {
2006     if (!tableName.equals(htd.getTableName())) {
2007       throw new IllegalArgumentException("the specified table name '" + tableName +
2008         "' doesn't match with the HTD one: " + htd.getTableName());
2009     }
2010 
2011     executeCallable(new MasterCallable<Void>(getConnection()) {
2012       @Override
2013       public Void call(int callTimeout) throws ServiceException {
2014         ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd);
2015         master.modifyTable(null, request);
2016         return null;
2017       }
2018     });
2019   }
2020 
2021   public void modifyTable(final byte[] tableName, final HTableDescriptor htd)
2022   throws IOException {
2023     modifyTable(TableName.valueOf(tableName), htd);
2024   }
2025 
2026   public void modifyTable(final String tableName, final HTableDescriptor htd)
2027   throws IOException {
2028     modifyTable(TableName.valueOf(tableName), htd);
2029   }
2030 
2031   /**
2032    * @param tableNameOrRegionName Name of a table or name of a region.
2033    * @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
2034    * @return a pair of HRegionInfo and ServerName if <code>tableNameOrRegionName</code> is
2035    *  a verified region name (we call {@link  MetaReader#getRegion( CatalogTracker, byte[])}
2036    *  else null.
2037    * Throw an exception if <code>tableNameOrRegionName</code> is null.
2038    * @throws IOException
2039    */
2040   Pair<HRegionInfo, ServerName> getRegion(final byte[] tableNameOrRegionName,
2041       final CatalogTracker ct) throws IOException {
2042     if (tableNameOrRegionName == null) {
2043       throw new IllegalArgumentException("Pass a table name or region name");
2044     }
2045     Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, tableNameOrRegionName);
2046     if (pair == null) {
2047       final AtomicReference<Pair<HRegionInfo, ServerName>> result =
2048         new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
2049       final String encodedName = Bytes.toString(tableNameOrRegionName);
2050       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
2051         @Override
2052         public boolean processRow(Result data) throws IOException {
2053           HRegionInfo info = HRegionInfo.getHRegionInfo(data);
2054           if (info == null) {
2055             LOG.warn("No serialized HRegionInfo in " + data);
2056             return true;
2057           }
2058           if (!encodedName.equals(info.getEncodedName())) return true;
2059           ServerName sn = HRegionInfo.getServerName(data);
2060           result.set(new Pair<HRegionInfo, ServerName>(info, sn));
2061           return false; // found the region, stop
2062         }
2063       };
2064 
2065       MetaScanner.metaScan(conf, connection, visitor, null);
2066       pair = result.get();
2067     }
2068     return pair;
2069   }
2070 
2071   /**
2072    * If the input is a region name, it is returned as is. If it's an
2073    * encoded region name, the corresponding region is found from meta
2074    * and its region name is returned. If we can't find any region in
2075    * meta matching the input as either region name or encoded region
2076    * name, the input is returned as is. We don't throw unknown
2077    * region exception.
2078    */
2079   private byte[] getRegionName(
2080       final byte[] regionNameOrEncodedRegionName) throws IOException {
2081     if (Bytes.equals(regionNameOrEncodedRegionName,
2082         HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
2083           || Bytes.equals(regionNameOrEncodedRegionName,
2084             HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
2085       return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
2086     }
2087     CatalogTracker ct = getCatalogTracker();
2088     byte[] tmp = regionNameOrEncodedRegionName;
2089     try {
2090       Pair<HRegionInfo, ServerName> regionServerPair
2091         = getRegion(regionNameOrEncodedRegionName, ct);
2092       if (regionServerPair != null && regionServerPair.getFirst() != null) {
2093         tmp = regionServerPair.getFirst().getRegionName();
2094       }
2095     } finally {
2096       cleanupCatalogTracker(ct);
2097     }
2098     return tmp;
2099   }
2100 
2101   /**
2102    * Check if table exists or not
2103    * @param tableName Name of a table.
2104    * @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
2105    * @return tableName instance
2106    * @throws IOException if a remote or network exception occurs.
2107    * @throws TableNotFoundException if table does not exist.
2108    */
2109   //TODO rename this method
2110   private TableName checkTableExists(
2111       final TableName tableName, CatalogTracker ct)
2112       throws IOException {
2113     if (!MetaReader.tableExists(ct, tableName)) {
2114       throw new TableNotFoundException(tableName);
2115     }
2116     return tableName;
2117   }
2118 
2119   /**
2120    * Shuts down the HBase cluster
2121    * @throws IOException if a remote or network exception occurs
2122    */
2123   public synchronized void shutdown() throws IOException {
2124     executeCallable(new MasterCallable<Void>(getConnection()) {
2125       @Override
2126       public Void call(int callTimeout) throws ServiceException {
2127         master.shutdown(null,ShutdownRequest.newBuilder().build());
2128         return null;
2129       }
2130     });
2131   }
2132 
2133   /**
2134    * Shuts down the current HBase master only.
2135    * Does not shutdown the cluster.
2136    * @see #shutdown()
2137    * @throws IOException if a remote or network exception occurs
2138    */
2139   public synchronized void stopMaster() throws IOException {
2140     executeCallable(new MasterCallable<Void>(getConnection()) {
2141       @Override
2142       public Void call(int callTimeout) throws ServiceException {
2143         master.stopMaster(null,StopMasterRequest.newBuilder().build());
2144         return null;
2145       }
2146     });
2147   }
2148 
2149   /**
2150    * Stop the designated regionserver
2151    * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
2152    * <code>example.org:1234</code>
2153    * @throws IOException if a remote or network exception occurs
2154    */
2155   public synchronized void stopRegionServer(final String hostnamePort)
2156   throws IOException {
2157     String hostname = Addressing.parseHostname(hostnamePort);
2158     int port = Addressing.parsePort(hostnamePort);
2159     AdminService.BlockingInterface admin =
2160       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
2161     StopServerRequest request = RequestConverter.buildStopServerRequest(
2162       "Called by admin client " + this.connection.toString());
2163     try {
2164       admin.stopServer(null, request);
2165     } catch (ServiceException se) {
2166       throw ProtobufUtil.getRemoteException(se);
2167     }
2168   }
2169 
2170 
2171   /**
2172    * @return cluster status
2173    * @throws IOException if a remote or network exception occurs
2174    */
2175   public ClusterStatus getClusterStatus() throws IOException {
2176     return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
2177       @Override
2178       public ClusterStatus call(int callTimeout) throws ServiceException {
2179         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
2180         return ClusterStatus.convert(master.getClusterStatus(null,req).getClusterStatus());
2181       }
2182     });
2183   }
2184 
2185   private HRegionLocation getFirstMetaServerForTable(final TableName tableName)
2186   throws IOException {
2187     return connection.locateRegion(TableName.META_TABLE_NAME,
2188       HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
2189   }
2190 
2191   /**
2192    * @return Configuration used by the instance.
2193    */
2194   public Configuration getConfiguration() {
2195     return this.conf;
2196   }
2197 
2198   /**
2199    * Create a new namespace
2200    * @param descriptor descriptor which describes the new namespace
2201    * @throws IOException
2202    */
2203   public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
2204     executeCallable(new MasterCallable<Void>(getConnection()) {
2205       @Override
2206       public Void call(int callTimeout) throws Exception {
2207         master.createNamespace(null,
2208           CreateNamespaceRequest.newBuilder()
2209                 .setNamespaceDescriptor(ProtobufUtil
2210                     .toProtoNamespaceDescriptor(descriptor)).build());
2211         return null;
2212       }
2213     });
2214   }
2215 
2216   /**
2217    * Modify an existing namespace
2218    * @param descriptor descriptor which describes the new namespace
2219    * @throws IOException
2220    */
2221   public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
2222     executeCallable(new MasterCallable<Void>(getConnection()) {
2223       @Override
2224       public Void call(int callTimeout) throws Exception {
2225         master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
2226           setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
2227         return null;
2228       }
2229     });
2230   }
2231 
2232   /**
2233    * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
2234    * @param name namespace name
2235    * @throws IOException
2236    */
2237   public void deleteNamespace(final String name) throws IOException {
2238     executeCallable(new MasterCallable<Void>(getConnection()) {
2239       @Override
2240       public Void call(int callTimeout) throws Exception {
2241         master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
2242           setNamespaceName(name).build());
2243         return null;
2244       }
2245     });
2246   }
2247 
2248   /**
2249    * Get a namespace descriptor by name
2250    * @param name name of namespace descriptor
2251    * @return A descriptor
2252    * @throws IOException
2253    */
2254   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
2255     return
2256         executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
2257           @Override
2258           public NamespaceDescriptor call(int callTimeout) throws Exception {
2259             return ProtobufUtil.toNamespaceDescriptor(
2260               master.getNamespaceDescriptor(null, GetNamespaceDescriptorRequest.newBuilder().
2261                 setNamespaceName(name).build()).getNamespaceDescriptor());
2262           }
2263         });
2264   }
2265 
2266   /**
2267    * List available namespace descriptors
2268    * @return List of descriptors
2269    * @throws IOException
2270    */
2271   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
2272     return
2273         executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
2274           @Override
2275           public NamespaceDescriptor[] call(int callTimeout) throws Exception {
2276             List<HBaseProtos.NamespaceDescriptor> list =
2277               master.listNamespaceDescriptors(null, ListNamespaceDescriptorsRequest.newBuilder().
2278                 build()).getNamespaceDescriptorList();
2279             NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
2280             for(int i = 0; i < list.size(); i++) {
2281               res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
2282             }
2283             return res;
2284           }
2285         });
2286   }
2287 
2288   /**
2289    * Get list of table descriptors by namespace
2290    * @param name namespace name
2291    * @return A descriptor
2292    * @throws IOException
2293    */
2294   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
2295     return
2296         executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
2297           @Override
2298           public HTableDescriptor[] call(int callTimeout) throws Exception {
2299             List<TableSchema> list =
2300               master.listTableDescriptorsByNamespace(null, ListTableDescriptorsByNamespaceRequest.
2301                 newBuilder().setNamespaceName(name).build()).getTableSchemaList();
2302             HTableDescriptor[] res = new HTableDescriptor[list.size()];
2303             for(int i=0; i < list.size(); i++) {
2304 
2305               res[i] = HTableDescriptor.convert(list.get(i));
2306             }
2307             return res;
2308           }
2309         });
2310   }
2311 
2312   /**
2313    * Get list of table names by namespace
2314    * @param name namespace name
2315    * @return The list of table names in the namespace
2316    * @throws IOException
2317    */
2318   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
2319     return
2320         executeCallable(new MasterCallable<TableName[]>(getConnection()) {
2321           @Override
2322           public TableName[] call(int callTimeout) throws Exception {
2323             List<HBaseProtos.TableName> tableNames =
2324               master.listTableNamesByNamespace(null, ListTableNamesByNamespaceRequest.
2325                 newBuilder().setNamespaceName(name).build())
2326                 .getTableNameList();
2327             TableName[] result = new TableName[tableNames.size()];
2328             for (int i = 0; i < tableNames.size(); i++) {
2329               result[i] = ProtobufUtil.toTableName(tableNames.get(i));
2330             }
2331             return result;
2332           }
2333         });
2334   }
2335 
2336   /**
2337    * Check to see if HBase is running. Throw an exception if not.
2338    * We consider that HBase is running if ZooKeeper and Master are running.
2339    *
2340    * @param conf system configuration
2341    * @throws MasterNotRunningException if the master is not running
2342    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
2343    */
2344   public static void checkHBaseAvailable(Configuration conf)
2345     throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException {
2346     Configuration copyOfConf = HBaseConfiguration.create(conf);
2347 
2348     // We set it to make it fail as soon as possible if HBase is not available
2349     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
2350     copyOfConf.setInt("zookeeper.recovery.retry", 0);
2351 
2352     ConnectionManager.HConnectionImplementation connection
2353       = (ConnectionManager.HConnectionImplementation)
2354       HConnectionManager.getConnection(copyOfConf);
2355 
2356     try {
2357       // Check ZK first.
2358       // If the connection exists, we may have a connection to ZK that does
2359       //  not work anymore
2360       ZooKeeperKeepAliveConnection zkw = null;
2361       try {
2362         zkw = connection.getKeepAliveZooKeeperWatcher();
2363         zkw.getRecoverableZooKeeper().getZooKeeper().exists(
2364           zkw.baseZNode, false);
2365 
2366       } catch (IOException e) {
2367         throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2368       } catch (InterruptedException e) {
2369         throw (InterruptedIOException)
2370             new InterruptedIOException("Can't connect to ZooKeeper").initCause(e);
2371       } catch (KeeperException e) {
2372         throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2373       } finally {
2374         if (zkw != null) {
2375           zkw.close();
2376         }
2377       }
2378 
2379       // Check Master
2380       connection.isMasterRunning();
2381 
2382     } finally {
2383       connection.close();
2384     }
2385   }
2386 
2387   /**
2388    * get the regions of a given table.
2389    *
2390    * @param tableName the name of the table
2391    * @return Ordered list of {@link HRegionInfo}.
2392    * @throws IOException
2393    */
2394   public List<HRegionInfo> getTableRegions(final TableName tableName)
2395   throws IOException {
2396     CatalogTracker ct = getCatalogTracker();
2397     List<HRegionInfo> Regions = null;
2398     try {
2399       Regions = MetaReader.getTableRegions(ct, tableName, true);
2400     } finally {
2401       cleanupCatalogTracker(ct);
2402     }
2403     return Regions;
2404   }
2405 
2406   public List<HRegionInfo> getTableRegions(final byte[] tableName)
2407   throws IOException {
2408     return getTableRegions(TableName.valueOf(tableName));
2409   }
2410 
2411   @Override
2412   public synchronized void close() throws IOException {
2413     if (cleanupConnectionOnClose && this.connection != null && !this.closed) {
2414       this.connection.close();
2415       this.closed = true;
2416     }
2417   }
2418 
2419   /**
2420    * Get tableDescriptors
2421    * @param tableNames List of table names
2422    * @return HTD[] the tableDescriptor
2423    * @throws IOException if a remote or network exception occurs
2424    */
2425   public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
2426   throws IOException {
2427     return this.connection.getHTableDescriptorsByTableName(tableNames);
2428   }
2429 
2430   /**
2431    * Get tableDescriptors
2432    * @param names List of table names
2433    * @return HTD[] the tableDescriptor
2434    * @throws IOException if a remote or network exception occurs
2435    */
2436   public HTableDescriptor[] getTableDescriptors(List<String> names)
2437   throws IOException {
2438     List<TableName> tableNames = new ArrayList<TableName>(names.size());
2439     for(String name : names) {
2440       tableNames.add(TableName.valueOf(name));
2441     }
2442     return getTableDescriptorsByTableName(tableNames);
2443   }
2444 
2445   /**
2446    * Roll the log writer. That is, start writing log messages to a new file.
2447    *
2448    * @param serverName
2449    *          The servername of the regionserver. A server name is made of host,
2450    *          port and startcode. This is mandatory. Here is an example:
2451    *          <code> host187.example.com,60020,1289493121758</code>
2452    * @return If lots of logs, flush the returned regions so next time through
2453    * we can clean logs. Returns null if nothing to flush.  Names are actual
2454    * region names as returned by {@link HRegionInfo#getEncodedName()}
2455    * @throws IOException if a remote or network exception occurs
2456    * @throws FailedLogCloseException
2457    */
2458  public synchronized  byte[][] rollHLogWriter(String serverName)
2459       throws IOException, FailedLogCloseException {
2460     ServerName sn = ServerName.valueOf(serverName);
2461     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2462     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
2463     try {
2464       RollWALWriterResponse response = admin.rollWALWriter(null, request);
2465       int regionCount = response.getRegionToFlushCount();
2466       byte[][] regionsToFlush = new byte[regionCount][];
2467       for (int i = 0; i < regionCount; i++) {
2468         ByteString region = response.getRegionToFlush(i);
2469         regionsToFlush[i] = region.toByteArray();
2470       }
2471       return regionsToFlush;
2472     } catch (ServiceException se) {
2473       throw ProtobufUtil.getRemoteException(se);
2474     }
2475   }
2476 
2477   public String[] getMasterCoprocessors() {
2478     try {
2479       return getClusterStatus().getMasterCoprocessors();
2480     } catch (IOException e) {
2481       LOG.error("Could not getClusterStatus()",e);
2482       return null;
2483     }
2484   }
2485 
2486   /**
2487    * Get the current compaction state of a table or region.
2488    * It could be in a major compaction, a minor compaction, both, or none.
2489    *
2490    * @param tableNameOrRegionName table or region to major compact
2491    * @throws IOException if a remote or network exception occurs
2492    * @throws InterruptedException
2493    * @return the current compaction state
2494    */
2495   public CompactionState getCompactionState(final String tableNameOrRegionName)
2496       throws IOException, InterruptedException {
2497     return getCompactionState(Bytes.toBytes(tableNameOrRegionName));
2498   }
2499 
2500   /**
2501    * Get the current compaction state of a table or region.
2502    * It could be in a major compaction, a minor compaction, both, or none.
2503    *
2504    * @param tableNameOrRegionName table or region to major compact
2505    * @throws IOException if a remote or network exception occurs
2506    * @throws InterruptedException
2507    * @return the current compaction state
2508    */
2509   public CompactionState getCompactionState(final byte[] tableNameOrRegionName)
2510       throws IOException, InterruptedException {
2511     CompactionState state = CompactionState.NONE;
2512     CatalogTracker ct = getCatalogTracker();
2513     try {
2514       Pair<HRegionInfo, ServerName> regionServerPair
2515         = getRegion(tableNameOrRegionName, ct);
2516       if (regionServerPair != null) {
2517         if (regionServerPair.getSecond() == null) {
2518           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
2519         } else {
2520           ServerName sn = regionServerPair.getSecond();
2521           AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2522           GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
2523             regionServerPair.getFirst().getRegionName(), true);
2524           GetRegionInfoResponse response = admin.getRegionInfo(null, request);
2525           return response.getCompactionState();
2526         }
2527       } else {
2528         final TableName tableName =
2529             checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
2530         List<Pair<HRegionInfo, ServerName>> pairs =
2531           MetaReader.getTableRegionsAndLocations(ct, tableName);
2532         for (Pair<HRegionInfo, ServerName> pair: pairs) {
2533           if (pair.getFirst().isOffline()) continue;
2534           if (pair.getSecond() == null) continue;
2535           try {
2536             ServerName sn = pair.getSecond();
2537             AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2538             GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
2539               pair.getFirst().getRegionName(), true);
2540             GetRegionInfoResponse response = admin.getRegionInfo(null, request);
2541             switch (response.getCompactionState()) {
2542             case MAJOR_AND_MINOR:
2543               return CompactionState.MAJOR_AND_MINOR;
2544             case MAJOR:
2545               if (state == CompactionState.MINOR) {
2546                 return CompactionState.MAJOR_AND_MINOR;
2547               }
2548               state = CompactionState.MAJOR;
2549               break;
2550             case MINOR:
2551               if (state == CompactionState.MAJOR) {
2552                 return CompactionState.MAJOR_AND_MINOR;
2553               }
2554               state = CompactionState.MINOR;
2555               break;
2556             case NONE:
2557               default: // nothing, continue
2558             }
2559           } catch (NotServingRegionException e) {
2560             if (LOG.isDebugEnabled()) {
2561               LOG.debug("Trying to get compaction state of " +
2562                 pair.getFirst() + ": " +
2563                 StringUtils.stringifyException(e));
2564             }
2565           } catch (RemoteException e) {
2566             if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
2567               if (LOG.isDebugEnabled()) {
2568                 LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
2569                     + StringUtils.stringifyException(e));
2570               }
2571             } else {
2572               throw e;
2573             }
2574           }
2575         }
2576       }
2577     } catch (ServiceException se) {
2578       throw ProtobufUtil.getRemoteException(se);
2579     } finally {
2580       cleanupCatalogTracker(ct);
2581     }
2582     return state;
2583   }
2584 
2585   /**
2586    * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
2587    * taken. If the table is disabled, an offline snapshot is taken.
2588    * <p>
2589    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2590    * snapshot with the same name (even a different type or with different parameters) will fail with
2591    * a {@link SnapshotCreationException} indicating the duplicate naming.
2592    * <p>
2593    * Snapshot names follow the same naming constraints as tables in HBase. See
2594    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
2595    * @param snapshotName name of the snapshot to be created
2596    * @param tableName name of the table for which snapshot is created
2597    * @throws IOException if a remote or network exception occurs
2598    * @throws SnapshotCreationException if snapshot creation failed
2599    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2600    */
2601   public void snapshot(final String snapshotName,
2602                        final TableName tableName) throws IOException,
2603       SnapshotCreationException, IllegalArgumentException {
2604     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
2605   }
2606 
2607   public void snapshot(final String snapshotName,
2608                        final String tableName) throws IOException,
2609       SnapshotCreationException, IllegalArgumentException {
2610     snapshot(snapshotName, TableName.valueOf(tableName),
2611         SnapshotDescription.Type.FLUSH);
2612   }
2613 
2614   /**
2615    public void snapshot(final String snapshotName,
2616     * Create a timestamp consistent snapshot for the given table.
2617                         final byte[] tableName) throws IOException,
2618     * <p>
2619     * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2620     * snapshot with the same name (even a different type or with different parameters) will fail with
2621     * a {@link SnapshotCreationException} indicating the duplicate naming.
2622     * <p>
2623     * Snapshot names follow the same naming constraints as tables in HBase.
2624     * @param snapshotName name of the snapshot to be created
2625     * @param tableName name of the table for which snapshot is created
2626     * @throws IOException if a remote or network exception occurs
2627     * @throws SnapshotCreationException if snapshot creation failed
2628     * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2629     */
2630   public void snapshot(final byte[] snapshotName,
2631                        final TableName tableName) throws IOException,
2632       SnapshotCreationException, IllegalArgumentException {
2633     snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH);
2634   }
2635 
2636   public void snapshot(final byte[] snapshotName,
2637                        final byte[] tableName) throws IOException,
2638       SnapshotCreationException, IllegalArgumentException {
2639     snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName),
2640         SnapshotDescription.Type.FLUSH);
2641   }
2642 
2643   /**
2644    * Create typed snapshot of the table.
2645    * <p>
2646    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2647    * snapshot with the same name (even a different type or with different parameters) will fail with
2648    * a {@link SnapshotCreationException} indicating the duplicate naming.
2649    * <p>
2650    * Snapshot names follow the same naming constraints as tables in HBase. See
2651    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
2652    * <p>
2653    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
2654    *          snapshots stored on the cluster
2655    * @param tableName name of the table to snapshot
2656    * @param type type of snapshot to take
2657    * @throws IOException we fail to reach the master
2658    * @throws SnapshotCreationException if snapshot creation failed
2659    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2660    */
2661   public void snapshot(final String snapshotName,
2662                        final TableName tableName,
2663                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2664       IllegalArgumentException {
2665     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
2666     builder.setTable(tableName.getNameAsString());
2667     builder.setName(snapshotName);
2668     builder.setType(type);
2669     snapshot(builder.build());
2670   }
2671 
2672   public void snapshot(final String snapshotName,
2673                        final String tableName,
2674                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2675       IllegalArgumentException {
2676     snapshot(snapshotName, TableName.valueOf(tableName), type);
2677   }
2678 
2679   public void snapshot(final String snapshotName,
2680                        final byte[] tableName,
2681                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2682       IllegalArgumentException {
2683     snapshot(snapshotName, TableName.valueOf(tableName), type);
2684   }
2685 
2686   /**
2687    * Take a snapshot and wait for the server to complete that snapshot (blocking).
2688    * <p>
2689    * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
2690    * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
2691    * time for a single cluster).
2692    * <p>
2693    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2694    * snapshot with the same name (even a different type or with different parameters) will fail with
2695    * a {@link SnapshotCreationException} indicating the duplicate naming.
2696    * <p>
2697    * Snapshot names follow the same naming constraints as tables in HBase. See
2698    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
2699    * <p>
2700    * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
2701    * unless you are sure about the type of snapshot that you want to take.
2702    * @param snapshot snapshot to take
2703    * @throws IOException or we lose contact with the master.
2704    * @throws SnapshotCreationException if snapshot failed to be taken
2705    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2706    */
2707   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
2708       IllegalArgumentException {
2709     // actually take the snapshot
2710     SnapshotResponse response = takeSnapshotAsync(snapshot);
2711     final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
2712         .build();
2713     IsSnapshotDoneResponse done = null;
2714     long start = EnvironmentEdgeManager.currentTimeMillis();
2715     long max = response.getExpectedTimeout();
2716     long maxPauseTime = max / this.numRetries;
2717     int tries = 0;
2718     LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
2719         ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
2720         maxPauseTime + " ms per retry)");
2721     while (tries == 0
2722         || ((EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done.getDone())) {
2723       try {
2724         // sleep a backoff <= pauseTime amount
2725         long sleep = getPauseTime(tries++);
2726         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2727         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
2728           "ms while waiting for snapshot completion.");
2729         Thread.sleep(sleep);
2730       } catch (InterruptedException e) {
2731         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
2732       }
2733       LOG.debug("Getting current status of snapshot from master...");
2734       done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
2735         @Override
2736         public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
2737           return master.isSnapshotDone(null, request);
2738         }
2739       });
2740     }
2741     if (!done.getDone()) {
2742       throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
2743           + "' wasn't completed in expectedTime:" + max + " ms", snapshot);
2744     }
2745   }
2746 
2747   /**
2748    * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
2749    * <p>
2750    * Only a single snapshot should be taken at a time, or results may be undefined.
2751    * @param snapshot snapshot to take
2752    * @return response from the server indicating the max time to wait for the snapshot
2753    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
2754    * @throws SnapshotCreationException if snapshot creation failed
2755    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2756    */
2757   public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
2758       SnapshotCreationException {
2759     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
2760     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
2761         .build();
2762     // run the snapshot on the master
2763     return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
2764       @Override
2765       public SnapshotResponse call(int callTimeout) throws ServiceException {
2766         return master.snapshot(null, request);
2767       }
2768     });
2769   }
2770 
2771   /**
2772    * Check the current state of the passed snapshot.
2773    * <p>
2774    * There are three possible states:
2775    * <ol>
2776    * <li>running - returns <tt>false</tt></li>
2777    * <li>finished - returns <tt>true</tt></li>
2778    * <li>finished with error - throws the exception that caused the snapshot to fail</li>
2779    * </ol>
2780    * <p>
2781    * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
2782    * run/started since the snapshot your are checking, you will recieve an
2783    * {@link UnknownSnapshotException}.
2784    * @param snapshot description of the snapshot to check
2785    * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
2786    *         running
2787    * @throws IOException if we have a network issue
2788    * @throws HBaseSnapshotException if the snapshot failed
2789    * @throws UnknownSnapshotException if the requested snapshot is unknown
2790    */
2791   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
2792       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
2793 
2794     return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
2795       @Override
2796       public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
2797         return master.isSnapshotDone(null,
2798           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
2799       }
2800     }).getDone();
2801   }
2802 
2803   /**
2804    * Restore the specified snapshot on the original table. (The table must be disabled)
2805    * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
2806    * is set to true, a snapshot of the current table is taken
2807    * before executing the restore operation.
2808    * In case of restore failure, the failsafe snapshot will be restored.
2809    * If the restore completes without problem the failsafe snapshot is deleted.
2810    *
2811    * @param snapshotName name of the snapshot to restore
2812    * @throws IOException if a remote or network exception occurs
2813    * @throws RestoreSnapshotException if snapshot failed to be restored
2814    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2815    */
2816   public void restoreSnapshot(final byte[] snapshotName)
2817       throws IOException, RestoreSnapshotException {
2818     restoreSnapshot(Bytes.toString(snapshotName));
2819   }
2820 
2821   /**
2822    * Restore the specified snapshot on the original table. (The table must be disabled)
2823    * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
2824    * is set to true, a snapshot of the current table is taken
2825    * before executing the restore operation.
2826    * In case of restore failure, the failsafe snapshot will be restored.
2827    * If the restore completes without problem the failsafe snapshot is deleted.
2828    *
2829    * @param snapshotName name of the snapshot to restore
2830    * @throws IOException if a remote or network exception occurs
2831    * @throws RestoreSnapshotException if snapshot failed to be restored
2832    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2833    */
2834   public void restoreSnapshot(final String snapshotName)
2835       throws IOException, RestoreSnapshotException {
2836     boolean takeFailSafeSnapshot =
2837       conf.getBoolean("hbase.snapshot.restore.take.failsafe.snapshot", false);
2838     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
2839   }
2840 
2841   /**
2842    * Restore the specified snapshot on the original table. (The table must be disabled)
2843    * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
2844    * before executing the restore operation.
2845    * In case of restore failure, the failsafe snapshot will be restored.
2846    * If the restore completes without problem the failsafe snapshot is deleted.
2847    *
2848    * The failsafe snapshot name is configurable by using the property
2849    * "hbase.snapshot.restore.failsafe.name".
2850    *
2851    * @param snapshotName name of the snapshot to restore
2852    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
2853    * @throws IOException if a remote or network exception occurs
2854    * @throws RestoreSnapshotException if snapshot failed to be restored
2855    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2856    */
2857   public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
2858       throws IOException, RestoreSnapshotException {
2859     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
2860   }
2861 
2862   /**
2863    * Restore the specified snapshot on the original table. (The table must be disabled)
2864    * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
2865    * before executing the restore operation.
2866    * In case of restore failure, the failsafe snapshot will be restored.
2867    * If the restore completes without problem the failsafe snapshot is deleted.
2868    *
2869    * The failsafe snapshot name is configurable by using the property
2870    * "hbase.snapshot.restore.failsafe.name".
2871    *
2872    * @param snapshotName name of the snapshot to restore
2873    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
2874    * @throws IOException if a remote or network exception occurs
2875    * @throws RestoreSnapshotException if snapshot failed to be restored
2876    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2877    */
2878   public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
2879       throws IOException, RestoreSnapshotException {
2880     TableName tableName = null;
2881     for (SnapshotDescription snapshotInfo: listSnapshots()) {
2882       if (snapshotInfo.getName().equals(snapshotName)) {
2883         tableName = TableName.valueOf(snapshotInfo.getTable());
2884         break;
2885       }
2886     }
2887 
2888     if (tableName == null) {
2889       throw new RestoreSnapshotException(
2890         "Unable to find the table name for snapshot=" + snapshotName);
2891     }
2892 
2893     // The table does not exists, switch to clone.
2894     if (!tableExists(tableName)) {
2895       try {
2896         cloneSnapshot(snapshotName, tableName);
2897       } catch (InterruptedException e) {
2898         throw new InterruptedIOException("Interrupted when restoring a nonexistent table: " +
2899           e.getMessage());
2900       }
2901       return;
2902     }
2903 
2904     // Check if the table is disabled
2905     if (!isTableDisabled(tableName)) {
2906       throw new TableNotDisabledException(tableName);
2907     }
2908 
2909     // Take a snapshot of the current state
2910     String failSafeSnapshotSnapshotName = null;
2911     if (takeFailSafeSnapshot) {
2912       failSafeSnapshotSnapshotName = conf.get("hbase.snapshot.restore.failsafe.name",
2913         "hbase-failsafe-{snapshot.name}-{restore.timestamp}");
2914       failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotName
2915         .replace("{snapshot.name}", snapshotName)
2916         .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
2917         .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTimeMillis()));
2918       LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
2919       snapshot(failSafeSnapshotSnapshotName, tableName);
2920     }
2921 
2922     try {
2923       // Restore snapshot
2924       internalRestoreSnapshot(snapshotName, tableName);
2925     } catch (IOException e) {
2926       // Somthing went wrong during the restore...
2927       // if the pre-restore snapshot is available try to rollback
2928       if (takeFailSafeSnapshot) {
2929         try {
2930           internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName);
2931           String msg = "Restore snapshot=" + snapshotName +
2932             " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded.";
2933           LOG.error(msg, e);
2934           throw new RestoreSnapshotException(msg, e);
2935         } catch (IOException ex) {
2936           String msg = "Failed to restore and rollback to snapshot=" + failSafeSnapshotSnapshotName;
2937           LOG.error(msg, ex);
2938           throw new RestoreSnapshotException(msg, e);
2939         }
2940       } else {
2941         throw new RestoreSnapshotException("Failed to restore snapshot=" + snapshotName, e);
2942       }
2943     }
2944 
2945     // If the restore is succeeded, delete the pre-restore snapshot
2946     if (takeFailSafeSnapshot) {
2947       try {
2948         LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
2949         deleteSnapshot(failSafeSnapshotSnapshotName);
2950       } catch (IOException e) {
2951         LOG.error("Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, e);
2952       }
2953     }
2954   }
2955 
2956   /**
2957    * Create a new table by cloning the snapshot content.
2958    *
2959    * @param snapshotName name of the snapshot to be cloned
2960    * @param tableName name of the table where the snapshot will be restored
2961    * @throws IOException if a remote or network exception occurs
2962    * @throws TableExistsException if table to be created already exists
2963    * @throws RestoreSnapshotException if snapshot failed to be cloned
2964    * @throws IllegalArgumentException if the specified table has not a valid name
2965    */
2966   public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
2967       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2968     cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
2969   }
2970 
2971   /**
2972    * Create a new table by cloning the snapshot content.
2973    *
2974    * @param snapshotName name of the snapshot to be cloned
2975    * @param tableName name of the table where the snapshot will be restored
2976    * @throws IOException if a remote or network exception occurs
2977    * @throws TableExistsException if table to be created already exists
2978    * @throws RestoreSnapshotException if snapshot failed to be cloned
2979    * @throws IllegalArgumentException if the specified table has not a valid name
2980    */
2981   public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
2982       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2983     cloneSnapshot(Bytes.toString(snapshotName), tableName);
2984   }
2985 
2986 
2987 
2988   /**
2989    * Create a new table by cloning the snapshot content.
2990    *
2991    * @param snapshotName name of the snapshot to be cloned
2992    * @param tableName name of the table where the snapshot will be restored
2993    * @throws IOException if a remote or network exception occurs
2994    * @throws TableExistsException if table to be created already exists
2995    * @throws RestoreSnapshotException if snapshot failed to be cloned
2996    * @throws IllegalArgumentException if the specified table has not a valid name
2997    */
2998   public void cloneSnapshot(final String snapshotName, final String tableName)
2999       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3000     cloneSnapshot(snapshotName, TableName.valueOf(tableName));
3001   }
3002 
3003   /**
3004    * Create a new table by cloning the snapshot content.
3005    *
3006    * @param snapshotName name of the snapshot to be cloned
3007    * @param tableName name of the table where the snapshot will be restored
3008    * @throws IOException if a remote or network exception occurs
3009    * @throws TableExistsException if table to be created already exists
3010    * @throws RestoreSnapshotException if snapshot failed to be cloned
3011    * @throws IllegalArgumentException if the specified table has not a valid name
3012    */
3013   public void cloneSnapshot(final String snapshotName, final TableName tableName)
3014       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3015     if (tableExists(tableName)) {
3016       throw new TableExistsException(tableName);
3017     }
3018     internalRestoreSnapshot(snapshotName, tableName);
3019     waitUntilTableIsEnabled(tableName);
3020   }
3021 
3022   /**
3023    * Execute a distributed procedure on a cluster.
3024    *
3025    * @param signature A distributed procedure is uniquely identified
3026    * by its signature (default the root ZK node name of the procedure).
3027    * @param instance The instance name of the procedure. For some procedures, this parameter is
3028    * optional.
3029    * @param props Property/Value pairs of properties passing to the procedure
3030    */
3031   public void execProcedure(String signature, String instance,
3032       Map<String, String> props) throws IOException {
3033     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3034     builder.setSignature(signature).setInstance(instance);
3035     for (Entry<String, String> entry : props.entrySet()) {
3036       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3037           .setValue(entry.getValue()).build();
3038       builder.addConfiguration(pair);
3039     }
3040 
3041     final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
3042         .setProcedure(builder.build()).build();
3043     // run the procedure on the master
3044     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
3045         getConnection()) {
3046       @Override
3047       public ExecProcedureResponse call(int callTimeout) throws ServiceException {
3048         return master.execProcedure(null, request);
3049       }
3050     });
3051 
3052     long start = EnvironmentEdgeManager.currentTimeMillis();
3053     long max = response.getExpectedTimeout();
3054     long maxPauseTime = max / this.numRetries;
3055     int tries = 0;
3056     LOG.debug("Waiting a max of " + max + " ms for procedure '" +
3057         signature + " : " + instance + "'' to complete. (max " + maxPauseTime + " ms per retry)");
3058     boolean done = false;
3059     while (tries == 0
3060         || ((EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done)) {
3061       try {
3062         // sleep a backoff <= pauseTime amount
3063         long sleep = getPauseTime(tries++);
3064         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3065         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
3066           "ms while waiting for procedure completion.");
3067         Thread.sleep(sleep);
3068       } catch (InterruptedException e) {
3069         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3070       }
3071       LOG.debug("Getting current status of procedure from master...");
3072       done = isProcedureFinished(signature, instance, props);
3073     }
3074     if (!done) {
3075       throw new IOException("Procedure '" + signature + " : " + instance
3076           + "' wasn't completed in expectedTime:" + max + " ms");
3077     }
3078   }
3079 
3080   /**
3081    * Check the current state of the specified procedure.
3082    * <p>
3083    * There are three possible states:
3084    * <ol>
3085    * <li>running - returns <tt>false</tt></li>
3086    * <li>finished - returns <tt>true</tt></li>
3087    * <li>finished with error - throws the exception that caused the procedure to fail</li>
3088    * </ol>
3089    * <p>
3090    *
3091    * @param signature The signature that uniquely identifies a procedure
3092    * @param instance The instance name of the procedure
3093    * @param props Property/Value pairs of properties passing to the procedure
3094    * @return true if the specified procedure is finished successfully, false if it is still running
3095    * @throws IOException if the specified procedure finished with error
3096    */
3097   public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
3098       throws IOException {
3099     final ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3100     builder.setSignature(signature).setInstance(instance);
3101     for (Entry<String, String> entry : props.entrySet()) {
3102       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3103           .setValue(entry.getValue()).build();
3104       builder.addConfiguration(pair);
3105     }
3106     final ProcedureDescription desc = builder.build();
3107     return executeCallable(
3108         new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
3109           @Override
3110           public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
3111             return master.isProcedureDone(null, IsProcedureDoneRequest
3112                 .newBuilder().setProcedure(desc).build());
3113           }
3114         }).getDone();
3115   }
3116 
3117   /**
3118    * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
3119    * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
3120    * create an HTable instance to this table before it is available.
3121    * @param snapshotName snapshot to restore
3122    * @param tableName table name to restore the snapshot on
3123    * @throws IOException if a remote or network exception occurs
3124    * @throws RestoreSnapshotException if snapshot failed to be restored
3125    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3126    */
3127   private void internalRestoreSnapshot(final String snapshotName, final TableName
3128       tableName)
3129       throws IOException, RestoreSnapshotException {
3130     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
3131         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
3132 
3133     // actually restore the snapshot
3134     internalRestoreSnapshotAsync(snapshot);
3135 
3136     final IsRestoreSnapshotDoneRequest request = IsRestoreSnapshotDoneRequest.newBuilder()
3137         .setSnapshot(snapshot).build();
3138     IsRestoreSnapshotDoneResponse done = IsRestoreSnapshotDoneResponse.newBuilder()
3139         .setDone(false).buildPartial();
3140     final long maxPauseTime = 5000;
3141     int tries = 0;
3142     while (!done.getDone()) {
3143       try {
3144         // sleep a backoff <= pauseTime amount
3145         long sleep = getPauseTime(tries++);
3146         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3147         LOG.debug(tries + ") Sleeping: " + sleep + " ms while we wait for snapshot restore to complete.");
3148         Thread.sleep(sleep);
3149       } catch (InterruptedException e) {
3150         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3151       }
3152       LOG.debug("Getting current status of snapshot restore from master...");
3153       done = executeCallable(new MasterCallable<IsRestoreSnapshotDoneResponse>(
3154           getConnection()) {
3155         @Override
3156         public IsRestoreSnapshotDoneResponse call(int callTimeout) throws ServiceException {
3157           return master.isRestoreSnapshotDone(null, request);
3158         }
3159       });
3160     }
3161     if (!done.getDone()) {
3162       throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
3163     }
3164   }
3165 
3166   /**
3167    * Execute Restore/Clone snapshot and wait for the server to complete (asynchronous)
3168    * <p>
3169    * Only a single snapshot should be restored at a time, or results may be undefined.
3170    * @param snapshot snapshot to restore
3171    * @return response from the server indicating the max time to wait for the snapshot
3172    * @throws IOException if a remote or network exception occurs
3173    * @throws RestoreSnapshotException if snapshot failed to be restored
3174    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3175    */
3176   private RestoreSnapshotResponse internalRestoreSnapshotAsync(final SnapshotDescription snapshot)
3177       throws IOException, RestoreSnapshotException {
3178     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
3179 
3180     final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
3181         .build();
3182 
3183     // run the snapshot restore on the master
3184     return executeCallable(new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
3185       @Override
3186       public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
3187         return master.restoreSnapshot(null, request);
3188       }
3189     });
3190   }
3191 
3192   /**
3193    * List completed snapshots.
3194    * @return a list of snapshot descriptors for completed snapshots
3195    * @throws IOException if a network error occurs
3196    */
3197   public List<SnapshotDescription> listSnapshots() throws IOException {
3198     return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
3199       @Override
3200       public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
3201         return master.getCompletedSnapshots(null, GetCompletedSnapshotsRequest.newBuilder().build())
3202             .getSnapshotsList();
3203       }
3204     });
3205   }
3206 
3207   /**
3208    * List all the completed snapshots matching the given regular expression.
3209    *
3210    * @param regex The regular expression to match against
3211    * @return - returns a List of SnapshotDescription
3212    * @throws IOException if a remote or network exception occurs
3213    */
3214   public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
3215     return listSnapshots(Pattern.compile(regex));
3216   }
3217 
3218   /**
3219    * List all the completed snapshots matching the given pattern.
3220    *
3221    * @param pattern The compiled regular expression to match against
3222    * @return - returns a List of SnapshotDescription
3223    * @throws IOException if a remote or network exception occurs
3224    */
3225   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
3226     List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>();
3227     List<SnapshotDescription> snapshots = listSnapshots();
3228     for (SnapshotDescription snapshot : snapshots) {
3229       if (pattern.matcher(snapshot.getName()).matches()) {
3230         matched.add(snapshot);
3231       }
3232     }
3233     return matched;
3234   }
3235 
3236   /**
3237    * Delete an existing snapshot.
3238    * @param snapshotName name of the snapshot
3239    * @throws IOException if a remote or network exception occurs
3240    */
3241   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
3242     deleteSnapshot(Bytes.toString(snapshotName));
3243   }
3244 
3245   /**
3246    * Delete an existing snapshot.
3247    * @param snapshotName name of the snapshot
3248    * @throws IOException if a remote or network exception occurs
3249    */
3250   public void deleteSnapshot(final String snapshotName) throws IOException {
3251     // make sure the snapshot is possibly valid
3252     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
3253     // do the delete
3254     executeCallable(new MasterCallable<Void>(getConnection()) {
3255       @Override
3256       public Void call(int callTimeout) throws ServiceException {
3257         master.deleteSnapshot(null,
3258           DeleteSnapshotRequest.newBuilder().
3259             setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build());
3260         return null;
3261       }
3262     });
3263   }
3264 
3265   /**
3266    * Delete existing snapshots whose names match the pattern passed.
3267    * @param regex The regular expression to match against
3268    * @throws IOException if a remote or network exception occurs
3269    */
3270   public void deleteSnapshots(final String regex) throws IOException {
3271     deleteSnapshots(Pattern.compile(regex));
3272   }
3273 
3274   /**
3275    * Delete existing snapshots whose names match the pattern passed.
3276    * @param pattern pattern for names of the snapshot to match
3277    * @throws IOException if a remote or network exception occurs
3278    */
3279   public void deleteSnapshots(final Pattern pattern) throws IOException {
3280     List<SnapshotDescription> snapshots = listSnapshots(pattern);
3281     for (final SnapshotDescription snapshot : snapshots) {
3282       // do the delete
3283       executeCallable(new MasterCallable<Void>(getConnection()) {
3284         @Override
3285         public Void call(int callTimeout) throws ServiceException {
3286           this.master.deleteSnapshot(null,
3287             DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot).build());
3288           return null;
3289         }
3290       });
3291     }
3292   }
3293 
3294   /**
3295    * Parent of {@link MasterCallable} and {@link MasterCallable}.
3296    * Has common methods.
3297    * @param <V>
3298    */
3299   abstract static class MasterCallable<V> implements RetryingCallable<V>, Closeable {
3300     protected HConnection connection;
3301     protected MasterKeepAliveConnection master;
3302 
3303     public MasterCallable(final HConnection connection) {
3304       this.connection = connection;
3305     }
3306 
3307     @Override
3308     public void prepare(boolean reload) throws IOException {
3309       this.master = this.connection.getKeepAliveMasterService();
3310     }
3311 
3312     @Override
3313     public void close() throws IOException {
3314       // The above prepare could fail but this would still be called though masterAdmin is null
3315       if (this.master != null) this.master.close();
3316     }
3317 
3318     @Override
3319     public void throwable(Throwable t, boolean retrying) {
3320     }
3321 
3322     @Override
3323     public String getExceptionMessageAdditionalDetail() {
3324       return "";
3325     }
3326 
3327     @Override
3328     public long sleep(long pause, int tries) {
3329       return ConnectionUtils.getPauseTime(pause, tries);
3330     }
3331   }
3332 
3333   private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
3334     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
3335     try {
3336       return caller.callWithRetries(callable, operationTimeout);
3337     } finally {
3338       callable.close();
3339     }
3340   }
3341 
3342   /**
3343    * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
3344    * connected to the active master.
3345    *
3346    * <p>
3347    * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
3348    * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
3349    * </p>
3350    *
3351    * <div style="background-color: #cccccc; padding: 2px">
3352    * <blockquote><pre>
3353    * CoprocessorRpcChannel channel = myAdmin.coprocessorService();
3354    * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
3355    * MyCallRequest request = MyCallRequest.newBuilder()
3356    *     ...
3357    *     .build();
3358    * MyCallResponse response = service.myCall(null, request);
3359    * </pre></blockquote></div>
3360    *
3361    * @return A MasterCoprocessorRpcChannel instance
3362    */
3363   public CoprocessorRpcChannel coprocessorService() {
3364     return new MasterCoprocessorRpcChannel(connection);
3365   }
3366 }