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.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.HashMap;
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.concurrent.ExecutionException;
34  import java.util.concurrent.Future;
35  import java.util.concurrent.TimeUnit;
36  import java.util.concurrent.TimeoutException;
37  import java.util.regex.Pattern;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.hbase.Abortable;
43  import org.apache.hadoop.hbase.ClusterStatus;
44  import org.apache.hadoop.hbase.DoNotRetryIOException;
45  import org.apache.hadoop.hbase.HBaseConfiguration;
46  import org.apache.hadoop.hbase.HColumnDescriptor;
47  import org.apache.hadoop.hbase.HConstants;
48  import org.apache.hadoop.hbase.HRegionInfo;
49  import org.apache.hadoop.hbase.HRegionLocation;
50  import org.apache.hadoop.hbase.HTableDescriptor;
51  import org.apache.hadoop.hbase.MasterNotRunningException;
52  import org.apache.hadoop.hbase.MetaTableAccessor;
53  import org.apache.hadoop.hbase.NamespaceDescriptor;
54  import org.apache.hadoop.hbase.NotServingRegionException;
55  import org.apache.hadoop.hbase.RegionLocations;
56  import org.apache.hadoop.hbase.ServerName;
57  import org.apache.hadoop.hbase.TableExistsException;
58  import org.apache.hadoop.hbase.TableName;
59  import org.apache.hadoop.hbase.TableNotDisabledException;
60  import org.apache.hadoop.hbase.TableNotFoundException;
61  import org.apache.hadoop.hbase.UnknownRegionException;
62  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
63  import org.apache.hadoop.hbase.classification.InterfaceAudience;
64  import org.apache.hadoop.hbase.classification.InterfaceStability;
65  import org.apache.hadoop.hbase.exceptions.DeserializationException;
66  import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
67  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
68  import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
69  import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
70  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
71  import org.apache.hadoop.hbase.protobuf.RequestConverter;
72  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
73  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
74  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
75  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
76  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
77  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
78  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
79  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
80  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
81  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
82  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
83  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
84  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
85  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
86  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
87  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
88  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
89  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
90  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
91  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
92  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
93  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
94  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
95  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
96  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
97  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
98  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
99  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
100 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
101 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
102 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
103 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
104 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
105 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
106 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
107 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
108 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
109 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
110 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
111 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
112 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
113 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
114 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
115 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
116 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
117 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
118 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
119 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
121 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
122 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
123 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
124 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
125 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
126 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
127 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
128 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
129 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
130 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
131 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
132 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
133 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
134 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
135 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
136 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
137 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
138 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
139 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
140 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
141 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
142 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
143 import org.apache.hadoop.hbase.quotas.QuotaFilter;
144 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
145 import org.apache.hadoop.hbase.quotas.QuotaSettings;
146 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
147 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
148 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
149 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
150 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
151 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
152 import org.apache.hadoop.hbase.util.Addressing;
153 import org.apache.hadoop.hbase.util.Bytes;
154 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
155 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
156 import org.apache.hadoop.hbase.util.Pair;
157 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
158 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
159 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
160 import org.apache.hadoop.ipc.RemoteException;
161 import org.apache.hadoop.util.StringUtils;
162 import org.apache.zookeeper.KeeperException;
163 
164 import com.google.common.annotations.VisibleForTesting;
165 import com.google.protobuf.ByteString;
166 import com.google.protobuf.ServiceException;
167 
168 /**
169  * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
170  * this is an HBase-internal class as defined in
171  * https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/InterfaceClassification.html
172  * There are no guarantees for backwards source / binary compatibility and methods or class can
173  * change or go away without deprecation.
174  * Use {@link Connection#getAdmin()} to obtain an instance of {@link Admin} instead of constructing
175  * an HBaseAdmin directly.
176  *
177  * <p>Connection should be an <i>unmanaged</i> connection obtained via
178  * {@link ConnectionFactory#createConnection(Configuration)}
179  *
180  * @see ConnectionFactory
181  * @see Connection
182  * @see Admin
183  */
184 @InterfaceAudience.Private
185 @InterfaceStability.Evolving
186 public class HBaseAdmin implements Admin {
187   private static final Log LOG = LogFactory.getLog(HBaseAdmin.class);
188 
189   private static final String ZK_IDENTIFIER_PREFIX =  "hbase-admin-on-";
190 
191   private ClusterConnection connection;
192 
193   private volatile Configuration conf;
194   private final long pause;
195   private final int numRetries;
196   // Some operations can take a long time such as disable of big table.
197   // numRetries is for 'normal' stuff... Multiply by this factor when
198   // want to wait a long time.
199   private final int retryLongerMultiplier;
200   private final int syncWaitTimeout;
201   private boolean aborted;
202   private boolean cleanupConnectionOnClose = false; // close the connection in close()
203   private boolean closed = false;
204   private int operationTimeout;
205 
206   private RpcRetryingCallerFactory rpcCallerFactory;
207 
208   private NonceGenerator ng;
209 
210   /**
211    * Constructor.
212    * See {@link #HBaseAdmin(Connection connection)}
213    *
214    * @param c Configuration object. Copied internally.
215    * @deprecated Constructing HBaseAdmin objects manually has been deprecated.
216    * Use {@link Connection#getAdmin()} to obtain an instance of {@link Admin} instead.
217    */
218   @Deprecated
219   public HBaseAdmin(Configuration c)
220   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
221     this(ConnectionFactory.createConnection(new Configuration(c)));
222     this.cleanupConnectionOnClose = true;
223   }
224 
225   @Override
226   public int getOperationTimeout() {
227     return operationTimeout;
228   }
229 
230 
231   /**
232    * Constructor for externally managed Connections.
233    * The connection to master will be created when required by admin functions.
234    *
235    * @param connection The Connection instance to use
236    * @throws MasterNotRunningException
237    * @throws ZooKeeperConnectionException are not
238    *  thrown anymore but kept into the interface for backward api compatibility
239    * @deprecated Constructing HBaseAdmin objects manually has been deprecated.
240    * Use {@link Connection#getAdmin()} to obtain an instance of {@link Admin} instead.
241    */
242   @Deprecated
243   public HBaseAdmin(Connection connection)
244       throws MasterNotRunningException, ZooKeeperConnectionException {
245     this((ClusterConnection)connection);
246   }
247 
248   HBaseAdmin(ClusterConnection connection) {
249     this.conf = connection.getConfiguration();
250     this.connection = connection;
251 
252     this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
253         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
254     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
255         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
256     this.retryLongerMultiplier = this.conf.getInt(
257         "hbase.client.retries.longer.multiplier", 10);
258     this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
259         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
260     this.syncWaitTimeout = this.conf.getInt(
261       "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
262 
263     this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
264 
265     this.ng = this.connection.getNonceGenerator();
266   }
267 
268   @Override
269   public void abort(String why, Throwable e) {
270     // Currently does nothing but throw the passed message and exception
271     this.aborted = true;
272     throw new RuntimeException(why, e);
273   }
274 
275   @Override
276   public boolean isAborted(){
277     return this.aborted;
278   }
279 
280   /** @return HConnection used by this object. */
281   @Override
282   public HConnection getConnection() {
283     return connection;
284   }
285 
286   /** @return - true if the master server is running. Throws an exception
287    *  otherwise.
288    * @throws ZooKeeperConnectionException
289    * @throws MasterNotRunningException
290    * @deprecated this has been deprecated without a replacement
291    */
292   @Deprecated
293   public boolean isMasterRunning()
294   throws MasterNotRunningException, ZooKeeperConnectionException {
295     return connection.isMasterRunning();
296   }
297 
298   /**
299    * @param tableName Table to check.
300    * @return True if table exists already.
301    * @throws IOException
302    */
303   @Override
304   public boolean tableExists(final TableName tableName) throws IOException {
305     return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
306       @Override
307       public Boolean call(int callTimeout) throws ServiceException, IOException {
308         return MetaTableAccessor.tableExists(connection, tableName);
309       }
310     });
311   }
312 
313   public boolean tableExists(final byte[] tableName)
314   throws IOException {
315     return tableExists(TableName.valueOf(tableName));
316   }
317 
318   public boolean tableExists(final String tableName)
319   throws IOException {
320     return tableExists(TableName.valueOf(tableName));
321   }
322 
323   @Override
324   public HTableDescriptor[] listTables() throws IOException {
325     return listTables((Pattern)null, false);
326   }
327 
328   @Override
329   public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
330     return listTables(pattern, false);
331   }
332 
333   @Override
334   public HTableDescriptor[] listTables(String regex) throws IOException {
335     return listTables(Pattern.compile(regex), false);
336   }
337 
338   @Override
339   public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
340       throws IOException {
341     return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
342       @Override
343       public HTableDescriptor[] call(int callTimeout) throws ServiceException {
344         GetTableDescriptorsRequest req =
345             RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
346         return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
347       }
348     });
349   }
350 
351   @Override
352   public HTableDescriptor[] listTables(String regex, boolean includeSysTables)
353       throws IOException {
354     return listTables(Pattern.compile(regex), includeSysTables);
355   }
356 
357   /**
358    * List all of the names of userspace tables.
359    * @return String[] table names
360    * @throws IOException if a remote or network exception occurs
361    * @deprecated Use {@link Admin#listTableNames()} instead
362    */
363   @Deprecated
364   public String[] getTableNames() throws IOException {
365     TableName[] tableNames = listTableNames();
366     String[] result = new String[tableNames.length];
367     for (int i = 0; i < tableNames.length; i++) {
368       result[i] = tableNames[i].getNameAsString();
369     }
370     return result;
371   }
372 
373   /**
374    * List all of the names of userspace tables matching the given regular expression.
375    * @param pattern The regular expression to match against
376    * @return String[] table names
377    * @throws IOException if a remote or network exception occurs
378    * @deprecated Use {@link Admin#listTableNames(Pattern)} instead.
379    */
380   @Deprecated
381   public String[] getTableNames(Pattern pattern) throws IOException {
382     TableName[] tableNames = listTableNames(pattern);
383     String[] result = new String[tableNames.length];
384     for (int i = 0; i < tableNames.length; i++) {
385       result[i] = tableNames[i].getNameAsString();
386     }
387     return result;
388   }
389 
390   /**
391    * List all of the names of userspace tables matching the given regular expression.
392    * @param regex The regular expression to match against
393    * @return String[] table names
394    * @throws IOException if a remote or network exception occurs
395    * @deprecated Use {@link Admin#listTableNames(Pattern)} instead.
396    */
397   @Deprecated
398   public String[] getTableNames(String regex) throws IOException {
399     return getTableNames(Pattern.compile(regex));
400   }
401 
402   @Override
403   public TableName[] listTableNames() throws IOException {
404     return listTableNames((Pattern)null, false);
405   }
406 
407   @Override
408   public TableName[] listTableNames(Pattern pattern) throws IOException {
409     return listTableNames(pattern, false);
410   }
411 
412   @Override
413   public TableName[] listTableNames(String regex) throws IOException {
414     return listTableNames(Pattern.compile(regex), false);
415   }
416 
417   @Override
418   public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables)
419       throws IOException {
420     return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
421       @Override
422       public TableName[] call(int callTimeout) throws ServiceException {
423         GetTableNamesRequest req =
424             RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
425         return ProtobufUtil.getTableNameArray(master.getTableNames(null, req)
426             .getTableNamesList());
427       }
428     });
429   }
430 
431   @Override
432   public TableName[] listTableNames(final String regex, final boolean includeSysTables)
433       throws IOException {
434     return listTableNames(Pattern.compile(regex), includeSysTables);
435   }
436 
437   /**
438    * Method for getting the tableDescriptor
439    * @param tableName as a byte []
440    * @return the tableDescriptor
441    * @throws TableNotFoundException
442    * @throws IOException if a remote or network exception occurs
443    */
444   @Override
445   public HTableDescriptor getTableDescriptor(final TableName tableName)
446   throws TableNotFoundException, IOException {
447     if (tableName == null) return null;
448     HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(getConnection()) {
449       @Override
450       public HTableDescriptor call(int callTimeout) throws ServiceException {
451         GetTableDescriptorsResponse htds;
452         GetTableDescriptorsRequest req =
453             RequestConverter.buildGetTableDescriptorsRequest(tableName);
454         htds = master.getTableDescriptors(null, req);
455 
456         if (!htds.getTableSchemaList().isEmpty()) {
457           return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
458         }
459         return null;
460       }
461     });
462     if (htd != null) {
463       return htd;
464     }
465     throw new TableNotFoundException(tableName.getNameAsString());
466   }
467 
468   public HTableDescriptor getTableDescriptor(final byte[] tableName)
469   throws TableNotFoundException, IOException {
470     return getTableDescriptor(TableName.valueOf(tableName));
471   }
472 
473   private long getPauseTime(int tries) {
474     int triesCount = tries;
475     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
476       triesCount = HConstants.RETRY_BACKOFF.length - 1;
477     }
478     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
479   }
480 
481   /**
482    * Creates a new table.
483    * Synchronous operation.
484    *
485    * @param desc table descriptor for table
486    *
487    * @throws IllegalArgumentException if the table name is reserved
488    * @throws MasterNotRunningException if master is not running
489    * @throws TableExistsException if table already exists (If concurrent
490    * threads, the table may have been created between test-for-existence
491    * and attempt-at-creation).
492    * @throws IOException if a remote or network exception occurs
493    */
494   @Override
495   public void createTable(HTableDescriptor desc)
496   throws IOException {
497     createTable(desc, null);
498   }
499 
500   /**
501    * Creates a new table with the specified number of regions.  The start key
502    * specified will become the end key of the first region of the table, and
503    * the end key specified will become the start key of the last region of the
504    * table (the first region has a null start key and the last region has a
505    * null end key).
506    *
507    * BigInteger math will be used to divide the key range specified into
508    * enough segments to make the required number of total regions.
509    *
510    * Synchronous operation.
511    *
512    * @param desc table descriptor for table
513    * @param startKey beginning of key range
514    * @param endKey end of key range
515    * @param numRegions the total number of regions to create
516    *
517    * @throws IllegalArgumentException if the table name is reserved
518    * @throws MasterNotRunningException if master is not running
519    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
520    * threads, the table may have been created between test-for-existence
521    * and attempt-at-creation).
522    * @throws IOException
523    */
524   @Override
525   public void createTable(HTableDescriptor desc, byte [] startKey,
526       byte [] endKey, int numRegions)
527   throws IOException {
528     if(numRegions < 3) {
529       throw new IllegalArgumentException("Must create at least three regions");
530     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
531       throw new IllegalArgumentException("Start key must be smaller than end key");
532     }
533     if (numRegions == 3) {
534       createTable(desc, new byte[][]{startKey, endKey});
535       return;
536     }
537     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
538     if(splitKeys == null || splitKeys.length != numRegions - 1) {
539       throw new IllegalArgumentException("Unable to split key range into enough regions");
540     }
541     createTable(desc, splitKeys);
542   }
543 
544   /**
545    * Creates a new table with an initial set of empty regions defined by the
546    * specified split keys.  The total number of regions created will be the
547    * number of split keys plus one. Synchronous operation.
548    * Note : Avoid passing empty split key.
549    *
550    * @param desc table descriptor for table
551    * @param splitKeys array of split keys for the initial regions of the table
552    *
553    * @throws IllegalArgumentException if the table name is reserved, if the split keys
554    * are repeated and if the split key has empty byte array.
555    * @throws MasterNotRunningException if master is not running
556    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
557    * threads, the table may have been created between test-for-existence
558    * and attempt-at-creation).
559    * @throws IOException
560    */
561   @Override
562   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
563       throws IOException {
564     Future<Void> future = createTableAsync(desc, splitKeys);
565     try {
566       // TODO: how long should we wait? spin forever?
567       future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
568     } catch (InterruptedException e) {
569       throw new InterruptedIOException("Interrupted when waiting" +
570           " for table to be enabled; meta scan was done");
571     } catch (TimeoutException e) {
572       throw new TimeoutIOException(e);
573     } catch (ExecutionException e) {
574       if (e.getCause() instanceof IOException) {
575         throw (IOException)e.getCause();
576       } else {
577         throw new IOException(e.getCause());
578       }
579     }
580   }
581 
582   /**
583    * Creates a new table but does not block and wait for it to come online.
584    * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
585    * It may throw ExecutionException if there was an error while executing the operation
586    * or TimeoutException in case the wait timeout was not long enough to allow the
587    * operation to complete.
588    *
589    * @param desc table descriptor for table
590    * @param splitKeys keys to check if the table has been created with all split keys
591    * @throws IllegalArgumentException Bad table name, if the split keys
592    *    are repeated and if the split key has empty byte array.
593    * @throws IOException if a remote or network exception occurs
594    * @return the result of the async creation. You can use Future.get(long, TimeUnit)
595    *    to wait on the operation to complete.
596    */
597   @Override
598   public Future<Void> createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys)
599       throws IOException {
600     if (desc.getTableName() == null) {
601       throw new IllegalArgumentException("TableName cannot be null");
602     }
603     if (splitKeys != null && splitKeys.length > 0) {
604       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
605       // Verify there are no duplicate split keys
606       byte[] lastKey = null;
607       for (byte[] splitKey : splitKeys) {
608         if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
609           throw new IllegalArgumentException(
610               "Empty split key must not be passed in the split keys.");
611         }
612         if (lastKey != null && Bytes.equals(splitKey, lastKey)) {
613           throw new IllegalArgumentException("All split keys must be unique, " +
614             "found duplicate: " + Bytes.toStringBinary(splitKey) +
615             ", " + Bytes.toStringBinary(lastKey));
616         }
617         lastKey = splitKey;
618       }
619     }
620 
621     CreateTableResponse response = executeCallable(
622         new MasterCallable<CreateTableResponse>(getConnection()) {
623       @Override
624       public CreateTableResponse call(int callTimeout) throws ServiceException {
625         CreateTableRequest request = RequestConverter.buildCreateTableRequest(
626           desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
627         return master.createTable(null, request);
628       }
629     });
630     return new CreateTableFuture(this, desc, splitKeys, response);
631   }
632 
633   private static class CreateTableFuture extends TableFuture<Void> {
634     private final HTableDescriptor desc;
635     private final byte[][] splitKeys;
636 
637     public CreateTableFuture(final HBaseAdmin admin, final HTableDescriptor desc,
638         final byte[][] splitKeys, final CreateTableResponse response) {
639       super(admin, desc.getTableName(),
640               (response != null && response.hasProcId()) ? response.getProcId() : null);
641       this.splitKeys = splitKeys;
642       this.desc = desc;
643     }
644 
645     @Override
646     protected HTableDescriptor getTableDescriptor() {
647       return desc;
648     }
649 
650     @Override
651     public String getOperationType() {
652       return "CREATE";
653     }
654 
655     @Override
656     protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
657       waitForTableEnabled(deadlineTs);
658       waitForAllRegionsOnline(deadlineTs, splitKeys);
659       return null;
660     }
661   }
662 
663   public void deleteTable(final String tableName) throws IOException {
664     deleteTable(TableName.valueOf(tableName));
665   }
666 
667   public void deleteTable(final byte[] tableName) throws IOException {
668     deleteTable(TableName.valueOf(tableName));
669   }
670 
671   /**
672    * Deletes a table.
673    * Synchronous operation.
674    *
675    * @param tableName name of table to delete
676    * @throws IOException if a remote or network exception occurs
677    */
678   @Override
679   public void deleteTable(final TableName tableName) throws IOException {
680     Future<Void> future = deleteTableAsync(tableName);
681     try {
682       future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
683     } catch (InterruptedException e) {
684       throw new InterruptedIOException("Interrupted when waiting for table to be deleted");
685     } catch (TimeoutException e) {
686       throw new TimeoutIOException(e);
687     } catch (ExecutionException e) {
688       if (e.getCause() instanceof IOException) {
689         throw (IOException)e.getCause();
690       } else {
691         throw new IOException(e.getCause());
692       }
693     }
694   }
695 
696   /**
697    * Deletes the table but does not block and wait for it be completely removed.
698    * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
699    * It may throw ExecutionException if there was an error while executing the operation
700    * or TimeoutException in case the wait timeout was not long enough to allow the
701    * operation to complete.
702    *
703    * @param tableName name of table to delete
704    * @throws IOException if a remote or network exception occurs
705    * @return the result of the async delete. You can use Future.get(long, TimeUnit)
706    *    to wait on the operation to complete.
707    */
708   @Override
709   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
710     DeleteTableResponse response = executeCallable(
711         new MasterCallable<DeleteTableResponse>(getConnection()) {
712       @Override
713       public DeleteTableResponse call(int callTimeout) throws ServiceException {
714         DeleteTableRequest req =
715             RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
716         return master.deleteTable(null,req);
717       }
718     });
719     return new DeleteTableFuture(this, tableName, response);
720   }
721 
722   private static class DeleteTableFuture extends TableFuture<Void> {
723     public DeleteTableFuture(final HBaseAdmin admin, final TableName tableName,
724         final DeleteTableResponse response) {
725       super(admin, tableName,
726               (response != null && response.hasProcId()) ? response.getProcId() : null);
727     }
728 
729     @Override
730     public String getOperationType() {
731       return "DELETE";
732     }
733 
734     @Override
735     protected Void waitOperationResult(final long deadlineTs)
736         throws IOException, TimeoutException {
737       waitTableNotFound(deadlineTs);
738       return null;
739     }
740 
741     @Override
742     protected Void postOperationResult(final Void result, final long deadlineTs)
743         throws IOException, TimeoutException {
744       // Delete cached information to prevent clients from using old locations
745       getAdmin().getConnection().clearRegionCache(getTableName());
746       return super.postOperationResult(result, deadlineTs);
747     }
748   }
749 
750   /**
751    * Deletes tables matching the passed in pattern and wait on completion.
752    *
753    * Warning: Use this method carefully, there is no prompting and the effect is
754    * immediate. Consider using {@link #listTables(java.lang.String)} and
755    * {@link #deleteTable(byte[])}
756    *
757    * @param regex The regular expression to match table names against
758    * @return Table descriptors for tables that couldn't be deleted
759    * @throws IOException
760    * @see #deleteTables(java.util.regex.Pattern)
761    * @see #deleteTable(java.lang.String)
762    */
763   @Override
764   public HTableDescriptor[] deleteTables(String regex) throws IOException {
765     return deleteTables(Pattern.compile(regex));
766   }
767 
768   /**
769    * Delete tables matching the passed in pattern and wait on completion.
770    *
771    * Warning: Use this method carefully, there is no prompting and the effect is
772    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
773    * {@link #deleteTable(byte[])}
774    *
775    * @param pattern The pattern to match table names against
776    * @return Table descriptors for tables that couldn't be deleted
777    * @throws IOException
778    */
779   @Override
780   public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
781     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
782     for (HTableDescriptor table : listTables(pattern)) {
783       try {
784         deleteTable(table.getTableName());
785       } catch (IOException ex) {
786         LOG.info("Failed to delete table " + table.getTableName(), ex);
787         failed.add(table);
788       }
789     }
790     return failed.toArray(new HTableDescriptor[failed.size()]);
791   }
792 
793   /**
794    * Truncate a table. Synchronous operation.
795    * @param tableName name of table to truncate
796    * @param preserveSplits True if the splits should be preserved
797    * @throws IOException if a remote or network exception occurs
798    */
799   @Override
800   public void truncateTable(final TableName tableName, final boolean preserveSplits)
801       throws IOException {
802     Future<Void> future = truncateTableAsync(tableName, preserveSplits);
803     try {
804       future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
805     } catch (InterruptedException e) {
806       throw new InterruptedIOException("Interrupted when waiting for table " + tableName
807           + " to be enabled.");
808     } catch (TimeoutException e) {
809       throw new TimeoutIOException(e);
810     } catch (ExecutionException e) {
811       if (e.getCause() instanceof IOException) {
812         throw (IOException) e.getCause();
813       } else {
814         throw new IOException(e.getCause());
815       }
816     }
817   }
818 
819   /**
820    * Truncate the table but does not block and wait for it be completely enabled. You can use
821    * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
822    * ExecutionException if there was an error while executing the operation or TimeoutException in
823    * case the wait timeout was not long enough to allow the operation to complete. Asynchronous
824    * operation.
825    * @param tableName name of table to delete
826    * @param preserveSplits true if the splits should be preserved
827    * @throws IOException if a remote or network exception occurs
828    * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the
829    *         operation to complete.
830    */
831   @Override
832   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
833       throws IOException {
834     TruncateTableResponse response =
835         executeCallable(new MasterCallable<TruncateTableResponse>(getConnection()) {
836           @Override
837           public TruncateTableResponse call(int callTimeout) throws ServiceException {
838             LOG.info("Started truncating " + tableName);
839             TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
840               tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
841             return master.truncateTable(null, req);
842           }
843         });
844     return new TruncateTableFuture(this, tableName, preserveSplits, response);
845   }
846 
847   private static class TruncateTableFuture extends TableFuture<Void> {
848     private final boolean preserveSplits;
849 
850     public TruncateTableFuture(final HBaseAdmin admin, final TableName tableName,
851         final boolean preserveSplits, final TruncateTableResponse response) {
852       super(admin, tableName,
853              (response != null && response.hasProcId()) ? response.getProcId() : null);
854       this.preserveSplits = preserveSplits;
855     }
856 
857     @Override
858     public String getOperationType() {
859       return "TRUNCATE";
860     }
861 
862     @Override
863     protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
864       waitForTableEnabled(deadlineTs);
865       // once the table is enabled, we know the operation is done. so we can fetch the splitKeys
866       byte[][] splitKeys = preserveSplits ? getAdmin().getTableSplits(getTableName()) : null;
867       waitForAllRegionsOnline(deadlineTs, splitKeys);
868       return null;
869     }
870   }
871 
872   private byte[][] getTableSplits(final TableName tableName) throws IOException {
873     byte[][] splits = null;
874     try (RegionLocator locator = getConnection().getRegionLocator(tableName)) {
875       byte[][] startKeys = locator.getStartKeys();
876       if (startKeys.length == 1) {
877         return splits;
878       }
879       splits = new byte[startKeys.length - 1][];
880       for (int i = 1; i < startKeys.length; i++) {
881         splits[i - 1] = startKeys[i];
882       }
883     }
884     return splits;
885   }
886 
887 
888   /**
889    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
890    * and {@link #isTableEnabled(byte[])} instead.
891    * The table has to be in disabled state for it to be enabled.
892    * @param tableName name of the table
893    * @throws IOException if a remote or network exception occurs
894    * There could be couple types of IOException
895    * TableNotFoundException means the table doesn't exist.
896    * TableNotDisabledException means the table isn't in disabled state.
897    * @see #isTableEnabled(byte[])
898    * @see #disableTable(byte[])
899    * @see #enableTableAsync(byte[])
900    */
901   @Override
902   public void enableTable(final TableName tableName)
903   throws IOException {
904     Future<Void> future = enableTableAsync(tableName);
905     try {
906       future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
907     } catch (InterruptedException e) {
908       throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
909     } catch (TimeoutException e) {
910       throw new TimeoutIOException(e);
911     } catch (ExecutionException e) {
912       if (e.getCause() instanceof IOException) {
913         throw (IOException)e.getCause();
914       } else {
915         throw new IOException(e.getCause());
916       }
917     }
918   }
919 
920   public void enableTable(final byte[] tableName)
921   throws IOException {
922     enableTable(TableName.valueOf(tableName));
923   }
924 
925   public void enableTable(final String tableName)
926   throws IOException {
927     enableTable(TableName.valueOf(tableName));
928   }
929 
930   /**
931    * Wait for the table to be enabled and available
932    * If enabling the table exceeds the retry period, an exception is thrown.
933    * @param tableName name of the table
934    * @throws IOException if a remote or network exception occurs or
935    *    table is not enabled after the retries period.
936    */
937   private void waitUntilTableIsEnabled(final TableName tableName) throws IOException {
938     boolean enabled = false;
939     long start = EnvironmentEdgeManager.currentTime();
940     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
941       try {
942         enabled = isTableEnabled(tableName);
943       } catch (TableNotFoundException tnfe) {
944         // wait for table to be created
945         enabled = false;
946       }
947       enabled = enabled && isTableAvailable(tableName);
948       if (enabled) {
949         break;
950       }
951       long sleep = getPauseTime(tries);
952       if (LOG.isDebugEnabled()) {
953         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
954           "enabled in " + tableName);
955       }
956       try {
957         Thread.sleep(sleep);
958       } catch (InterruptedException e) {
959         // Do this conversion rather than let it out because do not want to
960         // change the method signature.
961         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
962       }
963     }
964     if (!enabled) {
965       long msec = EnvironmentEdgeManager.currentTime() - start;
966       throw new IOException("Table '" + tableName +
967         "' not yet enabled, after " + msec + "ms.");
968     }
969   }
970 
971   public void enableTableAsync(final byte[] tableName)
972   throws IOException {
973     enableTable(TableName.valueOf(tableName));
974   }
975 
976   public void enableTableAsync(final String tableName)
977   throws IOException {
978     enableTableAsync(TableName.valueOf(tableName));
979   }
980 
981   /**
982    * Enable the table but does not block and wait for it be completely enabled.
983    * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
984    * It may throw ExecutionException if there was an error while executing the operation
985    * or TimeoutException in case the wait timeout was not long enough to allow the
986    * operation to complete.
987    *
988    * @param tableName name of table to delete
989    * @throws IOException if a remote or network exception occurs
990    * @return the result of the async enable. You can use Future.get(long, TimeUnit)
991    *    to wait on the operation to complete.
992    */
993   @Override
994   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
995     TableName.isLegalFullyQualifiedTableName(tableName.getName());
996     EnableTableResponse response = executeCallable(
997         new MasterCallable<EnableTableResponse>(getConnection()) {
998       @Override
999       public EnableTableResponse call(int callTimeout) throws ServiceException {
1000         LOG.info("Started enable of " + tableName);
1001         EnableTableRequest req =
1002             RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
1003         return master.enableTable(null,req);
1004       }
1005     });
1006     return new EnableTableFuture(this, tableName, response);
1007   }
1008 
1009   private static class EnableTableFuture extends TableFuture<Void> {
1010     public EnableTableFuture(final HBaseAdmin admin, final TableName tableName,
1011         final EnableTableResponse response) {
1012       super(admin, tableName,
1013               (response != null && response.hasProcId()) ? response.getProcId() : null);
1014     }
1015 
1016     @Override
1017     public String getOperationType() {
1018       return "ENABLE";
1019     }
1020 
1021     @Override
1022     protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
1023       waitForTableEnabled(deadlineTs);
1024       return null;
1025     }
1026   }
1027 
1028   /**
1029    * Enable tables matching the passed in pattern and wait on completion.
1030    *
1031    * Warning: Use this method carefully, there is no prompting and the effect is
1032    * immediate. Consider using {@link #listTables(java.lang.String)} and
1033    * {@link #enableTable(byte[])}
1034    *
1035    * @param regex The regular expression to match table names against
1036    * @throws IOException
1037    * @see #enableTables(java.util.regex.Pattern)
1038    * @see #enableTable(java.lang.String)
1039    */
1040   @Override
1041   public HTableDescriptor[] enableTables(String regex) throws IOException {
1042     return enableTables(Pattern.compile(regex));
1043   }
1044 
1045   /**
1046    * Enable tables matching the passed in pattern and wait on completion.
1047    *
1048    * Warning: Use this method carefully, there is no prompting and the effect is
1049    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
1050    * {@link #enableTable(byte[])}
1051    *
1052    * @param pattern The pattern to match table names against
1053    * @throws IOException
1054    */
1055   @Override
1056   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
1057     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
1058     for (HTableDescriptor table : listTables(pattern)) {
1059       if (isTableDisabled(table.getTableName())) {
1060         try {
1061           enableTable(table.getTableName());
1062         } catch (IOException ex) {
1063           LOG.info("Failed to enable table " + table.getTableName(), ex);
1064           failed.add(table);
1065         }
1066       }
1067     }
1068     return failed.toArray(new HTableDescriptor[failed.size()]);
1069   }
1070 
1071   public void disableTableAsync(final byte[] tableName) throws IOException {
1072     disableTableAsync(TableName.valueOf(tableName));
1073   }
1074 
1075   public void disableTableAsync(final String tableName) throws IOException {
1076     disableTableAsync(TableName.valueOf(tableName));
1077   }
1078 
1079   /**
1080    * Disable table and wait on completion.  May timeout eventually.  Use
1081    * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
1082    * instead.
1083    * The table has to be in enabled state for it to be disabled.
1084    * @param tableName
1085    * @throws IOException
1086    * There could be couple types of IOException
1087    * TableNotFoundException means the table doesn't exist.
1088    * TableNotEnabledException means the table isn't in enabled state.
1089    */
1090   @Override
1091   public void disableTable(final TableName tableName)
1092   throws IOException {
1093     Future<Void> future = disableTableAsync(tableName);
1094     try {
1095       future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
1096     } catch (InterruptedException e) {
1097       throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
1098     } catch (TimeoutException e) {
1099       throw new TimeoutIOException(e);
1100     } catch (ExecutionException e) {
1101       if (e.getCause() instanceof IOException) {
1102         throw (IOException)e.getCause();
1103       } else {
1104         throw new IOException(e.getCause());
1105       }
1106     }
1107   }
1108 
1109   public void disableTable(final byte[] tableName)
1110   throws IOException {
1111     disableTable(TableName.valueOf(tableName));
1112   }
1113 
1114   public void disableTable(final String tableName)
1115   throws IOException {
1116     disableTable(TableName.valueOf(tableName));
1117   }
1118 
1119   /**
1120    * Disable the table but does not block and wait for it be completely disabled.
1121    * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
1122    * It may throw ExecutionException if there was an error while executing the operation
1123    * or TimeoutException in case the wait timeout was not long enough to allow the
1124    * operation to complete.
1125    *
1126    * @param tableName name of table to delete
1127    * @throws IOException if a remote or network exception occurs
1128    * @return the result of the async disable. You can use Future.get(long, TimeUnit)
1129    *    to wait on the operation to complete.
1130    */
1131   @Override
1132   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
1133     TableName.isLegalFullyQualifiedTableName(tableName.getName());
1134     DisableTableResponse response = executeCallable(
1135         new MasterCallable<DisableTableResponse>(getConnection()) {
1136       @Override
1137       public DisableTableResponse call(int callTimeout) throws ServiceException {
1138         LOG.info("Started disable of " + tableName);
1139         DisableTableRequest req =
1140             RequestConverter.buildDisableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
1141         return master.disableTable(null, req);
1142       }
1143     });
1144     return new DisableTableFuture(this, tableName, response);
1145   }
1146 
1147   private static class DisableTableFuture extends TableFuture<Void> {
1148     public DisableTableFuture(final HBaseAdmin admin, final TableName tableName,
1149         final DisableTableResponse response) {
1150       super(admin, tableName,
1151               (response != null && response.hasProcId()) ? response.getProcId() : null);
1152     }
1153 
1154     @Override
1155     public String getOperationType() {
1156       return "DISABLE";
1157     }
1158 
1159     @Override
1160     protected Void waitOperationResult(final long deadlineTs)
1161         throws IOException, TimeoutException {
1162       waitForTableDisabled(deadlineTs);
1163       return null;
1164     }
1165   }
1166 
1167   /**
1168    * Disable tables matching the passed in pattern and wait on completion.
1169    *
1170    * Warning: Use this method carefully, there is no prompting and the effect is
1171    * immediate. Consider using {@link #listTables(java.lang.String)} and
1172    * {@link #disableTable(byte[])}
1173    *
1174    * @param regex The regular expression to match table names against
1175    * @return Table descriptors for tables that couldn't be disabled
1176    * @throws IOException
1177    * @see #disableTables(java.util.regex.Pattern)
1178    * @see #disableTable(java.lang.String)
1179    */
1180   @Override
1181   public HTableDescriptor[] disableTables(String regex) throws IOException {
1182     return disableTables(Pattern.compile(regex));
1183   }
1184 
1185   /**
1186    * Disable tables matching the passed in pattern and wait on completion.
1187    *
1188    * Warning: Use this method carefully, there is no prompting and the effect is
1189    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
1190    * {@link #disableTable(byte[])}
1191    *
1192    * @param pattern The pattern to match table names against
1193    * @return Table descriptors for tables that couldn't be disabled
1194    * @throws IOException
1195    */
1196   @Override
1197   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
1198     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
1199     for (HTableDescriptor table : listTables(pattern)) {
1200       if (isTableEnabled(table.getTableName())) {
1201         try {
1202           disableTable(table.getTableName());
1203         } catch (IOException ex) {
1204           LOG.info("Failed to disable table " + table.getTableName(), ex);
1205           failed.add(table);
1206         }
1207       }
1208     }
1209     return failed.toArray(new HTableDescriptor[failed.size()]);
1210   }
1211 
1212   /*
1213    * Checks whether table exists. If not, throws TableNotFoundException
1214    * @param tableName
1215    */
1216   private void checkTableExistence(TableName tableName) throws IOException {
1217     if (!tableExists(tableName)) {
1218       throw new TableNotFoundException(tableName);
1219     }
1220   }
1221 
1222   /**
1223    * @param tableName name of table to check
1224    * @return true if table is on-line
1225    * @throws IOException if a remote or network exception occurs
1226    */
1227   @Override
1228   public boolean isTableEnabled(final TableName tableName) throws IOException {
1229     checkTableExistence(tableName);
1230     return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
1231       @Override
1232       public Boolean call(int callTimeout) throws ServiceException, IOException {
1233         TableState tableState = MetaTableAccessor.getTableState(connection, tableName);
1234         if (tableState == null)
1235           throw new TableNotFoundException(tableName);
1236         return tableState.inStates(TableState.State.ENABLED);
1237       }
1238     });
1239   }
1240 
1241   public boolean isTableEnabled(byte[] tableName) throws IOException {
1242     return isTableEnabled(TableName.valueOf(tableName));
1243   }
1244 
1245   public boolean isTableEnabled(String tableName) throws IOException {
1246     return isTableEnabled(TableName.valueOf(tableName));
1247   }
1248 
1249 
1250 
1251   /**
1252    * @param tableName name of table to check
1253    * @return true if table is off-line
1254    * @throws IOException if a remote or network exception occurs
1255    */
1256   @Override
1257   public boolean isTableDisabled(TableName tableName) throws IOException {
1258     checkTableExistence(tableName);
1259     return connection.isTableDisabled(tableName);
1260   }
1261 
1262   public boolean isTableDisabled(byte[] tableName) throws IOException {
1263     return isTableDisabled(TableName.valueOf(tableName));
1264   }
1265 
1266   public boolean isTableDisabled(String tableName) throws IOException {
1267     return isTableDisabled(TableName.valueOf(tableName));
1268   }
1269 
1270   /**
1271    * @param tableName name of table to check
1272    * @return true if all regions of the table are available
1273    * @throws IOException if a remote or network exception occurs
1274    */
1275   @Override
1276   public boolean isTableAvailable(TableName tableName) throws IOException {
1277     return connection.isTableAvailable(tableName);
1278   }
1279 
1280   public boolean isTableAvailable(byte[] tableName) throws IOException {
1281     return isTableAvailable(TableName.valueOf(tableName));
1282   }
1283 
1284   public boolean isTableAvailable(String tableName) throws IOException {
1285     return isTableAvailable(TableName.valueOf(tableName));
1286   }
1287 
1288   /**
1289    * Use this api to check if the table has been created with the specified number of
1290    * splitkeys which was used while creating the given table.
1291    * Note : If this api is used after a table's region gets splitted, the api may return
1292    * false.
1293    * @param tableName
1294    *          name of table to check
1295    * @param splitKeys
1296    *          keys to check if the table has been created with all split keys
1297    * @throws IOException
1298    *           if a remote or network excpetion occurs
1299    */
1300   @Override
1301   public boolean isTableAvailable(TableName tableName,
1302                                   byte[][] splitKeys) throws IOException {
1303     return connection.isTableAvailable(tableName, splitKeys);
1304   }
1305 
1306   public boolean isTableAvailable(byte[] tableName,
1307                                   byte[][] splitKeys) throws IOException {
1308     return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1309   }
1310 
1311   public boolean isTableAvailable(String tableName,
1312                                   byte[][] splitKeys) throws IOException {
1313     return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1314   }
1315 
1316   /**
1317    * Get the status of alter command - indicates how many regions have received
1318    * the updated schema Asynchronous operation.
1319    *
1320    * @param tableName TableName instance
1321    * @return Pair indicating the number of regions updated Pair.getFirst() is the
1322    *         regions that are yet to be updated Pair.getSecond() is the total number
1323    *         of regions of the table
1324    * @throws IOException
1325    *           if a remote or network exception occurs
1326    */
1327   @Override
1328   public Pair<Integer, Integer> getAlterStatus(final TableName tableName)
1329   throws IOException {
1330     return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
1331       @Override
1332       public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
1333         GetSchemaAlterStatusRequest req = RequestConverter
1334             .buildGetSchemaAlterStatusRequest(tableName);
1335         GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req);
1336         Pair<Integer, Integer> pair = new Pair<Integer, Integer>(Integer.valueOf(ret
1337             .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
1338         return pair;
1339       }
1340     });
1341   }
1342 
1343   /**
1344    * Get the status of alter command - indicates how many regions have received
1345    * the updated schema Asynchronous operation.
1346    *
1347    * @param tableName
1348    *          name of the table to get the status of
1349    * @return Pair indicating the number of regions updated Pair.getFirst() is the
1350    *         regions that are yet to be updated Pair.getSecond() is the total number
1351    *         of regions of the table
1352    * @throws IOException
1353    *           if a remote or network exception occurs
1354    */
1355   @Override
1356   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
1357    throws IOException {
1358     return getAlterStatus(TableName.valueOf(tableName));
1359   }
1360 
1361   /**
1362    * Add a column family to an existing table.
1363    * Asynchronous operation.
1364    *
1365    * @param tableName name of the table to add column family to
1366    * @param columnFamily column family descriptor of column family to be added
1367    * @throws IOException if a remote or network exception occurs
1368    * @deprecated As of release 2.0.0.
1369    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1370    *             This will be removed in HBase 3.0.0.
1371    *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
1372    */
1373   @Deprecated
1374   public void addColumn(final byte[] tableName, HColumnDescriptor columnFamily)
1375   throws IOException {
1376     addColumnFamily(TableName.valueOf(tableName), columnFamily);
1377   }
1378 
1379   /**
1380    * Add a column family to an existing table.
1381    * Asynchronous operation.
1382    *
1383    * @param tableName name of the table to add column family to
1384    * @param columnFamily column family descriptor of column family to be added
1385    * @throws IOException if a remote or network exception occurs
1386    * @deprecated As of release 2.0.0.
1387    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1388    *             This will be removed in HBase 3.0.0.
1389    *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
1390    */
1391   @Deprecated
1392   public void addColumn(final String tableName, HColumnDescriptor columnFamily)
1393   throws IOException {
1394     addColumnFamily(TableName.valueOf(tableName), columnFamily);
1395   }
1396 
1397   /**
1398    * Add a column family to an existing table.
1399    * Asynchronous operation.
1400    *
1401    * @param tableName name of the table to add column family to
1402    * @param columnFamily column family descriptor of column family to be added
1403    * @throws IOException if a remote or network exception occurs
1404    * @deprecated As of release 2.0.0.
1405    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1406    *             This will be removed in HBase 3.0.0.
1407    *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
1408    */
1409   @Override
1410   @Deprecated
1411   public void addColumn(final TableName tableName, final HColumnDescriptor columnFamily)
1412   throws IOException {
1413     addColumnFamily(tableName, columnFamily);
1414   }
1415 
1416   @Override
1417   public void addColumnFamily(final TableName tableName, final HColumnDescriptor columnFamily)
1418   throws IOException {
1419     executeCallable(new MasterCallable<Void>(getConnection()) {
1420       @Override
1421       public Void call(int callTimeout) throws ServiceException {
1422         AddColumnRequest req = RequestConverter.buildAddColumnRequest(
1423           tableName, columnFamily, ng.getNonceGroup(), ng.newNonce());
1424         master.addColumn(null, req);
1425         return null;
1426       }
1427     });
1428   }
1429 
1430   /**
1431    * Delete a column family from a table.
1432    * Asynchronous operation.
1433    *
1434    * @param tableName name of table
1435    * @param columnFamily name of column family to be deleted
1436    * @throws IOException if a remote or network exception occurs
1437    * @deprecated As of release 2.0.0.
1438    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1439    *             This will be removed in HBase 3.0.0.
1440    *             Use {@link #deleteColumnFamily(TableName, byte[])}.
1441    */
1442   @Deprecated
1443   public void deleteColumn(final byte[] tableName, final String columnFamily)
1444   throws IOException {
1445     deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily));
1446   }
1447 
1448   /**
1449    * Delete a column family from a table.
1450    * Asynchronous operation.
1451    *
1452    * @param tableName name of table
1453    * @param columnFamily name of column family to be deleted
1454    * @throws IOException if a remote or network exception occurs
1455    * @deprecated As of release 2.0.0.
1456    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1457    *             This will be removed in HBase 3.0.0.
1458    *             Use {@link #deleteColumnFamily(TableName, byte[])}.
1459    */
1460   @Deprecated
1461   public void deleteColumn(final String tableName, final String columnFamily)
1462   throws IOException {
1463     deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily));
1464   }
1465 
1466   /**
1467    * Delete a column family from a table.
1468    * Asynchronous operation.
1469    *
1470    * @param tableName name of table
1471    * @param columnFamily name of column family to be deleted
1472    * @throws IOException if a remote or network exception occurs
1473    * @deprecated As of release 2.0.0.
1474    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1475    *             This will be removed in HBase 3.0.0.
1476    *             Use {@link #deleteColumnFamily(TableName, byte[])}.
1477    */
1478   @Override
1479   @Deprecated
1480   public void deleteColumn(final TableName tableName, final byte[] columnFamily)
1481   throws IOException {
1482     deleteColumnFamily(tableName, columnFamily);
1483   }
1484 
1485   @Override
1486   public void deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
1487   throws IOException {
1488     executeCallable(new MasterCallable<Void>(getConnection()) {
1489       @Override
1490       public Void call(int callTimeout) throws ServiceException {
1491         DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(
1492           tableName, columnFamily, ng.getNonceGroup(), ng.newNonce());
1493         master.deleteColumn(null, req);
1494         return null;
1495       }
1496     });
1497   }
1498 
1499   /**
1500    * Modify an existing column family on a table.
1501    * Asynchronous operation.
1502    *
1503    * @param tableName name of table
1504    * @param columnFamily new column family descriptor to use
1505    * @throws IOException if a remote or network exception occurs
1506    * @deprecated As of release 2.0.0.
1507    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1508    *             This will be removed in HBase 3.0.0.
1509    *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
1510    */
1511   @Deprecated
1512   public void modifyColumn(final String tableName, HColumnDescriptor columnFamily)
1513   throws IOException {
1514     modifyColumnFamily(TableName.valueOf(tableName), columnFamily);
1515   }
1516 
1517   /**
1518    * Modify an existing column family on a table.
1519    * Asynchronous operation.
1520    *
1521    * @param tableName name of table
1522    * @param columnFamily new column family descriptor to use
1523    * @throws IOException if a remote or network exception occurs
1524    * @deprecated As of release 2.0.0.
1525    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1526    *             This will be removed in HBase 3.0.0.
1527    *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
1528    */
1529   @Deprecated
1530   public void modifyColumn(final byte[] tableName, HColumnDescriptor columnFamily)
1531   throws IOException {
1532     modifyColumnFamily(TableName.valueOf(tableName), columnFamily);
1533   }
1534 
1535   /**
1536    * Modify an existing column family on a table.
1537    * Asynchronous operation.
1538    *
1539    * @param tableName name of table
1540    * @param columnFamily new column family descriptor to use
1541    * @throws IOException if a remote or network exception occurs
1542    * @deprecated As of release 2.0.0.
1543    *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
1544    *             This will be removed in HBase 3.0.0.
1545    *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
1546    */
1547   @Override
1548   @Deprecated
1549   public void modifyColumn(final TableName tableName, final HColumnDescriptor columnFamily)
1550   throws IOException {
1551     modifyColumnFamily(tableName, columnFamily);
1552   }
1553 
1554   @Override
1555   public void modifyColumnFamily(final TableName tableName, final HColumnDescriptor columnFamily)
1556   throws IOException {
1557     executeCallable(new MasterCallable<Void>(getConnection()) {
1558       @Override
1559       public Void call(int callTimeout) throws ServiceException {
1560         ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(
1561           tableName, columnFamily, ng.getNonceGroup(), ng.newNonce());
1562         master.modifyColumn(null,req);
1563         return null;
1564       }
1565     });
1566   }
1567 
1568   /**
1569    * Close a region. For expert-admins.  Runs close on the regionserver.  The
1570    * master will not be informed of the close.
1571    * @param regionname region name to close
1572    * @param serverName If supplied, we'll use this location rather than
1573    * the one currently in <code>hbase:meta</code>
1574    * @throws IOException if a remote or network exception occurs
1575    */
1576   @Override
1577   public void closeRegion(final String regionname, final String serverName)
1578   throws IOException {
1579     closeRegion(Bytes.toBytes(regionname), serverName);
1580   }
1581 
1582   /**
1583    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1584    * master will not be informed of the close.
1585    * @param regionname region name to close
1586    * @param serverName The servername of the regionserver.  If passed null we
1587    * will use servername found in the hbase:meta table. A server name
1588    * is made of host, port and startcode.  Here is an example:
1589    * <code> host187.example.com,60020,1289493121758</code>
1590    * @throws IOException if a remote or network exception occurs
1591    */
1592   @Override
1593   public void closeRegion(final byte [] regionname, final String serverName)
1594       throws IOException {
1595     if (serverName != null) {
1596       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
1597       if (pair == null || pair.getFirst() == null) {
1598         throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1599       } else {
1600         closeRegion(ServerName.valueOf(serverName), pair.getFirst());
1601       }
1602     } else {
1603       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
1604       if (pair == null) {
1605         throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1606       } else if (pair.getSecond() == null) {
1607         throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1608       } else {
1609         closeRegion(pair.getSecond(), pair.getFirst());
1610       }
1611     }
1612   }
1613 
1614   /**
1615    * For expert-admins. Runs close on the regionserver. Closes a region based on
1616    * the encoded region name. The region server name is mandatory. If the
1617    * servername is provided then based on the online regions in the specified
1618    * regionserver the specified region will be closed. The master will not be
1619    * informed of the close. Note that the regionname is the encoded regionname.
1620    *
1621    * @param encodedRegionName
1622    *          The encoded region name; i.e. the hash that makes up the region
1623    *          name suffix: e.g. if regionname is
1624    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
1625    *          , then the encoded region name is:
1626    *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
1627    * @param serverName
1628    *          The servername of the regionserver. A server name is made of host,
1629    *          port and startcode. This is mandatory. Here is an example:
1630    *          <code> host187.example.com,60020,1289493121758</code>
1631    * @return true if the region was closed, false if not.
1632    * @throws IOException
1633    *           if a remote or network exception occurs
1634    */
1635   @Override
1636   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1637       final String serverName) throws IOException {
1638     if (null == serverName || ("").equals(serverName.trim())) {
1639       throw new IllegalArgumentException(
1640           "The servername cannot be null or empty.");
1641     }
1642     ServerName sn = ServerName.valueOf(serverName);
1643     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1644     // Close the region without updating zk state.
1645     CloseRegionRequest request =
1646       RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
1647     try {
1648       CloseRegionResponse response = admin.closeRegion(null, request);
1649       boolean isRegionClosed = response.getClosed();
1650       if (false == isRegionClosed) {
1651         LOG.error("Not able to close the region " + encodedRegionName + ".");
1652       }
1653       return isRegionClosed;
1654     } catch (ServiceException se) {
1655       throw ProtobufUtil.getRemoteException(se);
1656     }
1657   }
1658 
1659   /**
1660    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1661    * master will not be informed of the close.
1662    * @param sn
1663    * @param hri
1664    * @throws IOException
1665    */
1666   @Override
1667   public void closeRegion(final ServerName sn, final HRegionInfo hri)
1668   throws IOException {
1669     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1670     // Close the region without updating zk state.
1671     ProtobufUtil.closeRegion(admin, sn, hri.getRegionName());
1672   }
1673 
1674   /**
1675    * Get all the online regions on a region server.
1676    */
1677   @Override
1678   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
1679     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1680     return ProtobufUtil.getOnlineRegions(admin);
1681   }
1682 
1683   /**
1684    * {@inheritDoc}
1685    */
1686   @Override
1687   public void flush(final TableName tableName) throws IOException {
1688     checkTableExists(tableName);
1689     if (isTableDisabled(tableName)) {
1690       LOG.info("Table is disabled: " + tableName.getNameAsString());
1691       return;
1692     }
1693     execProcedure("flush-table-proc", tableName.getNameAsString(),
1694       new HashMap<String, String>());
1695   }
1696 
1697   /**
1698    * {@inheritDoc}
1699    */
1700   @Override
1701   public void flushRegion(final byte[] regionName) throws IOException {
1702     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
1703     if (regionServerPair == null) {
1704       throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
1705     }
1706     if (regionServerPair.getSecond() == null) {
1707       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
1708     }
1709     flush(regionServerPair.getSecond(), regionServerPair.getFirst());
1710   }
1711 
1712   /**
1713    * @deprecated Use {@link #flush(org.apache.hadoop.hbase.TableName)} or {@link #flushRegion
1714    * (byte[])} instead.
1715    */
1716   @Deprecated
1717   public void flush(final String tableNameOrRegionName)
1718   throws IOException, InterruptedException {
1719     flush(Bytes.toBytes(tableNameOrRegionName));
1720   }
1721 
1722   /**
1723    * @deprecated Use {@link #flush(org.apache.hadoop.hbase.TableName)} or {@link #flushRegion
1724    * (byte[])} instead.
1725    */
1726   @Deprecated
1727   public void flush(final byte[] tableNameOrRegionName)
1728   throws IOException, InterruptedException {
1729     try {
1730       flushRegion(tableNameOrRegionName);
1731     } catch (IllegalArgumentException e) {
1732       // Unknown region.  Try table.
1733       flush(TableName.valueOf(tableNameOrRegionName));
1734     }
1735   }
1736 
1737   private void flush(final ServerName sn, final HRegionInfo hri)
1738   throws IOException {
1739     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1740     FlushRegionRequest request =
1741       RequestConverter.buildFlushRegionRequest(hri.getRegionName());
1742     try {
1743       admin.flushRegion(null, request);
1744     } catch (ServiceException se) {
1745       throw ProtobufUtil.getRemoteException(se);
1746     }
1747   }
1748 
1749   /**
1750    * {@inheritDoc}
1751    */
1752   @Override
1753   public void compact(final TableName tableName)
1754     throws IOException {
1755     compact(tableName, null, false);
1756   }
1757 
1758   /**
1759    * {@inheritDoc}
1760    */
1761   @Override
1762   public void compactRegion(final byte[] regionName)
1763     throws IOException {
1764     compactRegion(regionName, null, false);
1765   }
1766 
1767   /**
1768    * @deprecated Use {@link #compact(org.apache.hadoop.hbase.TableName)} or {@link #compactRegion
1769    * (byte[])} instead.
1770    */
1771   @Deprecated
1772   public void compact(final String tableNameOrRegionName)
1773   throws IOException {
1774     compact(Bytes.toBytes(tableNameOrRegionName));
1775   }
1776 
1777   /**
1778    * @deprecated Use {@link #compact(org.apache.hadoop.hbase.TableName)} or {@link #compactRegion
1779    * (byte[])} instead.
1780    */
1781   @Deprecated
1782   public void compact(final byte[] tableNameOrRegionName)
1783   throws IOException {
1784     try {
1785       compactRegion(tableNameOrRegionName, null, false);
1786     } catch (IllegalArgumentException e) {
1787       compact(TableName.valueOf(tableNameOrRegionName), null, false);
1788     }
1789   }
1790 
1791   /**
1792    * {@inheritDoc}
1793    */
1794   @Override
1795   public void compact(final TableName tableName, final byte[] columnFamily)
1796     throws IOException {
1797     compact(tableName, columnFamily, false);
1798   }
1799 
1800   /**
1801    * {@inheritDoc}
1802    */
1803   @Override
1804   public void compactRegion(final byte[] regionName, final byte[] columnFamily)
1805     throws IOException {
1806     compactRegion(regionName, columnFamily, false);
1807   }
1808 
1809   /**
1810    * @deprecated Use {@link #compact(org.apache.hadoop.hbase.TableName)} or {@link #compactRegion
1811    * (byte[], byte[])} instead.
1812    */
1813   @Deprecated
1814   public void compact(String tableOrRegionName, String columnFamily)
1815     throws IOException {
1816     compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
1817   }
1818 
1819   /**
1820    * @deprecated Use {@link #compact(org.apache.hadoop.hbase.TableName)} or {@link #compactRegion
1821    * (byte[], byte[])} instead.
1822    */
1823   @Deprecated
1824   public void compact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
1825   throws IOException {
1826     try {
1827       compactRegion(tableNameOrRegionName, columnFamily, false);
1828     } catch (IllegalArgumentException e) {
1829       // Bad region, try table
1830       compact(TableName.valueOf(tableNameOrRegionName), columnFamily, false);
1831     }
1832   }
1833 
1834   /**
1835    * {@inheritDoc}
1836    */
1837   @Override
1838   public void compactRegionServer(final ServerName sn, boolean major)
1839   throws IOException, InterruptedException {
1840     for (HRegionInfo region : getOnlineRegions(sn)) {
1841       compact(sn, region, major, null);
1842     }
1843   }
1844 
1845   /**
1846    * {@inheritDoc}
1847    */
1848   @Override
1849   public void majorCompact(final TableName tableName)
1850   throws IOException {
1851     compact(tableName, null, true);
1852   }
1853 
1854   /**
1855    * {@inheritDoc}
1856    */
1857   @Override
1858   public void majorCompactRegion(final byte[] regionName)
1859   throws IOException {
1860     compactRegion(regionName, null, true);
1861   }
1862 
1863   /**
1864    * @deprecated Use {@link #majorCompact(org.apache.hadoop.hbase.TableName)} or {@link
1865    * #majorCompactRegion(byte[])} instead.
1866    */
1867   @Deprecated
1868   public void majorCompact(final String tableNameOrRegionName)
1869   throws IOException {
1870     majorCompact(Bytes.toBytes(tableNameOrRegionName));
1871   }
1872 
1873   /**
1874    * @deprecated Use {@link #majorCompact(org.apache.hadoop.hbase.TableName)} or {@link
1875    * #majorCompactRegion(byte[])} instead.
1876    */
1877   @Deprecated
1878   public void majorCompact(final byte[] tableNameOrRegionName)
1879   throws IOException {
1880     try {
1881       compactRegion(tableNameOrRegionName, null, true);
1882     } catch (IllegalArgumentException e) {
1883       // Invalid region, try table
1884       compact(TableName.valueOf(tableNameOrRegionName), null, true);
1885     }
1886   }
1887 
1888   /**
1889    * {@inheritDoc}
1890    */
1891   @Override
1892   public void majorCompact(final TableName tableName, final byte[] columnFamily)
1893   throws IOException {
1894     compact(tableName, columnFamily, true);
1895   }
1896 
1897   /**
1898    * {@inheritDoc}
1899    */
1900   @Override
1901   public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
1902   throws IOException {
1903     compactRegion(regionName, columnFamily, true);
1904   }
1905 
1906   /**
1907    * @deprecated Use {@link #majorCompact(org.apache.hadoop.hbase.TableName,
1908    * byte[])} or {@link #majorCompactRegion(byte[], byte[])} instead.
1909    */
1910   @Deprecated
1911   public void majorCompact(final String tableNameOrRegionName, final String columnFamily)
1912   throws IOException {
1913     majorCompact(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(columnFamily));
1914   }
1915 
1916   /**
1917    * @deprecated Use {@link #majorCompact(org.apache.hadoop.hbase.TableName,
1918    * byte[])} or {@link #majorCompactRegion(byte[], byte[])} instead.
1919    */
1920   @Deprecated
1921   public void majorCompact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
1922   throws IOException {
1923     try {
1924       compactRegion(tableNameOrRegionName, columnFamily, true);
1925     } catch (IllegalArgumentException e) {
1926       // Invalid region, try table
1927       compact(TableName.valueOf(tableNameOrRegionName), columnFamily, true);
1928     }
1929   }
1930 
1931   /**
1932    * Compact a table.
1933    * Asynchronous operation.
1934    *
1935    * @param tableName table or region to compact
1936    * @param columnFamily column family within a table or region
1937    * @param major True if we are to do a major compaction.
1938    * @throws IOException if a remote or network exception occurs
1939    * @throws InterruptedException
1940    */
1941   private void compact(final TableName tableName, final byte[] columnFamily,final boolean major)
1942   throws IOException {
1943     ZooKeeperWatcher zookeeper = null;
1944     try {
1945       checkTableExists(tableName);
1946       zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
1947           new ThrowableAbortable());
1948       List<Pair<HRegionInfo, ServerName>> pairs;
1949       if (TableName.META_TABLE_NAME.equals(tableName)) {
1950         pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
1951       } else {
1952         pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
1953       }
1954       for (Pair<HRegionInfo, ServerName> pair: pairs) {
1955         if (pair.getFirst().isOffline()) continue;
1956         if (pair.getSecond() == null) continue;
1957         try {
1958           compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
1959         } catch (NotServingRegionException e) {
1960           if (LOG.isDebugEnabled()) {
1961             LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1962               pair.getFirst() + ": " +
1963               StringUtils.stringifyException(e));
1964           }
1965         }
1966       }
1967     } finally {
1968       if (zookeeper != null) {
1969         zookeeper.close();
1970       }
1971     }
1972   }
1973 
1974   /**
1975    * Compact an individual region.
1976    * Asynchronous operation.
1977    *
1978    * @param regionName region to compact
1979    * @param columnFamily column family within a table or region
1980    * @param major True if we are to do a major compaction.
1981    * @throws IOException if a remote or network exception occurs
1982    * @throws InterruptedException
1983    */
1984   private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
1985   throws IOException {
1986     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
1987     if (regionServerPair == null) {
1988       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
1989     }
1990     if (regionServerPair.getSecond() == null) {
1991       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
1992     }
1993     compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
1994   }
1995 
1996   private void compact(final ServerName sn, final HRegionInfo hri,
1997       final boolean major, final byte [] family)
1998   throws IOException {
1999     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2000     CompactRegionRequest request =
2001       RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
2002     try {
2003       admin.compactRegion(null, request);
2004     } catch (ServiceException se) {
2005       throw ProtobufUtil.getRemoteException(se);
2006     }
2007   }
2008 
2009   /**
2010    * Move the region <code>r</code> to <code>dest</code>.
2011    * @param encodedRegionName The encoded region name; i.e. the hash that makes
2012    * up the region name suffix: e.g. if regionname is
2013    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
2014    * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
2015    * @param destServerName The servername of the destination regionserver.  If
2016    * passed the empty byte array we'll assign to a random server.  A server name
2017    * is made of host, port and startcode.  Here is an example:
2018    * <code> host187.example.com,60020,1289493121758</code>
2019    * @throws UnknownRegionException Thrown if we can't find a region named
2020    * <code>encodedRegionName</code>
2021    */
2022   @Override
2023   public void move(final byte [] encodedRegionName, final byte [] destServerName)
2024       throws IOException {
2025 
2026     executeCallable(new MasterCallable<Void>(getConnection()) {
2027       @Override
2028       public Void call(int callTimeout) throws ServiceException {
2029         try {
2030           MoveRegionRequest request =
2031               RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
2032             master.moveRegion(null, request);
2033         } catch (DeserializationException de) {
2034           LOG.error("Could not parse destination server name: " + de);
2035           throw new ServiceException(new DoNotRetryIOException(de));
2036         }
2037         return null;
2038       }
2039     });
2040   }
2041 
2042   /**
2043    * @param regionName
2044    *          Region name to assign.
2045    * @throws MasterNotRunningException
2046    * @throws ZooKeeperConnectionException
2047    * @throws IOException
2048    */
2049   @Override
2050   public void assign(final byte[] regionName) throws MasterNotRunningException,
2051       ZooKeeperConnectionException, IOException {
2052     final byte[] toBeAssigned = getRegionName(regionName);
2053     executeCallable(new MasterCallable<Void>(getConnection()) {
2054       @Override
2055       public Void call(int callTimeout) throws ServiceException {
2056         AssignRegionRequest request =
2057           RequestConverter.buildAssignRegionRequest(toBeAssigned);
2058         master.assignRegion(null,request);
2059         return null;
2060       }
2061     });
2062   }
2063 
2064   /**
2065    * Unassign a region from current hosting regionserver.  Region will then be
2066    * assigned to a regionserver chosen at random.  Region could be reassigned
2067    * back to the same server.  Use {@link #move(byte[], byte[])} if you want
2068    * to control the region movement.
2069    * @param regionName Region to unassign. Will clear any existing RegionPlan
2070    * if one found.
2071    * @param force If true, force unassign (Will remove region from
2072    * regions-in-transition too if present. If results in double assignment
2073    * use hbck -fix to resolve. To be used by experts).
2074    * @throws MasterNotRunningException
2075    * @throws ZooKeeperConnectionException
2076    * @throws IOException
2077    */
2078   @Override
2079   public void unassign(final byte [] regionName, final boolean force)
2080   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
2081     final byte[] toBeUnassigned = getRegionName(regionName);
2082     executeCallable(new MasterCallable<Void>(getConnection()) {
2083       @Override
2084       public Void call(int callTimeout) throws ServiceException {
2085         UnassignRegionRequest request =
2086           RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
2087         master.unassignRegion(null, request);
2088         return null;
2089       }
2090     });
2091   }
2092 
2093   /**
2094    * Offline specified region from master's in-memory state. It will not attempt to reassign the
2095    * region as in unassign. This API can be used when a region not served by any region server and
2096    * still online as per Master's in memory state. If this API is incorrectly used on active region
2097    * then master will loose track of that region.
2098    *
2099    * This is a special method that should be used by experts or hbck.
2100    *
2101    * @param regionName
2102    *          Region to offline.
2103    * @throws IOException
2104    */
2105   @Override
2106   public void offline(final byte [] regionName)
2107   throws IOException {
2108     executeCallable(new MasterCallable<Void>(getConnection()) {
2109       @Override
2110       public Void call(int callTimeout) throws ServiceException {
2111         master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
2112         return null;
2113       }
2114     });
2115   }
2116 
2117   /**
2118    * Turn the load balancer on or off.
2119    * @param on If true, enable balancer. If false, disable balancer.
2120    * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
2121    * @return Previous balancer value
2122    */
2123   @Override
2124   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
2125   throws IOException {
2126     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2127       @Override
2128       public Boolean call(int callTimeout) throws ServiceException {
2129         SetBalancerRunningRequest req =
2130             RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
2131         return master.setBalancerRunning(null, req).getPrevBalanceValue();
2132       }
2133     });
2134   }
2135 
2136   /**
2137    * Invoke the balancer.  Will run the balancer and if regions to move, it will
2138    * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
2139    * logs.
2140    * @return True if balancer ran, false otherwise.
2141    */
2142   @Override
2143   public boolean balancer() throws IOException {
2144     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2145       @Override
2146       public Boolean call(int callTimeout) throws ServiceException {
2147         return master.balance(null, RequestConverter.buildBalanceRequest()).getBalancerRan();
2148       }
2149     });
2150   }
2151 
2152   /**
2153    * Query the state of the balancer from the Master. It's not a guarantee that the balancer is
2154    * actually running this very moment, but that it will run.
2155    *
2156    * @return True if the balancer is enabled, false otherwise.
2157    */
2158   @Override
2159   public boolean isBalancerEnabled() throws IOException {
2160     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2161       @Override
2162       public Boolean call(int callTimeout) throws ServiceException {
2163         return master.isBalancerEnabled(null, RequestConverter.buildIsBalancerEnabledRequest())
2164             .getEnabled();
2165       }
2166     });
2167   }
2168 
2169   /**
2170    * Enable/Disable the catalog janitor
2171    * @param enable if true enables the catalog janitor
2172    * @return the previous state
2173    * @throws MasterNotRunningException
2174    */
2175   @Override
2176   public boolean enableCatalogJanitor(final boolean enable)
2177       throws IOException {
2178     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2179       @Override
2180       public Boolean call(int callTimeout) throws ServiceException {
2181         return master.enableCatalogJanitor(null,
2182           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
2183       }
2184     });
2185   }
2186 
2187   /**
2188    * Ask for a scan of the catalog table
2189    * @return the number of entries cleaned
2190    * @throws MasterNotRunningException
2191    */
2192   @Override
2193   public int runCatalogScan() throws IOException {
2194     return executeCallable(new MasterCallable<Integer>(getConnection()) {
2195       @Override
2196       public Integer call(int callTimeout) throws ServiceException {
2197         return master.runCatalogScan(null,
2198           RequestConverter.buildCatalogScanRequest()).getScanResult();
2199       }
2200     });
2201   }
2202 
2203   /**
2204    * Query on the catalog janitor state (Enabled/Disabled?)
2205    * @throws org.apache.hadoop.hbase.MasterNotRunningException
2206    */
2207   @Override
2208   public boolean isCatalogJanitorEnabled() throws IOException {
2209     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2210       @Override
2211       public Boolean call(int callTimeout) throws ServiceException {
2212         return master.isCatalogJanitorEnabled(null,
2213           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
2214       }
2215     });
2216   }
2217 
2218   /**
2219    * Merge two regions. Asynchronous operation.
2220    * @param encodedNameOfRegionA encoded name of region a
2221    * @param encodedNameOfRegionB encoded name of region b
2222    * @param forcible true if do a compulsory merge, otherwise we will only merge
2223    *          two adjacent regions
2224    * @throws IOException
2225    */
2226   @Override
2227   public void mergeRegions(final byte[] encodedNameOfRegionA,
2228       final byte[] encodedNameOfRegionB, final boolean forcible)
2229       throws IOException {
2230     Pair<HRegionInfo, ServerName> pair = getRegion(encodedNameOfRegionA);
2231     if (pair != null && pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
2232       throw new IllegalArgumentException("Can't invoke merge on non-default regions directly");
2233     pair = getRegion(encodedNameOfRegionB);
2234     if (pair != null && pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
2235       throw new IllegalArgumentException("Can't invoke merge on non-default regions directly");
2236     executeCallable(new MasterCallable<Void>(getConnection()) {
2237       @Override
2238       public Void call(int callTimeout) throws ServiceException {
2239         try {
2240           DispatchMergingRegionsRequest request = RequestConverter
2241               .buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
2242                 encodedNameOfRegionB, forcible);
2243           master.dispatchMergingRegions(null, request);
2244         } catch (DeserializationException de) {
2245           LOG.error("Could not parse destination server name: " + de);
2246         }
2247         return null;
2248       }
2249     });
2250   }
2251 
2252   /**
2253    * {@inheritDoc}
2254    */
2255   @Override
2256   public void split(final TableName tableName)
2257     throws IOException {
2258     split(tableName, null);
2259   }
2260 
2261   /**
2262    * {@inheritDoc}
2263    */
2264   @Override
2265   public void splitRegion(final byte[] regionName)
2266     throws IOException {
2267     splitRegion(regionName, null);
2268   }
2269 
2270   /**
2271    * @deprecated Use {@link #split(org.apache.hadoop.hbase.TableName)} or {@link #splitRegion
2272    * (byte[])} instead.
2273    */
2274   @Deprecated
2275   public void split(final String tableNameOrRegionName)
2276   throws IOException, InterruptedException {
2277     split(Bytes.toBytes(tableNameOrRegionName));
2278   }
2279 
2280   /**
2281    * @deprecated Use {@link #split(org.apache.hadoop.hbase.TableName)} or {@link #splitRegion
2282    * (byte[])} instead.
2283    */
2284   @Deprecated
2285   public void split(final byte[] tableNameOrRegionName)
2286   throws IOException, InterruptedException {
2287     split(tableNameOrRegionName, null);
2288   }
2289 
2290   /**
2291    * {@inheritDoc}
2292    */
2293   @Override
2294   public void split(final TableName tableName, final byte [] splitPoint)
2295   throws IOException {
2296     ZooKeeperWatcher zookeeper = null;
2297     try {
2298       checkTableExists(tableName);
2299       zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2300         new ThrowableAbortable());
2301       List<Pair<HRegionInfo, ServerName>> pairs;
2302       if (TableName.META_TABLE_NAME.equals(tableName)) {
2303         pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
2304       } else {
2305         pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
2306       }
2307       for (Pair<HRegionInfo, ServerName> pair: pairs) {
2308         // May not be a server for a particular row
2309         if (pair.getSecond() == null) continue;
2310         HRegionInfo r = pair.getFirst();
2311         // check for parents
2312         if (r.isSplitParent()) continue;
2313         // if a split point given, only split that particular region
2314         if (r.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
2315            (splitPoint != null && !r.containsRow(splitPoint))) continue;
2316         // call out to region server to do split now
2317         split(pair.getSecond(), pair.getFirst(), splitPoint);
2318       }
2319     } finally {
2320       if (zookeeper != null) {
2321         zookeeper.close();
2322       }
2323     }
2324   }
2325 
2326   /**
2327    * {@inheritDoc}
2328    */
2329   @Override
2330   public void splitRegion(final byte[] regionName, final byte [] splitPoint)
2331   throws IOException {
2332     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
2333     if (regionServerPair == null) {
2334       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
2335     }
2336     if (regionServerPair.getFirst() != null &&
2337         regionServerPair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
2338       throw new IllegalArgumentException("Can't split replicas directly. "
2339           + "Replicas are auto-split when their primary is split.");
2340     }
2341     if (regionServerPair.getSecond() == null) {
2342       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
2343     }
2344     split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
2345   }
2346 
2347   /**
2348    * @deprecated Use {@link #split(org.apache.hadoop.hbase.TableName,
2349    * byte[])} or {@link #splitRegion(byte[], byte[])} instead.
2350    */
2351   @Deprecated
2352   public void split(final String tableNameOrRegionName,
2353     final String splitPoint) throws IOException {
2354     split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
2355   }
2356 
2357   /**
2358    * @deprecated Use {@link #split(org.apache.hadoop.hbase.TableName,
2359    * byte[])} or {@link #splitRegion(byte[], byte[])} instead.
2360    */
2361   @Deprecated
2362   public void split(final byte[] tableNameOrRegionName,
2363       final byte [] splitPoint) throws IOException {
2364     try {
2365       splitRegion(tableNameOrRegionName, splitPoint);
2366     } catch (IllegalArgumentException e) {
2367       // Bad region, try table
2368       split(TableName.valueOf(tableNameOrRegionName), splitPoint);
2369     }
2370   }
2371 
2372   @VisibleForTesting
2373   public void split(final ServerName sn, final HRegionInfo hri,
2374       byte[] splitPoint) throws IOException {
2375     if (hri.getStartKey() != null && splitPoint != null &&
2376          Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
2377        throw new IOException("should not give a splitkey which equals to startkey!");
2378     }
2379     // TODO: This is not executed via retries
2380     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2381     ProtobufUtil.split(admin, hri, splitPoint);
2382   }
2383 
2384   /**
2385    * Modify an existing table, more IRB friendly version. Asynchronous operation.
2386    * This means that it may be a while before your schema change is updated across all of the
2387    * table. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
2388    * It may throw ExecutionException if there was an error while executing the operation
2389    * or TimeoutException in case the wait timeout was not long enough to allow the
2390    * operation to complete.
2391    *
2392    * @param tableName name of table.
2393    * @param htd modified description of the table
2394    * @throws IOException if a remote or network exception occurs
2395    * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
2396    *     operation to complete.
2397    */
2398   @Override
2399   public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
2400   throws IOException {
2401     if (!tableName.equals(htd.getTableName())) {
2402       throw new IllegalArgumentException("the specified table name '" + tableName +
2403         "' doesn't match with the HTD one: " + htd.getTableName());
2404     }
2405 
2406     ModifyTableResponse response = executeCallable(
2407         new MasterCallable<ModifyTableResponse>(getConnection()) {
2408       @Override
2409       public ModifyTableResponse call(int callTimeout) throws ServiceException {
2410         ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
2411           tableName, htd, ng.getNonceGroup(), ng.newNonce());
2412         return master.modifyTable(null, request);
2413       }
2414     });
2415 
2416     return new ModifyTableFuture(this, tableName, response);
2417   }
2418 
2419   private static class ModifyTableFuture extends TableFuture<Void> {
2420     public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
2421         final ModifyTableResponse response) {
2422       super(admin, tableName,
2423           (response != null && response.hasProcId()) ? response.getProcId() : null);
2424     }
2425 
2426     @Override
2427     public String getOperationType() {
2428       return "MODIFY";
2429     }
2430 
2431     @Override
2432     protected Void postOperationResult(final Void result, final long deadlineTs)
2433         throws IOException, TimeoutException {
2434       // The modify operation on the table is asynchronous on the server side irrespective
2435       // of whether Procedure V2 is supported or not. So, we wait in the client till
2436       // all regions get updated.
2437       waitForSchemaUpdate(deadlineTs);
2438       return result;
2439     }
2440   }
2441 
2442   public void modifyTable(final byte[] tableName, final HTableDescriptor htd)
2443   throws IOException {
2444     modifyTable(TableName.valueOf(tableName), htd);
2445   }
2446 
2447   public void modifyTable(final String tableName, final HTableDescriptor htd)
2448   throws IOException {
2449     modifyTable(TableName.valueOf(tableName), htd);
2450   }
2451 
2452   /**
2453    * @param regionName Name of a region.
2454    * @return a pair of HRegionInfo and ServerName if <code>regionName</code> is
2455    *  a verified region name (we call {@link
2456    *  MetaTableAccessor#getRegion(HConnection, byte[])}
2457    *  else null.
2458    * Throw IllegalArgumentException if <code>regionName</code> is null.
2459    * @throws IOException
2460    */
2461   Pair<HRegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
2462     if (regionName == null) {
2463       throw new IllegalArgumentException("Pass a table name or region name");
2464     }
2465     Pair<HRegionInfo, ServerName> pair =
2466       MetaTableAccessor.getRegion(connection, regionName);
2467     if (pair == null) {
2468       final AtomicReference<Pair<HRegionInfo, ServerName>> result =
2469         new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
2470       final String encodedName = Bytes.toString(regionName);
2471       MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
2472         @Override
2473         public boolean visit(Result data) throws IOException {
2474           HRegionInfo info = HRegionInfo.getHRegionInfo(data);
2475           if (info == null) {
2476             LOG.warn("No serialized HRegionInfo in " + data);
2477             return true;
2478           }
2479           RegionLocations rl = MetaTableAccessor.getRegionLocations(data);
2480           boolean matched = false;
2481           ServerName sn = null;
2482           if (rl != null) {
2483             for (HRegionLocation h : rl.getRegionLocations()) {
2484               if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) {
2485                 sn = h.getServerName();
2486                 info = h.getRegionInfo();
2487                 matched = true;
2488               }
2489             }
2490           }
2491           if (!matched) return true;
2492           result.set(new Pair<HRegionInfo, ServerName>(info, sn));
2493           return false; // found the region, stop
2494         }
2495       };
2496 
2497       MetaTableAccessor.fullScanRegions(connection, visitor);
2498       pair = result.get();
2499     }
2500     return pair;
2501   }
2502 
2503   /**
2504    * If the input is a region name, it is returned as is. If it's an
2505    * encoded region name, the corresponding region is found from meta
2506    * and its region name is returned. If we can't find any region in
2507    * meta matching the input as either region name or encoded region
2508    * name, the input is returned as is. We don't throw unknown
2509    * region exception.
2510    */
2511   private byte[] getRegionName(
2512       final byte[] regionNameOrEncodedRegionName) throws IOException {
2513     if (Bytes.equals(regionNameOrEncodedRegionName,
2514         HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
2515           || Bytes.equals(regionNameOrEncodedRegionName,
2516             HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
2517       return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
2518     }
2519     byte[] tmp = regionNameOrEncodedRegionName;
2520     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
2521     if (regionServerPair != null && regionServerPair.getFirst() != null) {
2522       tmp = regionServerPair.getFirst().getRegionName();
2523     }
2524     return tmp;
2525   }
2526 
2527   /**
2528    * Check if table exists or not
2529    * @param tableName Name of a table.
2530    * @return tableName instance
2531    * @throws IOException if a remote or network exception occurs.
2532    * @throws TableNotFoundException if table does not exist.
2533    */
2534   private TableName checkTableExists(final TableName tableName)
2535       throws IOException {
2536     return executeCallable(new ConnectionCallable<TableName>(getConnection()) {
2537       @Override
2538       public TableName call(int callTimeout) throws ServiceException, IOException {
2539         if (!MetaTableAccessor.tableExists(connection, tableName)) {
2540           throw new TableNotFoundException(tableName);
2541         }
2542         return tableName;
2543       }
2544     });
2545   }
2546 
2547   /**
2548    * Shuts down the HBase cluster
2549    * @throws IOException if a remote or network exception occurs
2550    */
2551   @Override
2552   public synchronized void shutdown() throws IOException {
2553     executeCallable(new MasterCallable<Void>(getConnection()) {
2554       @Override
2555       public Void call(int callTimeout) throws ServiceException {
2556         master.shutdown(null,ShutdownRequest.newBuilder().build());
2557         return null;
2558       }
2559     });
2560   }
2561 
2562   /**
2563    * Shuts down the current HBase master only.
2564    * Does not shutdown the cluster.
2565    * @see #shutdown()
2566    * @throws IOException if a remote or network exception occurs
2567    */
2568   @Override
2569   public synchronized void stopMaster() throws IOException {
2570     executeCallable(new MasterCallable<Void>(getConnection()) {
2571       @Override
2572       public Void call(int callTimeout) throws ServiceException {
2573         master.stopMaster(null, StopMasterRequest.newBuilder().build());
2574         return null;
2575       }
2576     });
2577   }
2578 
2579   /**
2580    * Stop the designated regionserver
2581    * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
2582    * <code>example.org:1234</code>
2583    * @throws IOException if a remote or network exception occurs
2584    */
2585   @Override
2586   public synchronized void stopRegionServer(final String hostnamePort)
2587   throws IOException {
2588     String hostname = Addressing.parseHostname(hostnamePort);
2589     int port = Addressing.parsePort(hostnamePort);
2590     AdminService.BlockingInterface admin =
2591       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
2592     StopServerRequest request = RequestConverter.buildStopServerRequest(
2593       "Called by admin client " + this.connection.toString());
2594     try {
2595       admin.stopServer(null, request);
2596     } catch (ServiceException se) {
2597       throw ProtobufUtil.getRemoteException(se);
2598     }
2599   }
2600 
2601 
2602   /**
2603    * @return cluster status
2604    * @throws IOException if a remote or network exception occurs
2605    */
2606   @Override
2607   public ClusterStatus getClusterStatus() throws IOException {
2608     return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
2609       @Override
2610       public ClusterStatus call(int callTimeout) throws ServiceException {
2611         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
2612         return ClusterStatus.convert(master.getClusterStatus(null, req).getClusterStatus());
2613       }
2614     });
2615   }
2616 
2617   /**
2618    * @return Configuration used by the instance.
2619    */
2620   @Override
2621   public Configuration getConfiguration() {
2622     return this.conf;
2623   }
2624 
2625   /**
2626    * Create a new namespace
2627    * @param descriptor descriptor which describes the new namespace
2628    * @throws IOException
2629    */
2630   @Override
2631   public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
2632     executeCallable(new MasterCallable<Void>(getConnection()) {
2633       @Override
2634       public Void call(int callTimeout) throws Exception {
2635         master.createNamespace(null,
2636           CreateNamespaceRequest.newBuilder()
2637             .setNamespaceDescriptor(ProtobufUtil
2638               .toProtoNamespaceDescriptor(descriptor)).build()
2639         );
2640         return null;
2641       }
2642     });
2643   }
2644 
2645   /**
2646    * Modify an existing namespace
2647    * @param descriptor descriptor which describes the new namespace
2648    * @throws IOException
2649    */
2650   @Override
2651   public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
2652     executeCallable(new MasterCallable<Void>(getConnection()) {
2653       @Override
2654       public Void call(int callTimeout) throws Exception {
2655         master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
2656           setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
2657         return null;
2658       }
2659     });
2660   }
2661 
2662   /**
2663    * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
2664    * @param name namespace name
2665    * @throws IOException
2666    */
2667   @Override
2668   public void deleteNamespace(final String name) throws IOException {
2669     executeCallable(new MasterCallable<Void>(getConnection()) {
2670       @Override
2671       public Void call(int callTimeout) throws Exception {
2672         master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
2673           setNamespaceName(name).build());
2674         return null;
2675       }
2676     });
2677   }
2678 
2679   /**
2680    * Get a namespace descriptor by name
2681    * @param name name of namespace descriptor
2682    * @return A descriptor
2683    * @throws IOException
2684    */
2685   @Override
2686   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
2687     return
2688         executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
2689           @Override
2690           public NamespaceDescriptor call(int callTimeout) throws Exception {
2691             return ProtobufUtil.toNamespaceDescriptor(
2692               master.getNamespaceDescriptor(null, GetNamespaceDescriptorRequest.newBuilder().
2693                 setNamespaceName(name).build()).getNamespaceDescriptor());
2694           }
2695         });
2696   }
2697 
2698   /**
2699    * List available namespace descriptors
2700    * @return List of descriptors
2701    * @throws IOException
2702    */
2703   @Override
2704   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
2705     return
2706         executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
2707           @Override
2708           public NamespaceDescriptor[] call(int callTimeout) throws Exception {
2709             List<HBaseProtos.NamespaceDescriptor> list =
2710               master.listNamespaceDescriptors(null, ListNamespaceDescriptorsRequest.newBuilder().
2711                 build()).getNamespaceDescriptorList();
2712             NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
2713             for(int i = 0; i < list.size(); i++) {
2714               res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
2715             }
2716             return res;
2717           }
2718         });
2719   }
2720 
2721   /**
2722    * Get list of table descriptors by namespace
2723    * @param name namespace name
2724    * @return A descriptor
2725    * @throws IOException
2726    */
2727   @Override
2728   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
2729     return
2730         executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
2731           @Override
2732           public HTableDescriptor[] call(int callTimeout) throws Exception {
2733             List<TableSchema> list =
2734               master.listTableDescriptorsByNamespace(null, ListTableDescriptorsByNamespaceRequest.
2735                 newBuilder().setNamespaceName(name).build()).getTableSchemaList();
2736             HTableDescriptor[] res = new HTableDescriptor[list.size()];
2737             for(int i=0; i < list.size(); i++) {
2738 
2739               res[i] = HTableDescriptor.convert(list.get(i));
2740             }
2741             return res;
2742           }
2743         });
2744   }
2745 
2746   /**
2747    * Get list of table names by namespace
2748    * @param name namespace name
2749    * @return The list of table names in the namespace
2750    * @throws IOException
2751    */
2752   @Override
2753   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
2754     return
2755         executeCallable(new MasterCallable<TableName[]>(getConnection()) {
2756           @Override
2757           public TableName[] call(int callTimeout) throws Exception {
2758             List<HBaseProtos.TableName> tableNames =
2759               master.listTableNamesByNamespace(null, ListTableNamesByNamespaceRequest.
2760                 newBuilder().setNamespaceName(name).build())
2761                 .getTableNameList();
2762             TableName[] result = new TableName[tableNames.size()];
2763             for (int i = 0; i < tableNames.size(); i++) {
2764               result[i] = ProtobufUtil.toTableName(tableNames.get(i));
2765             }
2766             return result;
2767           }
2768         });
2769   }
2770 
2771   /**
2772    * Check to see if HBase is running. Throw an exception if not.
2773    * @param conf system configuration
2774    * @throws MasterNotRunningException if the master is not running
2775    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
2776    */
2777   // Used by tests and by the Merge tool. Merge tool uses it to figure if HBase is up or not.
2778   public static void checkHBaseAvailable(Configuration conf)
2779   throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException {
2780     Configuration copyOfConf = HBaseConfiguration.create(conf);
2781     // We set it to make it fail as soon as possible if HBase is not available
2782     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
2783     copyOfConf.setInt("zookeeper.recovery.retry", 0);
2784     try (ClusterConnection connection =
2785         (ClusterConnection)ConnectionFactory.createConnection(copyOfConf)) {
2786         // Check ZK first.
2787         // If the connection exists, we may have a connection to ZK that does not work anymore
2788         ZooKeeperKeepAliveConnection zkw = null;
2789         try {
2790           // This is NASTY. FIX!!!! Dependent on internal implementation! TODO
2791           zkw = ((ConnectionImplementation)connection).
2792             getKeepAliveZooKeeperWatcher();
2793           zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false);
2794         } catch (IOException e) {
2795           throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2796         } catch (InterruptedException e) {
2797           throw (InterruptedIOException)
2798             new InterruptedIOException("Can't connect to ZooKeeper").initCause(e);
2799         } catch (KeeperException e) {
2800           throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2801         } finally {
2802           if (zkw != null) {
2803             zkw.close();
2804           }
2805         }
2806       connection.isMasterRunning();
2807     }
2808   }
2809 
2810   /**
2811    * get the regions of a given table.
2812    *
2813    * @param tableName the name of the table
2814    * @return Ordered list of {@link HRegionInfo}.
2815    * @throws IOException
2816    */
2817   @Override
2818   public List<HRegionInfo> getTableRegions(final TableName tableName)
2819   throws IOException {
2820     ZooKeeperWatcher zookeeper =
2821       new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2822         new ThrowableAbortable());
2823     List<HRegionInfo> regions = null;
2824     try {
2825       if (TableName.META_TABLE_NAME.equals(tableName)) {
2826         regions = new MetaTableLocator().getMetaRegions(zookeeper);
2827       } else {
2828         regions = MetaTableAccessor.getTableRegions(connection, tableName, true);
2829       }
2830     } finally {
2831       zookeeper.close();
2832     }
2833     return regions;
2834   }
2835 
2836   public List<HRegionInfo> getTableRegions(final byte[] tableName)
2837   throws IOException {
2838     return getTableRegions(TableName.valueOf(tableName));
2839   }
2840 
2841   @Override
2842   public synchronized void close() throws IOException {
2843     if (cleanupConnectionOnClose && this.connection != null && !this.closed) {
2844       this.connection.close();
2845       this.closed = true;
2846     }
2847   }
2848 
2849   /**
2850    * Get tableDescriptors
2851    * @param tableNames List of table names
2852    * @return HTD[] the tableDescriptor
2853    * @throws IOException if a remote or network exception occurs
2854    */
2855   @Override
2856   public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
2857   throws IOException {
2858     return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
2859       @Override
2860       public HTableDescriptor[] call(int callTimeout) throws Exception {
2861         GetTableDescriptorsRequest req =
2862             RequestConverter.buildGetTableDescriptorsRequest(tableNames);
2863           return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
2864       }
2865     });
2866   }
2867 
2868   /**
2869    * Get tableDescriptor
2870    * @param tableName one table name
2871    * @return HTD the HTableDescriptor or null if the table not exists
2872    * @throws IOException if a remote or network exception occurs
2873    */
2874   private HTableDescriptor getTableDescriptorByTableName(TableName tableName)
2875       throws IOException {
2876     List<TableName> tableNames = new ArrayList<TableName>(1);
2877     tableNames.add(tableName);
2878 
2879     HTableDescriptor[] htdl = getTableDescriptorsByTableName(tableNames);
2880 
2881     if (htdl == null || htdl.length == 0) {
2882       return null;
2883     }
2884     else {
2885       return htdl[0];
2886     }
2887   }
2888 
2889   /**
2890    * Get tableDescriptors
2891    * @param names List of table names
2892    * @return HTD[] the tableDescriptor
2893    * @throws IOException if a remote or network exception occurs
2894    */
2895   @Override
2896   public HTableDescriptor[] getTableDescriptors(List<String> names)
2897   throws IOException {
2898     List<TableName> tableNames = new ArrayList<TableName>(names.size());
2899     for(String name : names) {
2900       tableNames.add(TableName.valueOf(name));
2901     }
2902     return getTableDescriptorsByTableName(tableNames);
2903   }
2904 
2905   private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
2906       FailedLogCloseException {
2907     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2908     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
2909     try {
2910       return admin.rollWALWriter(null, request);
2911     } catch (ServiceException se) {
2912       throw ProtobufUtil.getRemoteException(se);
2913     }
2914   }
2915 
2916   /**
2917    * Roll the log writer. I.e. when using a file system based write ahead log,
2918    * start writing log messages to a new file.
2919    *
2920    * Note that when talking to a version 1.0+ HBase deployment, the rolling is asynchronous.
2921    * This method will return as soon as the roll is requested and the return value will
2922    * always be null. Additionally, the named region server may schedule store flushes at the
2923    * request of the wal handling the roll request.
2924    *
2925    * When talking to a 0.98 or older HBase deployment, the rolling is synchronous and the
2926    * return value may be either null or a list of encoded region names.
2927    *
2928    * @param serverName
2929    *          The servername of the regionserver. A server name is made of host,
2930    *          port and startcode. This is mandatory. Here is an example:
2931    *          <code> host187.example.com,60020,1289493121758</code>
2932    * @return a set of {@link HRegionInfo#getEncodedName()} that would allow the wal to
2933    *         clean up some underlying files. null if there's nothing to flush.
2934    * @throws IOException if a remote or network exception occurs
2935    * @throws FailedLogCloseException
2936    * @deprecated use {@link #rollWALWriter(ServerName)}
2937    */
2938   @Deprecated
2939   public synchronized byte[][] rollHLogWriter(String serverName)
2940       throws IOException, FailedLogCloseException {
2941     ServerName sn = ServerName.valueOf(serverName);
2942     final RollWALWriterResponse response = rollWALWriterImpl(sn);
2943     int regionCount = response.getRegionToFlushCount();
2944     if (0 == regionCount) {
2945       return null;
2946     }
2947     byte[][] regionsToFlush = new byte[regionCount][];
2948     for (int i = 0; i < regionCount; i++) {
2949       ByteString region = response.getRegionToFlush(i);
2950       regionsToFlush[i] = region.toByteArray();
2951     }
2952     return regionsToFlush;
2953   }
2954 
2955   @Override
2956   public synchronized void rollWALWriter(ServerName serverName)
2957       throws IOException, FailedLogCloseException {
2958     rollWALWriterImpl(serverName);
2959   }
2960 
2961   @Override
2962   public String[] getMasterCoprocessors() {
2963     try {
2964       return getClusterStatus().getMasterCoprocessors();
2965     } catch (IOException e) {
2966       LOG.error("Could not getClusterStatus()",e);
2967       return null;
2968     }
2969   }
2970 
2971   /**
2972    * {@inheritDoc}
2973    */
2974   @Override
2975   public CompactionState getCompactionState(final TableName tableName)
2976   throws IOException {
2977     CompactionState state = CompactionState.NONE;
2978     ZooKeeperWatcher zookeeper =
2979       new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2980         new ThrowableAbortable());
2981     try {
2982       checkTableExists(tableName);
2983       List<Pair<HRegionInfo, ServerName>> pairs;
2984       if (TableName.META_TABLE_NAME.equals(tableName)) {
2985         pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
2986       } else {
2987         pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
2988       }
2989       for (Pair<HRegionInfo, ServerName> pair: pairs) {
2990         if (pair.getFirst().isOffline()) continue;
2991         if (pair.getSecond() == null) continue;
2992         try {
2993           ServerName sn = pair.getSecond();
2994           AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2995           GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
2996             pair.getFirst().getRegionName(), true);
2997           GetRegionInfoResponse response = admin.getRegionInfo(null, request);
2998           switch (response.getCompactionState()) {
2999           case MAJOR_AND_MINOR:
3000             return CompactionState.MAJOR_AND_MINOR;
3001           case MAJOR:
3002             if (state == CompactionState.MINOR) {
3003               return CompactionState.MAJOR_AND_MINOR;
3004             }
3005             state = CompactionState.MAJOR;
3006             break;
3007           case MINOR:
3008             if (state == CompactionState.MAJOR) {
3009               return CompactionState.MAJOR_AND_MINOR;
3010             }
3011             state = CompactionState.MINOR;
3012             break;
3013           case NONE:
3014           default: // nothing, continue
3015           }
3016         } catch (NotServingRegionException e) {
3017           if (LOG.isDebugEnabled()) {
3018             LOG.debug("Trying to get compaction state of " +
3019               pair.getFirst() + ": " +
3020               StringUtils.stringifyException(e));
3021           }
3022         } catch (RemoteException e) {
3023           if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
3024             if (LOG.isDebugEnabled()) {
3025               LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
3026                 + StringUtils.stringifyException(e));
3027             }
3028           } else {
3029             throw e;
3030           }
3031         }
3032       }
3033     } catch (ServiceException se) {
3034       throw ProtobufUtil.getRemoteException(se);
3035     } finally {
3036       zookeeper.close();
3037     }
3038     return state;
3039   }
3040 
3041   /**
3042    * {@inheritDoc}
3043    */
3044   @Override
3045   public CompactionState getCompactionStateForRegion(final byte[] regionName)
3046   throws IOException {
3047     try {
3048       Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
3049       if (regionServerPair == null) {
3050         throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
3051       }
3052       if (regionServerPair.getSecond() == null) {
3053         throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
3054       }
3055       ServerName sn = regionServerPair.getSecond();
3056       AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
3057       GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
3058         regionServerPair.getFirst().getRegionName(), true);
3059       GetRegionInfoResponse response = admin.getRegionInfo(null, request);
3060       return response.getCompactionState();
3061     } catch (ServiceException se) {
3062       throw ProtobufUtil.getRemoteException(se);
3063     }
3064   }
3065 
3066   /**
3067    * @deprecated Use {@link #getCompactionState(org.apache.hadoop.hbase.TableName)} or {@link
3068    * #getCompactionStateForRegion(byte[])} instead.
3069    */
3070   @Deprecated
3071   public CompactionState getCompactionState(final String tableNameOrRegionName)
3072   throws IOException, InterruptedException {
3073     return getCompactionState(Bytes.toBytes(tableNameOrRegionName));
3074   }
3075 
3076   /**
3077    * @deprecated Use {@link #getCompactionState(org.apache.hadoop.hbase.TableName)} or {@link
3078    * #getCompactionStateForRegion(byte[])} instead.
3079    */
3080   @Deprecated
3081   public CompactionState getCompactionState(final byte[] tableNameOrRegionName)
3082   throws IOException, InterruptedException {
3083     try {
3084       return getCompactionStateForRegion(tableNameOrRegionName);
3085     } catch (IllegalArgumentException e) {
3086       // Invalid region, try table
3087       return getCompactionState(TableName.valueOf(tableNameOrRegionName));
3088     }
3089   }
3090 
3091   /**
3092    * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
3093    * taken. If the table is disabled, an offline snapshot is taken.
3094    * <p>
3095    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
3096    * snapshot with the same name (even a different type or with different parameters) will fail with
3097    * a {@link SnapshotCreationException} indicating the duplicate naming.
3098    * <p>
3099    * Snapshot names follow the same naming constraints as tables in HBase. See
3100    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
3101    * @param snapshotName name of the snapshot to be created
3102    * @param tableName name of the table for which snapshot is created
3103    * @throws IOException if a remote or network exception occurs
3104    * @throws SnapshotCreationException if snapshot creation failed
3105    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
3106    */
3107   @Override
3108   public void snapshot(final String snapshotName,
3109                        final TableName tableName) throws IOException,
3110       SnapshotCreationException, IllegalArgumentException {
3111     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
3112   }
3113 
3114   public void snapshot(final String snapshotName,
3115                        final String tableName) throws IOException,
3116       SnapshotCreationException, IllegalArgumentException {
3117     snapshot(snapshotName, TableName.valueOf(tableName),
3118         SnapshotDescription.Type.FLUSH);
3119   }
3120 
3121   /**
3122    * Create snapshot for the given table of given flush type.
3123    * <p>
3124    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
3125    * snapshot with the same name (even a different type or with different parameters) will fail with
3126    * a {@link SnapshotCreationException} indicating the duplicate naming.
3127    * <p>
3128    * Snapshot names follow the same naming constraints as tables in HBase.
3129    * @param snapshotName name of the snapshot to be created
3130    * @param tableName name of the table for which snapshot is created
3131    * @param flushType if the snapshot should be taken without flush memstore first
3132    * @throws IOException if a remote or network exception occurs
3133    * @throws SnapshotCreationException if snapshot creation failed
3134    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
3135    */
3136    public void snapshot(final byte[] snapshotName, final byte[] tableName,
3137                        final SnapshotDescription.Type flushType) throws
3138       IOException, SnapshotCreationException, IllegalArgumentException {
3139       snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName), flushType);
3140   }
3141   /**
3142    public void snapshot(final String snapshotName,
3143     * Create a timestamp consistent snapshot for the given table.
3144                         final byte[] tableName) throws IOException,
3145     * <p>
3146     * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
3147     * snapshot with the same name (even a different type or with different parameters) will fail
3148     * with a {@link SnapshotCreationException} indicating the duplicate naming.
3149     * <p>
3150     * Snapshot names follow the same naming constraints as tables in HBase.
3151     * @param snapshotName name of the snapshot to be created
3152     * @param tableName name of the table for which snapshot is created
3153     * @throws IOException if a remote or network exception occurs
3154     * @throws SnapshotCreationException if snapshot creation failed
3155     * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
3156     */
3157   @Override
3158   public void snapshot(final byte[] snapshotName,
3159                        final TableName tableName) throws IOException,
3160       SnapshotCreationException, IllegalArgumentException {
3161     snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH);
3162   }
3163 
3164   public void snapshot(final byte[] snapshotName,
3165                        final byte[] tableName) throws IOException,
3166       SnapshotCreationException, IllegalArgumentException {
3167     snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName),
3168       SnapshotDescription.Type.FLUSH);
3169   }
3170 
3171   /**
3172    * Create typed snapshot of the table.
3173    * <p>
3174    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
3175    * snapshot with the same name (even a different type or with different parameters) will fail with
3176    * a {@link SnapshotCreationException} indicating the duplicate naming.
3177    * <p>
3178    * Snapshot names follow the same naming constraints as tables in HBase. See
3179    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
3180    * <p>
3181    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
3182    *          snapshots stored on the cluster
3183    * @param tableName name of the table to snapshot
3184    * @param type type of snapshot to take
3185    * @throws IOException we fail to reach the master
3186    * @throws SnapshotCreationException if snapshot creation failed
3187    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
3188    */
3189   @Override
3190   public void snapshot(final String snapshotName,
3191                        final TableName tableName,
3192                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
3193       IllegalArgumentException {
3194     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
3195     builder.setTable(tableName.getNameAsString());
3196     builder.setName(snapshotName);
3197     builder.setType(type);
3198     snapshot(builder.build());
3199   }
3200 
3201   public void snapshot(final String snapshotName,
3202                        final String tableName,
3203                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
3204       IllegalArgumentException {
3205     snapshot(snapshotName, TableName.valueOf(tableName), type);
3206   }
3207 
3208   public void snapshot(final String snapshotName,
3209                        final byte[] tableName,
3210                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
3211       IllegalArgumentException {
3212     snapshot(snapshotName, TableName.valueOf(tableName), type);
3213   }
3214 
3215   /**
3216    * Take a snapshot and wait for the server to complete that snapshot (blocking).
3217    * <p>
3218    * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
3219    * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
3220    * time for a single cluster).
3221    * <p>
3222    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
3223    * snapshot with the same name (even a different type or with different parameters) will fail with
3224    * a {@link SnapshotCreationException} indicating the duplicate naming.
3225    * <p>
3226    * Snapshot names follow the same naming constraints as tables in HBase. See
3227    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
3228    * <p>
3229    * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
3230    * unless you are sure about the type of snapshot that you want to take.
3231    * @param snapshot snapshot to take
3232    * @throws IOException or we lose contact with the master.
3233    * @throws SnapshotCreationException if snapshot failed to be taken
3234    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
3235    */
3236   @Override
3237   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
3238       IllegalArgumentException {
3239     // actually take the snapshot
3240     SnapshotResponse response = takeSnapshotAsync(snapshot);
3241     final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
3242         .build();
3243     IsSnapshotDoneResponse done = null;
3244     long start = EnvironmentEdgeManager.currentTime();
3245     long max = response.getExpectedTimeout();
3246     long maxPauseTime = max / this.numRetries;
3247     int tries = 0;
3248     LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
3249         ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
3250         maxPauseTime + " ms per retry)");
3251     while (tries == 0
3252         || ((EnvironmentEdgeManager.currentTime() - start) < max && !done.getDone())) {
3253       try {
3254         // sleep a backoff <= pauseTime amount
3255         long sleep = getPauseTime(tries++);
3256         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3257         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
3258           "ms while waiting for snapshot completion.");
3259         Thread.sleep(sleep);
3260       } catch (InterruptedException e) {
3261         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3262       }
3263       LOG.debug("Getting current status of snapshot from master...");
3264       done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
3265         @Override
3266         public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
3267           return master.isSnapshotDone(null, request);
3268         }
3269       });
3270     }
3271     if (!done.getDone()) {
3272       throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
3273           + "' wasn't completed in expectedTime:" + max + " ms", snapshot);
3274     }
3275   }
3276 
3277   /**
3278    * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
3279    * <p>
3280    * Only a single snapshot should be taken at a time, or results may be undefined.
3281    * @param snapshot snapshot to take
3282    * @return response from the server indicating the max time to wait for the snapshot
3283    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
3284    * @throws SnapshotCreationException if snapshot creation failed
3285    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
3286    */
3287   @Override
3288   public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
3289       SnapshotCreationException {
3290     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
3291     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
3292         .build();
3293     // run the snapshot on the master
3294     return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
3295       @Override
3296       public SnapshotResponse call(int callTimeout) throws ServiceException {
3297         return master.snapshot(null, request);
3298       }
3299     });
3300   }
3301 
3302   /**
3303    * Check the current state of the passed snapshot.
3304    * <p>
3305    * There are three possible states:
3306    * <ol>
3307    * <li>running - returns <tt>false</tt></li>
3308    * <li>finished - returns <tt>true</tt></li>
3309    * <li>finished with error - throws the exception that caused the snapshot to fail</li>
3310    * </ol>
3311    * <p>
3312    * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
3313    * run/started since the snapshot your are checking, you will recieve an
3314    * {@link UnknownSnapshotException}.
3315    * @param snapshot description of the snapshot to check
3316    * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
3317    *         running
3318    * @throws IOException if we have a network issue
3319    * @throws HBaseSnapshotException if the snapshot failed
3320    * @throws UnknownSnapshotException if the requested snapshot is unknown
3321    */
3322   @Override
3323   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
3324       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
3325 
3326     return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
3327       @Override
3328       public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
3329         return master.isSnapshotDone(null,
3330           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
3331       }
3332     }).getDone();
3333   }
3334 
3335   /**
3336    * Restore the specified snapshot on the original table. (The table must be disabled)
3337    * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
3338    * is set to true, a snapshot of the current table is taken
3339    * before executing the restore operation.
3340    * In case of restore failure, the failsafe snapshot will be restored.
3341    * If the restore completes without problem the failsafe snapshot is deleted.
3342    *
3343    * @param snapshotName name of the snapshot to restore
3344    * @throws IOException if a remote or network exception occurs
3345    * @throws RestoreSnapshotException if snapshot failed to be restored
3346    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3347    */
3348   @Override
3349   public void restoreSnapshot(final byte[] snapshotName)
3350       throws IOException, RestoreSnapshotException {
3351     restoreSnapshot(Bytes.toString(snapshotName));
3352   }
3353 
3354   /**
3355    * Restore the specified snapshot on the original table. (The table must be disabled)
3356    * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
3357    * is set to true, a snapshot of the current table is taken
3358    * before executing the restore operation.
3359    * In case of restore failure, the failsafe snapshot will be restored.
3360    * If the restore completes without problem the failsafe snapshot is deleted.
3361    *
3362    * @param snapshotName name of the snapshot to restore
3363    * @throws IOException if a remote or network exception occurs
3364    * @throws RestoreSnapshotException if snapshot failed to be restored
3365    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3366    */
3367   @Override
3368   public void restoreSnapshot(final String snapshotName)
3369       throws IOException, RestoreSnapshotException {
3370     boolean takeFailSafeSnapshot =
3371       conf.getBoolean("hbase.snapshot.restore.take.failsafe.snapshot", false);
3372     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
3373   }
3374 
3375   /**
3376    * Restore the specified snapshot on the original table. (The table must be disabled)
3377    * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
3378    * before executing the restore operation.
3379    * In case of restore failure, the failsafe snapshot will be restored.
3380    * If the restore completes without problem the failsafe snapshot is deleted.
3381    *
3382    * The failsafe snapshot name is configurable by using the property
3383    * "hbase.snapshot.restore.failsafe.name".
3384    *
3385    * @param snapshotName name of the snapshot to restore
3386    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
3387    * @throws IOException if a remote or network exception occurs
3388    * @throws RestoreSnapshotException if snapshot failed to be restored
3389    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3390    */
3391   @Override
3392   public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
3393       throws IOException, RestoreSnapshotException {
3394     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
3395   }
3396 
3397   /**
3398    * Restore the specified snapshot on the original table. (The table must be disabled)
3399    * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
3400    * before executing the restore operation.
3401    * In case of restore failure, the failsafe snapshot will be restored.
3402    * If the restore completes without problem the failsafe snapshot is deleted.
3403    *
3404    * The failsafe snapshot name is configurable by using the property
3405    * "hbase.snapshot.restore.failsafe.name".
3406    *
3407    * @param snapshotName name of the snapshot to restore
3408    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
3409    * @throws IOException if a remote or network exception occurs
3410    * @throws RestoreSnapshotException if snapshot failed to be restored
3411    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3412    */
3413   @Override
3414   public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
3415       throws IOException, RestoreSnapshotException {
3416     TableName tableName = null;
3417     for (SnapshotDescription snapshotInfo: listSnapshots()) {
3418       if (snapshotInfo.getName().equals(snapshotName)) {
3419         tableName = TableName.valueOf(snapshotInfo.getTable());
3420         break;
3421       }
3422     }
3423 
3424     if (tableName == null) {
3425       throw new RestoreSnapshotException(
3426         "Unable to find the table name for snapshot=" + snapshotName);
3427     }
3428 
3429     // The table does not exists, switch to clone.
3430     if (!tableExists(tableName)) {
3431       cloneSnapshot(snapshotName, tableName);
3432       return;
3433     }
3434 
3435     // Check if the table is disabled
3436     if (!isTableDisabled(tableName)) {
3437       throw new TableNotDisabledException(tableName);
3438     }
3439 
3440     // Take a snapshot of the current state
3441     String failSafeSnapshotSnapshotName = null;
3442     if (takeFailSafeSnapshot) {
3443       failSafeSnapshotSnapshotName = conf.get("hbase.snapshot.restore.failsafe.name",
3444         "hbase-failsafe-{snapshot.name}-{restore.timestamp}");
3445       failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotName
3446         .replace("{snapshot.name}", snapshotName)
3447         .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
3448         .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTime()));
3449       LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
3450       snapshot(failSafeSnapshotSnapshotName, tableName);
3451     }
3452 
3453     try {
3454       // Restore snapshot
3455       internalRestoreSnapshot(snapshotName, tableName);
3456     } catch (IOException e) {
3457       // Somthing went wrong during the restore...
3458       // if the pre-restore snapshot is available try to rollback
3459       if (takeFailSafeSnapshot) {
3460         try {
3461           internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName);
3462           String msg = "Restore snapshot=" + snapshotName +
3463             " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded.";
3464           LOG.error(msg, e);
3465           throw new RestoreSnapshotException(msg, e);
3466         } catch (IOException ex) {
3467           String msg = "Failed to restore and rollback to snapshot=" + failSafeSnapshotSnapshotName;
3468           LOG.error(msg, ex);
3469           throw new RestoreSnapshotException(msg, e);
3470         }
3471       } else {
3472         throw new RestoreSnapshotException("Failed to restore snapshot=" + snapshotName, e);
3473       }
3474     }
3475 
3476     // If the restore is succeeded, delete the pre-restore snapshot
3477     if (takeFailSafeSnapshot) {
3478       try {
3479         LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
3480         deleteSnapshot(failSafeSnapshotSnapshotName);
3481       } catch (IOException e) {
3482         LOG.error("Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, e);
3483       }
3484     }
3485   }
3486 
3487   /**
3488    * Create a new table by cloning the snapshot content.
3489    *
3490    * @param snapshotName name of the snapshot to be cloned
3491    * @param tableName name of the table where the snapshot will be restored
3492    * @throws IOException if a remote or network exception occurs
3493    * @throws TableExistsException if table to be created already exists
3494    * @throws RestoreSnapshotException if snapshot failed to be cloned
3495    * @throws IllegalArgumentException if the specified table has not a valid name
3496    */
3497   public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
3498       throws IOException, TableExistsException, RestoreSnapshotException {
3499     cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
3500   }
3501 
3502   /**
3503    * Create a new table by cloning the snapshot content.
3504    *
3505    * @param snapshotName name of the snapshot to be cloned
3506    * @param tableName name of the table where the snapshot will be restored
3507    * @throws IOException if a remote or network exception occurs
3508    * @throws TableExistsException if table to be created already exists
3509    * @throws RestoreSnapshotException if snapshot failed to be cloned
3510    * @throws IllegalArgumentException if the specified table has not a valid name
3511    */
3512   @Override
3513   public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
3514       throws IOException, TableExistsException, RestoreSnapshotException {
3515     cloneSnapshot(Bytes.toString(snapshotName), tableName);
3516   }
3517 
3518 
3519 
3520   /**
3521    * Create a new table by cloning the snapshot content.
3522    *
3523    * @param snapshotName name of the snapshot to be cloned
3524    * @param tableName name of the table where the snapshot will be restored
3525    * @throws IOException if a remote or network exception occurs
3526    * @throws TableExistsException if table to be created already exists
3527    * @throws RestoreSnapshotException if snapshot failed to be cloned
3528    * @throws IllegalArgumentException if the specified table has not a valid name
3529    */
3530   public void cloneSnapshot(final String snapshotName, final String tableName)
3531       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3532     cloneSnapshot(snapshotName, TableName.valueOf(tableName));
3533   }
3534 
3535   /**
3536    * Create a new table by cloning the snapshot content.
3537    *
3538    * @param snapshotName name of the snapshot to be cloned
3539    * @param tableName name of the table where the snapshot will be restored
3540    * @throws IOException if a remote or network exception occurs
3541    * @throws TableExistsException if table to be created already exists
3542    * @throws RestoreSnapshotException if snapshot failed to be cloned
3543    * @throws IllegalArgumentException if the specified table has not a valid name
3544    */
3545   @Override
3546   public void cloneSnapshot(final String snapshotName, final TableName tableName)
3547       throws IOException, TableExistsException, RestoreSnapshotException {
3548     if (tableExists(tableName)) {
3549       throw new TableExistsException(tableName);
3550     }
3551     internalRestoreSnapshot(snapshotName, tableName);
3552     waitUntilTableIsEnabled(tableName);
3553   }
3554 
3555   /**
3556    * Execute a distributed procedure on a cluster synchronously with return data
3557    *
3558    * @param signature A distributed procedure is uniquely identified
3559    * by its signature (default the root ZK node name of the procedure).
3560    * @param instance The instance name of the procedure. For some procedures, this parameter is
3561    * optional.
3562    * @param props Property/Value pairs of properties passing to the procedure
3563    * @return data returned after procedure execution. null if no return data.
3564    * @throws IOException
3565    */
3566   @Override
3567   public byte[] execProcedureWithRet(String signature, String instance,
3568       Map<String, String> props) throws IOException {
3569     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3570     builder.setSignature(signature).setInstance(instance);
3571     for (Entry<String, String> entry : props.entrySet()) {
3572       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3573           .setValue(entry.getValue()).build();
3574       builder.addConfiguration(pair);
3575     }
3576 
3577     final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
3578         .setProcedure(builder.build()).build();
3579     // run the procedure on the master
3580     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
3581         getConnection()) {
3582       @Override
3583       public ExecProcedureResponse call(int callTimeout) throws ServiceException {
3584         return master.execProcedureWithRet(null, request);
3585       }
3586     });
3587 
3588     return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
3589   }
3590   /**
3591    * Execute a distributed procedure on a cluster.
3592    *
3593    * @param signature A distributed procedure is uniquely identified
3594    * by its signature (default the root ZK node name of the procedure).
3595    * @param instance The instance name of the procedure. For some procedures, this parameter is
3596    * optional.
3597    * @param props Property/Value pairs of properties passing to the procedure
3598    * @throws IOException
3599    */
3600   @Override
3601   public void execProcedure(String signature, String instance,
3602       Map<String, String> props) throws IOException {
3603     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3604     builder.setSignature(signature).setInstance(instance);
3605     for (Entry<String, String> entry : props.entrySet()) {
3606       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3607           .setValue(entry.getValue()).build();
3608       builder.addConfiguration(pair);
3609     }
3610 
3611     final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
3612         .setProcedure(builder.build()).build();
3613     // run the procedure on the master
3614     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
3615         getConnection()) {
3616       @Override
3617       public ExecProcedureResponse call(int callTimeout) throws ServiceException {
3618         return master.execProcedure(null, request);
3619       }
3620     });
3621 
3622     long start = EnvironmentEdgeManager.currentTime();
3623     long max = response.getExpectedTimeout();
3624     long maxPauseTime = max / this.numRetries;
3625     int tries = 0;
3626     LOG.debug("Waiting a max of " + max + " ms for procedure '" +
3627         signature + " : " + instance + "'' to complete. (max " + maxPauseTime + " ms per retry)");
3628     boolean done = false;
3629     while (tries == 0
3630         || ((EnvironmentEdgeManager.currentTime() - start) < max && !done)) {
3631       try {
3632         // sleep a backoff <= pauseTime amount
3633         long sleep = getPauseTime(tries++);
3634         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3635         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
3636           "ms while waiting for procedure completion.");
3637         Thread.sleep(sleep);
3638       } catch (InterruptedException e) {
3639         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3640       }
3641       LOG.debug("Getting current status of procedure from master...");
3642       done = isProcedureFinished(signature, instance, props);
3643     }
3644     if (!done) {
3645       throw new IOException("Procedure '" + signature + " : " + instance
3646           + "' wasn't completed in expectedTime:" + max + " ms");
3647     }
3648   }
3649 
3650   /**
3651    * Check the current state of the specified procedure.
3652    * <p>
3653    * There are three possible states:
3654    * <ol>
3655    * <li>running - returns <tt>false</tt></li>
3656    * <li>finished - returns <tt>true</tt></li>
3657    * <li>finished with error - throws the exception that caused the procedure to fail</li>
3658    * </ol>
3659    * <p>
3660    *
3661    * @param signature The signature that uniquely identifies a procedure
3662    * @param instance The instance name of the procedure
3663    * @param props Property/Value pairs of properties passing to the procedure
3664    * @return true if the specified procedure is finished successfully, false if it is still running
3665    * @throws IOException if the specified procedure finished with error
3666    */
3667   @Override
3668   public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
3669       throws IOException {
3670     final ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3671     builder.setSignature(signature).setInstance(instance);
3672     for (Entry<String, String> entry : props.entrySet()) {
3673       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3674           .setValue(entry.getValue()).build();
3675       builder.addConfiguration(pair);
3676     }
3677     final ProcedureDescription desc = builder.build();
3678     return executeCallable(
3679         new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
3680           @Override
3681           public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
3682             return master.isProcedureDone(null, IsProcedureDoneRequest
3683                 .newBuilder().setProcedure(desc).build());
3684           }
3685         }).getDone();
3686   }
3687 
3688   /**
3689    * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
3690    * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
3691    * create an HTable instance to this table before it is available.
3692    * @param snapshotName snapshot to restore
3693    * @param tableName table name to restore the snapshot on
3694    * @throws IOException if a remote or network exception occurs
3695    * @throws RestoreSnapshotException if snapshot failed to be restored
3696    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3697    */
3698   private void internalRestoreSnapshot(final String snapshotName, final TableName
3699       tableName)
3700       throws IOException, RestoreSnapshotException {
3701     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
3702         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
3703 
3704     // actually restore the snapshot
3705     internalRestoreSnapshotAsync(snapshot);
3706 
3707     final IsRestoreSnapshotDoneRequest request = IsRestoreSnapshotDoneRequest.newBuilder()
3708         .setSnapshot(snapshot).build();
3709     IsRestoreSnapshotDoneResponse done = IsRestoreSnapshotDoneResponse.newBuilder()
3710         .setDone(false).buildPartial();
3711     final long maxPauseTime = 5000;
3712     int tries = 0;
3713     while (!done.getDone()) {
3714       try {
3715         // sleep a backoff <= pauseTime amount
3716         long sleep = getPauseTime(tries++);
3717         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3718         LOG.debug(tries + ") Sleeping: " + sleep
3719             + " ms while we wait for snapshot restore to complete.");
3720         Thread.sleep(sleep);
3721       } catch (InterruptedException e) {
3722         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3723       }
3724       LOG.debug("Getting current status of snapshot restore from master...");
3725       done = executeCallable(new MasterCallable<IsRestoreSnapshotDoneResponse>(
3726           getConnection()) {
3727         @Override
3728         public IsRestoreSnapshotDoneResponse call(int callTimeout) throws ServiceException {
3729           return master.isRestoreSnapshotDone(null, request);
3730         }
3731       });
3732     }
3733     if (!done.getDone()) {
3734       throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
3735     }
3736   }
3737 
3738   /**
3739    * Execute Restore/Clone snapshot and wait for the server to complete (asynchronous)
3740    * <p>
3741    * Only a single snapshot should be restored at a time, or results may be undefined.
3742    * @param snapshot snapshot to restore
3743    * @return response from the server indicating the max time to wait for the snapshot
3744    * @throws IOException if a remote or network exception occurs
3745    * @throws RestoreSnapshotException if snapshot failed to be restored
3746    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3747    */
3748   private RestoreSnapshotResponse internalRestoreSnapshotAsync(final SnapshotDescription snapshot)
3749       throws IOException, RestoreSnapshotException {
3750     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
3751 
3752     final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
3753         .build();
3754 
3755     // run the snapshot restore on the master
3756     return executeCallable(new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
3757       @Override
3758       public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
3759         return master.restoreSnapshot(null, request);
3760       }
3761     });
3762   }
3763 
3764   /**
3765    * List completed snapshots.
3766    * @return a list of snapshot descriptors for completed snapshots
3767    * @throws IOException if a network error occurs
3768    */
3769   @Override
3770   public List<SnapshotDescription> listSnapshots() throws IOException {
3771     return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
3772       @Override
3773       public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
3774         return master.getCompletedSnapshots(null, GetCompletedSnapshotsRequest.newBuilder().build())
3775             .getSnapshotsList();
3776       }
3777     });
3778   }
3779 
3780   /**
3781    * List all the completed snapshots matching the given regular expression.
3782    *
3783    * @param regex The regular expression to match against
3784    * @return - returns a List of SnapshotDescription
3785    * @throws IOException if a remote or network exception occurs
3786    */
3787   @Override
3788   public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
3789     return listSnapshots(Pattern.compile(regex));
3790   }
3791 
3792   /**
3793    * List all the completed snapshots matching the given pattern.
3794    *
3795    * @param pattern The compiled regular expression to match against
3796    * @return - returns a List of SnapshotDescription
3797    * @throws IOException if a remote or network exception occurs
3798    */
3799   @Override
3800   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
3801     List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>();
3802     List<SnapshotDescription> snapshots = listSnapshots();
3803     for (SnapshotDescription snapshot : snapshots) {
3804       if (pattern.matcher(snapshot.getName()).matches()) {
3805         matched.add(snapshot);
3806       }
3807     }
3808     return matched;
3809   }
3810 
3811   /**
3812    * List all the completed snapshots matching the given table name regular expression and snapshot
3813    * name regular expression.
3814    * @param tableNameRegex The table name regular expression to match against
3815    * @param snapshotNameRegex The snapshot name regular expression to match against
3816    * @return returns a List of completed SnapshotDescription
3817    * @throws IOException if a remote or network exception occurs
3818    */
3819   @Override
3820   public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
3821       String snapshotNameRegex) throws IOException {
3822     return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
3823   }
3824 
3825   /**
3826    * List all the completed snapshots matching the given table name regular expression and snapshot
3827    * name regular expression.
3828    * @param tableNamePattern The compiled table name regular expression to match against
3829    * @param snapshotNamePattern The compiled snapshot name regular expression to match against
3830    * @return returns a List of completed SnapshotDescription
3831    * @throws IOException if a remote or network exception occurs
3832    */
3833   @Override
3834   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
3835       Pattern snapshotNamePattern) throws IOException {
3836     TableName[] tableNames = listTableNames(tableNamePattern);
3837 
3838     List<SnapshotDescription> tableSnapshots = new LinkedList<SnapshotDescription>();
3839     List<SnapshotDescription> snapshots = listSnapshots(snapshotNamePattern);
3840 
3841     List<TableName> listOfTableNames = Arrays.asList(tableNames);
3842     for (SnapshotDescription snapshot : snapshots) {
3843       if (listOfTableNames.contains(TableName.valueOf(snapshot.getTable()))) {
3844         tableSnapshots.add(snapshot);
3845       }
3846     }
3847     return tableSnapshots;
3848   }
3849 
3850   /**
3851    * Delete an existing snapshot.
3852    * @param snapshotName name of the snapshot
3853    * @throws IOException if a remote or network exception occurs
3854    */
3855   @Override
3856   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
3857     deleteSnapshot(Bytes.toString(snapshotName));
3858   }
3859 
3860   /**
3861    * Delete an existing snapshot.
3862    * @param snapshotName name of the snapshot
3863    * @throws IOException if a remote or network exception occurs
3864    */
3865   @Override
3866   public void deleteSnapshot(final String snapshotName) throws IOException {
3867     // make sure the snapshot is possibly valid
3868     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
3869     // do the delete
3870     executeCallable(new MasterCallable<Void>(getConnection()) {
3871       @Override
3872       public Void call(int callTimeout) throws ServiceException {
3873         master.deleteSnapshot(null,
3874           DeleteSnapshotRequest.newBuilder().
3875             setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build()
3876         );
3877         return null;
3878       }
3879     });
3880   }
3881 
3882   /**
3883    * Delete existing snapshots whose names match the pattern passed.
3884    * @param regex The regular expression to match against
3885    * @throws IOException if a remote or network exception occurs
3886    */
3887   @Override
3888   public void deleteSnapshots(final String regex) throws IOException {
3889     deleteSnapshots(Pattern.compile(regex));
3890   }
3891 
3892   /**
3893    * Delete existing snapshots whose names match the pattern passed.
3894    * @param pattern pattern for names of the snapshot to match
3895    * @throws IOException if a remote or network exception occurs
3896    */
3897   @Override
3898   public void deleteSnapshots(final Pattern pattern) throws IOException {
3899     List<SnapshotDescription> snapshots = listSnapshots(pattern);
3900     for (final SnapshotDescription snapshot : snapshots) {
3901       try {
3902         internalDeleteSnapshot(snapshot);
3903       } catch (IOException ex) {
3904         LOG.info(
3905           "Failed to delete snapshot " + snapshot.getName() + " for table " + snapshot.getTable(),
3906           ex);
3907       }
3908     }
3909   }
3910 
3911   private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
3912     executeCallable(new MasterCallable<Void>(getConnection()) {
3913       @Override
3914       public Void call(int callTimeout) throws ServiceException {
3915         this.master.deleteSnapshot(null, DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot)
3916             .build());
3917         return null;
3918       }
3919     });
3920   }
3921 
3922   /**
3923    * Delete all existing snapshots matching the given table name regular expression and snapshot
3924    * name regular expression.
3925    * @param tableNameRegex The table name regular expression to match against
3926    * @param snapshotNameRegex The snapshot name regular expression to match against
3927    * @throws IOException if a remote or network exception occurs
3928    */
3929   @Override
3930   public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
3931       throws IOException {
3932     deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
3933   }
3934 
3935   /**
3936    * Delete all existing snapshots matching the given table name regular expression and snapshot
3937    * name regular expression.
3938    * @param tableNamePattern The compiled table name regular expression to match against
3939    * @param snapshotNamePattern The compiled snapshot name regular expression to match against
3940    * @throws IOException if a remote or network exception occurs
3941    */
3942   @Override
3943   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
3944       throws IOException {
3945     List<SnapshotDescription> snapshots = listTableSnapshots(tableNamePattern, snapshotNamePattern);
3946     for (SnapshotDescription snapshot : snapshots) {
3947       try {
3948         internalDeleteSnapshot(snapshot);
3949         LOG.debug("Successfully deleted snapshot: " + snapshot.getName());
3950       } catch (IOException e) {
3951         LOG.error("Failed to delete snapshot: " + snapshot.getName(), e);
3952       }
3953     }
3954   }
3955 
3956   /**
3957    * Apply the new quota settings.
3958    *
3959    * @param quota the quota settings
3960    * @throws IOException if a remote or network exception occurs
3961    */
3962   @Override
3963   public void setQuota(final QuotaSettings quota) throws IOException {
3964     executeCallable(new MasterCallable<Void>(getConnection()) {
3965       @Override
3966       public Void call(int callTimeout) throws ServiceException {
3967         this.master.setQuota(null, QuotaSettings.buildSetQuotaRequestProto(quota));
3968         return null;
3969       }
3970     });
3971   }
3972 
3973   /**
3974    * Return a Quota Scanner to list the quotas based on the filter.
3975    *
3976    * @param filter the quota settings filter
3977    * @return the quota scanner
3978    * @throws IOException if a remote or network exception occurs
3979    */
3980   @Override
3981   public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
3982     return QuotaRetriever.open(conf, filter);
3983   }
3984 
3985   private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable)
3986       throws IOException {
3987     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
3988     try {
3989       return caller.callWithRetries(callable, operationTimeout);
3990     } finally {
3991       callable.close();
3992     }
3993   }
3994 
3995   /**
3996    * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
3997    * connected to the active master.
3998    *
3999    * <p>
4000    * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
4001    * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
4002    * </p>
4003    *
4004    * <div style="background-color: #cccccc; padding: 2px">
4005    * <blockquote><pre>
4006    * CoprocessorRpcChannel channel = myAdmin.coprocessorService();
4007    * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
4008    * MyCallRequest request = MyCallRequest.newBuilder()
4009    *     ...
4010    *     .build();
4011    * MyCallResponse response = service.myCall(null, request);
4012    * </pre></blockquote></div>
4013    *
4014    * @return A MasterCoprocessorRpcChannel instance
4015    */
4016   @Override
4017   public CoprocessorRpcChannel coprocessorService() {
4018     return new MasterCoprocessorRpcChannel(connection);
4019   }
4020 
4021   /**
4022    * Simple {@link Abortable}, throwing RuntimeException on abort.
4023    */
4024   private static class ThrowableAbortable implements Abortable {
4025 
4026     @Override
4027     public void abort(String why, Throwable e) {
4028       throw new RuntimeException(why, e);
4029     }
4030 
4031     @Override
4032     public boolean isAborted() {
4033       return true;
4034     }
4035   }
4036 
4037   /**
4038    * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
4039    * connected to the passed region server.
4040    *
4041    * <p>
4042    * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
4043    * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
4044    * </p>
4045    *
4046    * <div style="background-color: #cccccc; padding: 2px">
4047    * <blockquote><pre>
4048    * CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName);
4049    * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
4050    * MyCallRequest request = MyCallRequest.newBuilder()
4051    *     ...
4052    *     .build();
4053    * MyCallResponse response = service.myCall(null, request);
4054    * </pre></blockquote></div>
4055    *
4056    * @param sn the server name to which the endpoint call is made
4057    * @return A RegionServerCoprocessorRpcChannel instance
4058    */
4059   @Override
4060   public CoprocessorRpcChannel coprocessorService(ServerName sn) {
4061     return new RegionServerCoprocessorRpcChannel(connection, sn);
4062   }
4063 
4064   @Override
4065   public void updateConfiguration(ServerName server) throws IOException {
4066     try {
4067       this.connection.getAdmin(server).updateConfiguration(null,
4068         UpdateConfigurationRequest.getDefaultInstance());
4069     } catch (ServiceException e) {
4070       throw ProtobufUtil.getRemoteException(e);
4071     }
4072   }
4073 
4074   @Override
4075   public void updateConfiguration() throws IOException {
4076     for (ServerName server : this.getClusterStatus().getServers()) {
4077       updateConfiguration(server);
4078     }
4079   }
4080 
4081   @Override
4082   public int getMasterInfoPort() throws IOException {
4083     // TODO: Fix!  Reaching into internal implementation!!!!
4084     ConnectionImplementation connection =
4085         (ConnectionImplementation)this.connection;
4086     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
4087     try {
4088       return MasterAddressTracker.getMasterInfoPort(zkw);
4089     } catch (KeeperException e) {
4090       throw new IOException("Failed to get master info port from MasterAddressTracker", e);
4091     }
4092   }
4093 
4094   @Override
4095   public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
4096     return executeCallable(new MasterCallable<Long>(getConnection()) {
4097       @Override
4098       public Long call(int callTimeout) throws ServiceException {
4099         MajorCompactionTimestampRequest req =
4100             MajorCompactionTimestampRequest.newBuilder()
4101                 .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
4102         return master.getLastMajorCompactionTimestamp(null, req).getCompactionTimestamp();
4103       }
4104     });
4105   }
4106 
4107   @Override
4108   public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException {
4109     return executeCallable(new MasterCallable<Long>(getConnection()) {
4110       @Override
4111       public Long call(int callTimeout) throws ServiceException {
4112         MajorCompactionTimestampForRegionRequest req =
4113             MajorCompactionTimestampForRegionRequest
4114                 .newBuilder()
4115                 .setRegion(
4116                   RequestConverter
4117                       .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
4118         return master.getLastMajorCompactionTimestampForRegion(null, req).getCompactionTimestamp();
4119       }
4120     });
4121   }
4122 
4123   /**
4124    * {@inheritDoc}
4125    */
4126   @Override
4127   public void compactMob(final TableName tableName, final byte[] columnFamily)
4128     throws IOException, InterruptedException {
4129     checkTableNameNotNull(tableName);
4130     checkFamilyNameNotNull(columnFamily);
4131     validateMobColumnFamily(tableName, columnFamily);
4132     compactMob(tableName, columnFamily, false);
4133   }
4134 
4135   /**
4136    * {@inheritDoc}
4137    */
4138   @Override
4139   public void compactMobs(final TableName tableName) throws IOException, InterruptedException {
4140     checkTableNameNotNull(tableName);
4141     compactMob(tableName, null, false);
4142   }
4143 
4144   /**
4145    * {@inheritDoc}
4146    */
4147   @Override
4148   public void majorCompactMob(final TableName tableName, final byte[] columnFamily)
4149     throws IOException, InterruptedException {
4150     checkTableNameNotNull(tableName);
4151     checkFamilyNameNotNull(columnFamily);
4152     validateMobColumnFamily(tableName, columnFamily);
4153     compactMob(tableName, columnFamily, true);
4154   }
4155 
4156   /**
4157    * {@inheritDoc}
4158    */
4159   @Override
4160   public void majorCompactMobs(final TableName tableName) throws IOException, InterruptedException {
4161     checkTableNameNotNull(tableName);
4162     compactMob(tableName, null, true);
4163   }
4164 
4165   /**
4166    * {@inheritDoc}
4167    */
4168   @Override
4169   public CompactionState getMobCompactionState(TableName tableName) throws IOException {
4170     checkTableNameNotNull(tableName);
4171     try {
4172       ServerName master = getClusterStatus().getMaster();
4173       HRegionInfo info = new HRegionInfo(tableName, Bytes.toBytes(".mob"),
4174         HConstants.EMPTY_END_ROW, false, 0);
4175       GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
4176         info.getRegionName(), true);
4177       GetRegionInfoResponse response = this.connection.getAdmin(master)
4178         .getRegionInfo(null, request);
4179       return response.getCompactionState();
4180     } catch (ServiceException se) {
4181       throw ProtobufUtil.getRemoteException(se);
4182     }
4183   }
4184 
4185   /**
4186    * Compacts the mob files in a mob-enabled column family. Asynchronous operation.
4187    * @param tableName The table to compact.
4188    * @param columnFamily The column family to compact. If it is null, all the mob-enabled
4189    *        column families in this table will be compacted.
4190    * @param major Whether to select all the mob files in the compaction.
4191    * @throws IOException
4192    * @throws InterruptedException
4193    */
4194   private void compactMob(final TableName tableName, final byte[] columnFamily, boolean major)
4195     throws IOException, InterruptedException {
4196     // get the mob region info, this is a dummy region.
4197     HRegionInfo info = new HRegionInfo(tableName, Bytes.toBytes(".mob"), HConstants.EMPTY_END_ROW,
4198       false, 0);
4199     ServerName master = getClusterStatus().getMaster();
4200     compact(master, info, major, columnFamily);
4201   }
4202 
4203   private void checkTableNameNotNull(TableName tableName) {
4204     if (tableName == null) {
4205       throw new IllegalArgumentException("TableName cannot be null");
4206     }
4207   }
4208 
4209   private void checkFamilyNameNotNull(byte[] columnFamily) {
4210     if (columnFamily == null) {
4211       throw new IllegalArgumentException("The column family name cannot be null");
4212     }
4213   }
4214 
4215   private void validateMobColumnFamily(TableName tableName, byte[] columnFamily)
4216     throws IOException {
4217     HTableDescriptor htd = getTableDescriptor(tableName);
4218     HColumnDescriptor family = htd.getFamily(columnFamily);
4219     if (family == null || !family.isMobEnabled()) {
4220       throw new IllegalArgumentException("Column family " + columnFamily
4221               + " is not a mob column family");
4222     }
4223   }
4224 
4225   /**
4226    * Future that waits on a procedure result.
4227    * Returned by the async version of the Admin calls,
4228    * and used internally by the sync calls to wait on the result of the procedure.
4229    */
4230   @InterfaceAudience.Private
4231   @InterfaceStability.Evolving
4232   protected static class ProcedureFuture<V> implements Future<V> {
4233     private ExecutionException exception = null;
4234     private boolean procResultFound = false;
4235     private boolean done = false;
4236     private V result = null;
4237 
4238     private final HBaseAdmin admin;
4239     private final Long procId;
4240 
4241     public ProcedureFuture(final HBaseAdmin admin, final Long procId) {
4242       this.admin = admin;
4243       this.procId = procId;
4244     }
4245 
4246     @Override
4247     public boolean cancel(boolean mayInterruptIfRunning) {
4248       throw new UnsupportedOperationException();
4249     }
4250 
4251     @Override
4252     public boolean isCancelled() {
4253       // TODO: Abort not implemented yet
4254       return false;
4255     }
4256 
4257     @Override
4258     public V get() throws InterruptedException, ExecutionException {
4259       // TODO: should we ever spin forever?
4260       throw new UnsupportedOperationException();
4261     }
4262 
4263     @Override
4264     public V get(long timeout, TimeUnit unit)
4265         throws InterruptedException, ExecutionException, TimeoutException {
4266       if (!done) {
4267         long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
4268         try {
4269           try {
4270             // if the master support procedures, try to wait the result
4271             if (procId != null) {
4272               result = waitProcedureResult(procId, deadlineTs);
4273             }
4274             // if we don't have a proc result, try the compatibility wait
4275             if (!procResultFound) {
4276               result = waitOperationResult(deadlineTs);
4277             }
4278             result = postOperationResult(result, deadlineTs);
4279             done = true;
4280           } catch (IOException e) {
4281             result = postOperationFailure(e, deadlineTs);
4282             done = true;
4283           }
4284         } catch (IOException e) {
4285           exception = new ExecutionException(e);
4286           done = true;
4287         }
4288       }
4289       if (exception != null) {
4290         throw exception;
4291       }
4292       return result;
4293     }
4294 
4295     @Override
4296     public boolean isDone() {
4297       return done;
4298     }
4299 
4300     protected HBaseAdmin getAdmin() {
4301       return admin;
4302     }
4303 
4304     private V waitProcedureResult(long procId, long deadlineTs)
4305         throws IOException, TimeoutException, InterruptedException {
4306       GetProcedureResultRequest request = GetProcedureResultRequest.newBuilder()
4307           .setProcId(procId)
4308           .build();
4309 
4310       int tries = 0;
4311       IOException serviceEx = null;
4312       while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
4313         GetProcedureResultResponse response = null;
4314         try {
4315           // Try to fetch the result
4316           response = getProcedureResult(request);
4317         } catch (IOException e) {
4318           serviceEx = unwrapException(e);
4319 
4320           // the master may be down
4321           LOG.warn("failed to get the procedure result procId=" + procId, serviceEx);
4322 
4323           // Not much to do, if we have a DoNotRetryIOException
4324           if (serviceEx instanceof DoNotRetryIOException) {
4325             // TODO: looks like there is no way to unwrap this exception and get the proper
4326             // UnsupportedOperationException aside from looking at the message.
4327             // anyway, if we fail here we just failover to the compatibility side
4328             // and that is always a valid solution.
4329             LOG.warn("Proc-v2 is unsupported on this master: " + serviceEx.getMessage(), serviceEx);
4330             procResultFound = false;
4331             return null;
4332           }
4333         }
4334 
4335         // If the procedure is no longer running, we should have a result
4336         if (response != null && response.getState() != GetProcedureResultResponse.State.RUNNING) {
4337           procResultFound = response.getState() != GetProcedureResultResponse.State.NOT_FOUND;
4338           return convertResult(response);
4339         }
4340 
4341         try {
4342           Thread.sleep(getAdmin().getPauseTime(tries++));
4343         } catch (InterruptedException e) {
4344           throw new InterruptedException(
4345             "Interrupted while waiting for the result of proc " + procId);
4346         }
4347       }
4348       if (serviceEx != null) {
4349         throw serviceEx;
4350       } else {
4351         throw new TimeoutException("The procedure " + procId + " is still running");
4352       }
4353     }
4354 
4355     private static IOException unwrapException(IOException e) {
4356       if (e instanceof RemoteException) {
4357         return ((RemoteException)e).unwrapRemoteException();
4358       }
4359       return e;
4360     }
4361 
4362     protected GetProcedureResultResponse getProcedureResult(final GetProcedureResultRequest request)
4363         throws IOException {
4364       return admin.executeCallable(new MasterCallable<GetProcedureResultResponse>(
4365           admin.getConnection()) {
4366         @Override
4367         public GetProcedureResultResponse call(int callTimeout) throws ServiceException {
4368           return master.getProcedureResult(null, request);
4369         }
4370       });
4371     }
4372 
4373     /**
4374      * Convert the procedure result response to a specified type.
4375      * @param response the procedure result object to parse
4376      * @return the result data of the procedure.
4377      */
4378     protected V convertResult(final GetProcedureResultResponse response) throws IOException {
4379       if (response.hasException()) {
4380         throw ForeignExceptionUtil.toIOException(response.getException());
4381       }
4382       return null;
4383     }
4384 
4385     /**
4386      * Fallback implementation in case the procedure is not supported by the server.
4387      * It should try to wait until the operation is completed.
4388      * @param deadlineTs the timestamp after which this method should throw a TimeoutException
4389      * @return the result data of the operation
4390      */
4391     protected V waitOperationResult(final long deadlineTs)
4392         throws IOException, TimeoutException {
4393       return null;
4394     }
4395 
4396     /**
4397      * Called after the operation is completed and the result fetched. this allows to perform extra
4398      * steps after the procedure is completed. it allows to apply transformations to the result that
4399      * will be returned by get().
4400      * @param result the result of the procedure
4401      * @param deadlineTs the timestamp after which this method should throw a TimeoutException
4402      * @return the result of the procedure, which may be the same as the passed one
4403      */
4404     protected V postOperationResult(final V result, final long deadlineTs)
4405         throws IOException, TimeoutException {
4406       return result;
4407     }
4408 
4409     /**
4410      * Called after the operation is terminated with a failure.
4411      * this allows to perform extra steps after the procedure is terminated.
4412      * it allows to apply transformations to the result that will be returned by get().
4413      * The default implementation will rethrow the exception
4414      * @param exception the exception got from fetching the result
4415      * @param deadlineTs the timestamp after which this method should throw a TimeoutException
4416      * @return the result of the procedure, which may be the same as the passed one
4417      */
4418     protected V postOperationFailure(final IOException exception, final long deadlineTs)
4419         throws IOException, TimeoutException {
4420       throw exception;
4421     }
4422 
4423     protected interface WaitForStateCallable {
4424       boolean checkState(int tries) throws IOException;
4425       void throwInterruptedException() throws InterruptedIOException;
4426       void throwTimeoutException(long elapsed) throws TimeoutException;
4427     }
4428 
4429     protected void waitForState(final long deadlineTs, final WaitForStateCallable callable)
4430         throws IOException, TimeoutException {
4431       int tries = 0;
4432       IOException serverEx = null;
4433       long startTime = EnvironmentEdgeManager.currentTime();
4434       while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
4435         serverEx = null;
4436         try {
4437           if (callable.checkState(tries)) {
4438             return;
4439           }
4440         } catch (IOException e) {
4441           serverEx = e;
4442         }
4443         try {
4444           Thread.sleep(getAdmin().getPauseTime(tries++));
4445         } catch (InterruptedException e) {
4446           callable.throwInterruptedException();
4447         }
4448       }
4449       if (serverEx != null) {
4450         throw unwrapException(serverEx);
4451       } else {
4452         callable.throwTimeoutException(EnvironmentEdgeManager.currentTime() - startTime);
4453       }
4454     }
4455   }
4456 
4457   @InterfaceAudience.Private
4458   @InterfaceStability.Evolving
4459   protected static abstract class TableFuture<V> extends ProcedureFuture<V> {
4460     private final TableName tableName;
4461 
4462     public TableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) {
4463       super(admin, procId);
4464       this.tableName = tableName;
4465     }
4466 
4467     /**
4468      * @return the table name
4469      */
4470     protected TableName getTableName() {
4471       return tableName;
4472     }
4473 
4474     /**
4475      * @return the table descriptor
4476      */
4477     protected HTableDescriptor getTableDescriptor() throws IOException {
4478       return getAdmin().getTableDescriptorByTableName(getTableName());
4479     }
4480 
4481     /**
4482      * @return the operation type like CREATE, DELETE, DISABLE etc.
4483      */
4484     public abstract String getOperationType();
4485 
4486     /**
4487      * @return a description of the operation
4488      */
4489     protected String getDescription() {
4490       return "Operation: " + getOperationType() + ", "
4491           + "Table Name: " + tableName.getNameWithNamespaceInclAsString();
4492 
4493     };
4494 
4495     protected abstract class TableWaitForStateCallable implements WaitForStateCallable {
4496       @Override
4497       public void throwInterruptedException() throws InterruptedIOException {
4498         throw new InterruptedIOException("Interrupted while waiting for operation: "
4499             + getOperationType() + " on table: " + tableName.getNameWithNamespaceInclAsString());
4500       }
4501 
4502       @Override
4503       public void throwTimeoutException(long elapsedTime) throws TimeoutException {
4504         throw new TimeoutException("The operation: " + getOperationType() + " on table: " +
4505             tableName.getNameAsString() + " not completed after " + elapsedTime + "msec");
4506       }
4507     }
4508 
4509     @Override
4510     protected V postOperationResult(final V result, final long deadlineTs)
4511         throws IOException, TimeoutException {
4512       LOG.info(getDescription() + " completed");
4513       return super.postOperationResult(result, deadlineTs);
4514     }
4515 
4516     @Override
4517     protected V postOperationFailure(final IOException exception, final long deadlineTs)
4518         throws IOException, TimeoutException {
4519       LOG.info(getDescription() + " failed with " + exception.getMessage());
4520       return super.postOperationFailure(exception, deadlineTs);
4521     }
4522 
4523     protected void waitForTableEnabled(final long deadlineTs)
4524         throws IOException, TimeoutException {
4525       waitForState(deadlineTs, new TableWaitForStateCallable() {
4526         @Override
4527         public boolean checkState(int tries) throws IOException {
4528           try {
4529             if (getAdmin().isTableAvailable(tableName)) {
4530               return true;
4531             }
4532           } catch (TableNotFoundException tnfe) {
4533             LOG.debug("Table " + tableName.getNameWithNamespaceInclAsString()
4534                 + " was not enabled, sleeping. tries=" + tries);
4535           }
4536           return false;
4537         }
4538       });
4539     }
4540 
4541     protected void waitForTableDisabled(final long deadlineTs)
4542         throws IOException, TimeoutException {
4543       waitForState(deadlineTs, new TableWaitForStateCallable() {
4544         @Override
4545         public boolean checkState(int tries) throws IOException {
4546           return getAdmin().isTableDisabled(tableName);
4547         }
4548       });
4549     }
4550 
4551     protected void waitTableNotFound(final long deadlineTs)
4552         throws IOException, TimeoutException {
4553       waitForState(deadlineTs, new TableWaitForStateCallable() {
4554         @Override
4555         public boolean checkState(int tries) throws IOException {
4556           return !getAdmin().tableExists(tableName);
4557         }
4558       });
4559     }
4560 
4561     protected void waitForSchemaUpdate(final long deadlineTs)
4562         throws IOException, TimeoutException {
4563       waitForState(deadlineTs, new TableWaitForStateCallable() {
4564         @Override
4565         public boolean checkState(int tries) throws IOException {
4566           return getAdmin().getAlterStatus(tableName).getFirst() == 0;
4567         }
4568       });
4569     }
4570 
4571     protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
4572         throws IOException, TimeoutException {
4573       final HTableDescriptor desc = getTableDescriptor();
4574       final AtomicInteger actualRegCount = new AtomicInteger(0);
4575       final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
4576         @Override
4577         public boolean visit(Result rowResult) throws IOException {
4578           RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
4579           if (list == null) {
4580             LOG.warn("No serialized HRegionInfo in " + rowResult);
4581             return true;
4582           }
4583           HRegionLocation l = list.getRegionLocation();
4584           if (l == null) {
4585             return true;
4586           }
4587           if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
4588             return false;
4589           }
4590           if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
4591           HRegionLocation[] locations = list.getRegionLocations();
4592           for (HRegionLocation location : locations) {
4593             if (location == null) continue;
4594             ServerName serverName = location.getServerName();
4595             // Make sure that regions are assigned to server
4596             if (serverName != null && serverName.getHostAndPort() != null) {
4597               actualRegCount.incrementAndGet();
4598             }
4599           }
4600           return true;
4601         }
4602       };
4603 
4604       int tries = 0;
4605       int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
4606       while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
4607         actualRegCount.set(0);
4608         MetaTableAccessor.scanMetaForTableRegions(getAdmin().getConnection(), visitor,
4609           desc.getTableName());
4610         if (actualRegCount.get() == numRegs) {
4611           // all the regions are online
4612           return;
4613         }
4614 
4615         try {
4616           Thread.sleep(getAdmin().getPauseTime(tries++));
4617         } catch (InterruptedException e) {
4618           throw new InterruptedIOException("Interrupted when opening" + " regions; "
4619               + actualRegCount.get() + " of " + numRegs + " regions processed so far");
4620         }
4621       }
4622       throw new TimeoutException("Only " + actualRegCount.get() + " of " + numRegs
4623           + " regions are online; retries exhausted.");
4624     }
4625   }
4626 }