View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.client;
20  
21  import java.io.Closeable;
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.HashMap;
27  import java.util.LinkedList;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.Map.Entry;
31  import java.util.concurrent.Callable;
32  import java.util.concurrent.ExecutionException;
33  import java.util.concurrent.Future;
34  import java.util.concurrent.TimeUnit;
35  import java.util.concurrent.TimeoutException;
36  import java.util.concurrent.atomic.AtomicInteger;
37  import java.util.concurrent.atomic.AtomicReference;
38  import java.util.regex.Pattern;
39
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.Abortable;
44  import org.apache.hadoop.hbase.ClusterStatus;
45  import org.apache.hadoop.hbase.DoNotRetryIOException;
46  import org.apache.hadoop.hbase.HBaseConfiguration;
47  import org.apache.hadoop.hbase.HColumnDescriptor;
48  import org.apache.hadoop.hbase.HConstants;
49  import org.apache.hadoop.hbase.HRegionInfo;
50  import org.apache.hadoop.hbase.HRegionLocation;
51  import org.apache.hadoop.hbase.HTableDescriptor;
52  import org.apache.hadoop.hbase.MasterNotRunningException;
53  import org.apache.hadoop.hbase.MetaTableAccessor;
54  import org.apache.hadoop.hbase.NamespaceDescriptor;
55  import org.apache.hadoop.hbase.NotServingRegionException;
56  import org.apache.hadoop.hbase.ProcedureInfo;
57  import org.apache.hadoop.hbase.ProcedureUtil;
58  import org.apache.hadoop.hbase.RegionLocations;
59  import org.apache.hadoop.hbase.ServerName;
60  import org.apache.hadoop.hbase.TableExistsException;
61  import org.apache.hadoop.hbase.TableName;
62  import org.apache.hadoop.hbase.TableNotDisabledException;
63  import org.apache.hadoop.hbase.TableNotFoundException;
64  import org.apache.hadoop.hbase.UnknownRegionException;
65  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
66  import org.apache.hadoop.hbase.classification.InterfaceAudience;
67  import org.apache.hadoop.hbase.classification.InterfaceStability;
68  import org.apache.hadoop.hbase.client.security.SecurityCapability;
69  import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
70  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
71  import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
72  import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
73  import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
74  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
75  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
76  import org.apache.hadoop.hbase.protobuf.RequestConverter;
77  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
78  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
79  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
80  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
81  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
82  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
83  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
84  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
85  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
86  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
87  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
88  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
89  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
90  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
91  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
92  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
93  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
94  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
95  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
96  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureResponse;
97  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
98  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
99  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
100 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
101 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
102 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
103 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
104 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
105 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
106 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
107 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
108 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
109 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
110 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
111 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
112 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
113 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
114 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
115 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
116 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
117 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
118 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
119 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
121 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
122 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
123 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
124 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
125 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
126 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
127 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
128 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
129 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
130 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
131 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
132 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
133 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
134 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
135 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
136 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresRequest;
137 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
138 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
139 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
140 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
141 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
142 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
143 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
144 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
145 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
146 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
147 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
148 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
149 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
150 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
151 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
152 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
153 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
154 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
155 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
156 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
157 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
158 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
159 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
160 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
161 import org.apache.hadoop.hbase.quotas.QuotaFilter;
162 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
163 import org.apache.hadoop.hbase.quotas.QuotaSettings;
164 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
165 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
166 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
167 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
168 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
169 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
170 import org.apache.hadoop.hbase.util.Addressing;
171 import org.apache.hadoop.hbase.util.Bytes;
172 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
173 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
174 import org.apache.hadoop.hbase.util.Pair;
175 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
176 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
177 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
178 import org.apache.hadoop.ipc.RemoteException;
179 import org.apache.hadoop.util.StringUtils;
180 import org.apache.zookeeper.KeeperException;
181
182 import com.google.common.annotations.VisibleForTesting;
183 import com.google.protobuf.ServiceException;
184
185 /**
186  * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
187  * this is an HBase-internal class as defined in
188  * https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/InterfaceClassification.html
189  * There are no guarantees for backwards source / binary compatibility and methods or class can
190  * change or go away without deprecation.
191  * Use {@link Connection#getAdmin()} to obtain an instance of {@link Admin} instead of constructing
192  * an HBaseAdmin directly.
193  *
194  * <p>Connection should be an <i>unmanaged</i> connection obtained via
195  * {@link ConnectionFactory#createConnection(Configuration)}
196  *
197  * @see ConnectionFactory
198  * @see Connection
199  * @see Admin
200  */
201 @InterfaceAudience.Private
202 @InterfaceStability.Evolving
203 public class HBaseAdmin implements Admin {
204   private static final Log LOG = LogFactory.getLog(HBaseAdmin.class);
205
206   private static final String ZK_IDENTIFIER_PREFIX =  "hbase-admin-on-";
207
208   private ClusterConnection connection;
209
210   private volatile Configuration conf;
211   private final long pause;
212   private final int numRetries;
213   private final int syncWaitTimeout;
214   private boolean aborted;
215   private int operationTimeout;
216   private int rpcTimeout;
217
218   private RpcRetryingCallerFactory rpcCallerFactory;
219   private RpcControllerFactory rpcControllerFactory;
220
221   private NonceGenerator ng;
222
223   @Override
224   public int getOperationTimeout() {
225     return operationTimeout;
226   }
227
228   HBaseAdmin(ClusterConnection connection) throws IOException {
229     this.conf = connection.getConfiguration();
230     this.connection = connection;
231
232     // TODO: receive ConnectionConfiguration here rather than re-parsing these configs every time.
233     this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
234         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
235     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
236         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
237     this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
238         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
239     this.rpcTimeout = this.conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
240         HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
241     this.syncWaitTimeout = this.conf.getInt(
242       "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
243
244     this.rpcCallerFactory = connection.getRpcRetryingCallerFactory();
245     this.rpcControllerFactory = connection.getRpcControllerFactory();
246
247     this.ng = this.connection.getNonceGenerator();
248   }
249
250   @Override
251   public void abort(String why, Throwable e) {
252     // Currently does nothing but throw the passed message and exception
253     this.aborted = true;
254     throw new RuntimeException(why, e);
255   }
256
257   @Override
258   public boolean isAborted() {
259     return this.aborted;
260   }
261
262   @Override
263   public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
264   throws IOException {
265     return get(abortProcedureAsync(procId, mayInterruptIfRunning), this.syncWaitTimeout,
266       TimeUnit.MILLISECONDS);
267   }
268
269   @Override
270   public Future<Boolean> abortProcedureAsync(final long procId, final boolean mayInterruptIfRunning)
271       throws IOException {
272     Boolean abortProcResponse =
273         executeCallable(new MasterCallable<AbortProcedureResponse>(getConnection(),
274             getRpcControllerFactory()) {
275       @Override
276       protected AbortProcedureResponse rpcCall() throws Exception {
277         AbortProcedureRequest abortProcRequest =
278             AbortProcedureRequest.newBuilder().setProcId(procId).build();
279         return master.abortProcedure(getRpcController(), abortProcRequest);
280       }
281     }).getIsProcedureAborted();
282     return new AbortProcedureFuture(this, procId, abortProcResponse);
283   }
284
285   private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
286     private boolean isAbortInProgress;
287
288     public AbortProcedureFuture(
289         final HBaseAdmin admin,
290         final Long procId,
291         final Boolean abortProcResponse) {
292       super(admin, procId);
293       this.isAbortInProgress = abortProcResponse;
294     }
295
296     @Override
297     public Boolean get(long timeout, TimeUnit unit)
298         throws InterruptedException, ExecutionException, TimeoutException {
299       if (!this.isAbortInProgress) {
300         return false;
301       }
302       super.get(timeout, unit);
303       return true;
304     }
305   }
306
307   /** @return Connection used by this object. */
308   @Override
309   public Connection getConnection() {
310     return connection;
311   }
312
313   @Override
314   public boolean tableExists(final TableName tableName) throws IOException {
315     return executeCallable(new RpcRetryingCallable<Boolean>() {
316       @Override
317       protected Boolean rpcCall(int callTimeout) throws Exception {
318         return MetaTableAccessor.tableExists(connection, tableName);
319       }
320     });
321   }
322
323   @Override
324   public HTableDescriptor[] listTables() throws IOException {
325     return listTables((Pattern)null, false);
326   }
327
328   @Override
329   public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
330     return listTables(pattern, false);
331   }
332
333   @Override
334   public HTableDescriptor[] listTables(String regex) throws IOException {
335     return listTables(Pattern.compile(regex), false);
336   }
337
338   @Override
339   public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
340       throws IOException {
341     return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
342         getRpcControllerFactory()) {
343       @Override
344       protected HTableDescriptor[] rpcCall() throws Exception {
345         GetTableDescriptorsRequest req =
346             RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
347         return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(getRpcController(),
348             req));
349       }
350     });
351   }
352
353   @Override
354   public HTableDescriptor[] listTables(String regex, boolean includeSysTables)
355       throws IOException {
356     return listTables(Pattern.compile(regex), includeSysTables);
357   }
358
359   @Override
360   public TableName[] listTableNames() throws IOException {
361     return listTableNames((Pattern)null, false);
362   }
363
364   @Override
365   public TableName[] listTableNames(Pattern pattern) throws IOException {
366     return listTableNames(pattern, false);
367   }
368
369   @Override
370   public TableName[] listTableNames(String regex) throws IOException {
371     return listTableNames(Pattern.compile(regex), false);
372   }
373
374   @Override
375   public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables)
376       throws IOException {
377     return executeCallable(new MasterCallable<TableName[]>(getConnection(),
378         getRpcControllerFactory()) {
379       @Override
380       protected TableName[] rpcCall() throws Exception {
381         GetTableNamesRequest req =
382             RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
383         return ProtobufUtil.getTableNameArray(master.getTableNames(getRpcController(), req)
384             .getTableNamesList());
385       }
386     });
387   }
388
389   @Override
390   public TableName[] listTableNames(final String regex, final boolean includeSysTables)
391       throws IOException {
392     return listTableNames(Pattern.compile(regex), includeSysTables);
393   }
394
395   @Override
396   public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
397     return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
398        operationTimeout, rpcTimeout);
399   }
400
401   static HTableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
402       RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
403       int operationTimeout, int rpcTimeout) throws IOException {
404     if (tableName == null) return null;
405     HTableDescriptor htd =
406         executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
407       @Override
408       protected HTableDescriptor rpcCall() throws Exception {
409         GetTableDescriptorsRequest req =
410             RequestConverter.buildGetTableDescriptorsRequest(tableName);
411         GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
412         if (!htds.getTableSchemaList().isEmpty()) {
413           return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
414         }
415         return null;
416       }
417     }, rpcCallerFactory, operationTimeout, rpcTimeout);
418     if (htd != null) {
419       return htd;
420     }
421     throw new TableNotFoundException(tableName.getNameAsString());
422   }
423
424   private long getPauseTime(int tries) {
425     int triesCount = tries;
426     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
427       triesCount = HConstants.RETRY_BACKOFF.length - 1;
428     }
429     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
430   }
431
432   @Override
433   public void createTable(HTableDescriptor desc)
434   throws IOException {
435     createTable(desc, null);
436   }
437
438   @Override
439   public void createTable(HTableDescriptor desc, byte [] startKey,
440       byte [] endKey, int numRegions)
441   throws IOException {
442     if(numRegions < 3) {
443       throw new IllegalArgumentException("Must create at least three regions");
444     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
445       throw new IllegalArgumentException("Start key must be smaller than end key");
446     }
447     if (numRegions == 3) {
448       createTable(desc, new byte[][]{startKey, endKey});
449       return;
450     }
451     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
452     if(splitKeys == null || splitKeys.length != numRegions - 1) {
453       throw new IllegalArgumentException("Unable to split key range into enough regions");
454     }
455     createTable(desc, splitKeys);
456   }
457
458   @Override
459   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
460       throws IOException {
461     get(createTableAsync(desc, splitKeys), syncWaitTimeout, TimeUnit.MILLISECONDS);
462   }
463
464   @Override
465   public Future<Void> createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys)
466       throws IOException {
467     if (desc.getTableName() == null) {
468       throw new IllegalArgumentException("TableName cannot be null");
469     }
470     if (splitKeys != null && splitKeys.length > 0) {
471       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
472       // Verify there are no duplicate split keys
473       byte[] lastKey = null;
474       for (byte[] splitKey : splitKeys) {
475         if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
476           throw new IllegalArgumentException(
477               "Empty split key must not be passed in the split keys.");
478         }
479         if (lastKey != null && Bytes.equals(splitKey, lastKey)) {
480           throw new IllegalArgumentException("All split keys must be unique, " +
481             "found duplicate: " + Bytes.toStringBinary(splitKey) +
482             ", " + Bytes.toStringBinary(lastKey));
483         }
484         lastKey = splitKey;
485       }
486     }
487
488     CreateTableResponse response = executeCallable(
489       new MasterCallable<CreateTableResponse>(getConnection(), getRpcControllerFactory()) {
490         @Override
491         protected CreateTableResponse rpcCall() throws Exception {
492           setPriority(desc.getTableName());
493           CreateTableRequest request = RequestConverter.buildCreateTableRequest(
494             desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
495           return master.createTable(getRpcController(), request);
496         }
497       });
498     return new CreateTableFuture(this, desc, splitKeys, response);
499   }
500
501   private static class CreateTableFuture extends TableFuture<Void> {
502     private final HTableDescriptor desc;
503     private final byte[][] splitKeys;
504
505     public CreateTableFuture(final HBaseAdmin admin, final HTableDescriptor desc,
506         final byte[][] splitKeys, final CreateTableResponse response) {
507       super(admin, desc.getTableName(),
508               (response != null && response.hasProcId()) ? response.getProcId() : null);
509       this.splitKeys = splitKeys;
510       this.desc = desc;
511     }
512
513     @Override
514     protected HTableDescriptor getTableDescriptor() {
515       return desc;
516     }
517
518     @Override
519     public String getOperationType() {
520       return "CREATE";
521     }
522
523     @Override
524     protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
525       waitForTableEnabled(deadlineTs);
526       waitForAllRegionsOnline(deadlineTs, splitKeys);
527       return null;
528     }
529   }
530
531   @Override
532   public void deleteTable(final TableName tableName) throws IOException {
533     get(deleteTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
534   }
535
536   @Override
537   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
538     DeleteTableResponse response = executeCallable(
539       new MasterCallable<DeleteTableResponse>(getConnection(), getRpcControllerFactory()) {
540         @Override
541         protected DeleteTableResponse rpcCall() throws Exception {
542           setPriority(tableName);
543           DeleteTableRequest req =
544               RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
545           return master.deleteTable(getRpcController(), req);
546         }
547       });
548     return new DeleteTableFuture(this, tableName, response);
549   }
550
551   private static class DeleteTableFuture extends TableFuture<Void> {
552     public DeleteTableFuture(final HBaseAdmin admin, final TableName tableName,
553         final DeleteTableResponse response) {
554       super(admin, tableName,
555               (response != null && response.hasProcId()) ? response.getProcId() : null);
556     }
557
558     @Override
559     public String getOperationType() {
560       return "DELETE";
561     }
562
563     @Override
564     protected Void waitOperationResult(final long deadlineTs)
565         throws IOException, TimeoutException {
566       waitTableNotFound(deadlineTs);
567       return null;
568     }
569
570     @Override
571     protected Void postOperationResult(final Void result, final long deadlineTs)
572         throws IOException, TimeoutException {
573       // Delete cached information to prevent clients from using old locations
574       ((ClusterConnection) getAdmin().getConnection()).clearRegionCache(getTableName());
575       return super.postOperationResult(result, deadlineTs);
576     }
577   }
578
579   @Override
580   public HTableDescriptor[] deleteTables(String regex) throws IOException {
581     return deleteTables(Pattern.compile(regex));
582   }
583
584   /**
585    * Delete tables matching the passed in pattern and wait on completion.
586    *
587    * Warning: Use this method carefully, there is no prompting and the effect is
588    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
589    * {@link #deleteTable(TableName)}
590    *
591    * @param pattern The pattern to match table names against
592    * @return Table descriptors for tables that couldn't be deleted
593    * @throws IOException
594    */
595   @Override
596   public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
597     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
598     for (HTableDescriptor table : listTables(pattern)) {
599       try {
600         deleteTable(table.getTableName());
601       } catch (IOException ex) {
602         LOG.info("Failed to delete table " + table.getTableName(), ex);
603         failed.add(table);
604       }
605     }
606     return failed.toArray(new HTableDescriptor[failed.size()]);
607   }
608
609   @Override
610   public void truncateTable(final TableName tableName, final boolean preserveSplits)
611       throws IOException {
612     get(truncateTableAsync(tableName, preserveSplits), syncWaitTimeout, TimeUnit.MILLISECONDS);
613   }
614
615   @Override
616   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
617       throws IOException {
618     TruncateTableResponse response =
619         executeCallable(new MasterCallable<TruncateTableResponse>(getConnection(),
620             getRpcControllerFactory()) {
621           @Override
622           protected TruncateTableResponse rpcCall() throws Exception {
623             setPriority(tableName);
624             LOG.info("Started truncating " + tableName);
625             TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
626               tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
627             return master.truncateTable(getRpcController(), req);
628           }
629         });
630     return new TruncateTableFuture(this, tableName, preserveSplits, response);
631   }
632
633   private static class TruncateTableFuture extends TableFuture<Void> {
634     private final boolean preserveSplits;
635
636     public TruncateTableFuture(final HBaseAdmin admin, final TableName tableName,
637         final boolean preserveSplits, final TruncateTableResponse response) {
638       super(admin, tableName,
639              (response != null && response.hasProcId()) ? response.getProcId() : null);
640       this.preserveSplits = preserveSplits;
641     }
642
643     @Override
644     public String getOperationType() {
645       return "TRUNCATE";
646     }
647
648     @Override
649     protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
650       waitForTableEnabled(deadlineTs);
651       // once the table is enabled, we know the operation is done. so we can fetch the splitKeys
652       byte[][] splitKeys = preserveSplits ? getAdmin().getTableSplits(getTableName()) : null;
653       waitForAllRegionsOnline(deadlineTs, splitKeys);
654       return null;
655     }
656   }
657
658   private byte[][] getTableSplits(final TableName tableName) throws IOException {
659     byte[][] splits = null;
660     try (RegionLocator locator = getConnection().getRegionLocator(tableName)) {
661       byte[][] startKeys = locator.getStartKeys();
662       if (startKeys.length == 1) {
663         return splits;
664       }
665       splits = new byte[startKeys.length - 1][];
666       for (int i = 1; i < startKeys.length; i++) {
667         splits[i - 1] = startKeys[i];
668       }
669     }
670     return splits;
671   }
672
673   @Override
674   public void enableTable(final TableName tableName)
675   throws IOException {
676     get(enableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
677   }
678
679   @Override
680   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
681     TableName.isLegalFullyQualifiedTableName(tableName.getName());
682     EnableTableResponse response = executeCallable(
683       new MasterCallable<EnableTableResponse>(getConnection(), getRpcControllerFactory()) {
684         @Override
685         protected EnableTableResponse rpcCall() throws Exception {
686           setPriority(tableName);
687           LOG.info("Started enable of " + tableName);
688           EnableTableRequest req =
689               RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
690           return master.enableTable(getRpcController(),req);
691         }
692       });
693     return new EnableTableFuture(this, tableName, response);
694   }
695
696   private static class EnableTableFuture extends TableFuture<Void> {
697     public EnableTableFuture(final HBaseAdmin admin, final TableName tableName,
698         final EnableTableResponse response) {
699       super(admin, tableName,
700               (response != null && response.hasProcId()) ? response.getProcId() : null);
701     }
702
703     @Override
704     public String getOperationType() {
705       return "ENABLE";
706     }
707
708     @Override
709     protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
710       waitForTableEnabled(deadlineTs);
711       return null;
712     }
713   }
714
715   @Override
716   public HTableDescriptor[] enableTables(String regex) throws IOException {
717     return enableTables(Pattern.compile(regex));
718   }
719
720   @Override
721   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
722     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
723     for (HTableDescriptor table : listTables(pattern)) {
724       if (isTableDisabled(table.getTableName())) {
725         try {
726           enableTable(table.getTableName());
727         } catch (IOException ex) {
728           LOG.info("Failed to enable table " + table.getTableName(), ex);
729           failed.add(table);
730         }
731       }
732     }
733     return failed.toArray(new HTableDescriptor[failed.size()]);
734   }
735
736   @Override
737   public void disableTable(final TableName tableName)
738   throws IOException {
739     get(disableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
740   }
741
742   @Override
743   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
744     TableName.isLegalFullyQualifiedTableName(tableName.getName());
745     DisableTableResponse response = executeCallable(
746       new MasterCallable<DisableTableResponse>(getConnection(), getRpcControllerFactory()) {
747         @Override
748         protected DisableTableResponse rpcCall() throws Exception {
749           setPriority(tableName);
750           LOG.info("Started disable of " + tableName);
751           DisableTableRequest req =
752               RequestConverter.buildDisableTableRequest(
753                 tableName, ng.getNonceGroup(), ng.newNonce());
754           return master.disableTable(getRpcController(), req);
755         }
756       });
757     return new DisableTableFuture(this, tableName, response);
758   }
759
760   private static class DisableTableFuture extends TableFuture<Void> {
761     public DisableTableFuture(final HBaseAdmin admin, final TableName tableName,
762         final DisableTableResponse response) {
763       super(admin, tableName,
764               (response != null && response.hasProcId()) ? response.getProcId() : null);
765     }
766
767     @Override
768     public String getOperationType() {
769       return "DISABLE";
770     }
771
772     @Override
773     protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException {
774       waitForTableDisabled(deadlineTs);
775       return null;
776     }
777   }
778
779   @Override
780   public HTableDescriptor[] disableTables(String regex) throws IOException {
781     return disableTables(Pattern.compile(regex));
782   }
783
784   @Override
785   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
786     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
787     for (HTableDescriptor table : listTables(pattern)) {
788       if (isTableEnabled(table.getTableName())) {
789         try {
790           disableTable(table.getTableName());
791         } catch (IOException ex) {
792           LOG.info("Failed to disable table " + table.getTableName(), ex);
793           failed.add(table);
794         }
795       }
796     }
797     return failed.toArray(new HTableDescriptor[failed.size()]);
798   }
799
800   @Override
801   public boolean isTableEnabled(final TableName tableName) throws IOException {
802     checkTableExists(tableName);
803     return executeCallable(new RpcRetryingCallable<Boolean>() {
804       @Override
805       protected Boolean rpcCall(int callTimeout) throws Exception {
806         TableState tableState = MetaTableAccessor.getTableState(getConnection(), tableName);
807         if (tableState == null) {
808           throw new TableNotFoundException(tableName);
809         }
810         return tableState.inStates(TableState.State.ENABLED);
811       }
812     });
813   }
814
815   @Override
816   public boolean isTableDisabled(TableName tableName) throws IOException {
817     checkTableExists(tableName);
818     return connection.isTableDisabled(tableName);
819   }
820
821   @Override
822   public boolean isTableAvailable(TableName tableName) throws IOException {
823     return connection.isTableAvailable(tableName, null);
824   }
825
826   @Override
827   public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
828     return connection.isTableAvailable(tableName, splitKeys);
829   }
830
831   @Override
832   public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
833     return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection(),
834         getRpcControllerFactory()) {
835       @Override
836       protected Pair<Integer, Integer> rpcCall() throws Exception {
837         setPriority(tableName);
838         GetSchemaAlterStatusRequest req = RequestConverter
839             .buildGetSchemaAlterStatusRequest(tableName);
840         GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(getRpcController(), req);
841         Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
842             ret.getTotalRegions());
843         return pair;
844       }
845     });
846   }
847
848   @Override
849   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException {
850     return getAlterStatus(TableName.valueOf(tableName));
851   }
852
853   /**
854    * {@inheritDoc}
855    * @deprecated Since 2.0. Will be removed in 3.0. Use
856    *     {@link #addColumnFamily(TableName, HColumnDescriptor)} instead.
857    */
858   @Override
859   @Deprecated
860   public void addColumn(final TableName tableName, final HColumnDescriptor columnFamily)
861   throws IOException {
862     addColumnFamily(tableName, columnFamily);
863   }
864
865   @Override
866   public Future<Void> addColumnFamily(final TableName tableName,
867       final HColumnDescriptor columnFamily) throws IOException {
868     AddColumnResponse response =
869         executeCallable(new MasterCallable<AddColumnResponse>(getConnection(),
870             getRpcControllerFactory()) {
871           @Override
872           protected AddColumnResponse rpcCall() throws Exception {
873             setPriority(tableName);
874             AddColumnRequest req =
875                 RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
876                   ng.newNonce());
877             return master.addColumn(getRpcController(), req);
878           }
879         });
880     return new AddColumnFamilyFuture(this, tableName, response);
881   }
882
883   private static class AddColumnFamilyFuture extends ModifyTableFuture {
884     public AddColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName,
885         final AddColumnResponse response) {
886       super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId()
887           : null);
888     }
889
890     @Override
891     public String getOperationType() {
892       return "ADD_COLUMN_FAMILY";
893     }
894   }
895
896   /**
897    * {@inheritDoc}
898    * @deprecated Since 2.0. Will be removed in 3.0. Use
899    *     {@link #deleteColumnFamily(TableName, byte[])} instead.
900    */
901   @Override
902   @Deprecated
903   public void deleteColumn(final TableName tableName, final byte[] columnFamily)
904   throws IOException {
905     deleteColumnFamily(tableName, columnFamily);
906   }
907
908   @Override
909   public Future<Void> deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
910       throws IOException {
911     DeleteColumnResponse response =
912         executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection(),
913             getRpcControllerFactory()) {
914           @Override
915           protected DeleteColumnResponse rpcCall() throws Exception {
916             setPriority(tableName);
917             DeleteColumnRequest req =
918                 RequestConverter.buildDeleteColumnRequest(tableName, columnFamily,
919                   ng.getNonceGroup(), ng.newNonce());
920             master.deleteColumn(getRpcController(), req);
921             return null;
922           }
923         });
924     return new DeleteColumnFamilyFuture(this, tableName, response);
925   }
926
927   private static class DeleteColumnFamilyFuture extends ModifyTableFuture {
928     public DeleteColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName,
929         final DeleteColumnResponse response) {
930       super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId()
931           : null);
932     }
933
934     @Override
935     public String getOperationType() {
936       return "DELETE_COLUMN_FAMILY";
937     }
938   }
939
940   /**
941    * {@inheritDoc}
942    * @deprecated As of 2.0. Will be removed in 3.0. Use
943    *     {@link #modifyColumnFamily(TableName, HColumnDescriptor)} instead.
944    */
945   @Override
946   @Deprecated
947   public void modifyColumn(final TableName tableName, final HColumnDescriptor columnFamily)
948   throws IOException {
949     modifyColumnFamily(tableName, columnFamily);
950   }
951
952   @Override
953   public Future<Void> modifyColumnFamily(final TableName tableName,
954       final HColumnDescriptor columnFamily) throws IOException {
955     ModifyColumnResponse response =
956         executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection(),
957             getRpcControllerFactory()) {
958           @Override
959           protected ModifyColumnResponse rpcCall() throws Exception {
960             setPriority(tableName);
961             ModifyColumnRequest req =
962                 RequestConverter.buildModifyColumnRequest(tableName, columnFamily,
963                   ng.getNonceGroup(), ng.newNonce());
964             master.modifyColumn(getRpcController(), req);
965             return null;
966           }
967         });
968     return new ModifyColumnFamilyFuture(this, tableName, response);
969   }
970
971   private static class ModifyColumnFamilyFuture extends ModifyTableFuture {
972     public ModifyColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName,
973         final ModifyColumnResponse response) {
974       super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId()
975           : null);
976     }
977
978     @Override
979     public String getOperationType() {
980       return "MODIFY_COLUMN_FAMILY";
981     }
982   }
983
984   @Override
985   public void closeRegion(final String regionname, final String serverName) throws IOException {
986     closeRegion(Bytes.toBytes(regionname), serverName);
987   }
988
989   @Override
990   public void closeRegion(final byte [] regionname, final String serverName) throws IOException {
991     if (serverName != null) {
992       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
993       if (pair == null || pair.getFirst() == null) {
994         throw new UnknownRegionException(Bytes.toStringBinary(regionname));
995       } else {
996         closeRegion(ServerName.valueOf(serverName), pair.getFirst());
997       }
998     } else {
999       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
1000       if (pair == null) {
1001         throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1002       } else if (pair.getSecond() == null) {
1003         throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1004       } else {
1005         closeRegion(pair.getSecond(), pair.getFirst());
1006       }
1007     }
1008   }
1009
1010   @Override
1011   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1012       final String serverName)
1013   throws IOException {
1014     if (null == serverName || ("").equals(serverName.trim())) {
1015       throw new IllegalArgumentException("The servername cannot be null or empty.");
1016     }
1017     ServerName sn = ServerName.valueOf(serverName);
1018     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1019     // Close the region without updating zk state.
1020     CloseRegionRequest request =
1021       RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
1022     // TODO: There is no timeout on this controller. Set one!
1023     PayloadCarryingRpcController controller = this.rpcControllerFactory.newController();
1024     try {
1025       CloseRegionResponse response = admin.closeRegion(controller, request);
1026       boolean closed = response.getClosed();
1027       if (false == closed) {
1028         LOG.error("Not able to close the region " + encodedRegionName + ".");
1029       }
1030       return closed;
1031     } catch (Exception e) {
1032       throw ProtobufUtil.handleRemoteException(e);
1033     }
1034   }
1035
1036   @Override
1037   public void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException {
1038     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1039     // TODO: There is no timeout on this controller. Set one!
1040     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1041
1042     // Close the region without updating zk state.
1043     ProtobufUtil.closeRegion(controller, admin, sn, hri.getRegionName());
1044   }
1045
1046   @Override
1047   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
1048     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1049     // TODO: There is no timeout on this controller. Set one!
1050     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1051     return ProtobufUtil.getOnlineRegions(controller, admin);
1052   }
1053
1054   @Override
1055   public void flush(final TableName tableName) throws IOException {
1056     checkTableExists(tableName);
1057     if (isTableDisabled(tableName)) {
1058       LOG.info("Table is disabled: " + tableName.getNameAsString());
1059       return;
1060     }
1061     execProcedure("flush-table-proc", tableName.getNameAsString(),
1062       new HashMap<String, String>());
1063   }
1064
1065   @Override
1066   public void flushRegion(final byte[] regionName) throws IOException {
1067     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
1068     if (regionServerPair == null) {
1069       throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
1070     }
1071     if (regionServerPair.getSecond() == null) {
1072       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
1073     }
1074     final HRegionInfo hRegionInfo = regionServerPair.getFirst();
1075     ServerName serverName = regionServerPair.getSecond();
1076     final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
1077     Callable<Void> callable = new Callable<Void>() {
1078       @Override
1079       public Void call() throws Exception {
1080         // TODO: There is no timeout on this controller. Set one!
1081         PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1082         FlushRegionRequest request =
1083             RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
1084         admin.flushRegion(controller, request);
1085         return null;
1086       }
1087     };
1088     ProtobufUtil.call(callable);
1089   }
1090
1091   /**
1092    * {@inheritDoc}
1093    */
1094   @Override
1095   public void compact(final TableName tableName)
1096     throws IOException {
1097     compact(tableName, null, false, CompactType.NORMAL);
1098   }
1099
1100   @Override
1101   public void compactRegion(final byte[] regionName)
1102     throws IOException {
1103     compactRegion(regionName, null, false);
1104   }
1105
1106   /**
1107    * {@inheritDoc}
1108    */
1109   @Override
1110   public void compact(final TableName tableName, final byte[] columnFamily)
1111     throws IOException {
1112     compact(tableName, columnFamily, false, CompactType.NORMAL);
1113   }
1114
1115   /**
1116    * {@inheritDoc}
1117    */
1118   @Override
1119   public void compactRegion(final byte[] regionName, final byte[] columnFamily)
1120     throws IOException {
1121     compactRegion(regionName, columnFamily, false);
1122   }
1123
1124   /**
1125    * {@inheritDoc}
1126    */
1127   @Override
1128   public void compactRegionServer(final ServerName sn, boolean major)
1129   throws IOException, InterruptedException {
1130     for (HRegionInfo region : getOnlineRegions(sn)) {
1131       compact(sn, region, major, null);
1132     }
1133   }
1134
1135   @Override
1136   public void majorCompact(final TableName tableName)
1137   throws IOException {
1138     compact(tableName, null, true, CompactType.NORMAL);
1139   }
1140
1141   @Override
1142   public void majorCompactRegion(final byte[] regionName)
1143   throws IOException {
1144     compactRegion(regionName, null, true);
1145   }
1146
1147   /**
1148    * {@inheritDoc}
1149    */
1150   @Override
1151   public void majorCompact(final TableName tableName, final byte[] columnFamily)
1152   throws IOException {
1153     compact(tableName, columnFamily, true, CompactType.NORMAL);
1154   }
1155
1156   @Override
1157   public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
1158   throws IOException {
1159     compactRegion(regionName, columnFamily, true);
1160   }
1161
1162   /**
1163    * Compact a table.
1164    * Asynchronous operation.
1165    *
1166    * @param tableName table or region to compact
1167    * @param columnFamily column family within a table or region
1168    * @param major True if we are to do a major compaction.
1169    * @throws IOException if a remote or network exception occurs
1170    * @throws InterruptedException
1171    */
1172   private void compact(final TableName tableName, final byte[] columnFamily,final boolean major,
1173                        CompactType compactType) throws IOException {
1174     switch (compactType) {
1175       case MOB:
1176         ServerName master = getMasterAddress();
1177         compact(master, getMobRegionInfo(tableName), major, columnFamily);
1178         break;
1179       case NORMAL:
1180       default:
1181         ZooKeeperWatcher zookeeper = null;
1182         try {
1183           checkTableExists(tableName);
1184           zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
1185                   new ThrowableAbortable());
1186           List<Pair<HRegionInfo, ServerName>> pairs;
1187           if (TableName.META_TABLE_NAME.equals(tableName)) {
1188             pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
1189           } else {
1190             pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
1191           }
1192           for (Pair<HRegionInfo, ServerName> pair: pairs) {
1193             if (pair.getFirst().isOffline()) continue;
1194             if (pair.getSecond() == null) continue;
1195             try {
1196               compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
1197             } catch (NotServingRegionException e) {
1198               if (LOG.isDebugEnabled()) {
1199                 LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1200                         pair.getFirst() + ": " +
1201                         StringUtils.stringifyException(e));
1202               }
1203             }
1204           }
1205         } finally {
1206           if (zookeeper != null) {
1207             zookeeper.close();
1208           }
1209         }
1210         break;
1211     }
1212   }
1213
1214   /**
1215    * Compact an individual region.
1216    * Asynchronous operation.
1217    *
1218    * @param regionName region to compact
1219    * @param columnFamily column family within a table or region
1220    * @param major True if we are to do a major compaction.
1221    * @throws IOException if a remote or network exception occurs
1222    * @throws InterruptedException
1223    */
1224   private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
1225   throws IOException {
1226     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
1227     if (regionServerPair == null) {
1228       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
1229     }
1230     if (regionServerPair.getSecond() == null) {
1231       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
1232     }
1233     compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
1234   }
1235
1236   private void compact(final ServerName sn, final HRegionInfo hri,
1237       final boolean major, final byte [] family)
1238   throws IOException {
1239     final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1240     Callable<Void> callable = new Callable<Void>() {
1241       @Override
1242       public Void call() throws Exception {
1243         // TODO: There is no timeout on this controller. Set one!
1244         PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1245         CompactRegionRequest request =
1246             RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
1247         admin.compactRegion(controller, request);
1248         return null;
1249       }
1250     };
1251     ProtobufUtil.call(callable);
1252   }
1253
1254   @Override
1255   public void move(final byte [] encodedRegionName, final byte [] destServerName)
1256   throws IOException {
1257     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
1258       @Override
1259       protected Void rpcCall() throws Exception {
1260         setPriority(encodedRegionName);
1261         MoveRegionRequest request =
1262             RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
1263         master.moveRegion(getRpcController(), request);
1264         return null;
1265       }
1266     });
1267   }
1268
1269   @Override
1270   public void assign(final byte [] regionName) throws MasterNotRunningException,
1271       ZooKeeperConnectionException, IOException {
1272     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
1273       @Override
1274       protected Void rpcCall() throws Exception {
1275         setPriority(regionName);
1276         AssignRegionRequest request =
1277             RequestConverter.buildAssignRegionRequest(getRegionName(regionName));
1278         master.assignRegion(getRpcController(), request);
1279         return null;
1280       }
1281     });
1282   }
1283
1284   @Override
1285   public void unassign(final byte [] regionName, final boolean force)
1286   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
1287     final byte[] toBeUnassigned = getRegionName(regionName);
1288     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
1289       @Override
1290       protected Void rpcCall() throws Exception {
1291         setPriority(regionName);
1292         UnassignRegionRequest request =
1293             RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
1294         master.unassignRegion(getRpcController(), request);
1295         return null;
1296       }
1297     });
1298   }
1299
1300   @Override
1301   public void offline(final byte [] regionName)
1302   throws IOException {
1303     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
1304       @Override
1305       protected Void rpcCall() throws Exception {
1306         setPriority(regionName);
1307         master.offlineRegion(getRpcController(),
1308             RequestConverter.buildOfflineRegionRequest(regionName));
1309         return null;
1310       }
1311     });
1312   }
1313
1314   @Override
1315   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
1316   throws IOException {
1317     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1318       @Override
1319       protected Boolean rpcCall() throws Exception {
1320         SetBalancerRunningRequest req =
1321             RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
1322         return master.setBalancerRunning(getRpcController(), req).getPrevBalanceValue();
1323       }
1324     });
1325   }
1326
1327   @Override
1328   public boolean balancer() throws IOException {
1329     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1330       @Override
1331       protected Boolean rpcCall() throws Exception {
1332         return master.balance(getRpcController(),
1333             RequestConverter.buildBalanceRequest(false)).getBalancerRan();
1334       }
1335     });
1336   }
1337
1338   @Override
1339   public boolean balancer(final boolean force) throws IOException {
1340     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1341       @Override
1342       protected Boolean rpcCall() throws Exception {
1343         return master.balance(getRpcController(),
1344             RequestConverter.buildBalanceRequest(force)).getBalancerRan();
1345       }
1346     });
1347   }
1348
1349   @Override
1350   public boolean isBalancerEnabled() throws IOException {
1351     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1352       @Override
1353       protected Boolean rpcCall() throws Exception {
1354         return master.isBalancerEnabled(getRpcController(),
1355           RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
1356       }
1357     });
1358   }
1359
1360   @Override
1361   public boolean normalize() throws IOException {
1362     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1363       @Override
1364       protected Boolean rpcCall() throws Exception {
1365         return master.normalize(getRpcController(),
1366             RequestConverter.buildNormalizeRequest()).getNormalizerRan();
1367       }
1368     });
1369   }
1370
1371   @Override
1372   public boolean isNormalizerEnabled() throws IOException {
1373     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1374       @Override
1375       protected Boolean rpcCall() throws Exception {
1376         return master.isNormalizerEnabled(getRpcController(),
1377           RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
1378       }
1379     });
1380   }
1381
1382   @Override
1383   public boolean setNormalizerRunning(final boolean on) throws IOException {
1384     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1385       @Override
1386       protected Boolean rpcCall() throws Exception {
1387         SetNormalizerRunningRequest req =
1388           RequestConverter.buildSetNormalizerRunningRequest(on);
1389         return master.setNormalizerRunning(getRpcController(), req).getPrevNormalizerValue();
1390       }
1391     });
1392   }
1393
1394   @Override
1395   public boolean enableCatalogJanitor(final boolean enable) throws IOException {
1396     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1397       @Override
1398       protected Boolean rpcCall() throws Exception {
1399         return master.enableCatalogJanitor(getRpcController(),
1400           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
1401       }
1402     });
1403   }
1404
1405   @Override
1406   public int runCatalogScan() throws IOException {
1407     return executeCallable(new MasterCallable<Integer>(getConnection(), getRpcControllerFactory()) {
1408       @Override
1409       protected Integer rpcCall() throws Exception {
1410         return master.runCatalogScan(getRpcController(),
1411           RequestConverter.buildCatalogScanRequest()).getScanResult();
1412       }
1413     });
1414   }
1415
1416   @Override
1417   public boolean isCatalogJanitorEnabled() throws IOException {
1418     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
1419       @Override
1420       protected Boolean rpcCall() throws Exception {
1421         return master.isCatalogJanitorEnabled(getRpcController(),
1422           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
1423       }
1424     });
1425   }
1426
1427   private boolean isEncodedRegionName(byte[] regionName) throws IOException {
1428     try {
1429       HRegionInfo.parseRegionName(regionName);
1430       return false;
1431     } catch (IOException e) {
1432       if (StringUtils.stringifyException(e)
1433         .contains(HRegionInfo.INVALID_REGION_NAME_FORMAT_MESSAGE)) {
1434         return true;
1435       }
1436       throw e;
1437     }
1438   }
1439
1440   /**
1441    * Merge two regions. Synchronous operation.
1442    * Note: It is not feasible to predict the length of merge.
1443    *   Therefore, this is for internal testing only.
1444    * @param nameOfRegionA encoded or full name of region a
1445    * @param nameOfRegionB encoded or full name of region b
1446    * @param forcible true if do a compulsory merge, otherwise we will only merge
1447    *          two adjacent regions
1448    * @throws IOException
1449    */
1450   @VisibleForTesting
1451   public void mergeRegionsSync(
1452       final byte[] nameOfRegionA,
1453       final byte[] nameOfRegionB,
1454       final boolean forcible) throws IOException {
1455     get(
1456       mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible),
1457       syncWaitTimeout,
1458       TimeUnit.MILLISECONDS);
1459   }
1460
1461   /**
1462    * Merge two regions. Asynchronous operation.
1463    * @param nameOfRegionA encoded or full name of region a
1464    * @param nameOfRegionB encoded or full name of region b
1465    * @param forcible true if do a compulsory merge, otherwise we will only merge
1466    *          two adjacent regions
1467    * @throws IOException
1468    * @deprecated Since 2.0. Will be removed in 3.0. Use
1469    *     {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
1470    */
1471   @Deprecated
1472   @Override
1473   public void mergeRegions(final byte[] nameOfRegionA,
1474       final byte[] nameOfRegionB, final boolean forcible)
1475       throws IOException {
1476     mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible);
1477   }
1478
1479   /**
1480    * Merge two regions. Asynchronous operation.
1481    * @param nameOfRegionA encoded or full name of region a
1482    * @param nameOfRegionB encoded or full name of region b
1483    * @param forcible true if do a compulsory merge, otherwise we will only merge
1484    *          two adjacent regions
1485    * @throws IOException
1486    */
1487   @Override
1488   public Future<Void> mergeRegionsAsync(
1489       final byte[] nameOfRegionA,
1490       final byte[] nameOfRegionB,
1491       final boolean forcible) throws IOException {
1492
1493     final byte[] encodedNameOfRegionA = isEncodedRegionName(nameOfRegionA) ?
1494       nameOfRegionA : HRegionInfo.encodeRegionName(nameOfRegionA).getBytes();
1495     final byte[] encodedNameOfRegionB = isEncodedRegionName(nameOfRegionB) ?
1496       nameOfRegionB : HRegionInfo.encodeRegionName(nameOfRegionB).getBytes();
1497
1498     TableName tableName;
1499     Pair<HRegionInfo, ServerName> pair = getRegion(nameOfRegionA);
1500
1501     if (pair != null) {
1502       if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
1503         throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
1504       }
1505       tableName = pair.getFirst().getTable();
1506     } else {
1507       throw new UnknownRegionException (
1508         "Can't invoke merge on unknown region " + Bytes.toStringBinary(encodedNameOfRegionA));
1509     }
1510
1511     pair = getRegion(nameOfRegionB);
1512     if (pair != null) {
1513       if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
1514         throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
1515       }
1516
1517       if (!tableName.equals(pair.getFirst().getTable())) {
1518         throw new IllegalArgumentException ("Cannot merge regions from two different tables " +
1519           tableName + " and " + pair.getFirst().getTable());
1520       }
1521     } else {
1522       throw new UnknownRegionException (
1523         "Can't invoke merge on unknown region " + Bytes.toStringBinary(encodedNameOfRegionB));
1524     }
1525
1526     DispatchMergingRegionsResponse response =
1527         executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection(),
1528             getRpcControllerFactory()) {
1529       @Override
1530       protected DispatchMergingRegionsResponse rpcCall() throws Exception {
1531         DispatchMergingRegionsRequest request = RequestConverter
1532             .buildDispatchMergingRegionsRequest(
1533                 encodedNameOfRegionA,
1534                 encodedNameOfRegionB,
1535                 forcible,
1536                 ng.getNonceGroup(),
1537                 ng.newNonce());
1538         return master.dispatchMergingRegions(getRpcController(), request);
1539       }
1540     });
1541     return new DispatchMergingRegionsFuture(this, tableName, response);
1542   }
1543
1544   private static class DispatchMergingRegionsFuture extends TableFuture<Void> {
1545     public DispatchMergingRegionsFuture(
1546         final HBaseAdmin admin,
1547         final TableName tableName,
1548         final DispatchMergingRegionsResponse response) {
1549       super(admin, tableName,
1550           (response != null && response.hasProcId()) ? response.getProcId() : null);
1551     }
1552
1553     public DispatchMergingRegionsFuture(
1554         final HBaseAdmin admin,
1555         final TableName tableName,
1556         final Long procId) {
1557       super(admin, tableName, procId);
1558     }
1559
1560     @Override
1561     public String getOperationType() {
1562       return "MERGE_REGIONS";
1563     }
1564   }
1565
1566   @Override
1567   public void split(final TableName tableName) throws IOException {
1568     split(tableName, null);
1569   }
1570
1571   @Override
1572   public void splitRegion(final byte[] regionName) throws IOException {
1573     splitRegion(regionName, null);
1574   }
1575
1576   /**
1577    * {@inheritDoc}
1578    */
1579   @Override
1580   public void split(final TableName tableName, final byte [] splitPoint) throws IOException {
1581     ZooKeeperWatcher zookeeper = null;
1582     try {
1583       checkTableExists(tableName);
1584       zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
1585         new ThrowableAbortable());
1586       List<Pair<HRegionInfo, ServerName>> pairs;
1587       if (TableName.META_TABLE_NAME.equals(tableName)) {
1588         pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
1589       } else {
1590         pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
1591       }
1592       for (Pair<HRegionInfo, ServerName> pair: pairs) {
1593         // May not be a server for a particular row
1594         if (pair.getSecond() == null) continue;
1595         HRegionInfo r = pair.getFirst();
1596         // check for parents
1597         if (r.isSplitParent()) continue;
1598         // if a split point given, only split that particular region
1599         if (r.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
1600            (splitPoint != null && !r.containsRow(splitPoint))) continue;
1601         // call out to region server to do split now
1602         split(pair.getSecond(), pair.getFirst(), splitPoint);
1603       }
1604     } finally {
1605       if (zookeeper != null) {
1606         zookeeper.close();
1607       }
1608     }
1609   }
1610
1611   @Override
1612   public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
1613     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
1614     if (regionServerPair == null) {
1615       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
1616     }
1617     if (regionServerPair.getFirst() != null &&
1618         regionServerPair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
1619       throw new IllegalArgumentException("Can't split replicas directly. "
1620           + "Replicas are auto-split when their primary is split.");
1621     }
1622     if (regionServerPair.getSecond() == null) {
1623       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
1624     }
1625     split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
1626   }
1627
1628   @VisibleForTesting
1629   public void split(final ServerName sn, final HRegionInfo hri,
1630       byte[] splitPoint) throws IOException {
1631     if (hri.getStartKey() != null && splitPoint != null &&
1632          Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
1633        throw new IOException("should not give a splitkey which equals to startkey!");
1634     }
1635     // TODO: There is no timeout on this controller. Set one!
1636     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1637     controller.setPriority(hri.getTable());
1638
1639     // TODO: this does not do retries, it should. Set priority and timeout in controller
1640     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1641     ProtobufUtil.split(controller, admin, hri, splitPoint);
1642   }
1643
1644   @Override
1645   public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
1646       throws IOException {
1647     if (!tableName.equals(htd.getTableName())) {
1648       throw new IllegalArgumentException("the specified table name '" + tableName +
1649         "' doesn't match with the HTD one: " + htd.getTableName());
1650     }
1651     ModifyTableResponse response = executeCallable(
1652       new MasterCallable<ModifyTableResponse>(getConnection(), getRpcControllerFactory()) {
1653         @Override
1654         protected ModifyTableResponse rpcCall() throws Exception {
1655           setPriority(tableName);
1656           ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
1657             tableName, htd, ng.getNonceGroup(), ng.newNonce());
1658           return master.modifyTable(getRpcController(), request);
1659         }
1660       });
1661     return new ModifyTableFuture(this, tableName, response);
1662   }
1663
1664   private static class ModifyTableFuture extends TableFuture<Void> {
1665     public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
1666         final ModifyTableResponse response) {
1667       super(admin, tableName,
1668           (response != null && response.hasProcId()) ? response.getProcId() : null);
1669     }
1670
1671     public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) {
1672       super(admin, tableName, procId);
1673     }
1674
1675     @Override
1676     public String getOperationType() {
1677       return "MODIFY";
1678     }
1679
1680     @Override
1681     protected Void postOperationResult(final Void result, final long deadlineTs)
1682         throws IOException, TimeoutException {
1683       // The modify operation on the table is asynchronous on the server side irrespective
1684       // of whether Procedure V2 is supported or not. So, we wait in the client till
1685       // all regions get updated.
1686       waitForSchemaUpdate(deadlineTs);
1687       return result;
1688     }
1689   }
1690
1691   /**
1692    * @param regionName Name of a region.
1693    * @return a pair of HRegionInfo and ServerName if <code>regionName</code> is
1694    *  a verified region name (we call {@link
1695    *  MetaTableAccessor#getRegionLocation(Connection, byte[])}
1696    *  else null.
1697    * Throw IllegalArgumentException if <code>regionName</code> is null.
1698    * @throws IOException
1699    */
1700   Pair<HRegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
1701     if (regionName == null) {
1702       throw new IllegalArgumentException("Pass a table name or region name");
1703     }
1704     Pair<HRegionInfo, ServerName> pair =
1705       MetaTableAccessor.getRegion(connection, regionName);
1706     if (pair == null) {
1707       final AtomicReference<Pair<HRegionInfo, ServerName>> result =
1708         new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
1709       final String encodedName = Bytes.toString(regionName);
1710       MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
1711         @Override
1712         public boolean visit(Result data) throws IOException {
1713           HRegionInfo info = MetaTableAccessor.getHRegionInfo(data);
1714           if (info == null) {
1715             LOG.warn("No serialized HRegionInfo in " + data);
1716             return true;
1717           }
1718           RegionLocations rl = MetaTableAccessor.getRegionLocations(data);
1719           boolean matched = false;
1720           ServerName sn = null;
1721           if (rl != null) {
1722             for (HRegionLocation h : rl.getRegionLocations()) {
1723               if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) {
1724                 sn = h.getServerName();
1725                 info = h.getRegionInfo();
1726                 matched = true;
1727               }
1728             }
1729           }
1730           if (!matched) return true;
1731           result.set(new Pair<HRegionInfo, ServerName>(info, sn));
1732           return false; // found the region, stop
1733         }
1734       };
1735
1736       MetaTableAccessor.fullScanRegions(connection, visitor);
1737       pair = result.get();
1738     }
1739     return pair;
1740   }
1741
1742   /**
1743    * If the input is a region name, it is returned as is. If it's an
1744    * encoded region name, the corresponding region is found from meta
1745    * and its region name is returned. If we can't find any region in
1746    * meta matching the input as either region name or encoded region
1747    * name, the input is returned as is. We don't throw unknown
1748    * region exception.
1749    */
1750   private byte[] getRegionName(
1751       final byte[] regionNameOrEncodedRegionName) throws IOException {
1752     if (Bytes.equals(regionNameOrEncodedRegionName,
1753         HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
1754           || Bytes.equals(regionNameOrEncodedRegionName,
1755             HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
1756       return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
1757     }
1758     byte[] tmp = regionNameOrEncodedRegionName;
1759     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
1760     if (regionServerPair != null && regionServerPair.getFirst() != null) {
1761       tmp = regionServerPair.getFirst().getRegionName();
1762     }
1763     return tmp;
1764   }
1765
1766   /**
1767    * Check if table exists or not
1768    * @param tableName Name of a table.
1769    * @return tableName instance
1770    * @throws IOException if a remote or network exception occurs.
1771    * @throws TableNotFoundException if table does not exist.
1772    */
1773   private TableName checkTableExists(final TableName tableName)
1774       throws IOException {
1775     return executeCallable(new RpcRetryingCallable<TableName>() {
1776       @Override
1777       protected TableName rpcCall(int callTimeout) throws Exception {
1778         if (!MetaTableAccessor.tableExists(connection, tableName)) {
1779           throw new TableNotFoundException(tableName);
1780         }
1781         return tableName;
1782       }
1783     });
1784   }
1785
1786   @Override
1787   public synchronized void shutdown() throws IOException {
1788     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
1789       @Override
1790       protected Void rpcCall() throws Exception {
1791         setPriority(HConstants.HIGH_QOS);
1792         master.shutdown(getRpcController(), ShutdownRequest.newBuilder().build());
1793         return null;
1794       }
1795     });
1796   }
1797
1798   @Override
1799   public synchronized void stopMaster() throws IOException {
1800     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
1801       @Override
1802       protected Void rpcCall() throws Exception {
1803         setPriority(HConstants.HIGH_QOS);
1804         master.stopMaster(getRpcController(), StopMasterRequest.newBuilder().build());
1805         return null;
1806       }
1807     });
1808   }
1809
1810   @Override
1811   public synchronized void stopRegionServer(final String hostnamePort)
1812   throws IOException {
1813     String hostname = Addressing.parseHostname(hostnamePort);
1814     int port = Addressing.parsePort(hostnamePort);
1815     final AdminService.BlockingInterface admin =
1816       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
1817     // TODO: There is no timeout on this controller. Set one!
1818     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1819     controller.setPriority(HConstants.HIGH_QOS);
1820     StopServerRequest request = RequestConverter.buildStopServerRequest(
1821         "Called by admin client " + this.connection.toString());
1822     try {
1823       admin.stopServer(controller, request);
1824     } catch (Exception e) {
1825       throw ProtobufUtil.handleRemoteException(e);
1826     }
1827   }
1828
1829   @Override
1830   public boolean isMasterInMaintenanceMode() throws IOException {
1831     return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection(),
1832         this.rpcControllerFactory) {
1833       @Override
1834       protected IsInMaintenanceModeResponse rpcCall() throws Exception {
1835         return master.isMasterInMaintenanceMode(getRpcController(),
1836             IsInMaintenanceModeRequest.newBuilder().build());
1837       }
1838     }).getInMaintenanceMode();
1839   }
1840
1841   @Override
1842   public ClusterStatus getClusterStatus() throws IOException {
1843     return executeCallable(new MasterCallable<ClusterStatus>(getConnection(),
1844         this.rpcControllerFactory) {
1845       @Override
1846       protected ClusterStatus rpcCall() throws Exception {
1847         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
1848         return ProtobufUtil.convert(master.getClusterStatus(getRpcController(), req).
1849             getClusterStatus());
1850       }
1851     });
1852   }
1853
1854   @Override
1855   public Configuration getConfiguration() {
1856     return this.conf;
1857   }
1858
1859   /**
1860    * Do a get with a timeout against the passed in <code>future<code>.
1861    */
1862   private static <T> T get(final Future<T> future, final long timeout, final TimeUnit units)
1863   throws IOException {
1864     try {
1865       // TODO: how long should we wait? Spin forever?
1866       return future.get(timeout, units);
1867     } catch (InterruptedException e) {
1868       throw new InterruptedIOException("Interrupt while waiting on " + future);
1869     } catch (TimeoutException e) {
1870       throw new TimeoutIOException(e);
1871     } catch (ExecutionException e) {
1872       if (e.getCause() instanceof IOException) {
1873         throw (IOException)e.getCause();
1874       } else {
1875         throw new IOException(e.getCause());
1876       }
1877     }
1878   }
1879
1880   @Override
1881   public void createNamespace(final NamespaceDescriptor descriptor)
1882   throws IOException {
1883     get(createNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
1884   }
1885
1886   @Override
1887   public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
1888       throws IOException {
1889     CreateNamespaceResponse response =
1890         executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection(),
1891             getRpcControllerFactory()) {
1892       @Override
1893       protected CreateNamespaceResponse rpcCall() throws Exception {
1894         return master.createNamespace(getRpcController(),
1895           CreateNamespaceRequest.newBuilder().setNamespaceDescriptor(ProtobufUtil.
1896               toProtoNamespaceDescriptor(descriptor)).build());
1897       }
1898     });
1899     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
1900       @Override
1901       public String getOperationType() {
1902         return "CREATE_NAMESPACE";
1903       }
1904     };
1905   }
1906
1907   @Override
1908   public void modifyNamespace(final NamespaceDescriptor descriptor)
1909   throws IOException {
1910     get(modifyNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
1911   }
1912
1913   @Override
1914   public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
1915       throws IOException {
1916     ModifyNamespaceResponse response =
1917         executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection(),
1918             getRpcControllerFactory()) {
1919       @Override
1920       protected ModifyNamespaceResponse rpcCall() throws Exception {
1921         // TODO: set priority based on NS?
1922         return master.modifyNamespace(getRpcController(), ModifyNamespaceRequest.newBuilder().
1923           setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
1924        }
1925     });
1926     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
1927       @Override
1928       public String getOperationType() {
1929         return "MODIFY_NAMESPACE";
1930       }
1931     };
1932   }
1933
1934   @Override
1935   public void deleteNamespace(final String name)
1936   throws IOException {
1937     get(deleteNamespaceAsync(name), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
1938   }
1939
1940   @Override
1941   public Future<Void> deleteNamespaceAsync(final String name)
1942       throws IOException {
1943     DeleteNamespaceResponse response =
1944         executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection(),
1945             getRpcControllerFactory()) {
1946       @Override
1947       protected DeleteNamespaceResponse rpcCall() throws Exception {
1948         // TODO: set priority based on NS?
1949         return master.deleteNamespace(getRpcController(), DeleteNamespaceRequest.newBuilder().
1950           setNamespaceName(name).build());
1951         }
1952       });
1953     return new NamespaceFuture(this, name, response.getProcId()) {
1954       @Override
1955       public String getOperationType() {
1956         return "DELETE_NAMESPACE";
1957       }
1958     };
1959   }
1960
1961   @Override
1962   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
1963     return executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection(),
1964         getRpcControllerFactory()) {
1965       @Override
1966       protected NamespaceDescriptor rpcCall() throws Exception {
1967         return ProtobufUtil.toNamespaceDescriptor(
1968             master.getNamespaceDescriptor(getRpcController(),
1969                 GetNamespaceDescriptorRequest.newBuilder().
1970                   setNamespaceName(name).build()).getNamespaceDescriptor());
1971       }
1972     });
1973   }
1974
1975   @Override
1976   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
1977     return executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection(),
1978         getRpcControllerFactory()) {
1979       @Override
1980       protected NamespaceDescriptor[] rpcCall() throws Exception {
1981         List<HBaseProtos.NamespaceDescriptor> list =
1982             master.listNamespaceDescriptors(getRpcController(),
1983               ListNamespaceDescriptorsRequest.newBuilder().build()).getNamespaceDescriptorList();
1984         NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
1985         for(int i = 0; i < list.size(); i++) {
1986           res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
1987         }
1988         return res;
1989       }
1990     });
1991   }
1992
1993   @Override
1994   public ProcedureInfo[] listProcedures() throws IOException {
1995     return executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection(),
1996         getRpcControllerFactory()) {
1997       @Override
1998       protected ProcedureInfo[] rpcCall() throws Exception {
1999         List<ProcedureProtos.Procedure> procList = master.listProcedures(
2000             getRpcController(), ListProceduresRequest.newBuilder().build()).getProcedureList();
2001         ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
2002         for (int i = 0; i < procList.size(); i++) {
2003           procInfoList[i] = ProcedureUtil.convert(procList.get(i));
2004         }
2005         return procInfoList;
2006       }
2007     });
2008   }
2009
2010   @Override
2011   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
2012     return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
2013         getRpcControllerFactory()) {
2014       @Override
2015       protected HTableDescriptor[] rpcCall() throws Exception {
2016         List<TableSchema> list =
2017             master.listTableDescriptorsByNamespace(getRpcController(),
2018                 ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
2019                 .build()).getTableSchemaList();
2020         HTableDescriptor[] res = new HTableDescriptor[list.size()];
2021         for(int i=0; i < list.size(); i++) {
2022
2023           res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
2024         }
2025         return res;
2026       }
2027     });
2028   }
2029
2030   @Override
2031   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
2032     return executeCallable(new MasterCallable<TableName[]>(getConnection(),
2033         getRpcControllerFactory()) {
2034       @Override
2035       protected TableName[] rpcCall() throws Exception {
2036         List<HBaseProtos.TableName> tableNames =
2037             master.listTableNamesByNamespace(getRpcController(), ListTableNamesByNamespaceRequest.
2038                 newBuilder().setNamespaceName(name).build())
2039             .getTableNameList();
2040         TableName[] result = new TableName[tableNames.size()];
2041         for (int i = 0; i < tableNames.size(); i++) {
2042           result[i] = ProtobufUtil.toTableName(tableNames.get(i));
2043         }
2044         return result;
2045       }
2046     });
2047   }
2048
2049   /**
2050    * Check to see if HBase is running. Throw an exception if not.
2051    * @param conf system configuration
2052    * @throws MasterNotRunningException if the master is not running
2053    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
2054    * @deprecated since hbase-2.0.0 because throws a ServiceException. We don't want to have
2055    * protobuf as part of our public API. Use {@link #available(Configuration)}
2056    */
2057   // Used by tests and by the Merge tool. Merge tool uses it to figure if HBase is up or not.
2058   // MOB uses it too.
2059   // NOTE: hbase-2.0.0 removes ServiceException from the throw.
2060   @Deprecated
2061   public static void checkHBaseAvailable(Configuration conf)
2062   throws MasterNotRunningException, ZooKeeperConnectionException, IOException,
2063   com.google.protobuf.ServiceException {
2064     available(conf);
2065   }
2066
2067   /**
2068    * Is HBase available? Throw an exception if not.
2069    * @param conf system configuration
2070    * @throws ZooKeeperConnectionException if unable to connect to zookeeper]
2071    */
2072   public static void available(final Configuration conf)
2073   throws ZooKeeperConnectionException, InterruptedIOException {
2074     Configuration copyOfConf = HBaseConfiguration.create(conf);
2075     // We set it to make it fail as soon as possible if HBase is not available
2076     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
2077     copyOfConf.setInt("zookeeper.recovery.retry", 0);
2078
2079     // Check ZK first.
2080     // If the connection exists, we may have a connection to ZK that does not work anymore
2081     try (ClusterConnection connection =
2082              (ClusterConnection) ConnectionFactory.createConnection(copyOfConf);
2083          ZooKeeperKeepAliveConnection zkw = ((ConnectionImplementation) connection).
2084              getKeepAliveZooKeeperWatcher();) {
2085       // This is NASTY. FIX!!!! Dependent on internal implementation! TODO
2086       zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false);
2087       connection.isMasterRunning();
2088     } catch (IOException e) {
2089       throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2090     } catch (InterruptedException e) {
2091       throw (InterruptedIOException)
2092           new InterruptedIOException("Can't connect to ZooKeeper").initCause(e);
2093     } catch (KeeperException e) {
2094       throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2095     }
2096   }
2097
2098   @Override
2099   public List<HRegionInfo> getTableRegions(final TableName tableName)
2100   throws IOException {
2101     ZooKeeperWatcher zookeeper =
2102       new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2103         new ThrowableAbortable());
2104     List<HRegionInfo> regions = null;
2105     try {
2106       if (TableName.META_TABLE_NAME.equals(tableName)) {
2107         regions = new MetaTableLocator().getMetaRegions(zookeeper);
2108       } else {
2109         regions = MetaTableAccessor.getTableRegions(connection, tableName, true);
2110       }
2111     } finally {
2112       zookeeper.close();
2113     }
2114     return regions;
2115   }
2116
2117   @Override
2118   public synchronized void close() throws IOException {
2119   }
2120
2121   @Override
2122   public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
2123   throws IOException {
2124     return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
2125         getRpcControllerFactory()) {
2126       @Override
2127       protected HTableDescriptor[] rpcCall() throws Exception {
2128         GetTableDescriptorsRequest req =
2129             RequestConverter.buildGetTableDescriptorsRequest(tableNames);
2130           return ProtobufUtil.
2131               getHTableDescriptorArray(master.getTableDescriptors(getRpcController(), req));
2132       }
2133     });
2134   }
2135
2136   /**
2137    * Get tableDescriptor
2138    * @param tableName one table name
2139    * @return HTD the HTableDescriptor or null if the table not exists
2140    * @throws IOException if a remote or network exception occurs
2141    */
2142   private HTableDescriptor getTableDescriptorByTableName(TableName tableName)
2143       throws IOException {
2144     List<TableName> tableNames = new ArrayList<TableName>(1);
2145     tableNames.add(tableName);
2146
2147     HTableDescriptor[] htdl = getTableDescriptorsByTableName(tableNames);
2148
2149     if (htdl == null || htdl.length == 0) {
2150       return null;
2151     }
2152     else {
2153       return htdl[0];
2154     }
2155   }
2156
2157   @Override
2158   public HTableDescriptor[] getTableDescriptors(List<String> names)
2159   throws IOException {
2160     List<TableName> tableNames = new ArrayList<TableName>(names.size());
2161     for(String name : names) {
2162       tableNames.add(TableName.valueOf(name));
2163     }
2164     return getTableDescriptorsByTableName(tableNames);
2165   }
2166
2167   private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
2168       FailedLogCloseException {
2169     final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2170     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
2171     // TODO: There is no timeout on this controller. Set one!
2172     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2173     try {
2174       return admin.rollWALWriter(controller, request);
2175     } catch (ServiceException e) {
2176       throw ProtobufUtil.handleRemoteException(e);
2177     }
2178   }
2179
2180   /**
2181    * Roll the log writer. I.e. when using a file system based write ahead log,
2182    * start writing log messages to a new file.
2183    *
2184    * Note that when talking to a version 1.0+ HBase deployment, the rolling is asynchronous.
2185    * This method will return as soon as the roll is requested and the return value will
2186    * always be null. Additionally, the named region server may schedule store flushes at the
2187    * request of the wal handling the roll request.
2188    *
2189    * When talking to a 0.98 or older HBase deployment, the rolling is synchronous and the
2190    * return value may be either null or a list of encoded region names.
2191    *
2192    * @param serverName
2193    *          The servername of the regionserver. A server name is made of host,
2194    *          port and startcode. This is mandatory. Here is an example:
2195    *          <code> host187.example.com,60020,1289493121758</code>
2196    * @return a set of {@link HRegionInfo#getEncodedName()} that would allow the wal to
2197    *         clean up some underlying files. null if there's nothing to flush.
2198    * @throws IOException if a remote or network exception occurs
2199    * @throws FailedLogCloseException
2200    * @deprecated use {@link #rollWALWriter(ServerName)}
2201    */
2202   @Deprecated
2203   public synchronized byte[][] rollHLogWriter(String serverName)
2204       throws IOException, FailedLogCloseException {
2205     ServerName sn = ServerName.valueOf(serverName);
2206     final RollWALWriterResponse response = rollWALWriterImpl(sn);
2207     int regionCount = response.getRegionToFlushCount();
2208     if (0 == regionCount) {
2209       return null;
2210     }
2211     byte[][] regionsToFlush = new byte[regionCount][];
2212     for (int i = 0; i < regionCount; i++) {
2213       regionsToFlush[i] = ProtobufUtil.toBytes(response.getRegionToFlush(i));
2214     }
2215     return regionsToFlush;
2216   }
2217
2218   @Override
2219   public synchronized void rollWALWriter(ServerName serverName)
2220       throws IOException, FailedLogCloseException {
2221     rollWALWriterImpl(serverName);
2222   }
2223
2224   @Override
2225   public String[] getMasterCoprocessors() {
2226     try {
2227       return getClusterStatus().getMasterCoprocessors();
2228     } catch (IOException e) {
2229       LOG.error("Could not getClusterStatus()",e);
2230       return null;
2231     }
2232   }
2233
2234   @Override
2235   public CompactionState getCompactionState(final TableName tableName)
2236   throws IOException {
2237     return getCompactionState(tableName, CompactType.NORMAL);
2238   }
2239
2240   @Override
2241   public CompactionState getCompactionStateForRegion(final byte[] regionName)
2242   throws IOException {
2243     final Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
2244     if (regionServerPair == null) {
2245       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
2246     }
2247     if (regionServerPair.getSecond() == null) {
2248       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
2249     }
2250     ServerName sn = regionServerPair.getSecond();
2251     final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2252     // TODO: There is no timeout on this controller. Set one!
2253     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2254     GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
2255       regionServerPair.getFirst().getRegionName(), true);
2256     GetRegionInfoResponse response;
2257     try {
2258       response = admin.getRegionInfo(controller, request);
2259     } catch (ServiceException e) {
2260       throw ProtobufUtil.handleRemoteException(e);
2261     }
2262     if (response.getCompactionState() != null) {
2263       return ProtobufUtil.createCompactionState(response.getCompactionState());
2264     }
2265     return null;
2266   }
2267
2268   @Override
2269   public void snapshot(final String snapshotName,
2270                        final TableName tableName) throws IOException,
2271       SnapshotCreationException, IllegalArgumentException {
2272     snapshot(snapshotName, tableName, SnapshotType.FLUSH);
2273   }
2274
2275   @Override
2276   public void snapshot(final byte[] snapshotName, final TableName tableName)
2277       throws IOException, SnapshotCreationException, IllegalArgumentException {
2278     snapshot(Bytes.toString(snapshotName), tableName, SnapshotType.FLUSH);
2279   }
2280
2281   @Override
2282   public void snapshot(final String snapshotName, final TableName tableName,
2283       SnapshotType type)
2284       throws IOException, SnapshotCreationException, IllegalArgumentException {
2285     snapshot(new SnapshotDescription(snapshotName, tableName.getNameAsString(), type));
2286   }
2287
2288   @Override
2289   public void snapshot(SnapshotDescription snapshotDesc)
2290       throws IOException, SnapshotCreationException, IllegalArgumentException {
2291     // actually take the snapshot
2292     HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
2293     SnapshotResponse response = asyncSnapshot(snapshot);
2294     final IsSnapshotDoneRequest request =
2295         IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build();
2296     IsSnapshotDoneResponse done = null;
2297     long start = EnvironmentEdgeManager.currentTime();
2298     long max = response.getExpectedTimeout();
2299     long maxPauseTime = max / this.numRetries;
2300     int tries = 0;
2301     LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
2302         ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
2303         maxPauseTime + " ms per retry)");
2304     while (tries == 0
2305         || ((EnvironmentEdgeManager.currentTime() - start) < max && !done.getDone())) {
2306       try {
2307         // sleep a backoff <= pauseTime amount
2308         long sleep = getPauseTime(tries++);
2309         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2310         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
2311           "ms while waiting for snapshot completion.");
2312         Thread.sleep(sleep);
2313       } catch (InterruptedException e) {
2314         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
2315       }
2316       LOG.debug("Getting current status of snapshot from master...");
2317       done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
2318           getRpcControllerFactory()) {
2319         @Override
2320         protected IsSnapshotDoneResponse rpcCall() throws Exception {
2321           return master.isSnapshotDone(getRpcController(), request);
2322         }
2323       });
2324     }
2325     if (!done.getDone()) {
2326       throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
2327           + "' wasn't completed in expectedTime:" + max + " ms", snapshotDesc);
2328     }
2329   }
2330
2331   @Override
2332   public void takeSnapshotAsync(SnapshotDescription snapshotDesc) throws IOException,
2333       SnapshotCreationException {
2334     HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
2335     asyncSnapshot(snapshot);
2336   }
2337
2338   private HBaseProtos.SnapshotDescription
2339       createHBaseProtosSnapshotDesc(SnapshotDescription snapshotDesc) {
2340     HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
2341     if (snapshotDesc.getTable() != null) {
2342       builder.setTable(snapshotDesc.getTable());
2343     }
2344     if (snapshotDesc.getName() != null) {
2345       builder.setName(snapshotDesc.getName());
2346     }
2347     if (snapshotDesc.getOwner() != null) {
2348       builder.setOwner(snapshotDesc.getOwner());
2349     }
2350     if (snapshotDesc.getCreationTime() != -1) {
2351       builder.setCreationTime(snapshotDesc.getCreationTime());
2352     }
2353     if (snapshotDesc.getVersion() != -1) {
2354       builder.setVersion(snapshotDesc.getVersion());
2355     }
2356     builder.setType(ProtobufUtil.createProtosSnapShotDescType(snapshotDesc.getType()));
2357     HBaseProtos.SnapshotDescription snapshot = builder.build();
2358     return snapshot;
2359   }
2360
2361   private SnapshotResponse asyncSnapshot(HBaseProtos.SnapshotDescription snapshot)
2362       throws IOException {
2363     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
2364     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
2365         .build();
2366     // run the snapshot on the master
2367     return executeCallable(new MasterCallable<SnapshotResponse>(getConnection(),
2368         getRpcControllerFactory()) {
2369       @Override
2370       protected SnapshotResponse rpcCall() throws Exception {
2371         return master.snapshot(getRpcController(), request);
2372       }
2373     });
2374   }
2375
2376   @Override
2377   public boolean isSnapshotFinished(final SnapshotDescription snapshotDesc)
2378       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
2379     final HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
2380     return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
2381         getRpcControllerFactory()) {
2382       @Override
2383       protected IsSnapshotDoneResponse rpcCall() throws Exception {
2384         return master.isSnapshotDone(getRpcController(),
2385           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
2386       }
2387     }).getDone();
2388   }
2389
2390   @Override
2391   public void restoreSnapshot(final byte[] snapshotName)
2392       throws IOException, RestoreSnapshotException {
2393     restoreSnapshot(Bytes.toString(snapshotName));
2394   }
2395
2396   @Override
2397   public void restoreSnapshot(final String snapshotName)
2398       throws IOException, RestoreSnapshotException {
2399     boolean takeFailSafeSnapshot =
2400       conf.getBoolean("hbase.snapshot.restore.take.failsafe.snapshot", false);
2401     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
2402   }
2403
2404   @Override
2405   public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
2406       throws IOException, RestoreSnapshotException {
2407     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
2408   }
2409
2410   /*
2411    * Check whether the snapshot exists and contains disabled table
2412    *
2413    * @param snapshotName name of the snapshot to restore
2414    * @throws IOException if a remote or network exception occurs
2415    * @throws RestoreSnapshotException if no valid snapshot is found
2416    */
2417   private TableName getTableNameBeforeRestoreSnapshot(final String snapshotName)
2418       throws IOException, RestoreSnapshotException {
2419     TableName tableName = null;
2420     for (SnapshotDescription snapshotInfo: listSnapshots()) {
2421       if (snapshotInfo.getName().equals(snapshotName)) {
2422         tableName = TableName.valueOf(snapshotInfo.getTable());
2423         break;
2424       }
2425     }
2426
2427     if (tableName == null) {
2428       throw new RestoreSnapshotException(
2429         "Unable to find the table name for snapshot=" + snapshotName);
2430     }
2431     return tableName;
2432   }
2433
2434   @Override
2435   public void restoreSnapshot(final String snapshotName, final boolean takeFailSafeSnapshot)
2436       throws IOException, RestoreSnapshotException {
2437     TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
2438
2439     // The table does not exists, switch to clone.
2440     if (!tableExists(tableName)) {
2441       cloneSnapshot(snapshotName, tableName);
2442       return;
2443     }
2444
2445     // Check if the table is disabled
2446     if (!isTableDisabled(tableName)) {
2447       throw new TableNotDisabledException(tableName);
2448     }
2449
2450     // Take a snapshot of the current state
2451     String failSafeSnapshotSnapshotName = null;
2452     if (takeFailSafeSnapshot) {
2453       failSafeSnapshotSnapshotName = conf.get("hbase.snapshot.restore.failsafe.name",
2454         "hbase-failsafe-{snapshot.name}-{restore.timestamp}");
2455       failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotName
2456         .replace("{snapshot.name}", snapshotName)
2457         .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
2458         .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTime()));
2459       LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
2460       snapshot(failSafeSnapshotSnapshotName, tableName);
2461     }
2462
2463     try {
2464       // Restore snapshot
2465       get(
2466         internalRestoreSnapshotAsync(snapshotName, tableName),
2467         syncWaitTimeout,
2468         TimeUnit.MILLISECONDS);
2469     } catch (IOException e) {
2470       // Somthing went wrong during the restore...
2471       // if the pre-restore snapshot is available try to rollback
2472       if (takeFailSafeSnapshot) {
2473         try {
2474           get(
2475             internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, tableName),
2476             syncWaitTimeout,
2477             TimeUnit.MILLISECONDS);
2478           String msg = "Restore snapshot=" + snapshotName +
2479             " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded.";
2480           LOG.error(msg, e);
2481           throw new RestoreSnapshotException(msg, e);
2482         } catch (IOException ex) {
2483           String msg = "Failed to restore and rollback to snapshot=" + failSafeSnapshotSnapshotName;
2484           LOG.error(msg, ex);
2485           throw new RestoreSnapshotException(msg, e);
2486         }
2487       } else {
2488         throw new RestoreSnapshotException("Failed to restore snapshot=" + snapshotName, e);
2489       }
2490     }
2491
2492     // If the restore is succeeded, delete the pre-restore snapshot
2493     if (takeFailSafeSnapshot) {
2494       try {
2495         LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
2496         deleteSnapshot(failSafeSnapshotSnapshotName);
2497       } catch (IOException e) {
2498         LOG.error("Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, e);
2499       }
2500     }
2501   }
2502
2503   @Override
2504   public Future<Void> restoreSnapshotAsync(final String snapshotName)
2505       throws IOException, RestoreSnapshotException {
2506     TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
2507
2508     // The table does not exists, switch to clone.
2509     if (!tableExists(tableName)) {
2510       return cloneSnapshotAsync(snapshotName, tableName);
2511     }
2512
2513     // Check if the table is disabled
2514     if (!isTableDisabled(tableName)) {
2515       throw new TableNotDisabledException(tableName);
2516     }
2517
2518     return internalRestoreSnapshotAsync(snapshotName, tableName);
2519   }
2520
2521   @Override
2522   public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
2523       throws IOException, TableExistsException, RestoreSnapshotException {
2524     cloneSnapshot(Bytes.toString(snapshotName), tableName);
2525   }
2526
2527   @Override
2528   public void cloneSnapshot(final String snapshotName, final TableName tableName)
2529       throws IOException, TableExistsException, RestoreSnapshotException {
2530     if (tableExists(tableName)) {
2531       throw new TableExistsException(tableName);
2532     }
2533     get(
2534       internalRestoreSnapshotAsync(snapshotName, tableName),
2535       Integer.MAX_VALUE,
2536       TimeUnit.MILLISECONDS);
2537   }
2538
2539   @Override
2540   public Future<Void> cloneSnapshotAsync(final String snapshotName, final TableName tableName)
2541       throws IOException, TableExistsException {
2542     if (tableExists(tableName)) {
2543       throw new TableExistsException(tableName);
2544     }
2545     return internalRestoreSnapshotAsync(snapshotName, tableName);
2546   }
2547
2548   @Override
2549   public byte[] execProcedureWithRet(String signature, String instance, Map<String, String> props)
2550       throws IOException {
2551     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
2552     builder.setSignature(signature).setInstance(instance);
2553     for (Entry<String, String> entry : props.entrySet()) {
2554       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
2555           .setValue(entry.getValue()).build();
2556       builder.addConfiguration(pair);
2557     }
2558
2559     final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
2560         .setProcedure(builder.build()).build();
2561     // run the procedure on the master
2562     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
2563         getConnection(), getRpcControllerFactory()) {
2564       @Override
2565       protected ExecProcedureResponse rpcCall() throws Exception {
2566         return master.execProcedureWithRet(getRpcController(), request);
2567       }
2568     });
2569
2570     return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
2571   }
2572
2573   @Override
2574   public void execProcedure(String signature, String instance, Map<String, String> props)
2575       throws IOException {
2576     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
2577     builder.setSignature(signature).setInstance(instance);
2578     for (Entry<String, String> entry : props.entrySet()) {
2579       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
2580           .setValue(entry.getValue()).build();
2581       builder.addConfiguration(pair);
2582     }
2583
2584     final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
2585         .setProcedure(builder.build()).build();
2586     // run the procedure on the master
2587     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
2588         getConnection(), getRpcControllerFactory()) {
2589       @Override
2590       protected ExecProcedureResponse rpcCall() throws Exception {
2591         return master.execProcedure(getRpcController(), request);
2592       }
2593     });
2594
2595     long start = EnvironmentEdgeManager.currentTime();
2596     long max = response.getExpectedTimeout();
2597     long maxPauseTime = max / this.numRetries;
2598     int tries = 0;
2599     LOG.debug("Waiting a max of " + max + " ms for procedure '" +
2600         signature + " : " + instance + "'' to complete. (max " + maxPauseTime + " ms per retry)");
2601     boolean done = false;
2602     while (tries == 0
2603         || ((EnvironmentEdgeManager.currentTime() - start) < max && !done)) {
2604       try {
2605         // sleep a backoff <= pauseTime amount
2606         long sleep = getPauseTime(tries++);
2607         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2608         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
2609           "ms while waiting for procedure completion.");
2610         Thread.sleep(sleep);
2611       } catch (InterruptedException e) {
2612         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
2613       }
2614       LOG.debug("Getting current status of procedure from master...");
2615       done = isProcedureFinished(signature, instance, props);
2616     }
2617     if (!done) {
2618       throw new IOException("Procedure '" + signature + " : " + instance
2619           + "' wasn't completed in expectedTime:" + max + " ms");
2620     }
2621   }
2622
2623   @Override
2624   public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
2625       throws IOException {
2626     final ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
2627     builder.setSignature(signature).setInstance(instance);
2628     for (Entry<String, String> entry : props.entrySet()) {
2629       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
2630           .setValue(entry.getValue()).build();
2631       builder.addConfiguration(pair);
2632     }
2633     final ProcedureDescription desc = builder.build();
2634     return executeCallable(
2635         new MasterCallable<IsProcedureDoneResponse>(getConnection(), getRpcControllerFactory()) {
2636           @Override
2637           protected IsProcedureDoneResponse rpcCall() throws Exception {
2638             return master.isProcedureDone(getRpcController(), IsProcedureDoneRequest
2639                 .newBuilder().setProcedure(desc).build());
2640           }
2641         }).getDone();
2642   }
2643
2644   /**
2645    * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
2646    * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
2647    * create an HTable instance to this table before it is available.
2648    * @param snapshotName snapshot to restore
2649    * @param tableName table name to restore the snapshot on
2650    * @throws IOException if a remote or network exception occurs
2651    * @throws RestoreSnapshotException if snapshot failed to be restored
2652    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2653    */
2654   private Future<Void> internalRestoreSnapshotAsync(
2655       final String snapshotName,
2656       final TableName tableName) throws IOException, RestoreSnapshotException {
2657     final HBaseProtos.SnapshotDescription snapshot = HBaseProtos.SnapshotDescription.newBuilder()
2658         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
2659
2660     // actually restore the snapshot
2661     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
2662
2663     RestoreSnapshotResponse response = executeCallable(
2664         new MasterCallable<RestoreSnapshotResponse>(getConnection(), getRpcControllerFactory()) {
2665       @Override
2666       protected RestoreSnapshotResponse rpcCall() throws Exception {
2667         final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder()
2668             .setSnapshot(snapshot)
2669             .setNonceGroup(ng.getNonceGroup())
2670             .setNonce(ng.newNonce())
2671             .build();
2672         return master.restoreSnapshot(getRpcController(), request);
2673       }
2674     });
2675
2676     return new RestoreSnapshotFuture(
2677       this, snapshot, TableName.valueOf(snapshot.getTable()), response);
2678   }
2679
2680   private static class RestoreSnapshotFuture extends TableFuture<Void> {
2681     public RestoreSnapshotFuture(
2682         final HBaseAdmin admin,
2683         final HBaseProtos.SnapshotDescription snapshot,
2684         final TableName tableName,
2685         final RestoreSnapshotResponse response) {
2686       super(admin, tableName,
2687           (response != null && response.hasProcId()) ? response.getProcId() : null);
2688
2689       if (response != null && !response.hasProcId()) {
2690         throw new UnsupportedOperationException("Client could not call old version of Server");
2691       }
2692     }
2693
2694     public RestoreSnapshotFuture(
2695         final HBaseAdmin admin,
2696         final TableName tableName,
2697         final Long procId) {
2698       super(admin, tableName, procId);
2699     }
2700
2701     @Override
2702     public String getOperationType() {
2703       return "MODIFY";
2704     }
2705   }
2706
2707   @Override
2708   public List<SnapshotDescription> listSnapshots() throws IOException {
2709     return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection(),
2710         getRpcControllerFactory()) {
2711       @Override
2712       protected List<SnapshotDescription> rpcCall() throws Exception {
2713         List<HBaseProtos.SnapshotDescription> snapshotsList = master
2714             .getCompletedSnapshots(getRpcController(),
2715                 GetCompletedSnapshotsRequest.newBuilder().build())
2716             .getSnapshotsList();
2717         List<SnapshotDescription> result = new ArrayList<SnapshotDescription>(snapshotsList.size());
2718         for (HBaseProtos.SnapshotDescription snapshot : snapshotsList) {
2719           result.add(new SnapshotDescription(snapshot.getName(), snapshot.getTable(),
2720               ProtobufUtil.createSnapshotType(snapshot.getType()), snapshot.getOwner(),
2721               snapshot.getCreationTime(), snapshot.getVersion()));
2722         }
2723         return result;
2724       }
2725     });
2726   }
2727
2728   @Override
2729   public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
2730     return listSnapshots(Pattern.compile(regex));
2731   }
2732
2733   @Override
2734   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
2735     List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>();
2736     List<SnapshotDescription> snapshots = listSnapshots();
2737     for (SnapshotDescription snapshot : snapshots) {
2738       if (pattern.matcher(snapshot.getName()).matches()) {
2739         matched.add(snapshot);
2740       }
2741     }
2742     return matched;
2743   }
2744
2745   @Override
2746   public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
2747       String snapshotNameRegex) throws IOException {
2748     return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
2749   }
2750
2751   @Override
2752   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
2753       Pattern snapshotNamePattern) throws IOException {
2754     TableName[] tableNames = listTableNames(tableNamePattern);
2755
2756     List<SnapshotDescription> tableSnapshots = new LinkedList<SnapshotDescription>();
2757     List<SnapshotDescription> snapshots = listSnapshots(snapshotNamePattern);
2758
2759     List<TableName> listOfTableNames = Arrays.asList(tableNames);
2760     for (SnapshotDescription snapshot : snapshots) {
2761       if (listOfTableNames.contains(TableName.valueOf(snapshot.getTable()))) {
2762         tableSnapshots.add(snapshot);
2763       }
2764     }
2765     return tableSnapshots;
2766   }
2767
2768   @Override
2769   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
2770     deleteSnapshot(Bytes.toString(snapshotName));
2771   }
2772
2773   @Override
2774   public void deleteSnapshot(final String snapshotName) throws IOException {
2775     // make sure the snapshot is possibly valid
2776     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
2777     // do the delete
2778     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
2779       @Override
2780       protected Void rpcCall() throws Exception {
2781         master.deleteSnapshot(getRpcController(),
2782           DeleteSnapshotRequest.newBuilder().setSnapshot(
2783                 HBaseProtos.SnapshotDescription.newBuilder().setName(snapshotName).build())
2784               .build()
2785         );
2786         return null;
2787       }
2788     });
2789   }
2790
2791   @Override
2792   public void deleteSnapshots(final String regex) throws IOException {
2793     deleteSnapshots(Pattern.compile(regex));
2794   }
2795
2796   @Override
2797   public void deleteSnapshots(final Pattern pattern) throws IOException {
2798     List<SnapshotDescription> snapshots = listSnapshots(pattern);
2799     for (final SnapshotDescription snapshot : snapshots) {
2800       try {
2801         internalDeleteSnapshot(snapshot);
2802       } catch (IOException ex) {
2803         LOG.info(
2804           "Failed to delete snapshot " + snapshot.getName() + " for table " + snapshot.getTable(),
2805           ex);
2806       }
2807     }
2808   }
2809
2810   private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
2811     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
2812       @Override
2813       protected Void rpcCall() throws Exception {
2814         this.master.deleteSnapshot(getRpcController(), DeleteSnapshotRequest.newBuilder()
2815           .setSnapshot(createHBaseProtosSnapshotDesc(snapshot)).build());
2816         return null;
2817       }
2818     });
2819   }
2820
2821   @Override
2822   public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
2823       throws IOException {
2824     deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
2825   }
2826
2827   @Override
2828   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
2829       throws IOException {
2830     List<SnapshotDescription> snapshots = listTableSnapshots(tableNamePattern, snapshotNamePattern);
2831     for (SnapshotDescription snapshot : snapshots) {
2832       try {
2833         internalDeleteSnapshot(snapshot);
2834         LOG.debug("Successfully deleted snapshot: " + snapshot.getName());
2835       } catch (IOException e) {
2836         LOG.error("Failed to delete snapshot: " + snapshot.getName(), e);
2837       }
2838     }
2839   }
2840
2841   @Override
2842   public void setQuota(final QuotaSettings quota) throws IOException {
2843     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
2844       @Override
2845       protected Void rpcCall() throws Exception {
2846         this.master.setQuota(getRpcController(), QuotaSettings.buildSetQuotaRequestProto(quota));
2847         return null;
2848       }
2849     });
2850   }
2851
2852   @Override
2853   public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
2854     return QuotaRetriever.open(conf, filter);
2855   }
2856
2857   private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable)
2858       throws IOException {
2859     return executeCallable(callable, rpcCallerFactory, operationTimeout, rpcTimeout);
2860   }
2861
2862   static private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable,
2863              RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout)
2864   throws IOException {
2865     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller(rpcTimeout);
2866     try {
2867       return caller.callWithRetries(callable, operationTimeout);
2868     } finally {
2869       callable.close();
2870     }
2871   }
2872
2873   @Override
2874   public CoprocessorRpcChannel coprocessorService() {
2875     return new MasterCoprocessorRpcChannel(connection);
2876   }
2877
2878   /**
2879    * Simple {@link Abortable}, throwing RuntimeException on abort.
2880    */
2881   private static class ThrowableAbortable implements Abortable {
2882     @Override
2883     public void abort(String why, Throwable e) {
2884       throw new RuntimeException(why, e);
2885     }
2886
2887     @Override
2888     public boolean isAborted() {
2889       return true;
2890     }
2891   }
2892
2893   @Override
2894   public CoprocessorRpcChannel coprocessorService(ServerName sn) {
2895     return new RegionServerCoprocessorRpcChannel(connection, sn);
2896   }
2897
2898   @Override
2899   public void updateConfiguration(final ServerName server) throws IOException {
2900     final AdminService.BlockingInterface admin = this.connection.getAdmin(server);
2901     Callable<Void> callable = new Callable<Void>() {
2902       @Override
2903       public Void call() throws Exception {
2904         admin.updateConfiguration(null, UpdateConfigurationRequest.getDefaultInstance());
2905         return null;
2906       }
2907     };
2908     ProtobufUtil.call(callable);
2909   }
2910
2911   @Override
2912   public void updateConfiguration() throws IOException {
2913     for (ServerName server : this.getClusterStatus().getServers()) {
2914       updateConfiguration(server);
2915     }
2916   }
2917
2918   @Override
2919   public int getMasterInfoPort() throws IOException {
2920     // TODO: Fix!  Reaching into internal implementation!!!!
2921     ConnectionImplementation connection = (ConnectionImplementation)this.connection;
2922     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
2923     try {
2924       return MasterAddressTracker.getMasterInfoPort(zkw);
2925     } catch (KeeperException e) {
2926       throw new IOException("Failed to get master info port from MasterAddressTracker", e);
2927     }
2928   }
2929
2930   private ServerName getMasterAddress() throws IOException {
2931     // TODO: Fix!  Reaching into internal implementation!!!!
2932     ConnectionImplementation connection = (ConnectionImplementation)this.connection;
2933     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
2934     try {
2935       return MasterAddressTracker.getMasterAddress(zkw);
2936     } catch (KeeperException e) {
2937       throw new IOException("Failed to get master server name from MasterAddressTracker", e);
2938     }
2939   }
2940
2941   @Override
2942   public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
2943     return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
2944       @Override
2945       protected Long rpcCall() throws Exception {
2946         MajorCompactionTimestampRequest req =
2947             MajorCompactionTimestampRequest.newBuilder()
2948                 .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
2949         return master.getLastMajorCompactionTimestamp(getRpcController(), req).
2950             getCompactionTimestamp();
2951       }
2952     });
2953   }
2954
2955   @Override
2956   public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException {
2957     return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
2958       @Override
2959       protected Long rpcCall() throws Exception {
2960         MajorCompactionTimestampForRegionRequest req =
2961             MajorCompactionTimestampForRegionRequest.newBuilder().setRegion(RequestConverter
2962                       .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
2963         return master.getLastMajorCompactionTimestampForRegion(getRpcController(), req)
2964             .getCompactionTimestamp();
2965       }
2966     });
2967   }
2968
2969   /**
2970    * {@inheritDoc}
2971    */
2972   @Override
2973   public void compact(final TableName tableName, final byte[] columnFamily, CompactType compactType)
2974     throws IOException, InterruptedException {
2975     compact(tableName, columnFamily, false, compactType);
2976   }
2977
2978   /**
2979    * {@inheritDoc}
2980    */
2981   @Override
2982   public void compact(final TableName tableName, CompactType compactType)
2983     throws IOException, InterruptedException {
2984     compact(tableName, null, false, compactType);
2985   }
2986
2987   /**
2988    * {@inheritDoc}
2989    */
2990   @Override
2991   public void majorCompact(final TableName tableName, final byte[] columnFamily,
2992     CompactType compactType) throws IOException, InterruptedException {
2993     compact(tableName, columnFamily, true, compactType);
2994   }
2995
2996   /**
2997    * {@inheritDoc}
2998    */
2999   @Override
3000   public void majorCompact(final TableName tableName, CompactType compactType)
3001           throws IOException, InterruptedException {
3002     compact(tableName, null, true, compactType);
3003   }
3004
3005   /**
3006    * {@inheritDoc}
3007    */
3008   @Override
3009   public CompactionState getCompactionState(final TableName tableName,
3010     CompactType compactType) throws IOException {
3011     AdminProtos.GetRegionInfoResponse.CompactionState state =
3012         AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
3013     checkTableExists(tableName);
3014     // TODO: There is no timeout on this controller. Set one!
3015     final PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
3016     switch (compactType) {
3017       case MOB:
3018         final AdminProtos.AdminService.BlockingInterface masterAdmin =
3019           this.connection.getAdmin(getMasterAddress());
3020         Callable<AdminProtos.GetRegionInfoResponse.CompactionState> callable =
3021             new Callable<AdminProtos.GetRegionInfoResponse.CompactionState>() {
3022           @Override
3023           public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception {
3024             HRegionInfo info = getMobRegionInfo(tableName);
3025             GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
3026                 info.getRegionName(), true);
3027             GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request);
3028             return response.getCompactionState();
3029           }
3030         };
3031         state = ProtobufUtil.call(callable);
3032         break;
3033       case NORMAL:
3034       default:
3035         ZooKeeperWatcher zookeeper = null;
3036         try {
3037           List<Pair<HRegionInfo, ServerName>> pairs;
3038           if (TableName.META_TABLE_NAME.equals(tableName)) {
3039             zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
3040               new ThrowableAbortable());
3041             pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
3042           } else {
3043             pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
3044           }
3045           for (Pair<HRegionInfo, ServerName> pair: pairs) {
3046             if (pair.getFirst().isOffline()) continue;
3047             if (pair.getSecond() == null) continue;
3048             final ServerName sn = pair.getSecond();
3049             final byte [] regionName = pair.getFirst().getRegionName();
3050             final AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn);
3051             try {
3052               Callable<GetRegionInfoResponse> regionInfoCallable =
3053                   new Callable<GetRegionInfoResponse>() {
3054                 @Override
3055                 public GetRegionInfoResponse call() throws Exception {
3056                   GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
3057                       regionName, true);
3058                   return snAdmin.getRegionInfo(rpcController, request);
3059                 }
3060               };
3061               GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable);
3062               switch (response.getCompactionState()) {
3063                 case MAJOR_AND_MINOR:
3064                   return CompactionState.MAJOR_AND_MINOR;
3065                 case MAJOR:
3066                   if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MINOR) {
3067                     return CompactionState.MAJOR_AND_MINOR;
3068                   }
3069                   state = AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR;
3070                   break;
3071                 case MINOR:
3072                   if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR) {
3073                     return CompactionState.MAJOR_AND_MINOR;
3074                   }
3075                   state = AdminProtos.GetRegionInfoResponse.CompactionState.MINOR;
3076                   break;
3077                 case NONE:
3078                 default: // nothing, continue
3079               }
3080             } catch (NotServingRegionException e) {
3081               if (LOG.isDebugEnabled()) {
3082                 LOG.debug("Trying to get compaction state of " +
3083                         pair.getFirst() + ": " +
3084                         StringUtils.stringifyException(e));
3085               }
3086             } catch (RemoteException e) {
3087               if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
3088                 if (LOG.isDebugEnabled()) {
3089                   LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
3090                           + StringUtils.stringifyException(e));
3091                 }
3092               } else {
3093                 throw e;
3094               }
3095             }
3096           }
3097         } finally {
3098           if (zookeeper != null) {
3099             zookeeper.close();
3100           }
3101         }
3102         break;
3103     }
3104     if(state != null) {
3105       return ProtobufUtil.createCompactionState(state);
3106     }
3107     return null;
3108   }
3109
3110   /**
3111    * Future that waits on a procedure result.
3112    * Returned by the async version of the Admin calls,
3113    * and used internally by the sync calls to wait on the result of the procedure.
3114    */
3115   @InterfaceAudience.Private
3116   @InterfaceStability.Evolving
3117   protected static class ProcedureFuture<V> implements Future<V> {
3118     private ExecutionException exception = null;
3119     private boolean procResultFound = false;
3120     private boolean done = false;
3121     private boolean cancelled = false;
3122     private V result = null;
3123
3124     private final HBaseAdmin admin;
3125     private final Long procId;
3126
3127     public ProcedureFuture(final HBaseAdmin admin, final Long procId) {
3128       this.admin = admin;
3129       this.procId = procId;
3130     }
3131
3132     @Override
3133     public boolean cancel(boolean mayInterruptIfRunning) {
3134       AbortProcedureRequest abortProcRequest = AbortProcedureRequest.newBuilder()
3135           .setProcId(procId).setMayInterruptIfRunning(mayInterruptIfRunning).build();
3136       try {
3137         cancelled = abortProcedureResult(abortProcRequest).getIsProcedureAborted();
3138         if (cancelled) {
3139           done = true;
3140         }
3141       } catch (IOException e) {
3142         // Cancell thrown exception for some reason. At this time, we are not sure whether
3143         // the cancell succeeds or fails. We assume that it is failed, but print out a warning
3144         // for debugging purpose.
3145         LOG.warn(
3146           "Cancelling the procedure with procId=" + procId + " throws exception " + e.getMessage(),
3147           e);
3148         cancelled = false;
3149       }
3150       return cancelled;
3151     }
3152
3153     @Override
3154     public boolean isCancelled() {
3155       return cancelled;
3156     }
3157
3158     protected AbortProcedureResponse abortProcedureResult(
3159         final AbortProcedureRequest request) throws IOException {
3160       return admin.executeCallable(new MasterCallable<AbortProcedureResponse>(
3161           admin.getConnection(), admin.getRpcControllerFactory()) {
3162         @Override
3163         protected AbortProcedureResponse rpcCall() throws Exception {
3164           return master.abortProcedure(getRpcController(), request);
3165         }
3166       });
3167     }
3168
3169     @Override
3170     public V get() throws InterruptedException, ExecutionException {
3171       // TODO: should we ever spin forever?
3172       throw new UnsupportedOperationException();
3173     }
3174
3175     @Override
3176     public V get(long timeout, TimeUnit unit)
3177         throws InterruptedException, ExecutionException, TimeoutException {
3178       if (!done) {
3179         long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
3180         try {
3181           try {
3182             // if the master support procedures, try to wait the result
3183             if (procId != null) {
3184               result = waitProcedureResult(procId, deadlineTs);
3185             }
3186             // if we don't have a proc result, try the compatibility wait
3187             if (!procResultFound) {
3188               result = waitOperationResult(deadlineTs);
3189             }
3190             result = postOperationResult(result, deadlineTs);
3191             done = true;
3192           } catch (IOException e) {
3193             result = postOperationFailure(e, deadlineTs);
3194             done = true;
3195           }
3196         } catch (IOException e) {
3197           exception = new ExecutionException(e);
3198           done = true;
3199         }
3200       }
3201       if (exception != null) {
3202         throw exception;
3203       }
3204       return result;
3205     }
3206
3207     @Override
3208     public boolean isDone() {
3209       return done;
3210     }
3211
3212     protected HBaseAdmin getAdmin() {
3213       return admin;
3214     }
3215
3216     private V waitProcedureResult(long procId, long deadlineTs)
3217         throws IOException, TimeoutException, InterruptedException {
3218       GetProcedureResultRequest request = GetProcedureResultRequest.newBuilder()
3219           .setProcId(procId)
3220           .build();
3221
3222       int tries = 0;
3223       IOException serviceEx = null;
3224       while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
3225         GetProcedureResultResponse response = null;
3226         try {
3227           // Try to fetch the result
3228           response = getProcedureResult(request);
3229         } catch (IOException e) {
3230           serviceEx = unwrapException(e);
3231
3232           // the master may be down
3233           LOG.warn("failed to get the procedure result procId=" + procId, serviceEx);
3234
3235           // Not much to do, if we have a DoNotRetryIOException
3236           if (serviceEx instanceof DoNotRetryIOException) {
3237             // TODO: looks like there is no way to unwrap this exception and get the proper
3238             // UnsupportedOperationException aside from looking at the message.
3239             // anyway, if we fail here we just failover to the compatibility side
3240             // and that is always a valid solution.
3241             LOG.warn("Proc-v2 is unsupported on this master: " + serviceEx.getMessage(), serviceEx);
3242             procResultFound = false;
3243             return null;
3244           }
3245         }
3246
3247         // If the procedure is no longer running, we should have a result
3248         if (response != null && response.getState() != GetProcedureResultResponse.State.RUNNING) {
3249           procResultFound = response.getState() != GetProcedureResultResponse.State.NOT_FOUND;
3250           return convertResult(response);
3251         }
3252
3253         try {
3254           Thread.sleep(getAdmin().getPauseTime(tries++));
3255         } catch (InterruptedException e) {
3256           throw new InterruptedException(
3257             "Interrupted while waiting for the result of proc " + procId);
3258         }
3259       }
3260       if (serviceEx != null) {
3261         throw serviceEx;
3262       } else {
3263         throw new TimeoutException("The procedure " + procId + " is still running");
3264       }
3265     }
3266
3267     private static IOException unwrapException(IOException e) {
3268       if (e instanceof RemoteException) {
3269         return ((RemoteException)e).unwrapRemoteException();
3270       }
3271       return e;
3272     }
3273
3274     protected GetProcedureResultResponse getProcedureResult(final GetProcedureResultRequest request)
3275         throws IOException {
3276       return admin.executeCallable(new MasterCallable<GetProcedureResultResponse>(
3277           admin.getConnection(), admin.getRpcControllerFactory()) {
3278         @Override
3279         protected GetProcedureResultResponse rpcCall() throws Exception {
3280           return master.getProcedureResult(getRpcController(), request);
3281         }
3282       });
3283     }
3284
3285     /**
3286      * Convert the procedure result response to a specified type.
3287      * @param response the procedure result object to parse
3288      * @return the result data of the procedure.
3289      */
3290     protected V convertResult(final GetProcedureResultResponse response) throws IOException {
3291       if (response.hasException()) {
3292         throw ForeignExceptionUtil.toIOException(response.getException());
3293       }
3294       return null;
3295     }
3296
3297     /**
3298      * Fallback implementation in case the procedure is not supported by the server.
3299      * It should try to wait until the operation is completed.
3300      * @param deadlineTs the timestamp after which this method should throw a TimeoutException
3301      * @return the result data of the operation
3302      */
3303     protected V waitOperationResult(final long deadlineTs)
3304         throws IOException, TimeoutException {
3305       return null;
3306     }
3307
3308     /**
3309      * Called after the operation is completed and the result fetched. this allows to perform extra
3310      * steps after the procedure is completed. it allows to apply transformations to the result that
3311      * will be returned by get().
3312      * @param result the result of the procedure
3313      * @param deadlineTs the timestamp after which this method should throw a TimeoutException
3314      * @return the result of the procedure, which may be the same as the passed one
3315      */
3316     protected V postOperationResult(final V result, final long deadlineTs)
3317         throws IOException, TimeoutException {
3318       return result;
3319     }
3320
3321     /**
3322      * Called after the operation is terminated with a failure.
3323      * this allows to perform extra steps after the procedure is terminated.
3324      * it allows to apply transformations to the result that will be returned by get().
3325      * The default implementation will rethrow the exception
3326      * @param exception the exception got from fetching the result
3327      * @param deadlineTs the timestamp after which this method should throw a TimeoutException
3328      * @return the result of the procedure, which may be the same as the passed one
3329      */
3330     protected V postOperationFailure(final IOException exception, final long deadlineTs)
3331         throws IOException, TimeoutException {
3332       throw exception;
3333     }
3334
3335     protected interface WaitForStateCallable {
3336       boolean checkState(int tries) throws IOException;
3337       void throwInterruptedException() throws InterruptedIOException;
3338       void throwTimeoutException(long elapsed) throws TimeoutException;
3339     }
3340
3341     protected void waitForState(final long deadlineTs, final WaitForStateCallable callable)
3342         throws IOException, TimeoutException {
3343       int tries = 0;
3344       IOException serverEx = null;
3345       long startTime = EnvironmentEdgeManager.currentTime();
3346       while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
3347         serverEx = null;
3348         try {
3349           if (callable.checkState(tries)) {
3350             return;
3351           }
3352         } catch (IOException e) {
3353           serverEx = e;
3354         }
3355         try {
3356           Thread.sleep(getAdmin().getPauseTime(tries++));
3357         } catch (InterruptedException e) {
3358           callable.throwInterruptedException();
3359         }
3360       }
3361       if (serverEx != null) {
3362         throw unwrapException(serverEx);
3363       } else {
3364         callable.throwTimeoutException(EnvironmentEdgeManager.currentTime() - startTime);
3365       }
3366     }
3367   }
3368
3369   @InterfaceAudience.Private
3370   @InterfaceStability.Evolving
3371   protected static abstract class TableFuture<V> extends ProcedureFuture<V> {
3372     private final TableName tableName;
3373
3374     public TableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) {
3375       super(admin, procId);
3376       this.tableName = tableName;
3377     }
3378
3379     @Override
3380     public String toString() {
3381       return getDescription();
3382     }
3383
3384     /**
3385      * @return the table name
3386      */
3387     protected TableName getTableName() {
3388       return tableName;
3389     }
3390
3391     /**
3392      * @return the table descriptor
3393      */
3394     protected HTableDescriptor getTableDescriptor() throws IOException {
3395       return getAdmin().getTableDescriptorByTableName(getTableName());
3396     }
3397
3398     /**
3399      * @return the operation type like CREATE, DELETE, DISABLE etc.
3400      */
3401     public abstract String getOperationType();
3402
3403     /**
3404      * @return a description of the operation
3405      */
3406     protected String getDescription() {
3407       return "Operation: " + getOperationType() + ", "
3408           + "Table Name: " + tableName.getNameWithNamespaceInclAsString();
3409
3410     };
3411
3412     protected abstract class TableWaitForStateCallable implements WaitForStateCallable {
3413       @Override
3414       public void throwInterruptedException() throws InterruptedIOException {
3415         throw new InterruptedIOException("Interrupted while waiting for operation: "
3416             + getOperationType() + " on table: " + tableName.getNameWithNamespaceInclAsString());
3417       }
3418
3419       @Override
3420       public void throwTimeoutException(long elapsedTime) throws TimeoutException {
3421         throw new TimeoutException("The operation: " + getOperationType() + " on table: " +
3422             tableName.getNameAsString() + " has not completed after " + elapsedTime + "ms");
3423       }
3424     }
3425
3426     @Override
3427     protected V postOperationResult(final V result, final long deadlineTs)
3428         throws IOException, TimeoutException {
3429       LOG.info(getDescription() + " completed");
3430       return super.postOperationResult(result, deadlineTs);
3431     }
3432
3433     @Override
3434     protected V postOperationFailure(final IOException exception, final long deadlineTs)
3435         throws IOException, TimeoutException {
3436       LOG.info(getDescription() + " failed with " + exception.getMessage());
3437       return super.postOperationFailure(exception, deadlineTs);
3438     }
3439
3440     protected void waitForTableEnabled(final long deadlineTs)
3441         throws IOException, TimeoutException {
3442       waitForState(deadlineTs, new TableWaitForStateCallable() {
3443         @Override
3444         public boolean checkState(int tries) throws IOException {
3445           try {
3446             if (getAdmin().isTableAvailable(tableName)) {
3447               return true;
3448             }
3449           } catch (TableNotFoundException tnfe) {
3450             LOG.debug("Table " + tableName.getNameWithNamespaceInclAsString()
3451                 + " was not enabled, sleeping. tries=" + tries);
3452           }
3453           return false;
3454         }
3455       });
3456     }
3457
3458     protected void waitForTableDisabled(final long deadlineTs)
3459         throws IOException, TimeoutException {
3460       waitForState(deadlineTs, new TableWaitForStateCallable() {
3461         @Override
3462         public boolean checkState(int tries) throws IOException {
3463           return getAdmin().isTableDisabled(tableName);
3464         }
3465       });
3466     }
3467
3468     protected void waitTableNotFound(final long deadlineTs)
3469         throws IOException, TimeoutException {
3470       waitForState(deadlineTs, new TableWaitForStateCallable() {
3471         @Override
3472         public boolean checkState(int tries) throws IOException {
3473           return !getAdmin().tableExists(tableName);
3474         }
3475       });
3476     }
3477
3478     protected void waitForSchemaUpdate(final long deadlineTs)
3479         throws IOException, TimeoutException {
3480       waitForState(deadlineTs, new TableWaitForStateCallable() {
3481         @Override
3482         public boolean checkState(int tries) throws IOException {
3483           return getAdmin().getAlterStatus(tableName).getFirst() == 0;
3484         }
3485       });
3486     }
3487
3488     protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
3489         throws IOException, TimeoutException {
3490       final HTableDescriptor desc = getTableDescriptor();
3491       final AtomicInteger actualRegCount = new AtomicInteger(0);
3492       final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
3493         @Override
3494         public boolean visit(Result rowResult) throws IOException {
3495           RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
3496           if (list == null) {
3497             LOG.warn("No serialized HRegionInfo in " + rowResult);
3498             return true;
3499           }
3500           HRegionLocation l = list.getRegionLocation();
3501           if (l == null) {
3502             return true;
3503           }
3504           if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
3505             return false;
3506           }
3507           if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
3508           HRegionLocation[] locations = list.getRegionLocations();
3509           for (HRegionLocation location : locations) {
3510             if (location == null) continue;
3511             ServerName serverName = location.getServerName();
3512             // Make sure that regions are assigned to server
3513             if (serverName != null && serverName.getHostAndPort() != null) {
3514               actualRegCount.incrementAndGet();
3515             }
3516           }
3517           return true;
3518         }
3519       };
3520
3521       int tries = 0;
3522       int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
3523       while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
3524         actualRegCount.set(0);
3525         MetaTableAccessor.scanMetaForTableRegions(getAdmin().getConnection(), visitor,
3526           desc.getTableName());
3527         if (actualRegCount.get() == numRegs) {
3528           // all the regions are online
3529           return;
3530         }
3531
3532         try {
3533           Thread.sleep(getAdmin().getPauseTime(tries++));
3534         } catch (InterruptedException e) {
3535           throw new InterruptedIOException("Interrupted when opening" + " regions; "
3536               + actualRegCount.get() + " of " + numRegs + " regions processed so far");
3537         }
3538       }
3539       throw new TimeoutException("Only " + actualRegCount.get() + " of " + numRegs
3540           + " regions are online; retries exhausted.");
3541     }
3542   }
3543
3544   @InterfaceAudience.Private
3545   @InterfaceStability.Evolving
3546   protected static abstract class NamespaceFuture extends ProcedureFuture<Void> {
3547     private final String namespaceName;
3548
3549     public NamespaceFuture(final HBaseAdmin admin, final String namespaceName, final Long procId) {
3550       super(admin, procId);
3551       this.namespaceName = namespaceName;
3552     }
3553
3554     /**
3555      * @return the namespace name
3556      */
3557     protected String getNamespaceName() {
3558       return namespaceName;
3559     }
3560
3561     /**
3562      * @return the operation type like CREATE_NAMESPACE, DELETE_NAMESPACE, etc.
3563      */
3564     public abstract String getOperationType();
3565
3566     @Override
3567     public String toString() {
3568       return "Operation: " + getOperationType() + ", Namespace: " + getNamespaceName();
3569     }
3570   }
3571
3572   @Override
3573   public List<SecurityCapability> getSecurityCapabilities() throws IOException {
3574     try {
3575       return executeCallable(new MasterCallable<List<SecurityCapability>>(getConnection(),
3576           getRpcControllerFactory()) {
3577         @Override
3578         protected List<SecurityCapability> rpcCall() throws Exception {
3579           SecurityCapabilitiesRequest req = SecurityCapabilitiesRequest.newBuilder().build();
3580           return ProtobufUtil.toSecurityCapabilityList(
3581             master.getSecurityCapabilities(getRpcController(), req).getCapabilitiesList());
3582         }
3583       });
3584     } catch (IOException e) {
3585       if (e instanceof RemoteException) {
3586         e = ((RemoteException)e).unwrapRemoteException();
3587       }
3588       throw e;
3589     }
3590   }
3591
3592   @Override
3593   public boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
3594                                           final MasterSwitchType... switchTypes)
3595     throws IOException {
3596     return executeCallable(new MasterCallable<boolean[]>(getConnection(),
3597         getRpcControllerFactory()) {
3598       @Override
3599       protected boolean[] rpcCall() throws Exception {
3600         MasterProtos.SetSplitOrMergeEnabledResponse response =
3601             master.setSplitOrMergeEnabled(getRpcController(),
3602                 RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous,
3603                     switchTypes));
3604         boolean[] result = new boolean[switchTypes.length];
3605         int i = 0;
3606         for (Boolean prevValue : response.getPrevValueList()) {
3607           result[i++] = prevValue;
3608         }
3609         return result;
3610       }
3611     });
3612   }
3613
3614   @Override
3615   public boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException {
3616     return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
3617       @Override
3618       protected Boolean rpcCall() throws Exception {
3619         return master.isSplitOrMergeEnabled(getRpcController(),
3620           RequestConverter.buildIsSplitOrMergeEnabledRequest(switchType)).getEnabled();
3621       }
3622     });
3623   }
3624
3625   private HRegionInfo getMobRegionInfo(TableName tableName) {
3626     return new HRegionInfo(tableName, Bytes.toBytes(".mob"),
3627             HConstants.EMPTY_END_ROW, false, 0);
3628   }
3629
3630   private RpcControllerFactory getRpcControllerFactory() {
3631     return this.rpcControllerFactory;
3632   }
3633 }