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