001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase; 019 020import java.io.IOException; 021import java.security.PrivilegedAction; 022import java.util.ArrayList; 023import java.util.HashSet; 024import java.util.List; 025import java.util.Set; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.fs.FileSystem; 028import org.apache.hadoop.hbase.client.RegionReplicaUtil; 029import org.apache.hadoop.hbase.master.HMaster; 030import org.apache.hadoop.hbase.regionserver.HRegion; 031import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult; 032import org.apache.hadoop.hbase.regionserver.HRegionServer; 033import org.apache.hadoop.hbase.regionserver.Region; 034import org.apache.hadoop.hbase.security.User; 035import org.apache.hadoop.hbase.test.MetricsAssertHelper; 036import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 037import org.apache.hadoop.hbase.util.JVMClusterUtil; 038import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; 039import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; 040import org.apache.hadoop.hbase.util.Threads; 041import org.apache.yetus.audience.InterfaceAudience; 042import org.slf4j.Logger; 043import org.slf4j.LoggerFactory; 044 045import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; 046import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; 047import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; 048import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; 049 050/** 051 * This class creates a single process HBase cluster. each server. The master uses the 'default' 052 * FileSystem. The RegionServers, if we are running on DistributedFilesystem, create a FileSystem 053 * instance each and will close down their instance on the way out. 054 */ 055@InterfaceAudience.Public 056public class MiniHBaseCluster extends HBaseCluster { 057 private static final Logger LOG = LoggerFactory.getLogger(MiniHBaseCluster.class.getName()); 058 public LocalHBaseCluster hbaseCluster; 059 private static int index; 060 061 /** 062 * Start a MiniHBaseCluster. 063 * @param conf Configuration to be used for cluster 064 * @param numRegionServers initial number of region servers to start. 065 */ 066 public MiniHBaseCluster(Configuration conf, int numRegionServers) 067 throws IOException, InterruptedException { 068 this(conf, 1, numRegionServers); 069 } 070 071 /** 072 * Start a MiniHBaseCluster. 073 * @param conf Configuration to be used for cluster 074 * @param numMasters initial number of masters to start. 075 * @param numRegionServers initial number of region servers to start. 076 */ 077 public MiniHBaseCluster(Configuration conf, int numMasters, int numRegionServers) 078 throws IOException, InterruptedException { 079 this(conf, numMasters, numRegionServers, null, null); 080 } 081 082 /** 083 * Start a MiniHBaseCluster. 084 * @param conf Configuration to be used for cluster 085 * @param numMasters initial number of masters to start. 086 * @param numRegionServers initial number of region servers to start. 087 */ 088 public MiniHBaseCluster(Configuration conf, int numMasters, int numRegionServers, 089 Class<? extends HMaster> masterClass, 090 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass) 091 throws IOException, InterruptedException { 092 this(conf, numMasters, 0, numRegionServers, null, masterClass, regionserverClass); 093 } 094 095 /** 096 * @param rsPorts Ports that RegionServer should use; pass ports if you want to test cluster 097 * restart where for sure the regionservers come up on same address+port (but just 098 * with different startcode); by default mini hbase clusters choose new arbitrary 099 * ports on each cluster start. 100 */ 101 public MiniHBaseCluster(Configuration conf, int numMasters, int numAlwaysStandByMasters, 102 int numRegionServers, List<Integer> rsPorts, Class<? extends HMaster> masterClass, 103 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass) 104 throws IOException, InterruptedException { 105 super(conf); 106 107 // Hadoop 2 108 CompatibilityFactory.getInstance(MetricsAssertHelper.class).init(); 109 110 init(numMasters, numAlwaysStandByMasters, numRegionServers, rsPorts, masterClass, 111 regionserverClass); 112 this.initialClusterStatus = getClusterMetrics(); 113 } 114 115 public Configuration getConfiguration() { 116 return this.conf; 117 } 118 119 /** 120 * Subclass so can get at protected methods (none at moment). Also, creates a FileSystem instance 121 * per instantiation. Adds a shutdown own FileSystem on the way out. Shuts down own Filesystem 122 * only, not All filesystems as the FileSystem system exit hook does. 123 */ 124 public static class MiniHBaseClusterRegionServer extends HRegionServer { 125 private Thread shutdownThread = null; 126 private User user = null; 127 /** 128 * List of RegionServers killed so far. ServerName also comprises startCode of a server, so any 129 * restarted instances of the same server will have different ServerName and will not coincide 130 * with past dead ones. So there's no need to cleanup this list. 131 */ 132 static Set<ServerName> killedServers = new HashSet<>(); 133 134 public MiniHBaseClusterRegionServer(Configuration conf) 135 throws IOException, InterruptedException { 136 super(conf); 137 this.user = User.getCurrent(); 138 } 139 140 /* 141 * @param currentfs We return this if we did not make a new one. 142 * @param uniqueName Same name used to help identify the created fs. 143 * @return A new fs instance if we are up on DistributeFileSystem. 144 */ 145 146 @Override 147 protected void handleReportForDutyResponse(final RegionServerStartupResponse c) 148 throws IOException { 149 super.handleReportForDutyResponse(c); 150 // Run this thread to shutdown our filesystem on way out. 151 this.shutdownThread = new SingleFileSystemShutdownThread(getFileSystem()); 152 } 153 154 @Override 155 public void run() { 156 try { 157 this.user.runAs(new PrivilegedAction<Object>() { 158 @Override 159 public Object run() { 160 runRegionServer(); 161 return null; 162 } 163 }); 164 } catch (Throwable t) { 165 LOG.error("Exception in run", t); 166 } finally { 167 // Run this on the way out. 168 if (this.shutdownThread != null) { 169 this.shutdownThread.start(); 170 Threads.shutdown(this.shutdownThread, 30000); 171 } 172 } 173 } 174 175 private void runRegionServer() { 176 super.run(); 177 } 178 179 @Override 180 protected void kill() { 181 killedServers.add(getServerName()); 182 super.kill(); 183 } 184 185 @Override 186 public void abort(final String reason, final Throwable cause) { 187 this.user.runAs(new PrivilegedAction<Object>() { 188 @Override 189 public Object run() { 190 abortRegionServer(reason, cause); 191 return null; 192 } 193 }); 194 } 195 196 private void abortRegionServer(String reason, Throwable cause) { 197 super.abort(reason, cause); 198 } 199 } 200 201 /** 202 * Alternate shutdown hook. Just shuts down the passed fs, not all as default filesystem hook 203 * does. 204 */ 205 static class SingleFileSystemShutdownThread extends Thread { 206 private final FileSystem fs; 207 208 SingleFileSystemShutdownThread(final FileSystem fs) { 209 super("Shutdown of " + fs); 210 this.fs = fs; 211 } 212 213 @Override 214 public void run() { 215 try { 216 LOG.info("Hook closing fs=" + this.fs); 217 this.fs.close(); 218 } catch (IOException e) { 219 LOG.warn("Running hook", e); 220 } 221 } 222 } 223 224 private void init(final int nMasterNodes, final int numAlwaysStandByMasters, 225 final int nRegionNodes, List<Integer> rsPorts, Class<? extends HMaster> masterClass, 226 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass) 227 throws IOException, InterruptedException { 228 try { 229 if (masterClass == null) { 230 masterClass = HMaster.class; 231 } 232 if (regionserverClass == null) { 233 regionserverClass = MiniHBaseCluster.MiniHBaseClusterRegionServer.class; 234 } 235 236 // start up a LocalHBaseCluster 237 hbaseCluster = new LocalHBaseCluster(conf, nMasterNodes, numAlwaysStandByMasters, 0, 238 masterClass, regionserverClass); 239 240 // manually add the regionservers as other users 241 for (int i = 0; i < nRegionNodes; i++) { 242 Configuration rsConf = HBaseConfiguration.create(conf); 243 if (rsPorts != null) { 244 rsConf.setInt(HConstants.REGIONSERVER_PORT, rsPorts.get(i)); 245 } 246 User user = HBaseTestingUtility.getDifferentUser(rsConf, ".hfs." + index++); 247 hbaseCluster.addRegionServer(rsConf, i, user); 248 } 249 250 hbaseCluster.startup(); 251 } catch (IOException e) { 252 shutdown(); 253 throw e; 254 } catch (Throwable t) { 255 LOG.error("Error starting cluster", t); 256 shutdown(); 257 throw new IOException("Shutting down", t); 258 } 259 } 260 261 @Override 262 public void startRegionServer(String hostname, int port) throws IOException { 263 final Configuration newConf = HBaseConfiguration.create(conf); 264 newConf.setInt(HConstants.REGIONSERVER_PORT, port); 265 startRegionServer(newConf); 266 } 267 268 @Override 269 public void killRegionServer(ServerName serverName) throws IOException { 270 HRegionServer server = getRegionServer(getRegionServerIndex(serverName)); 271 if (server instanceof MiniHBaseClusterRegionServer) { 272 LOG.info("Killing " + server.toString()); 273 ((MiniHBaseClusterRegionServer) server).kill(); 274 } else { 275 abortRegionServer(getRegionServerIndex(serverName)); 276 } 277 } 278 279 @Override 280 public boolean isKilledRS(ServerName serverName) { 281 return MiniHBaseClusterRegionServer.killedServers.contains(serverName); 282 } 283 284 @Override 285 public void stopRegionServer(ServerName serverName) throws IOException { 286 stopRegionServer(getRegionServerIndex(serverName)); 287 } 288 289 @Override 290 public void suspendRegionServer(ServerName serverName) throws IOException { 291 suspendRegionServer(getRegionServerIndex(serverName)); 292 } 293 294 @Override 295 public void resumeRegionServer(ServerName serverName) throws IOException { 296 resumeRegionServer(getRegionServerIndex(serverName)); 297 } 298 299 @Override 300 public void waitForRegionServerToStop(ServerName serverName, long timeout) throws IOException { 301 // ignore timeout for now 302 waitOnRegionServer(getRegionServerIndex(serverName)); 303 } 304 305 @Override 306 public void startZkNode(String hostname, int port) throws IOException { 307 LOG.warn("Starting zookeeper nodes on mini cluster is not supported"); 308 } 309 310 @Override 311 public void killZkNode(ServerName serverName) throws IOException { 312 LOG.warn("Aborting zookeeper nodes on mini cluster is not supported"); 313 } 314 315 @Override 316 public void stopZkNode(ServerName serverName) throws IOException { 317 LOG.warn("Stopping zookeeper nodes on mini cluster is not supported"); 318 } 319 320 @Override 321 public void waitForZkNodeToStart(ServerName serverName, long timeout) throws IOException { 322 LOG.warn("Waiting for zookeeper nodes to start on mini cluster is not supported"); 323 } 324 325 @Override 326 public void waitForZkNodeToStop(ServerName serverName, long timeout) throws IOException { 327 LOG.warn("Waiting for zookeeper nodes to stop on mini cluster is not supported"); 328 } 329 330 @Override 331 public void startDataNode(ServerName serverName) throws IOException { 332 LOG.warn("Starting datanodes on mini cluster is not supported"); 333 } 334 335 @Override 336 public void killDataNode(ServerName serverName) throws IOException { 337 LOG.warn("Aborting datanodes on mini cluster is not supported"); 338 } 339 340 @Override 341 public void stopDataNode(ServerName serverName) throws IOException { 342 LOG.warn("Stopping datanodes on mini cluster is not supported"); 343 } 344 345 @Override 346 public void waitForDataNodeToStart(ServerName serverName, long timeout) throws IOException { 347 LOG.warn("Waiting for datanodes to start on mini cluster is not supported"); 348 } 349 350 @Override 351 public void waitForDataNodeToStop(ServerName serverName, long timeout) throws IOException { 352 LOG.warn("Waiting for datanodes to stop on mini cluster is not supported"); 353 } 354 355 @Override 356 public void startNameNode(ServerName serverName) throws IOException { 357 LOG.warn("Starting namenodes on mini cluster is not supported"); 358 } 359 360 @Override 361 public void killNameNode(ServerName serverName) throws IOException { 362 LOG.warn("Aborting namenodes on mini cluster is not supported"); 363 } 364 365 @Override 366 public void stopNameNode(ServerName serverName) throws IOException { 367 LOG.warn("Stopping namenodes on mini cluster is not supported"); 368 } 369 370 @Override 371 public void waitForNameNodeToStart(ServerName serverName, long timeout) throws IOException { 372 LOG.warn("Waiting for namenodes to start on mini cluster is not supported"); 373 } 374 375 @Override 376 public void waitForNameNodeToStop(ServerName serverName, long timeout) throws IOException { 377 LOG.warn("Waiting for namenodes to stop on mini cluster is not supported"); 378 } 379 380 @Override 381 public void startJournalNode(ServerName serverName) { 382 LOG.warn("Starting journalnodes on mini cluster is not supported"); 383 } 384 385 @Override 386 public void killJournalNode(ServerName serverName) { 387 LOG.warn("Aborting journalnodes on mini cluster is not supported"); 388 } 389 390 @Override 391 public void stopJournalNode(ServerName serverName) { 392 LOG.warn("Stopping journalnodes on mini cluster is not supported"); 393 } 394 395 @Override 396 public void waitForJournalNodeToStart(ServerName serverName, long timeout) { 397 LOG.warn("Waiting for journalnodes to start on mini cluster is not supported"); 398 } 399 400 @Override 401 public void waitForJournalNodeToStop(ServerName serverName, long timeout) { 402 LOG.warn("Waiting for journalnodes to stop on mini cluster is not supported"); 403 } 404 405 @Override 406 public void startMaster(String hostname, int port) throws IOException { 407 this.startMaster(); 408 } 409 410 @Override 411 public void killMaster(ServerName serverName) throws IOException { 412 abortMaster(getMasterIndex(serverName)); 413 } 414 415 @Override 416 public void stopMaster(ServerName serverName) throws IOException { 417 stopMaster(getMasterIndex(serverName)); 418 } 419 420 @Override 421 public void waitForMasterToStop(ServerName serverName, long timeout) throws IOException { 422 // ignore timeout for now 423 waitOnMaster(getMasterIndex(serverName)); 424 } 425 426 /** 427 * Starts a region server thread running 428 * @return New RegionServerThread 429 */ 430 public JVMClusterUtil.RegionServerThread startRegionServer() throws IOException { 431 final Configuration newConf = HBaseConfiguration.create(conf); 432 return startRegionServer(newConf); 433 } 434 435 private JVMClusterUtil.RegionServerThread startRegionServer(Configuration configuration) 436 throws IOException { 437 User rsUser = HBaseTestingUtility.getDifferentUser(configuration, ".hfs." + index++); 438 JVMClusterUtil.RegionServerThread t = null; 439 try { 440 t = 441 hbaseCluster.addRegionServer(configuration, hbaseCluster.getRegionServers().size(), rsUser); 442 t.start(); 443 t.waitForServerOnline(); 444 } catch (InterruptedException ie) { 445 throw new IOException("Interrupted adding regionserver to cluster", ie); 446 } 447 return t; 448 } 449 450 /** 451 * Starts a region server thread and waits until its processed by master. Throws an exception when 452 * it can't start a region server or when the region server is not processed by master within the 453 * timeout. 454 * @return New RegionServerThread 455 */ 456 public JVMClusterUtil.RegionServerThread startRegionServerAndWait(long timeout) 457 throws IOException { 458 459 JVMClusterUtil.RegionServerThread t = startRegionServer(); 460 ServerName rsServerName = t.getRegionServer().getServerName(); 461 462 long start = EnvironmentEdgeManager.currentTime(); 463 ClusterStatus clusterStatus = getClusterStatus(); 464 while ((EnvironmentEdgeManager.currentTime() - start) < timeout) { 465 if (clusterStatus != null && clusterStatus.getServers().contains(rsServerName)) { 466 return t; 467 } 468 Threads.sleep(100); 469 } 470 if (t.getRegionServer().isOnline()) { 471 throw new IOException("RS: " + rsServerName + " online, but not processed by master"); 472 } else { 473 throw new IOException("RS: " + rsServerName + " is offline"); 474 } 475 } 476 477 /** 478 * Cause a region server to exit doing basic clean up only on its way out. 479 * @param serverNumber Used as index into a list. 480 */ 481 public String abortRegionServer(int serverNumber) { 482 HRegionServer server = getRegionServer(serverNumber); 483 LOG.info("Aborting " + server.toString()); 484 server.abort("Aborting for tests", new Exception("Trace info")); 485 return server.toString(); 486 } 487 488 /** 489 * Shut down the specified region server cleanly 490 * @param serverNumber Used as index into a list. 491 * @return the region server that was stopped 492 */ 493 public JVMClusterUtil.RegionServerThread stopRegionServer(int serverNumber) { 494 return stopRegionServer(serverNumber, true); 495 } 496 497 /** 498 * Shut down the specified region server cleanly 499 * @param serverNumber Used as index into a list. 500 * @param shutdownFS True is we are to shutdown the filesystem as part of this regionserver's 501 * shutdown. Usually we do but you do not want to do this if you are running 502 * multiple regionservers in a test and you shut down one before end of the 503 * test. 504 * @return the region server that was stopped 505 */ 506 public JVMClusterUtil.RegionServerThread stopRegionServer(int serverNumber, 507 final boolean shutdownFS) { 508 JVMClusterUtil.RegionServerThread server = hbaseCluster.getRegionServers().get(serverNumber); 509 LOG.info("Stopping " + server.toString()); 510 server.getRegionServer().stop("Stopping rs " + serverNumber); 511 return server; 512 } 513 514 /** 515 * Suspend the specified region server 516 * @param serverNumber Used as index into a list. 517 */ 518 public JVMClusterUtil.RegionServerThread suspendRegionServer(int serverNumber) { 519 JVMClusterUtil.RegionServerThread server = hbaseCluster.getRegionServers().get(serverNumber); 520 LOG.info("Suspending {}", server.toString()); 521 server.suspend(); 522 return server; 523 } 524 525 /** 526 * Resume the specified region server 527 * @param serverNumber Used as index into a list. 528 */ 529 public JVMClusterUtil.RegionServerThread resumeRegionServer(int serverNumber) { 530 JVMClusterUtil.RegionServerThread server = hbaseCluster.getRegionServers().get(serverNumber); 531 LOG.info("Resuming {}", server.toString()); 532 server.resume(); 533 return server; 534 } 535 536 /** 537 * Wait for the specified region server to stop. Removes this thread from list of running threads. 538 * @return Name of region server that just went down. 539 */ 540 public String waitOnRegionServer(final int serverNumber) { 541 return this.hbaseCluster.waitOnRegionServer(serverNumber); 542 } 543 544 /** 545 * Starts a master thread running 546 * @return New RegionServerThread 547 */ 548 public JVMClusterUtil.MasterThread startMaster() throws IOException { 549 Configuration c = HBaseConfiguration.create(conf); 550 User user = HBaseTestingUtility.getDifferentUser(c, ".hfs." + index++); 551 552 JVMClusterUtil.MasterThread t = null; 553 try { 554 t = hbaseCluster.addMaster(c, hbaseCluster.getMasters().size(), user); 555 t.start(); 556 } catch (InterruptedException ie) { 557 throw new IOException("Interrupted adding master to cluster", ie); 558 } 559 conf.set(HConstants.MASTER_ADDRS_KEY, 560 hbaseCluster.getConfiguration().get(HConstants.MASTER_ADDRS_KEY)); 561 return t; 562 } 563 564 /** 565 * Returns the current active master, if available. 566 * @return the active HMaster, null if none is active. 567 */ 568 @Override 569 public MasterService.BlockingInterface getMasterAdminService() { 570 return this.hbaseCluster.getActiveMaster().getMasterRpcServices(); 571 } 572 573 /** 574 * Returns the current active master, if available. 575 * @return the active HMaster, null if none is active. 576 */ 577 public HMaster getMaster() { 578 return this.hbaseCluster.getActiveMaster(); 579 } 580 581 /** 582 * Returns the current active master thread, if available. 583 * @return the active MasterThread, null if none is active. 584 */ 585 public MasterThread getMasterThread() { 586 for (MasterThread mt : hbaseCluster.getLiveMasters()) { 587 if (mt.getMaster().isActiveMaster()) { 588 return mt; 589 } 590 } 591 return null; 592 } 593 594 /** 595 * Returns the master at the specified index, if available. 596 * @return the active HMaster, null if none is active. 597 */ 598 public HMaster getMaster(final int serverNumber) { 599 return this.hbaseCluster.getMaster(serverNumber); 600 } 601 602 /** 603 * Cause a master to exit without shutting down entire cluster. 604 * @param serverNumber Used as index into a list. 605 */ 606 public String abortMaster(int serverNumber) { 607 HMaster server = getMaster(serverNumber); 608 LOG.info("Aborting " + server.toString()); 609 server.abort("Aborting for tests", new Exception("Trace info")); 610 return server.toString(); 611 } 612 613 /** 614 * Shut down the specified master cleanly 615 * @param serverNumber Used as index into a list. 616 * @return the region server that was stopped 617 */ 618 public JVMClusterUtil.MasterThread stopMaster(int serverNumber) { 619 return stopMaster(serverNumber, true); 620 } 621 622 /** 623 * Shut down the specified master cleanly 624 * @param serverNumber Used as index into a list. 625 * @param shutdownFS True is we are to shutdown the filesystem as part of this master's 626 * shutdown. Usually we do but you do not want to do this if you are running 627 * multiple master in a test and you shut down one before end of the test. 628 * @return the master that was stopped 629 */ 630 public JVMClusterUtil.MasterThread stopMaster(int serverNumber, final boolean shutdownFS) { 631 JVMClusterUtil.MasterThread server = hbaseCluster.getMasters().get(serverNumber); 632 LOG.info("Stopping " + server.toString()); 633 server.getMaster().stop("Stopping master " + serverNumber); 634 return server; 635 } 636 637 /** 638 * Wait for the specified master to stop. Removes this thread from list of running threads. 639 * @return Name of master that just went down. 640 */ 641 public String waitOnMaster(final int serverNumber) { 642 return this.hbaseCluster.waitOnMaster(serverNumber); 643 } 644 645 /** 646 * Blocks until there is an active master and that master has completed initialization. 647 * @return true if an active master becomes available. false if there are no masters left. 648 */ 649 @Override 650 public boolean waitForActiveAndReadyMaster(long timeout) throws IOException { 651 List<JVMClusterUtil.MasterThread> mts; 652 long start = EnvironmentEdgeManager.currentTime(); 653 while ( 654 !(mts = getMasterThreads()).isEmpty() 655 && (EnvironmentEdgeManager.currentTime() - start) < timeout 656 ) { 657 for (JVMClusterUtil.MasterThread mt : mts) { 658 if (mt.getMaster().isActiveMaster() && mt.getMaster().isInitialized()) { 659 return true; 660 } 661 } 662 663 Threads.sleep(100); 664 } 665 return false; 666 } 667 668 /** Returns List of master threads. */ 669 public List<JVMClusterUtil.MasterThread> getMasterThreads() { 670 return this.hbaseCluster.getMasters(); 671 } 672 673 /** Returns List of live master threads (skips the aborted and the killed) */ 674 public List<JVMClusterUtil.MasterThread> getLiveMasterThreads() { 675 return this.hbaseCluster.getLiveMasters(); 676 } 677 678 /** 679 * Wait for Mini HBase Cluster to shut down. 680 */ 681 public void join() { 682 this.hbaseCluster.join(); 683 } 684 685 /** 686 * Shut down the mini HBase cluster 687 */ 688 @Override 689 public void shutdown() throws IOException { 690 if (this.hbaseCluster != null) { 691 this.hbaseCluster.shutdown(); 692 } 693 } 694 695 @Override 696 public void close() throws IOException { 697 } 698 699 /** 700 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use 701 * {@link #getClusterMetrics()} instead. 702 */ 703 @Deprecated 704 public ClusterStatus getClusterStatus() throws IOException { 705 HMaster master = getMaster(); 706 return master == null ? null : new ClusterStatus(master.getClusterMetrics()); 707 } 708 709 @Override 710 public ClusterMetrics getClusterMetrics() throws IOException { 711 HMaster master = getMaster(); 712 return master == null ? null : master.getClusterMetrics(); 713 } 714 715 private void executeFlush(HRegion region) throws IOException { 716 if (!RegionReplicaUtil.isDefaultReplica(region.getRegionInfo())) { 717 return; 718 } 719 // retry 5 times if we can not flush 720 for (int i = 0; i < 5; i++) { 721 FlushResult result = region.flush(true); 722 if (result.getResult() != FlushResult.Result.CANNOT_FLUSH) { 723 return; 724 } 725 Threads.sleep(1000); 726 } 727 } 728 729 /** 730 * Call flushCache on all regions on all participating regionservers. 731 */ 732 public void flushcache() throws IOException { 733 for (JVMClusterUtil.RegionServerThread t : this.hbaseCluster.getRegionServers()) { 734 for (HRegion r : t.getRegionServer().getOnlineRegionsLocalContext()) { 735 executeFlush(r); 736 } 737 } 738 } 739 740 /** 741 * Call flushCache on all regions of the specified table. 742 */ 743 public void flushcache(TableName tableName) throws IOException { 744 for (JVMClusterUtil.RegionServerThread t : this.hbaseCluster.getRegionServers()) { 745 for (HRegion r : t.getRegionServer().getOnlineRegionsLocalContext()) { 746 if (r.getTableDescriptor().getTableName().equals(tableName)) { 747 executeFlush(r); 748 } 749 } 750 } 751 } 752 753 /** 754 * Call flushCache on all regions on all participating regionservers. 755 */ 756 public void compact(boolean major) throws IOException { 757 for (JVMClusterUtil.RegionServerThread t : this.hbaseCluster.getRegionServers()) { 758 for (HRegion r : t.getRegionServer().getOnlineRegionsLocalContext()) { 759 if (RegionReplicaUtil.isDefaultReplica(r.getRegionInfo())) { 760 r.compact(major); 761 } 762 } 763 } 764 } 765 766 /** 767 * Call flushCache on all regions of the specified table. 768 */ 769 public void compact(TableName tableName, boolean major) throws IOException { 770 for (JVMClusterUtil.RegionServerThread t : this.hbaseCluster.getRegionServers()) { 771 for (HRegion r : t.getRegionServer().getOnlineRegionsLocalContext()) { 772 if (r.getTableDescriptor().getTableName().equals(tableName)) { 773 if (RegionReplicaUtil.isDefaultReplica(r.getRegionInfo())) { 774 r.compact(major); 775 } 776 } 777 } 778 } 779 } 780 781 /** Returns Number of live region servers in the cluster currently. */ 782 public int getNumLiveRegionServers() { 783 return this.hbaseCluster.getLiveRegionServers().size(); 784 } 785 786 /** 787 * @return List of region server threads. Does not return the master even though it is also a 788 * region server. 789 */ 790 public List<JVMClusterUtil.RegionServerThread> getRegionServerThreads() { 791 return this.hbaseCluster.getRegionServers(); 792 } 793 794 /** Returns List of live region server threads (skips the aborted and the killed) */ 795 public List<JVMClusterUtil.RegionServerThread> getLiveRegionServerThreads() { 796 return this.hbaseCluster.getLiveRegionServers(); 797 } 798 799 /** 800 * Grab a numbered region server of your choice. 801 * @return region server 802 */ 803 public HRegionServer getRegionServer(int serverNumber) { 804 return hbaseCluster.getRegionServer(serverNumber); 805 } 806 807 public HRegionServer getRegionServer(ServerName serverName) { 808 return hbaseCluster.getRegionServers().stream().map(t -> t.getRegionServer()) 809 .filter(r -> r.getServerName().equals(serverName)).findFirst().orElse(null); 810 } 811 812 public List<HRegion> getRegions(byte[] tableName) { 813 return getRegions(TableName.valueOf(tableName)); 814 } 815 816 public List<HRegion> getRegions(TableName tableName) { 817 List<HRegion> ret = new ArrayList<>(); 818 for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) { 819 HRegionServer hrs = rst.getRegionServer(); 820 for (Region region : hrs.getOnlineRegionsLocalContext()) { 821 if (region.getTableDescriptor().getTableName().equals(tableName)) { 822 ret.add((HRegion) region); 823 } 824 } 825 } 826 return ret; 827 } 828 829 /** 830 * @return Index into List of {@link MiniHBaseCluster#getRegionServerThreads()} of HRS carrying 831 * regionName. Returns -1 if none found. 832 */ 833 public int getServerWithMeta() { 834 return getServerWith(HRegionInfo.FIRST_META_REGIONINFO.getRegionName()); 835 } 836 837 /** 838 * Get the location of the specified region 839 * @param regionName Name of the region in bytes 840 * @return Index into List of {@link MiniHBaseCluster#getRegionServerThreads()} of HRS carrying 841 * hbase:meta. Returns -1 if none found. 842 */ 843 public int getServerWith(byte[] regionName) { 844 int index = -1; 845 int count = 0; 846 for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) { 847 HRegionServer hrs = rst.getRegionServer(); 848 if (!hrs.isStopped()) { 849 Region region = hrs.getOnlineRegion(regionName); 850 if (region != null) { 851 index = count; 852 break; 853 } 854 } 855 count++; 856 } 857 return index; 858 } 859 860 @Override 861 public ServerName getServerHoldingRegion(final TableName tn, byte[] regionName) 862 throws IOException { 863 // Assume there is only one master thread which is the active master. 864 // If there are multiple master threads, the backup master threads 865 // should hold some regions. Please refer to #countServedRegions 866 // to see how we find out all regions. 867 HMaster master = getMaster(); 868 Region region = master.getOnlineRegion(regionName); 869 if (region != null) { 870 return master.getServerName(); 871 } 872 int index = getServerWith(regionName); 873 if (index < 0) { 874 return null; 875 } 876 return getRegionServer(index).getServerName(); 877 } 878 879 /** 880 * Counts the total numbers of regions being served by the currently online region servers by 881 * asking each how many regions they have. Does not look at hbase:meta at all. Count includes 882 * catalog tables. 883 * @return number of regions being served by all region servers 884 */ 885 public long countServedRegions() { 886 long count = 0; 887 for (JVMClusterUtil.RegionServerThread rst : getLiveRegionServerThreads()) { 888 count += rst.getRegionServer().getNumberOfOnlineRegions(); 889 } 890 for (JVMClusterUtil.MasterThread mt : getLiveMasterThreads()) { 891 count += mt.getMaster().getNumberOfOnlineRegions(); 892 } 893 return count; 894 } 895 896 /** 897 * Do a simulated kill all masters and regionservers. Useful when it is impossible to bring the 898 * mini-cluster back for clean shutdown. 899 */ 900 public void killAll() { 901 // Do backups first. 902 MasterThread activeMaster = null; 903 for (MasterThread masterThread : getMasterThreads()) { 904 if (!masterThread.getMaster().isActiveMaster()) { 905 masterThread.getMaster().abort("killAll"); 906 } else { 907 activeMaster = masterThread; 908 } 909 } 910 // Do active after. 911 if (activeMaster != null) { 912 activeMaster.getMaster().abort("killAll"); 913 } 914 for (RegionServerThread rst : getRegionServerThreads()) { 915 rst.getRegionServer().abort("killAll"); 916 } 917 } 918 919 @Override 920 public void waitUntilShutDown() { 921 this.hbaseCluster.join(); 922 } 923 924 public List<HRegion> findRegionsForTable(TableName tableName) { 925 ArrayList<HRegion> ret = new ArrayList<>(); 926 for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) { 927 HRegionServer hrs = rst.getRegionServer(); 928 for (Region region : hrs.getRegions(tableName)) { 929 if (region.getTableDescriptor().getTableName().equals(tableName)) { 930 ret.add((HRegion) region); 931 } 932 } 933 } 934 return ret; 935 } 936 937 protected int getRegionServerIndex(ServerName serverName) { 938 // we have a small number of region servers, this should be fine for now. 939 List<RegionServerThread> servers = getRegionServerThreads(); 940 for (int i = 0; i < servers.size(); i++) { 941 if (servers.get(i).getRegionServer().getServerName().equals(serverName)) { 942 return i; 943 } 944 } 945 return -1; 946 } 947 948 protected int getMasterIndex(ServerName serverName) { 949 List<MasterThread> masters = getMasterThreads(); 950 for (int i = 0; i < masters.size(); i++) { 951 if (masters.get(i).getMaster().getServerName().equals(serverName)) { 952 return i; 953 } 954 } 955 return -1; 956 } 957 958 @Override 959 public AdminService.BlockingInterface getAdminProtocol(ServerName serverName) throws IOException { 960 return getRegionServer(getRegionServerIndex(serverName)).getRSRpcServices(); 961 } 962 963 @Override 964 public ClientService.BlockingInterface getClientProtocol(ServerName serverName) 965 throws IOException { 966 return getRegionServer(getRegionServerIndex(serverName)).getRSRpcServices(); 967 } 968}