View Javadoc

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