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