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