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