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 static org.junit.Assert.assertEquals; 021import static org.junit.Assert.assertTrue; 022import static org.junit.Assert.fail; 023 024import edu.umd.cs.findbugs.annotations.Nullable; 025import java.io.File; 026import java.io.IOException; 027import java.io.OutputStream; 028import java.lang.reflect.Field; 029import java.lang.reflect.Modifier; 030import java.net.BindException; 031import java.net.DatagramSocket; 032import java.net.InetAddress; 033import java.net.ServerSocket; 034import java.net.Socket; 035import java.net.UnknownHostException; 036import java.nio.charset.StandardCharsets; 037import java.security.MessageDigest; 038import java.util.ArrayList; 039import java.util.Arrays; 040import java.util.Collection; 041import java.util.Collections; 042import java.util.HashSet; 043import java.util.Iterator; 044import java.util.List; 045import java.util.Map; 046import java.util.NavigableSet; 047import java.util.Properties; 048import java.util.Random; 049import java.util.Set; 050import java.util.TreeSet; 051import java.util.concurrent.TimeUnit; 052import java.util.concurrent.atomic.AtomicReference; 053import java.util.stream.Collectors; 054import java.util.function.BooleanSupplier; 055import org.apache.commons.io.FileUtils; 056import org.apache.commons.lang3.RandomStringUtils; 057import org.apache.hadoop.conf.Configuration; 058import org.apache.hadoop.fs.FileSystem; 059import org.apache.hadoop.fs.Path; 060import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; 061import org.apache.hadoop.hbase.Waiter.Predicate; 062import org.apache.hadoop.hbase.client.Admin; 063import org.apache.hadoop.hbase.client.BufferedMutator; 064import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 065import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 066import org.apache.hadoop.hbase.client.Connection; 067import org.apache.hadoop.hbase.client.ConnectionFactory; 068import org.apache.hadoop.hbase.client.Consistency; 069import org.apache.hadoop.hbase.client.Delete; 070import org.apache.hadoop.hbase.client.Durability; 071import org.apache.hadoop.hbase.client.Get; 072import org.apache.hadoop.hbase.client.HBaseAdmin; 073import org.apache.hadoop.hbase.client.Hbck; 074import org.apache.hadoop.hbase.client.ImmutableHRegionInfo; 075import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor; 076import org.apache.hadoop.hbase.client.MasterRegistry; 077import org.apache.hadoop.hbase.client.Put; 078import org.apache.hadoop.hbase.client.RegionInfo; 079import org.apache.hadoop.hbase.client.RegionInfoBuilder; 080import org.apache.hadoop.hbase.client.RegionLocator; 081import org.apache.hadoop.hbase.client.Result; 082import org.apache.hadoop.hbase.client.ResultScanner; 083import org.apache.hadoop.hbase.client.Scan; 084import org.apache.hadoop.hbase.client.Table; 085import org.apache.hadoop.hbase.client.TableDescriptor; 086import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 087import org.apache.hadoop.hbase.client.TableState; 088import org.apache.hadoop.hbase.fs.HFileSystem; 089import org.apache.hadoop.hbase.io.compress.Compression; 090import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; 091import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; 092import org.apache.hadoop.hbase.io.hfile.BlockCache; 093import org.apache.hadoop.hbase.io.hfile.ChecksumUtil; 094import org.apache.hadoop.hbase.io.hfile.HFile; 095import org.apache.hadoop.hbase.ipc.RpcServerInterface; 096import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; 097import org.apache.hadoop.hbase.logging.Log4jUtils; 098import org.apache.hadoop.hbase.mapreduce.MapreduceTestingShim; 099import org.apache.hadoop.hbase.master.HMaster; 100import org.apache.hadoop.hbase.master.RegionState; 101import org.apache.hadoop.hbase.master.ServerManager; 102import org.apache.hadoop.hbase.master.assignment.AssignmentManager; 103import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; 104import org.apache.hadoop.hbase.master.assignment.RegionStateStore; 105import org.apache.hadoop.hbase.master.assignment.RegionStates; 106import org.apache.hadoop.hbase.mob.MobFileCache; 107import org.apache.hadoop.hbase.regionserver.BloomType; 108import org.apache.hadoop.hbase.regionserver.ChunkCreator; 109import org.apache.hadoop.hbase.regionserver.HRegion; 110import org.apache.hadoop.hbase.regionserver.HRegionServer; 111import org.apache.hadoop.hbase.regionserver.HStore; 112import org.apache.hadoop.hbase.regionserver.InternalScanner; 113import org.apache.hadoop.hbase.regionserver.MemStoreLAB; 114import org.apache.hadoop.hbase.regionserver.Region; 115import org.apache.hadoop.hbase.regionserver.RegionScanner; 116import org.apache.hadoop.hbase.regionserver.RegionServerServices; 117import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; 118import org.apache.hadoop.hbase.security.HBaseKerberosUtils; 119import org.apache.hadoop.hbase.security.User; 120import org.apache.hadoop.hbase.security.visibility.VisibilityLabelsCache; 121import org.apache.hadoop.hbase.trace.TraceUtil; 122import org.apache.hadoop.hbase.util.Bytes; 123import org.apache.hadoop.hbase.util.CommonFSUtils; 124import org.apache.hadoop.hbase.util.FSTableDescriptors; 125import org.apache.hadoop.hbase.util.FSUtils; 126import org.apache.hadoop.hbase.util.JVMClusterUtil; 127import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; 128import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; 129import org.apache.hadoop.hbase.util.Pair; 130import org.apache.hadoop.hbase.util.RegionSplitter; 131import org.apache.hadoop.hbase.util.RegionSplitter.SplitAlgorithm; 132import org.apache.hadoop.hbase.util.RetryCounter; 133import org.apache.hadoop.hbase.util.Threads; 134import org.apache.hadoop.hbase.wal.WAL; 135import org.apache.hadoop.hbase.wal.WALFactory; 136import org.apache.hadoop.hbase.zookeeper.EmptyWatcher; 137import org.apache.hadoop.hbase.zookeeper.ZKConfig; 138import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 139import org.apache.hadoop.hdfs.DFSClient; 140import org.apache.hadoop.hdfs.DistributedFileSystem; 141import org.apache.hadoop.hdfs.MiniDFSCluster; 142import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream; 143import org.apache.hadoop.mapred.JobConf; 144import org.apache.hadoop.mapred.MiniMRCluster; 145import org.apache.hadoop.mapred.TaskLog; 146import org.apache.hadoop.minikdc.MiniKdc; 147import org.apache.yetus.audience.InterfaceAudience; 148import org.apache.zookeeper.WatchedEvent; 149import org.apache.zookeeper.ZooKeeper; 150import org.apache.zookeeper.ZooKeeper.States; 151 152import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 153import org.apache.hbase.thirdparty.com.google.common.io.Closeables; 154import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 155 156/** 157 * Facility for testing HBase. Replacement for 158 * old HBaseTestCase and HBaseClusterTestCase functionality. 159 * Create an instance and keep it around testing HBase. This class is 160 * meant to be your one-stop shop for anything you might need testing. Manages 161 * one cluster at a time only. Managed cluster can be an in-process 162 * {@link MiniHBaseCluster}, or a deployed cluster of type {@code DistributedHBaseCluster}. 163 * Not all methods work with the real cluster. 164 * Depends on log4j being on classpath and 165 * hbase-site.xml for logging and test-run configuration. It does not set 166 * logging levels. 167 * In the configuration properties, default values for master-info-port and 168 * region-server-port are overridden such that a random port will be assigned (thus 169 * avoiding port contention if another local HBase instance is already running). 170 * <p>To preserve test data directories, pass the system property "hbase.testing.preserve.testdir" 171 * setting it to true. 172 */ 173@InterfaceAudience.Public 174@SuppressWarnings("deprecation") 175public class HBaseTestingUtility extends HBaseZKTestingUtility { 176 177 /** 178 * System property key to get test directory value. Name is as it is because mini dfs has 179 * hard-codings to put test data here. It should NOT be used directly in HBase, as it's a property 180 * used in mini dfs. 181 * @deprecated since 2.0.0 and will be removed in 3.0.0. Can be used only with mini dfs. 182 * @see <a href="https://issues.apache.org/jira/browse/HBASE-19410">HBASE-19410</a> 183 */ 184 @Deprecated 185 private static final String TEST_DIRECTORY_KEY = "test.build.data"; 186 187 public static final String REGIONS_PER_SERVER_KEY = "hbase.test.regions-per-server"; 188 /** 189 * The default number of regions per regionserver when creating a pre-split 190 * table. 191 */ 192 public static final int DEFAULT_REGIONS_PER_SERVER = 3; 193 194 195 public static final String PRESPLIT_TEST_TABLE_KEY = "hbase.test.pre-split-table"; 196 public static final boolean PRESPLIT_TEST_TABLE = true; 197 198 private MiniDFSCluster dfsCluster = null; 199 200 private volatile HBaseCluster hbaseCluster = null; 201 private MiniMRCluster mrCluster = null; 202 203 /** If there is a mini cluster running for this testing utility instance. */ 204 private volatile boolean miniClusterRunning; 205 206 private String hadoopLogDir; 207 208 /** Directory on test filesystem where we put the data for this instance of 209 * HBaseTestingUtility*/ 210 private Path dataTestDirOnTestFS = null; 211 212 /** 213 * Shared cluster connection. 214 */ 215 private volatile Connection connection; 216 217 /** Filesystem URI used for map-reduce mini-cluster setup */ 218 private static String FS_URI; 219 220 /** This is for unit tests parameterized with a single boolean. */ 221 public static final List<Object[]> MEMSTORETS_TAGS_PARAMETRIZED = memStoreTSAndTagsCombination(); 222 223 /** 224 * Checks to see if a specific port is available. 225 * 226 * @param port the port number to check for availability 227 * @return <tt>true</tt> if the port is available, or <tt>false</tt> if not 228 */ 229 public static boolean available(int port) { 230 ServerSocket ss = null; 231 DatagramSocket ds = null; 232 try { 233 ss = new ServerSocket(port); 234 ss.setReuseAddress(true); 235 ds = new DatagramSocket(port); 236 ds.setReuseAddress(true); 237 return true; 238 } catch (IOException e) { 239 // Do nothing 240 } finally { 241 if (ds != null) { 242 ds.close(); 243 } 244 245 if (ss != null) { 246 try { 247 ss.close(); 248 } catch (IOException e) { 249 /* should not be thrown */ 250 } 251 } 252 } 253 254 return false; 255 } 256 257 /** 258 * Create all combinations of Bloom filters and compression algorithms for 259 * testing. 260 */ 261 private static List<Object[]> bloomAndCompressionCombinations() { 262 List<Object[]> configurations = new ArrayList<>(); 263 for (Compression.Algorithm comprAlgo : 264 HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS) { 265 for (BloomType bloomType : BloomType.values()) { 266 configurations.add(new Object[] { comprAlgo, bloomType }); 267 } 268 } 269 return Collections.unmodifiableList(configurations); 270 } 271 272 /** 273 * Create combination of memstoreTS and tags 274 */ 275 private static List<Object[]> memStoreTSAndTagsCombination() { 276 List<Object[]> configurations = new ArrayList<>(); 277 configurations.add(new Object[] { false, false }); 278 configurations.add(new Object[] { false, true }); 279 configurations.add(new Object[] { true, false }); 280 configurations.add(new Object[] { true, true }); 281 return Collections.unmodifiableList(configurations); 282 } 283 284 public static List<Object[]> memStoreTSTagsAndOffheapCombination() { 285 List<Object[]> configurations = new ArrayList<>(); 286 configurations.add(new Object[] { false, false, true }); 287 configurations.add(new Object[] { false, false, false }); 288 configurations.add(new Object[] { false, true, true }); 289 configurations.add(new Object[] { false, true, false }); 290 configurations.add(new Object[] { true, false, true }); 291 configurations.add(new Object[] { true, false, false }); 292 configurations.add(new Object[] { true, true, true }); 293 configurations.add(new Object[] { true, true, false }); 294 return Collections.unmodifiableList(configurations); 295 } 296 297 public static final Collection<Object[]> BLOOM_AND_COMPRESSION_COMBINATIONS = 298 bloomAndCompressionCombinations(); 299 300 301 /** 302 * <p>Create an HBaseTestingUtility using a default configuration. 303 * 304 * <p>Initially, all tmp files are written to a local test data directory. 305 * Once {@link #startMiniDFSCluster} is called, either directly or via 306 * {@link #startMiniCluster()}, tmp data will be written to the DFS directory instead. 307 * 308 * <p>Previously, there was a distinction between the type of utility returned by 309 * {@link #createLocalHTU()} and this constructor; this is no longer the case. All 310 * HBaseTestingUtility objects will behave as local until a DFS cluster is started, 311 * at which point they will switch to using mini DFS for storage. 312 */ 313 public HBaseTestingUtility() { 314 this(HBaseConfiguration.create()); 315 } 316 317 /** 318 * <p>Create an HBaseTestingUtility using a given configuration. 319 * 320 * <p>Initially, all tmp files are written to a local test data directory. 321 * Once {@link #startMiniDFSCluster} is called, either directly or via 322 * {@link #startMiniCluster()}, tmp data will be written to the DFS directory instead. 323 * 324 * <p>Previously, there was a distinction between the type of utility returned by 325 * {@link #createLocalHTU()} and this constructor; this is no longer the case. All 326 * HBaseTestingUtility objects will behave as local until a DFS cluster is started, 327 * at which point they will switch to using mini DFS for storage. 328 * 329 * @param conf The configuration to use for further operations 330 */ 331 public HBaseTestingUtility(@Nullable Configuration conf) { 332 super(conf); 333 334 // a hbase checksum verification failure will cause unit tests to fail 335 ChecksumUtil.generateExceptionForChecksumFailureForTest(true); 336 337 // Save this for when setting default file:// breaks things 338 if (this.conf.get("fs.defaultFS") != null) { 339 this.conf.set("original.defaultFS", this.conf.get("fs.defaultFS")); 340 } 341 if (this.conf.get(HConstants.HBASE_DIR) != null) { 342 this.conf.set("original.hbase.dir", this.conf.get(HConstants.HBASE_DIR)); 343 } 344 // Every cluster is a local cluster until we start DFS 345 // Note that conf could be null, but this.conf will not be 346 String dataTestDir = getDataTestDir().toString(); 347 this.conf.set("fs.defaultFS","file:///"); 348 this.conf.set(HConstants.HBASE_DIR, "file://" + dataTestDir); 349 LOG.debug("Setting {} to {}", HConstants.HBASE_DIR, dataTestDir); 350 this.conf.setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE,false); 351 // If the value for random ports isn't set set it to true, thus making 352 // tests opt-out for random port assignment 353 this.conf.setBoolean(LocalHBaseCluster.ASSIGN_RANDOM_PORTS, 354 this.conf.getBoolean(LocalHBaseCluster.ASSIGN_RANDOM_PORTS, true)); 355 } 356 357 /** 358 * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #HBaseTestingUtility()} 359 * instead. 360 * @return a normal HBaseTestingUtility 361 * @see #HBaseTestingUtility() 362 * @see <a href="https://issues.apache.org/jira/browse/HBASE-19841">HBASE-19841</a> 363 */ 364 @Deprecated 365 public static HBaseTestingUtility createLocalHTU() { 366 return new HBaseTestingUtility(); 367 } 368 369 /** 370 * @deprecated since 2.0.0 and will be removed in 3.0.0. Use 371 * {@link #HBaseTestingUtility(Configuration)} instead. 372 * @return a normal HBaseTestingUtility 373 * @see #HBaseTestingUtility(Configuration) 374 * @see <a href="https://issues.apache.org/jira/browse/HBASE-19841">HBASE-19841</a> 375 */ 376 @Deprecated 377 public static HBaseTestingUtility createLocalHTU(Configuration c) { 378 return new HBaseTestingUtility(c); 379 } 380 381 /** 382 * Close both the region {@code r} and it's underlying WAL. For use in tests. 383 */ 384 public static void closeRegionAndWAL(final Region r) throws IOException { 385 closeRegionAndWAL((HRegion)r); 386 } 387 388 /** 389 * Close both the HRegion {@code r} and it's underlying WAL. For use in tests. 390 */ 391 public static void closeRegionAndWAL(final HRegion r) throws IOException { 392 if (r == null) return; 393 r.close(); 394 if (r.getWAL() == null) return; 395 r.getWAL().close(); 396 } 397 398 /** 399 * Returns this classes's instance of {@link Configuration}. Be careful how 400 * you use the returned Configuration since {@link Connection} instances 401 * can be shared. The Map of Connections is keyed by the Configuration. If 402 * say, a Connection was being used against a cluster that had been shutdown, 403 * see {@link #shutdownMiniCluster()}, then the Connection will no longer 404 * be wholesome. Rather than use the return direct, its usually best to 405 * make a copy and use that. Do 406 * <code>Configuration c = new Configuration(INSTANCE.getConfiguration());</code> 407 * @return Instance of Configuration. 408 */ 409 @Override 410 public Configuration getConfiguration() { 411 return super.getConfiguration(); 412 } 413 414 public void setHBaseCluster(HBaseCluster hbaseCluster) { 415 this.hbaseCluster = hbaseCluster; 416 } 417 418 /** 419 * Home our data in a dir under {@link #DEFAULT_BASE_TEST_DIRECTORY}. 420 * Give it a random name so can have many concurrent tests running if 421 * we need to. It needs to amend the {@link #TEST_DIRECTORY_KEY} 422 * System property, as it's what minidfscluster bases 423 * it data dir on. Moding a System property is not the way to do concurrent 424 * instances -- another instance could grab the temporary 425 * value unintentionally -- but not anything can do about it at moment; 426 * single instance only is how the minidfscluster works. 427 * 428 * We also create the underlying directory names for 429 * hadoop.log.dir, mapreduce.cluster.local.dir and hadoop.tmp.dir, and set the values 430 * in the conf, and as a system property for hadoop.tmp.dir (We do not create them!). 431 * 432 * @return The calculated data test build directory, if newly-created. 433 */ 434 @Override 435 protected Path setupDataTestDir() { 436 Path testPath = super.setupDataTestDir(); 437 if (null == testPath) { 438 return null; 439 } 440 441 createSubDirAndSystemProperty( 442 "hadoop.log.dir", 443 testPath, "hadoop-log-dir"); 444 445 // This is defaulted in core-default.xml to /tmp/hadoop-${user.name}, but 446 // we want our own value to ensure uniqueness on the same machine 447 createSubDirAndSystemProperty( 448 "hadoop.tmp.dir", 449 testPath, "hadoop-tmp-dir"); 450 451 // Read and modified in org.apache.hadoop.mapred.MiniMRCluster 452 createSubDir( 453 "mapreduce.cluster.local.dir", 454 testPath, "mapred-local-dir"); 455 return testPath; 456 } 457 458 private void createSubDirAndSystemProperty( 459 String propertyName, Path parent, String subDirName){ 460 461 String sysValue = System.getProperty(propertyName); 462 463 if (sysValue != null) { 464 // There is already a value set. So we do nothing but hope 465 // that there will be no conflicts 466 LOG.info("System.getProperty(\""+propertyName+"\") already set to: "+ 467 sysValue + " so I do NOT create it in " + parent); 468 String confValue = conf.get(propertyName); 469 if (confValue != null && !confValue.endsWith(sysValue)){ 470 LOG.warn( 471 propertyName + " property value differs in configuration and system: "+ 472 "Configuration="+confValue+" while System="+sysValue+ 473 " Erasing configuration value by system value." 474 ); 475 } 476 conf.set(propertyName, sysValue); 477 } else { 478 // Ok, it's not set, so we create it as a subdirectory 479 createSubDir(propertyName, parent, subDirName); 480 System.setProperty(propertyName, conf.get(propertyName)); 481 } 482 } 483 484 /** 485 * @return Where to write test data on the test filesystem; Returns working directory 486 * for the test filesystem by default 487 * @see #setupDataTestDirOnTestFS() 488 * @see #getTestFileSystem() 489 */ 490 private Path getBaseTestDirOnTestFS() throws IOException { 491 FileSystem fs = getTestFileSystem(); 492 return new Path(fs.getWorkingDirectory(), "test-data"); 493 } 494 495 /** 496 * @return META table descriptor 497 * @deprecated since 2.0 version and will be removed in 3.0 version. Currently for test only. 498 * use {@link #getMetaTableDescriptorBuilder()} 499 */ 500 @Deprecated 501 public HTableDescriptor getMetaTableDescriptor() { 502 return new ImmutableHTableDescriptor(getMetaTableDescriptorBuilder().build()); 503 } 504 505 /** 506 * @return META table descriptor 507 * @deprecated Since 2.3.0. No one should be using this internal. Used in testing only. 508 */ 509 @Deprecated 510 @VisibleForTesting 511 public TableDescriptorBuilder getMetaTableDescriptorBuilder() { 512 try { 513 return FSTableDescriptors.createMetaTableDescriptorBuilder(conf); 514 } catch (IOException e) { 515 throw new RuntimeException("Unable to create META table descriptor", e); 516 } 517 } 518 519 /** 520 * Returns a Path in the test filesystem, obtained from {@link #getTestFileSystem()} 521 * to write temporary test data. Call this method after setting up the mini dfs cluster 522 * if the test relies on it. 523 * @return a unique path in the test filesystem 524 */ 525 public Path getDataTestDirOnTestFS() throws IOException { 526 if (dataTestDirOnTestFS == null) { 527 setupDataTestDirOnTestFS(); 528 } 529 530 return dataTestDirOnTestFS; 531 } 532 533 /** 534 * Returns a Path in the test filesystem, obtained from {@link #getTestFileSystem()} 535 * to write temporary test data. Call this method after setting up the mini dfs cluster 536 * if the test relies on it. 537 * @return a unique path in the test filesystem 538 * @param subdirName name of the subdir to create under the base test dir 539 */ 540 public Path getDataTestDirOnTestFS(final String subdirName) throws IOException { 541 return new Path(getDataTestDirOnTestFS(), subdirName); 542 } 543 544 /** 545 * Sets up a path in test filesystem to be used by tests. 546 * Creates a new directory if not already setup. 547 */ 548 private void setupDataTestDirOnTestFS() throws IOException { 549 if (dataTestDirOnTestFS != null) { 550 LOG.warn("Data test on test fs dir already setup in " 551 + dataTestDirOnTestFS.toString()); 552 return; 553 } 554 dataTestDirOnTestFS = getNewDataTestDirOnTestFS(); 555 } 556 557 /** 558 * Sets up a new path in test filesystem to be used by tests. 559 */ 560 private Path getNewDataTestDirOnTestFS() throws IOException { 561 //The file system can be either local, mini dfs, or if the configuration 562 //is supplied externally, it can be an external cluster FS. If it is a local 563 //file system, the tests should use getBaseTestDir, otherwise, we can use 564 //the working directory, and create a unique sub dir there 565 FileSystem fs = getTestFileSystem(); 566 Path newDataTestDir; 567 String randomStr = getRandomUUID().toString(); 568 if (fs.getUri().getScheme().equals(FileSystem.getLocal(conf).getUri().getScheme())) { 569 newDataTestDir = new Path(getDataTestDir(), randomStr); 570 File dataTestDir = new File(newDataTestDir.toString()); 571 if (deleteOnExit()) dataTestDir.deleteOnExit(); 572 } else { 573 Path base = getBaseTestDirOnTestFS(); 574 newDataTestDir = new Path(base, randomStr); 575 if (deleteOnExit()) fs.deleteOnExit(newDataTestDir); 576 } 577 return newDataTestDir; 578 } 579 580 /** 581 * Cleans the test data directory on the test filesystem. 582 * @return True if we removed the test dirs 583 * @throws IOException 584 */ 585 public boolean cleanupDataTestDirOnTestFS() throws IOException { 586 boolean ret = getTestFileSystem().delete(dataTestDirOnTestFS, true); 587 if (ret) 588 dataTestDirOnTestFS = null; 589 return ret; 590 } 591 592 /** 593 * Cleans a subdirectory under the test data directory on the test filesystem. 594 * @return True if we removed child 595 * @throws IOException 596 */ 597 public boolean cleanupDataTestDirOnTestFS(String subdirName) throws IOException { 598 Path cpath = getDataTestDirOnTestFS(subdirName); 599 return getTestFileSystem().delete(cpath, true); 600 } 601 602 /** 603 * Start a minidfscluster. 604 * @param servers How many DNs to start. 605 * @throws Exception 606 * @see #shutdownMiniDFSCluster() 607 * @return The mini dfs cluster created. 608 */ 609 public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception { 610 return startMiniDFSCluster(servers, null); 611 } 612 613 /** 614 * Start a minidfscluster. 615 * This is useful if you want to run datanode on distinct hosts for things 616 * like HDFS block location verification. 617 * If you start MiniDFSCluster without host names, all instances of the 618 * datanodes will have the same host name. 619 * @param hosts hostnames DNs to run on. 620 * @throws Exception 621 * @see #shutdownMiniDFSCluster() 622 * @return The mini dfs cluster created. 623 */ 624 public MiniDFSCluster startMiniDFSCluster(final String hosts[]) 625 throws Exception { 626 if ( hosts != null && hosts.length != 0) { 627 return startMiniDFSCluster(hosts.length, hosts); 628 } else { 629 return startMiniDFSCluster(1, null); 630 } 631 } 632 633 /** 634 * Start a minidfscluster. 635 * Can only create one. 636 * @param servers How many DNs to start. 637 * @param hosts hostnames DNs to run on. 638 * @throws Exception 639 * @see #shutdownMiniDFSCluster() 640 * @return The mini dfs cluster created. 641 */ 642 public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[]) 643 throws Exception { 644 return startMiniDFSCluster(servers, null, hosts); 645 } 646 647 private void setFs() throws IOException { 648 if(this.dfsCluster == null){ 649 LOG.info("Skipping setting fs because dfsCluster is null"); 650 return; 651 } 652 FileSystem fs = this.dfsCluster.getFileSystem(); 653 CommonFSUtils.setFsDefault(this.conf, new Path(fs.getUri())); 654 655 // re-enable this check with dfs 656 conf.unset(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE); 657 } 658 659 public MiniDFSCluster startMiniDFSCluster(int servers, final String racks[], String hosts[]) 660 throws Exception { 661 createDirsAndSetProperties(); 662 EditLogFileOutputStream.setShouldSkipFsyncForTesting(true); 663 664 // Error level to skip some warnings specific to the minicluster. See HBASE-4709 665 Log4jUtils.setLogLevel(org.apache.hadoop.metrics2.util.MBeans.class.getName(), "ERROR"); 666 Log4jUtils.setLogLevel(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class.getName(), 667 "ERROR"); 668 669 TraceUtil.initTracer(conf); 670 671 this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true, 672 true, null, racks, hosts, null); 673 674 // Set this just-started cluster as our filesystem. 675 setFs(); 676 677 // Wait for the cluster to be totally up 678 this.dfsCluster.waitClusterUp(); 679 680 //reset the test directory for test file system 681 dataTestDirOnTestFS = null; 682 String dataTestDir = getDataTestDir().toString(); 683 conf.set(HConstants.HBASE_DIR, dataTestDir); 684 LOG.debug("Setting {} to {}", HConstants.HBASE_DIR, dataTestDir); 685 686 return this.dfsCluster; 687 } 688 689 public MiniDFSCluster startMiniDFSClusterForTestWAL(int namenodePort) throws IOException { 690 createDirsAndSetProperties(); 691 // Error level to skip some warnings specific to the minicluster. See HBASE-4709 692 Log4jUtils.setLogLevel(org.apache.hadoop.metrics2.util.MBeans.class.getName(), "ERROR"); 693 Log4jUtils.setLogLevel(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class.getName(), 694 "ERROR"); 695 dfsCluster = new MiniDFSCluster(namenodePort, conf, 5, false, true, true, null, 696 null, null, null); 697 return dfsCluster; 698 } 699 700 /** 701 * This is used before starting HDFS and map-reduce mini-clusters Run something like the below to 702 * check for the likes of '/tmp' references -- i.e. references outside of the test data dir -- in 703 * the conf. 704 * 705 * <pre> 706 * Configuration conf = TEST_UTIL.getConfiguration(); 707 * for (Iterator<Map.Entry<String, String>> i = conf.iterator(); i.hasNext();) { 708 * Map.Entry<String, String> e = i.next(); 709 * assertFalse(e.getKey() + " " + e.getValue(), e.getValue().contains("/tmp")); 710 * } 711 * </pre> 712 */ 713 private void createDirsAndSetProperties() throws IOException { 714 setupClusterTestDir(); 715 conf.set(TEST_DIRECTORY_KEY, clusterTestDir.getPath()); 716 System.setProperty(TEST_DIRECTORY_KEY, clusterTestDir.getPath()); 717 createDirAndSetProperty("test.cache.data"); 718 createDirAndSetProperty("hadoop.tmp.dir"); 719 hadoopLogDir = createDirAndSetProperty("hadoop.log.dir"); 720 createDirAndSetProperty("mapreduce.cluster.local.dir"); 721 createDirAndSetProperty("mapreduce.cluster.temp.dir"); 722 enableShortCircuit(); 723 724 Path root = getDataTestDirOnTestFS("hadoop"); 725 conf.set(MapreduceTestingShim.getMROutputDirProp(), 726 new Path(root, "mapred-output-dir").toString()); 727 conf.set("mapreduce.jobtracker.system.dir", new Path(root, "mapred-system-dir").toString()); 728 conf.set("mapreduce.jobtracker.staging.root.dir", 729 new Path(root, "mapreduce-jobtracker-staging-root-dir").toString()); 730 conf.set("mapreduce.job.working.dir", new Path(root, "mapred-working-dir").toString()); 731 conf.set("yarn.app.mapreduce.am.staging-dir", 732 new Path(root, "mapreduce-am-staging-root-dir").toString()); 733 734 // Frustrate yarn's and hdfs's attempts at writing /tmp. 735 // Below is fragile. Make it so we just interpolate any 'tmp' reference. 736 createDirAndSetProperty("yarn.node-labels.fs-store.root-dir"); 737 createDirAndSetProperty("yarn.node-attribute.fs-store.root-dir"); 738 createDirAndSetProperty("yarn.nodemanager.log-dirs"); 739 createDirAndSetProperty("yarn.nodemanager.remote-app-log-dir"); 740 createDirAndSetProperty("yarn.timeline-service.entity-group-fs-store.active-dir"); 741 createDirAndSetProperty("yarn.timeline-service.entity-group-fs-store.done-dir"); 742 createDirAndSetProperty("yarn.nodemanager.remote-app-log-dir"); 743 createDirAndSetProperty("dfs.journalnode.edits.dir"); 744 createDirAndSetProperty("dfs.datanode.shared.file.descriptor.paths"); 745 createDirAndSetProperty("nfs.dump.dir"); 746 createDirAndSetProperty("java.io.tmpdir"); 747 createDirAndSetProperty("dfs.journalnode.edits.dir"); 748 createDirAndSetProperty("dfs.provided.aliasmap.inmemory.leveldb.dir"); 749 createDirAndSetProperty("fs.s3a.committer.staging.tmp.path"); 750 } 751 752 /** 753 * Check whether the tests should assume NEW_VERSION_BEHAVIOR when creating 754 * new column families. Default to false. 755 */ 756 public boolean isNewVersionBehaviorEnabled(){ 757 final String propName = "hbase.tests.new.version.behavior"; 758 String v = System.getProperty(propName); 759 if (v != null){ 760 return Boolean.parseBoolean(v); 761 } 762 return false; 763 } 764 765 /** 766 * Get the HBase setting for dfs.client.read.shortcircuit from the conf or a system property. 767 * This allows to specify this parameter on the command line. 768 * If not set, default is true. 769 */ 770 public boolean isReadShortCircuitOn(){ 771 final String propName = "hbase.tests.use.shortcircuit.reads"; 772 String readOnProp = System.getProperty(propName); 773 if (readOnProp != null){ 774 return Boolean.parseBoolean(readOnProp); 775 } else { 776 return conf.getBoolean(propName, false); 777 } 778 } 779 780 /** Enable the short circuit read, unless configured differently. 781 * Set both HBase and HDFS settings, including skipping the hdfs checksum checks. 782 */ 783 private void enableShortCircuit() { 784 if (isReadShortCircuitOn()) { 785 String curUser = System.getProperty("user.name"); 786 LOG.info("read short circuit is ON for user " + curUser); 787 // read short circuit, for hdfs 788 conf.set("dfs.block.local-path-access.user", curUser); 789 // read short circuit, for hbase 790 conf.setBoolean("dfs.client.read.shortcircuit", true); 791 // Skip checking checksum, for the hdfs client and the datanode 792 conf.setBoolean("dfs.client.read.shortcircuit.skip.checksum", true); 793 } else { 794 LOG.info("read short circuit is OFF"); 795 } 796 } 797 798 private String createDirAndSetProperty(String property) { 799 return createDirAndSetProperty(property, property); 800 } 801 802 private String createDirAndSetProperty(final String relPath, String property) { 803 String path = getDataTestDir(relPath).toString(); 804 System.setProperty(property, path); 805 conf.set(property, path); 806 new File(path).mkdirs(); 807 LOG.info("Setting " + property + " to " + path + " in system properties and HBase conf"); 808 return path; 809 } 810 811 /** 812 * Shuts down instance created by call to {@link #startMiniDFSCluster(int)} 813 * or does nothing. 814 * @throws IOException 815 */ 816 public void shutdownMiniDFSCluster() throws IOException { 817 if (this.dfsCluster != null) { 818 // The below throws an exception per dn, AsynchronousCloseException. 819 this.dfsCluster.shutdown(); 820 dfsCluster = null; 821 dataTestDirOnTestFS = null; 822 CommonFSUtils.setFsDefault(this.conf, new Path("file:///")); 823 } 824 } 825 826 /** 827 * Start up a minicluster of hbase, dfs, and zookeeper where WAL's walDir is created separately. 828 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 829 * @param createWALDir Whether to create a new WAL directory. 830 * @return The mini HBase cluster created. 831 * @see #shutdownMiniCluster() 832 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 833 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 834 * @see #startMiniCluster(StartMiniClusterOption) 835 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 836 */ 837 @Deprecated 838 public MiniHBaseCluster startMiniCluster(boolean createWALDir) throws Exception { 839 StartMiniClusterOption option = StartMiniClusterOption.builder() 840 .createWALDir(createWALDir).build(); 841 return startMiniCluster(option); 842 } 843 844 /** 845 * Start up a minicluster of hbase, dfs, and zookeeper. 846 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 847 * @param numSlaves Slave node number, for both HBase region server and HDFS data node. 848 * @param createRootDir Whether to create a new root or data directory path. 849 * @return The mini HBase cluster created. 850 * @see #shutdownMiniCluster() 851 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 852 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 853 * @see #startMiniCluster(StartMiniClusterOption) 854 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 855 */ 856 @Deprecated 857 public MiniHBaseCluster startMiniCluster(int numSlaves, boolean createRootDir) 858 throws Exception { 859 StartMiniClusterOption option = StartMiniClusterOption.builder() 860 .numRegionServers(numSlaves).numDataNodes(numSlaves).createRootDir(createRootDir).build(); 861 return startMiniCluster(option); 862 } 863 864 /** 865 * Start up a minicluster of hbase, dfs, and zookeeper. 866 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 867 * @param numSlaves Slave node number, for both HBase region server and HDFS data node. 868 * @param createRootDir Whether to create a new root or data directory path. 869 * @param createWALDir Whether to create a new WAL directory. 870 * @return The mini HBase cluster created. 871 * @see #shutdownMiniCluster() 872 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 873 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 874 * @see #startMiniCluster(StartMiniClusterOption) 875 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 876 */ 877 @Deprecated 878 public MiniHBaseCluster startMiniCluster(int numSlaves, boolean createRootDir, 879 boolean createWALDir) throws Exception { 880 StartMiniClusterOption option = StartMiniClusterOption.builder() 881 .numRegionServers(numSlaves).numDataNodes(numSlaves).createRootDir(createRootDir) 882 .createWALDir(createWALDir).build(); 883 return startMiniCluster(option); 884 } 885 886 /** 887 * Start up a minicluster of hbase, dfs, and zookeeper. 888 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 889 * @param numMasters Master node number. 890 * @param numSlaves Slave node number, for both HBase region server and HDFS data node. 891 * @param createRootDir Whether to create a new root or data directory path. 892 * @return The mini HBase cluster created. 893 * @see #shutdownMiniCluster() 894 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 895 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 896 * @see #startMiniCluster(StartMiniClusterOption) 897 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 898 */ 899 @Deprecated 900 public MiniHBaseCluster startMiniCluster(int numMasters, int numSlaves, boolean createRootDir) 901 throws Exception { 902 StartMiniClusterOption option = StartMiniClusterOption.builder() 903 .numMasters(numMasters).numRegionServers(numSlaves).createRootDir(createRootDir) 904 .numDataNodes(numSlaves).build(); 905 return startMiniCluster(option); 906 } 907 908 /** 909 * Start up a minicluster of hbase, dfs, and zookeeper. 910 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 911 * @param numMasters Master node number. 912 * @param numSlaves Slave node number, for both HBase region server and HDFS data node. 913 * @return The mini HBase cluster created. 914 * @see #shutdownMiniCluster() 915 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 916 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 917 * @see #startMiniCluster(StartMiniClusterOption) 918 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 919 */ 920 @Deprecated 921 public MiniHBaseCluster startMiniCluster(int numMasters, int numSlaves) throws Exception { 922 StartMiniClusterOption option = StartMiniClusterOption.builder() 923 .numMasters(numMasters).numRegionServers(numSlaves).numDataNodes(numSlaves).build(); 924 return startMiniCluster(option); 925 } 926 927 /** 928 * Start up a minicluster of hbase, dfs, and zookeeper. 929 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 930 * @param numMasters Master node number. 931 * @param numSlaves Slave node number, for both HBase region server and HDFS data node. 932 * @param dataNodeHosts The hostnames of DataNodes to run on. If not null, its size will overwrite 933 * HDFS data node number. 934 * @param createRootDir Whether to create a new root or data directory path. 935 * @return The mini HBase cluster created. 936 * @see #shutdownMiniCluster() 937 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 938 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 939 * @see #startMiniCluster(StartMiniClusterOption) 940 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 941 */ 942 @Deprecated 943 public MiniHBaseCluster startMiniCluster(int numMasters, int numSlaves, String[] dataNodeHosts, 944 boolean createRootDir) throws Exception { 945 StartMiniClusterOption option = StartMiniClusterOption.builder() 946 .numMasters(numMasters).numRegionServers(numSlaves).createRootDir(createRootDir) 947 .numDataNodes(numSlaves).dataNodeHosts(dataNodeHosts).build(); 948 return startMiniCluster(option); 949 } 950 951 /** 952 * Start up a minicluster of hbase, dfs, and zookeeper. 953 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 954 * @param numMasters Master node number. 955 * @param numSlaves Slave node number, for both HBase region server and HDFS data node. 956 * @param dataNodeHosts The hostnames of DataNodes to run on. If not null, its size will overwrite 957 * HDFS data node number. 958 * @return The mini HBase cluster created. 959 * @see #shutdownMiniCluster() 960 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 961 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 962 * @see #startMiniCluster(StartMiniClusterOption) 963 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 964 */ 965 @Deprecated 966 public MiniHBaseCluster startMiniCluster(int numMasters, int numSlaves, String[] dataNodeHosts) 967 throws Exception { 968 StartMiniClusterOption option = StartMiniClusterOption.builder() 969 .numMasters(numMasters).numRegionServers(numSlaves) 970 .numDataNodes(numSlaves).dataNodeHosts(dataNodeHosts).build(); 971 return startMiniCluster(option); 972 } 973 974 /** 975 * Start up a minicluster of hbase, dfs, and zookeeper. 976 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 977 * @param numMasters Master node number. 978 * @param numRegionServers Number of region servers. 979 * @param numDataNodes Number of datanodes. 980 * @return The mini HBase cluster created. 981 * @see #shutdownMiniCluster() 982 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 983 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 984 * @see #startMiniCluster(StartMiniClusterOption) 985 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 986 */ 987 @Deprecated 988 public MiniHBaseCluster startMiniCluster(int numMasters, int numRegionServers, int numDataNodes) 989 throws Exception { 990 StartMiniClusterOption option = StartMiniClusterOption.builder() 991 .numMasters(numMasters).numRegionServers(numRegionServers).numDataNodes(numDataNodes) 992 .build(); 993 return startMiniCluster(option); 994 } 995 996 /** 997 * Start up a minicluster of hbase, dfs, and zookeeper. 998 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 999 * @param numMasters Master node number. 1000 * @param numSlaves Slave node number, for both HBase region server and HDFS data node. 1001 * @param dataNodeHosts The hostnames of DataNodes to run on. If not null, its size will overwrite 1002 * HDFS data node number. 1003 * @param masterClass The class to use as HMaster, or null for default. 1004 * @param rsClass The class to use as HRegionServer, or null for default. 1005 * @return The mini HBase cluster created. 1006 * @see #shutdownMiniCluster() 1007 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 1008 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 1009 * @see #startMiniCluster(StartMiniClusterOption) 1010 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 1011 */ 1012 @Deprecated 1013 public MiniHBaseCluster startMiniCluster(int numMasters, int numSlaves, String[] dataNodeHosts, 1014 Class<? extends HMaster> masterClass, 1015 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> rsClass) 1016 throws Exception { 1017 StartMiniClusterOption option = StartMiniClusterOption.builder() 1018 .numMasters(numMasters).masterClass(masterClass) 1019 .numRegionServers(numSlaves).rsClass(rsClass) 1020 .numDataNodes(numSlaves).dataNodeHosts(dataNodeHosts) 1021 .build(); 1022 return startMiniCluster(option); 1023 } 1024 1025 /** 1026 * Start up a minicluster of hbase, dfs, and zookeeper. 1027 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 1028 * @param numMasters Master node number. 1029 * @param numRegionServers Number of region servers. 1030 * @param numDataNodes Number of datanodes. 1031 * @param dataNodeHosts The hostnames of DataNodes to run on. If not null, its size will overwrite 1032 * HDFS data node number. 1033 * @param masterClass The class to use as HMaster, or null for default. 1034 * @param rsClass The class to use as HRegionServer, or null for default. 1035 * @return The mini HBase cluster created. 1036 * @see #shutdownMiniCluster() 1037 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 1038 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 1039 * @see #startMiniCluster(StartMiniClusterOption) 1040 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 1041 */ 1042 @Deprecated 1043 public MiniHBaseCluster startMiniCluster(int numMasters, int numRegionServers, int numDataNodes, 1044 String[] dataNodeHosts, Class<? extends HMaster> masterClass, 1045 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> rsClass) 1046 throws Exception { 1047 StartMiniClusterOption option = StartMiniClusterOption.builder() 1048 .numMasters(numMasters).masterClass(masterClass) 1049 .numRegionServers(numRegionServers).rsClass(rsClass) 1050 .numDataNodes(numDataNodes).dataNodeHosts(dataNodeHosts) 1051 .build(); 1052 return startMiniCluster(option); 1053 } 1054 1055 /** 1056 * Start up a minicluster of hbase, dfs, and zookeeper. 1057 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 1058 * @param numMasters Master node number. 1059 * @param numRegionServers Number of region servers. 1060 * @param numDataNodes Number of datanodes. 1061 * @param dataNodeHosts The hostnames of DataNodes to run on. If not null, its size will overwrite 1062 * HDFS data node number. 1063 * @param masterClass The class to use as HMaster, or null for default. 1064 * @param rsClass The class to use as HRegionServer, or null for default. 1065 * @param createRootDir Whether to create a new root or data directory path. 1066 * @param createWALDir Whether to create a new WAL directory. 1067 * @return The mini HBase cluster created. 1068 * @see #shutdownMiniCluster() 1069 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 1070 * {@link #startMiniCluster(StartMiniClusterOption)} instead. 1071 * @see #startMiniCluster(StartMiniClusterOption) 1072 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 1073 */ 1074 @Deprecated 1075 public MiniHBaseCluster startMiniCluster(int numMasters, int numRegionServers, int numDataNodes, 1076 String[] dataNodeHosts, Class<? extends HMaster> masterClass, 1077 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> rsClass, boolean createRootDir, 1078 boolean createWALDir) throws Exception { 1079 StartMiniClusterOption option = StartMiniClusterOption.builder() 1080 .numMasters(numMasters).masterClass(masterClass) 1081 .numRegionServers(numRegionServers).rsClass(rsClass) 1082 .numDataNodes(numDataNodes).dataNodeHosts(dataNodeHosts) 1083 .createRootDir(createRootDir).createWALDir(createWALDir) 1084 .build(); 1085 return startMiniCluster(option); 1086 } 1087 1088 /** 1089 * Start up a minicluster of hbase, dfs and zookeeper clusters with given slave node number. 1090 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 1091 * @param numSlaves slave node number, for both HBase region server and HDFS data node. 1092 * @see #startMiniCluster(StartMiniClusterOption option) 1093 * @see #shutdownMiniDFSCluster() 1094 */ 1095 public MiniHBaseCluster startMiniCluster(int numSlaves) throws Exception { 1096 StartMiniClusterOption option = StartMiniClusterOption.builder() 1097 .numRegionServers(numSlaves).numDataNodes(numSlaves).build(); 1098 return startMiniCluster(option); 1099 } 1100 1101 /** 1102 * Start up a minicluster of hbase, dfs and zookeeper all using default options. 1103 * Option default value can be found in {@link StartMiniClusterOption.Builder}. 1104 * @see #startMiniCluster(StartMiniClusterOption option) 1105 * @see #shutdownMiniDFSCluster() 1106 */ 1107 public MiniHBaseCluster startMiniCluster() throws Exception { 1108 return startMiniCluster(StartMiniClusterOption.builder().build()); 1109 } 1110 1111 /** 1112 * Start up a mini cluster of hbase, optionally dfs and zookeeper if needed. 1113 * It modifies Configuration. It homes the cluster data directory under a random 1114 * subdirectory in a directory under System property test.build.data, to be cleaned up on exit. 1115 * @see #shutdownMiniDFSCluster() 1116 */ 1117 public MiniHBaseCluster startMiniCluster(StartMiniClusterOption option) throws Exception { 1118 LOG.info("Starting up minicluster with option: {}", option); 1119 1120 // If we already put up a cluster, fail. 1121 if (miniClusterRunning) { 1122 throw new IllegalStateException("A mini-cluster is already running"); 1123 } 1124 miniClusterRunning = true; 1125 1126 setupClusterTestDir(); 1127 System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestDir.getPath()); 1128 1129 // Bring up mini dfs cluster. This spews a bunch of warnings about missing 1130 // scheme. Complaints are 'Scheme is undefined for build/test/data/dfs/name1'. 1131 if (dfsCluster == null) { 1132 LOG.info("STARTING DFS"); 1133 dfsCluster = startMiniDFSCluster(option.getNumDataNodes(), option.getDataNodeHosts()); 1134 } else { 1135 LOG.info("NOT STARTING DFS"); 1136 } 1137 1138 // Start up a zk cluster. 1139 if (getZkCluster() == null) { 1140 startMiniZKCluster(option.getNumZkServers()); 1141 } 1142 1143 // Start the MiniHBaseCluster 1144 return startMiniHBaseCluster(option); 1145 } 1146 1147 /** 1148 * Starts up mini hbase cluster. Usually you won't want this. You'll usually want 1149 * {@link #startMiniCluster()}. This is useful when doing stepped startup of clusters. 1150 * @return Reference to the hbase mini hbase cluster. 1151 * @see #startMiniCluster(StartMiniClusterOption) 1152 * @see #shutdownMiniHBaseCluster() 1153 */ 1154 public MiniHBaseCluster startMiniHBaseCluster(StartMiniClusterOption option) 1155 throws IOException, InterruptedException { 1156 // Now do the mini hbase cluster. Set the hbase.rootdir in config. 1157 createRootDir(option.isCreateRootDir()); 1158 if (option.isCreateWALDir()) { 1159 createWALRootDir(); 1160 } 1161 // Set the hbase.fs.tmp.dir config to make sure that we have some default value. This is 1162 // for tests that do not read hbase-defaults.xml 1163 setHBaseFsTmpDir(); 1164 1165 // These settings will make the server waits until this exact number of 1166 // regions servers are connected. 1167 if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1) == -1) { 1168 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, option.getNumRegionServers()); 1169 } 1170 if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1) == -1) { 1171 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, option.getNumRegionServers()); 1172 } 1173 1174 // Avoid log flooded with chore execution time, see HBASE-24646 for more details. 1175 Log4jUtils.setLogLevel(org.apache.hadoop.hbase.ScheduledChore.class.getName(), "INFO"); 1176 1177 Configuration c = new Configuration(this.conf); 1178 TraceUtil.initTracer(c); 1179 this.hbaseCluster = new MiniHBaseCluster(c, option.getNumMasters(), 1180 option.getNumAlwaysStandByMasters(), option.getNumRegionServers(), option.getRsPorts(), 1181 option.getMasterClass(), option.getRsClass()); 1182 // Populate the master address configuration from mini cluster configuration. 1183 conf.set(HConstants.MASTER_ADDRS_KEY, MasterRegistry.getMasterAddr(c)); 1184 // Don't leave here till we've done a successful scan of the hbase:meta 1185 try (Table t = getConnection().getTable(TableName.META_TABLE_NAME); 1186 ResultScanner s = t.getScanner(new Scan())) { 1187 for (;;) { 1188 if (s.next() == null) { 1189 break; 1190 } 1191 } 1192 } 1193 1194 1195 getAdmin(); // create immediately the hbaseAdmin 1196 LOG.info("Minicluster is up; activeMaster={}", getHBaseCluster().getMaster()); 1197 1198 return (MiniHBaseCluster) hbaseCluster; 1199 } 1200 1201 /** 1202 * Starts up mini hbase cluster using default options. 1203 * Default options can be found in {@link StartMiniClusterOption.Builder}. 1204 * @see #startMiniHBaseCluster(StartMiniClusterOption) 1205 * @see #shutdownMiniHBaseCluster() 1206 */ 1207 public MiniHBaseCluster startMiniHBaseCluster() throws IOException, InterruptedException { 1208 return startMiniHBaseCluster(StartMiniClusterOption.builder().build()); 1209 } 1210 1211 /** 1212 * Starts up mini hbase cluster. 1213 * Usually you won't want this. You'll usually want {@link #startMiniCluster()}. 1214 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 1215 * @param numMasters Master node number. 1216 * @param numRegionServers Number of region servers. 1217 * @return The mini HBase cluster created. 1218 * @see #shutdownMiniHBaseCluster() 1219 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 1220 * {@link #startMiniHBaseCluster(StartMiniClusterOption)} instead. 1221 * @see #startMiniHBaseCluster(StartMiniClusterOption) 1222 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 1223 */ 1224 @Deprecated 1225 public MiniHBaseCluster startMiniHBaseCluster(int numMasters, int numRegionServers) 1226 throws IOException, InterruptedException { 1227 StartMiniClusterOption option = StartMiniClusterOption.builder() 1228 .numMasters(numMasters).numRegionServers(numRegionServers).build(); 1229 return startMiniHBaseCluster(option); 1230 } 1231 1232 /** 1233 * Starts up mini hbase cluster. 1234 * Usually you won't want this. You'll usually want {@link #startMiniCluster()}. 1235 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 1236 * @param numMasters Master node number. 1237 * @param numRegionServers Number of region servers. 1238 * @param rsPorts Ports that RegionServer should use. 1239 * @return The mini HBase cluster created. 1240 * @see #shutdownMiniHBaseCluster() 1241 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 1242 * {@link #startMiniHBaseCluster(StartMiniClusterOption)} instead. 1243 * @see #startMiniHBaseCluster(StartMiniClusterOption) 1244 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 1245 */ 1246 @Deprecated 1247 public MiniHBaseCluster startMiniHBaseCluster(int numMasters, int numRegionServers, 1248 List<Integer> rsPorts) throws IOException, InterruptedException { 1249 StartMiniClusterOption option = StartMiniClusterOption.builder() 1250 .numMasters(numMasters).numRegionServers(numRegionServers).rsPorts(rsPorts).build(); 1251 return startMiniHBaseCluster(option); 1252 } 1253 1254 /** 1255 * Starts up mini hbase cluster. 1256 * Usually you won't want this. You'll usually want {@link #startMiniCluster()}. 1257 * All other options will use default values, defined in {@link StartMiniClusterOption.Builder}. 1258 * @param numMasters Master node number. 1259 * @param numRegionServers Number of region servers. 1260 * @param rsPorts Ports that RegionServer should use. 1261 * @param masterClass The class to use as HMaster, or null for default. 1262 * @param rsClass The class to use as HRegionServer, or null for default. 1263 * @param createRootDir Whether to create a new root or data directory path. 1264 * @param createWALDir Whether to create a new WAL directory. 1265 * @return The mini HBase cluster created. 1266 * @see #shutdownMiniHBaseCluster() 1267 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use 1268 * {@link #startMiniHBaseCluster(StartMiniClusterOption)} instead. 1269 * @see #startMiniHBaseCluster(StartMiniClusterOption) 1270 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21071">HBASE-21071</a> 1271 */ 1272 @Deprecated 1273 public MiniHBaseCluster startMiniHBaseCluster(int numMasters, int numRegionServers, 1274 List<Integer> rsPorts, Class<? extends HMaster> masterClass, 1275 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> rsClass, 1276 boolean createRootDir, boolean createWALDir) throws IOException, InterruptedException { 1277 StartMiniClusterOption option = StartMiniClusterOption.builder() 1278 .numMasters(numMasters).masterClass(masterClass) 1279 .numRegionServers(numRegionServers).rsClass(rsClass).rsPorts(rsPorts) 1280 .createRootDir(createRootDir).createWALDir(createWALDir).build(); 1281 return startMiniHBaseCluster(option); 1282 } 1283 1284 /** 1285 * Starts the hbase cluster up again after shutting it down previously in a 1286 * test. Use this if you want to keep dfs/zk up and just stop/start hbase. 1287 * @param servers number of region servers 1288 */ 1289 public void restartHBaseCluster(int servers) throws IOException, InterruptedException { 1290 this.restartHBaseCluster(servers, null); 1291 } 1292 1293 public void restartHBaseCluster(int servers, List<Integer> ports) 1294 throws IOException, InterruptedException { 1295 StartMiniClusterOption option = 1296 StartMiniClusterOption.builder().numRegionServers(servers).rsPorts(ports).build(); 1297 restartHBaseCluster(option); 1298 invalidateConnection(); 1299 } 1300 1301 public void restartHBaseCluster(StartMiniClusterOption option) 1302 throws IOException, InterruptedException { 1303 if (hbaseAdmin != null) { 1304 hbaseAdmin.close(); 1305 hbaseAdmin = null; 1306 } 1307 if (this.connection != null) { 1308 this.connection.close(); 1309 this.connection = null; 1310 } 1311 this.hbaseCluster = 1312 new MiniHBaseCluster(this.conf, option.getNumMasters(), option.getNumAlwaysStandByMasters(), 1313 option.getNumRegionServers(), option.getRsPorts(), option.getMasterClass(), 1314 option.getRsClass()); 1315 // Don't leave here till we've done a successful scan of the hbase:meta 1316 Connection conn = ConnectionFactory.createConnection(this.conf); 1317 Table t = conn.getTable(TableName.META_TABLE_NAME); 1318 ResultScanner s = t.getScanner(new Scan()); 1319 while (s.next() != null) { 1320 // do nothing 1321 } 1322 LOG.info("HBase has been restarted"); 1323 s.close(); 1324 t.close(); 1325 conn.close(); 1326 } 1327 1328 /** 1329 * @return Current mini hbase cluster. Only has something in it after a call 1330 * to {@link #startMiniCluster()}. 1331 * @see #startMiniCluster() 1332 */ 1333 public MiniHBaseCluster getMiniHBaseCluster() { 1334 if (this.hbaseCluster == null || this.hbaseCluster instanceof MiniHBaseCluster) { 1335 return (MiniHBaseCluster)this.hbaseCluster; 1336 } 1337 throw new RuntimeException(hbaseCluster + " not an instance of " + 1338 MiniHBaseCluster.class.getName()); 1339 } 1340 1341 /** 1342 * Stops mini hbase, zk, and hdfs clusters. 1343 * @see #startMiniCluster(int) 1344 */ 1345 public void shutdownMiniCluster() throws IOException { 1346 LOG.info("Shutting down minicluster"); 1347 shutdownMiniHBaseCluster(); 1348 shutdownMiniDFSCluster(); 1349 shutdownMiniZKCluster(); 1350 1351 cleanupTestDir(); 1352 miniClusterRunning = false; 1353 LOG.info("Minicluster is down"); 1354 } 1355 1356 /** 1357 * Shutdown HBase mini cluster.Does not shutdown zk or dfs if running. 1358 * @throws java.io.IOException in case command is unsuccessful 1359 */ 1360 public void shutdownMiniHBaseCluster() throws IOException { 1361 cleanup(); 1362 if (this.hbaseCluster != null) { 1363 this.hbaseCluster.shutdown(); 1364 // Wait till hbase is down before going on to shutdown zk. 1365 this.hbaseCluster.waitUntilShutDown(); 1366 this.hbaseCluster = null; 1367 } 1368 if (zooKeeperWatcher != null) { 1369 zooKeeperWatcher.close(); 1370 zooKeeperWatcher = null; 1371 } 1372 } 1373 1374 /** 1375 * Abruptly Shutdown HBase mini cluster. Does not shutdown zk or dfs if running. 1376 * @throws java.io.IOException throws in case command is unsuccessful 1377 */ 1378 public void killMiniHBaseCluster() throws IOException { 1379 cleanup(); 1380 if (this.hbaseCluster != null) { 1381 getMiniHBaseCluster().killAll(); 1382 this.hbaseCluster = null; 1383 } 1384 if (zooKeeperWatcher != null) { 1385 zooKeeperWatcher.close(); 1386 zooKeeperWatcher = null; 1387 } 1388 } 1389 1390 // close hbase admin, close current connection and reset MIN MAX configs for RS. 1391 private void cleanup() throws IOException { 1392 if (hbaseAdmin != null) { 1393 hbaseAdmin.close(); 1394 hbaseAdmin = null; 1395 } 1396 if (this.connection != null) { 1397 this.connection.close(); 1398 this.connection = null; 1399 } 1400 // unset the configuration for MIN and MAX RS to start 1401 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1); 1402 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1); 1403 } 1404 1405 /** 1406 * Returns the path to the default root dir the minicluster uses. If <code>create</code> 1407 * is true, a new root directory path is fetched irrespective of whether it has been fetched 1408 * before or not. If false, previous path is used. 1409 * Note: this does not cause the root dir to be created. 1410 * @return Fully qualified path for the default hbase root dir 1411 * @throws IOException 1412 */ 1413 public Path getDefaultRootDirPath(boolean create) throws IOException { 1414 if (!create) { 1415 return getDataTestDirOnTestFS(); 1416 } else { 1417 return getNewDataTestDirOnTestFS(); 1418 } 1419 } 1420 1421 /** 1422 * Same as {{@link HBaseTestingUtility#getDefaultRootDirPath(boolean create)} 1423 * except that <code>create</code> flag is false. 1424 * Note: this does not cause the root dir to be created. 1425 * @return Fully qualified path for the default hbase root dir 1426 * @throws IOException 1427 */ 1428 public Path getDefaultRootDirPath() throws IOException { 1429 return getDefaultRootDirPath(false); 1430 } 1431 1432 /** 1433 * Creates an hbase rootdir in user home directory. Also creates hbase 1434 * version file. Normally you won't make use of this method. Root hbasedir 1435 * is created for you as part of mini cluster startup. You'd only use this 1436 * method if you were doing manual operation. 1437 * @param create This flag decides whether to get a new 1438 * root or data directory path or not, if it has been fetched already. 1439 * Note : Directory will be made irrespective of whether path has been fetched or not. 1440 * If directory already exists, it will be overwritten 1441 * @return Fully qualified path to hbase root dir 1442 * @throws IOException 1443 */ 1444 public Path createRootDir(boolean create) throws IOException { 1445 FileSystem fs = FileSystem.get(this.conf); 1446 Path hbaseRootdir = getDefaultRootDirPath(create); 1447 CommonFSUtils.setRootDir(this.conf, hbaseRootdir); 1448 fs.mkdirs(hbaseRootdir); 1449 FSUtils.setVersion(fs, hbaseRootdir); 1450 return hbaseRootdir; 1451 } 1452 1453 /** 1454 * Same as {@link HBaseTestingUtility#createRootDir(boolean create)} 1455 * except that <code>create</code> flag is false. 1456 * @return Fully qualified path to hbase root dir 1457 * @throws IOException 1458 */ 1459 public Path createRootDir() throws IOException { 1460 return createRootDir(false); 1461 } 1462 1463 /** 1464 * Creates a hbase walDir in the user's home directory. 1465 * Normally you won't make use of this method. Root hbaseWALDir 1466 * is created for you as part of mini cluster startup. You'd only use this 1467 * method if you were doing manual operation. 1468 * 1469 * @return Fully qualified path to hbase root dir 1470 * @throws IOException 1471 */ 1472 public Path createWALRootDir() throws IOException { 1473 FileSystem fs = FileSystem.get(this.conf); 1474 Path walDir = getNewDataTestDirOnTestFS(); 1475 CommonFSUtils.setWALRootDir(this.conf, walDir); 1476 fs.mkdirs(walDir); 1477 return walDir; 1478 } 1479 1480 private void setHBaseFsTmpDir() throws IOException { 1481 String hbaseFsTmpDirInString = this.conf.get("hbase.fs.tmp.dir"); 1482 if (hbaseFsTmpDirInString == null) { 1483 this.conf.set("hbase.fs.tmp.dir", getDataTestDirOnTestFS("hbase-staging").toString()); 1484 LOG.info("Setting hbase.fs.tmp.dir to " + this.conf.get("hbase.fs.tmp.dir")); 1485 } else { 1486 LOG.info("The hbase.fs.tmp.dir is set to " + hbaseFsTmpDirInString); 1487 } 1488 } 1489 1490 /** 1491 * Flushes all caches in the mini hbase cluster 1492 * @throws IOException 1493 */ 1494 public void flush() throws IOException { 1495 getMiniHBaseCluster().flushcache(); 1496 } 1497 1498 /** 1499 * Flushes all caches in the mini hbase cluster 1500 * @throws IOException 1501 */ 1502 public void flush(TableName tableName) throws IOException { 1503 getMiniHBaseCluster().flushcache(tableName); 1504 } 1505 1506 /** 1507 * Compact all regions in the mini hbase cluster 1508 * @throws IOException 1509 */ 1510 public void compact(boolean major) throws IOException { 1511 getMiniHBaseCluster().compact(major); 1512 } 1513 1514 /** 1515 * Compact all of a table's reagion in the mini hbase cluster 1516 * @throws IOException 1517 */ 1518 public void compact(TableName tableName, boolean major) throws IOException { 1519 getMiniHBaseCluster().compact(tableName, major); 1520 } 1521 1522 /** 1523 * Create a table. 1524 * @param tableName 1525 * @param family 1526 * @return A Table instance for the created table. 1527 * @throws IOException 1528 */ 1529 public Table createTable(TableName tableName, String family) 1530 throws IOException{ 1531 return createTable(tableName, new String[]{family}); 1532 } 1533 1534 /** 1535 * Create a table. 1536 * @param tableName 1537 * @param families 1538 * @return A Table instance for the created table. 1539 * @throws IOException 1540 */ 1541 public Table createTable(TableName tableName, String[] families) 1542 throws IOException { 1543 List<byte[]> fams = new ArrayList<>(families.length); 1544 for (String family : families) { 1545 fams.add(Bytes.toBytes(family)); 1546 } 1547 return createTable(tableName, fams.toArray(new byte[0][])); 1548 } 1549 1550 /** 1551 * Create a table. 1552 * @param tableName 1553 * @param family 1554 * @return A Table instance for the created table. 1555 * @throws IOException 1556 */ 1557 public Table createTable(TableName tableName, byte[] family) 1558 throws IOException{ 1559 return createTable(tableName, new byte[][]{family}); 1560 } 1561 1562 /** 1563 * Create a table with multiple regions. 1564 * @param tableName 1565 * @param family 1566 * @param numRegions 1567 * @return A Table instance for the created table. 1568 * @throws IOException 1569 */ 1570 public Table createMultiRegionTable(TableName tableName, byte[] family, int numRegions) 1571 throws IOException { 1572 if (numRegions < 3) throw new IOException("Must create at least 3 regions"); 1573 byte[] startKey = Bytes.toBytes("aaaaa"); 1574 byte[] endKey = Bytes.toBytes("zzzzz"); 1575 byte[][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3); 1576 1577 return createTable(tableName, new byte[][] { family }, splitKeys); 1578 } 1579 1580 /** 1581 * Create a table. 1582 * @param tableName 1583 * @param families 1584 * @return A Table instance for the created table. 1585 * @throws IOException 1586 */ 1587 public Table createTable(TableName tableName, byte[][] families) 1588 throws IOException { 1589 return createTable(tableName, families, (byte[][]) null); 1590 } 1591 1592 /** 1593 * Create a table with multiple regions. 1594 * @param tableName 1595 * @param families 1596 * @return A Table instance for the created table. 1597 * @throws IOException 1598 */ 1599 public Table createMultiRegionTable(TableName tableName, byte[][] families) throws IOException { 1600 return createTable(tableName, families, KEYS_FOR_HBA_CREATE_TABLE); 1601 } 1602 1603 /** 1604 * Create a table with multiple regions. 1605 * @param tableName 1606 * @param replicaCount replica count. 1607 * @param families 1608 * @return A Table instance for the created table. 1609 * @throws IOException 1610 */ 1611 public Table createMultiRegionTable(TableName tableName, int replicaCount, byte[][] families) 1612 throws IOException { 1613 return createTable(tableName, families, KEYS_FOR_HBA_CREATE_TABLE, replicaCount); 1614 } 1615 1616 /** 1617 * Create a table. 1618 * @param tableName 1619 * @param families 1620 * @param splitKeys 1621 * @return A Table instance for the created table. 1622 * @throws IOException 1623 */ 1624 public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys) 1625 throws IOException { 1626 return createTable(tableName, families, splitKeys, 1, new Configuration(getConfiguration())); 1627 } 1628 1629 /** 1630 * Create a table. 1631 * @param tableName the table name 1632 * @param families the families 1633 * @param splitKeys the splitkeys 1634 * @param replicaCount the region replica count 1635 * @return A Table instance for the created table. 1636 * @throws IOException throws IOException 1637 */ 1638 public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys, 1639 int replicaCount) throws IOException { 1640 return createTable(tableName, families, splitKeys, replicaCount, 1641 new Configuration(getConfiguration())); 1642 } 1643 1644 public Table createTable(TableName tableName, byte[][] families, 1645 int numVersions, byte[] startKey, byte[] endKey, int numRegions) 1646 throws IOException{ 1647 HTableDescriptor desc = createTableDescriptor(tableName, families, numVersions); 1648 1649 getAdmin().createTable(desc, startKey, endKey, numRegions); 1650 // HBaseAdmin only waits for regions to appear in hbase:meta we 1651 // should wait until they are assigned 1652 waitUntilAllRegionsAssigned(tableName); 1653 return getConnection().getTable(tableName); 1654 } 1655 1656 /** 1657 * Create a table. 1658 * @param htd 1659 * @param families 1660 * @param c Configuration to use 1661 * @return A Table instance for the created table. 1662 * @throws IOException 1663 */ 1664 public Table createTable(TableDescriptor htd, byte[][] families, Configuration c) 1665 throws IOException { 1666 return createTable(htd, families, null, c); 1667 } 1668 1669 /** 1670 * Create a table. 1671 * @param htd table descriptor 1672 * @param families array of column families 1673 * @param splitKeys array of split keys 1674 * @param c Configuration to use 1675 * @return A Table instance for the created table. 1676 * @throws IOException if getAdmin or createTable fails 1677 */ 1678 public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys, 1679 Configuration c) throws IOException { 1680 // Disable blooms (they are on by default as of 0.95) but we disable them here because 1681 // tests have hard coded counts of what to expect in block cache, etc., and blooms being 1682 // on is interfering. 1683 return createTable(htd, families, splitKeys, BloomType.NONE, HConstants.DEFAULT_BLOCKSIZE, c); 1684 } 1685 1686 /** 1687 * Create a table. 1688 * @param htd table descriptor 1689 * @param families array of column families 1690 * @param splitKeys array of split keys 1691 * @param type Bloom type 1692 * @param blockSize block size 1693 * @param c Configuration to use 1694 * @return A Table instance for the created table. 1695 * @throws IOException if getAdmin or createTable fails 1696 */ 1697 1698 public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys, 1699 BloomType type, int blockSize, Configuration c) throws IOException { 1700 TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(htd); 1701 for (byte[] family : families) { 1702 ColumnFamilyDescriptorBuilder cfdb = ColumnFamilyDescriptorBuilder.newBuilder(family) 1703 .setBloomFilterType(type) 1704 .setBlocksize(blockSize); 1705 if (isNewVersionBehaviorEnabled()) { 1706 cfdb.setNewVersionBehavior(true); 1707 } 1708 builder.setColumnFamily(cfdb.build()); 1709 } 1710 TableDescriptor td = builder.build(); 1711 getAdmin().createTable(td, splitKeys); 1712 // HBaseAdmin only waits for regions to appear in hbase:meta 1713 // we should wait until they are assigned 1714 waitUntilAllRegionsAssigned(td.getTableName()); 1715 return getConnection().getTable(td.getTableName()); 1716 } 1717 1718 /** 1719 * Create a table. 1720 * @param htd table descriptor 1721 * @param splitRows array of split keys 1722 * @return A Table instance for the created table. 1723 * @throws IOException 1724 */ 1725 public Table createTable(TableDescriptor htd, byte[][] splitRows) 1726 throws IOException { 1727 TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(htd); 1728 if (isNewVersionBehaviorEnabled()) { 1729 for (ColumnFamilyDescriptor family : htd.getColumnFamilies()) { 1730 builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family) 1731 .setNewVersionBehavior(true).build()); 1732 } 1733 } 1734 getAdmin().createTable(builder.build(), splitRows); 1735 // HBaseAdmin only waits for regions to appear in hbase:meta 1736 // we should wait until they are assigned 1737 waitUntilAllRegionsAssigned(htd.getTableName()); 1738 return getConnection().getTable(htd.getTableName()); 1739 } 1740 1741 /** 1742 * Create a table. 1743 * @param tableName the table name 1744 * @param families the families 1745 * @param splitKeys the split keys 1746 * @param replicaCount the replica count 1747 * @param c Configuration to use 1748 * @return A Table instance for the created table. 1749 * @throws IOException 1750 */ 1751 public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys, 1752 int replicaCount, final Configuration c) throws IOException { 1753 HTableDescriptor htd = new HTableDescriptor(tableName); 1754 htd.setRegionReplication(replicaCount); 1755 return createTable(htd, families, splitKeys, c); 1756 } 1757 1758 /** 1759 * Create a table. 1760 * @param tableName 1761 * @param family 1762 * @param numVersions 1763 * @return A Table instance for the created table. 1764 * @throws IOException 1765 */ 1766 public Table createTable(TableName tableName, byte[] family, int numVersions) 1767 throws IOException { 1768 return createTable(tableName, new byte[][]{family}, numVersions); 1769 } 1770 1771 /** 1772 * Create a table. 1773 * @param tableName 1774 * @param families 1775 * @param numVersions 1776 * @return A Table instance for the created table. 1777 * @throws IOException 1778 */ 1779 public Table createTable(TableName tableName, byte[][] families, int numVersions) 1780 throws IOException { 1781 return createTable(tableName, families, numVersions, (byte[][]) null); 1782 } 1783 1784 /** 1785 * Create a table. 1786 * @param tableName 1787 * @param families 1788 * @param numVersions 1789 * @param splitKeys 1790 * @return A Table instance for the created table. 1791 * @throws IOException 1792 */ 1793 public Table createTable(TableName tableName, byte[][] families, int numVersions, 1794 byte[][] splitKeys) throws IOException { 1795 HTableDescriptor desc = new HTableDescriptor(tableName); 1796 for (byte[] family : families) { 1797 HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions); 1798 if (isNewVersionBehaviorEnabled()) { 1799 hcd.setNewVersionBehavior(true); 1800 } 1801 desc.addFamily(hcd); 1802 } 1803 getAdmin().createTable(desc, splitKeys); 1804 // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are 1805 // assigned 1806 waitUntilAllRegionsAssigned(tableName); 1807 return getConnection().getTable(tableName); 1808 } 1809 1810 /** 1811 * Create a table with multiple regions. 1812 * @param tableName 1813 * @param families 1814 * @param numVersions 1815 * @return A Table instance for the created table. 1816 * @throws IOException 1817 */ 1818 public Table createMultiRegionTable(TableName tableName, byte[][] families, int numVersions) 1819 throws IOException { 1820 return createTable(tableName, families, numVersions, KEYS_FOR_HBA_CREATE_TABLE); 1821 } 1822 1823 /** 1824 * Create a table. 1825 * @param tableName 1826 * @param families 1827 * @param numVersions 1828 * @param blockSize 1829 * @return A Table instance for the created table. 1830 * @throws IOException 1831 */ 1832 public Table createTable(TableName tableName, byte[][] families, 1833 int numVersions, int blockSize) throws IOException { 1834 HTableDescriptor desc = new HTableDescriptor(tableName); 1835 for (byte[] family : families) { 1836 HColumnDescriptor hcd = new HColumnDescriptor(family) 1837 .setMaxVersions(numVersions) 1838 .setBlocksize(blockSize); 1839 if (isNewVersionBehaviorEnabled()) { 1840 hcd.setNewVersionBehavior(true); 1841 } 1842 desc.addFamily(hcd); 1843 } 1844 getAdmin().createTable(desc); 1845 // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are 1846 // assigned 1847 waitUntilAllRegionsAssigned(tableName); 1848 return getConnection().getTable(tableName); 1849 } 1850 1851 public Table createTable(TableName tableName, byte[][] families, 1852 int numVersions, int blockSize, String cpName) throws IOException { 1853 HTableDescriptor desc = new HTableDescriptor(tableName); 1854 for (byte[] family : families) { 1855 HColumnDescriptor hcd = new HColumnDescriptor(family) 1856 .setMaxVersions(numVersions) 1857 .setBlocksize(blockSize); 1858 if (isNewVersionBehaviorEnabled()) { 1859 hcd.setNewVersionBehavior(true); 1860 } 1861 desc.addFamily(hcd); 1862 } 1863 if(cpName != null) { 1864 desc.addCoprocessor(cpName); 1865 } 1866 getAdmin().createTable(desc); 1867 // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are 1868 // assigned 1869 waitUntilAllRegionsAssigned(tableName); 1870 return getConnection().getTable(tableName); 1871 } 1872 1873 /** 1874 * Create a table. 1875 * @param tableName 1876 * @param families 1877 * @param numVersions 1878 * @return A Table instance for the created table. 1879 * @throws IOException 1880 */ 1881 public Table createTable(TableName tableName, byte[][] families, 1882 int[] numVersions) 1883 throws IOException { 1884 HTableDescriptor desc = new HTableDescriptor(tableName); 1885 int i = 0; 1886 for (byte[] family : families) { 1887 HColumnDescriptor hcd = new HColumnDescriptor(family) 1888 .setMaxVersions(numVersions[i]); 1889 if (isNewVersionBehaviorEnabled()) { 1890 hcd.setNewVersionBehavior(true); 1891 } 1892 desc.addFamily(hcd); 1893 i++; 1894 } 1895 getAdmin().createTable(desc); 1896 // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are 1897 // assigned 1898 waitUntilAllRegionsAssigned(tableName); 1899 return getConnection().getTable(tableName); 1900 } 1901 1902 /** 1903 * Create a table. 1904 * @param tableName 1905 * @param family 1906 * @param splitRows 1907 * @return A Table instance for the created table. 1908 * @throws IOException 1909 */ 1910 public Table createTable(TableName tableName, byte[] family, byte[][] splitRows) 1911 throws IOException { 1912 HTableDescriptor desc = new HTableDescriptor(tableName); 1913 HColumnDescriptor hcd = new HColumnDescriptor(family); 1914 if (isNewVersionBehaviorEnabled()) { 1915 hcd.setNewVersionBehavior(true); 1916 } 1917 desc.addFamily(hcd); 1918 getAdmin().createTable(desc, splitRows); 1919 // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are 1920 // assigned 1921 waitUntilAllRegionsAssigned(tableName); 1922 return getConnection().getTable(tableName); 1923 } 1924 1925 /** 1926 * Create a table with multiple regions. 1927 * @param tableName 1928 * @param family 1929 * @return A Table instance for the created table. 1930 * @throws IOException 1931 */ 1932 public Table createMultiRegionTable(TableName tableName, byte[] family) throws IOException { 1933 return createTable(tableName, family, KEYS_FOR_HBA_CREATE_TABLE); 1934 } 1935 1936 /** 1937 * Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}. 1938 */ 1939 @SuppressWarnings("serial") 1940 public static void modifyTableSync(Admin admin, TableDescriptor desc) 1941 throws IOException, InterruptedException { 1942 admin.modifyTable(desc); 1943 Pair<Integer, Integer> status = new Pair<Integer, Integer>() {{ 1944 setFirst(0); 1945 setSecond(0); 1946 }}; 1947 int i = 0; 1948 do { 1949 status = admin.getAlterStatus(desc.getTableName()); 1950 if (status.getSecond() != 0) { 1951 LOG.debug(status.getSecond() - status.getFirst() + "/" + status.getSecond() 1952 + " regions updated."); 1953 Thread.sleep(1 * 1000L); 1954 } else { 1955 LOG.debug("All regions updated."); 1956 break; 1957 } 1958 } while (status.getFirst() != 0 && i++ < 500); 1959 if (status.getFirst() != 0) { 1960 throw new IOException("Failed to update all regions even after 500 seconds."); 1961 } 1962 } 1963 1964 /** 1965 * Set the number of Region replicas. 1966 */ 1967 public static void setReplicas(Admin admin, TableName table, int replicaCount) 1968 throws IOException, InterruptedException { 1969 admin.disableTable(table); 1970 HTableDescriptor desc = new HTableDescriptor(admin.getTableDescriptor(table)); 1971 desc.setRegionReplication(replicaCount); 1972 admin.modifyTable(desc.getTableName(), desc); 1973 admin.enableTable(table); 1974 } 1975 1976 /** 1977 * Drop an existing table 1978 * @param tableName existing table 1979 */ 1980 public void deleteTable(TableName tableName) throws IOException { 1981 try { 1982 getAdmin().disableTable(tableName); 1983 } catch (TableNotEnabledException e) { 1984 LOG.debug("Table: " + tableName + " already disabled, so just deleting it."); 1985 } 1986 getAdmin().deleteTable(tableName); 1987 } 1988 1989 /** 1990 * Drop an existing table 1991 * @param tableName existing table 1992 */ 1993 public void deleteTableIfAny(TableName tableName) throws IOException { 1994 try { 1995 deleteTable(tableName); 1996 } catch (TableNotFoundException e) { 1997 // ignore 1998 } 1999 } 2000 2001 // ========================================================================== 2002 // Canned table and table descriptor creation 2003 // TODO replace HBaseTestCase 2004 2005 public final static byte [] fam1 = Bytes.toBytes("colfamily11"); 2006 public final static byte [] fam2 = Bytes.toBytes("colfamily21"); 2007 public final static byte [] fam3 = Bytes.toBytes("colfamily31"); 2008 public static final byte[][] COLUMNS = {fam1, fam2, fam3}; 2009 private static final int MAXVERSIONS = 3; 2010 2011 public static final char FIRST_CHAR = 'a'; 2012 public static final char LAST_CHAR = 'z'; 2013 public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR}; 2014 public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET); 2015 2016 /** 2017 * @deprecated since 2.0.0 and will be removed in 3.0.0. Use 2018 * {@link #createTableDescriptor(TableName, int, int, int, KeepDeletedCells)} instead. 2019 * @see #createTableDescriptor(TableName, int, int, int, KeepDeletedCells) 2020 * @see <a href="https://issues.apache.org/jira/browse/HBASE-13893">HBASE-13893</a> 2021 */ 2022 @Deprecated 2023 public HTableDescriptor createTableDescriptor(final String name, 2024 final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) { 2025 return this.createTableDescriptor(TableName.valueOf(name), minVersions, versions, ttl, 2026 keepDeleted); 2027 } 2028 2029 /** 2030 * Create a table of name <code>name</code>. 2031 * @param name Name to give table. 2032 * @return Column descriptor. 2033 * @deprecated since 2.0.0 and will be removed in 3.0.0. Use 2034 * {@link #createTableDescriptor(TableName, int, int, int, KeepDeletedCells)} instead. 2035 * @see #createTableDescriptor(TableName, int, int, int, KeepDeletedCells) 2036 * @see <a href="https://issues.apache.org/jira/browse/HBASE-13893">HBASE-13893</a> 2037 */ 2038 @Deprecated 2039 public HTableDescriptor createTableDescriptor(final String name) { 2040 return createTableDescriptor(TableName.valueOf(name), HColumnDescriptor.DEFAULT_MIN_VERSIONS, 2041 MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); 2042 } 2043 2044 public HTableDescriptor createTableDescriptor(final TableName name, 2045 final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) { 2046 HTableDescriptor htd = new HTableDescriptor(name); 2047 for (byte[] cfName : new byte[][]{ fam1, fam2, fam3 }) { 2048 HColumnDescriptor hcd = new HColumnDescriptor(cfName) 2049 .setMinVersions(minVersions) 2050 .setMaxVersions(versions) 2051 .setKeepDeletedCells(keepDeleted) 2052 .setBlockCacheEnabled(false) 2053 .setTimeToLive(ttl); 2054 if (isNewVersionBehaviorEnabled()) { 2055 hcd.setNewVersionBehavior(true); 2056 } 2057 htd.addFamily(hcd); 2058 } 2059 return htd; 2060 } 2061 2062 /** 2063 * Create a table of name <code>name</code>. 2064 * @param name Name to give table. 2065 * @return Column descriptor. 2066 */ 2067 public HTableDescriptor createTableDescriptor(final TableName name) { 2068 return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, 2069 MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); 2070 } 2071 2072 public HTableDescriptor createTableDescriptor(final TableName tableName, 2073 byte[] family) { 2074 return createTableDescriptor(tableName, new byte[][] {family}, 1); 2075 } 2076 2077 public HTableDescriptor createTableDescriptor(final TableName tableName, 2078 byte[][] families, int maxVersions) { 2079 HTableDescriptor desc = new HTableDescriptor(tableName); 2080 for (byte[] family : families) { 2081 HColumnDescriptor hcd = new HColumnDescriptor(family) 2082 .setMaxVersions(maxVersions); 2083 if (isNewVersionBehaviorEnabled()) { 2084 hcd.setNewVersionBehavior(true); 2085 } 2086 desc.addFamily(hcd); 2087 } 2088 return desc; 2089 } 2090 2091 /** 2092 * Create an HRegion that writes to the local tmp dirs 2093 * @param desc a table descriptor indicating which table the region belongs to 2094 * @param startKey the start boundary of the region 2095 * @param endKey the end boundary of the region 2096 * @return a region that writes to local dir for testing 2097 * @throws IOException 2098 */ 2099 public HRegion createLocalHRegion(TableDescriptor desc, byte [] startKey, 2100 byte [] endKey) 2101 throws IOException { 2102 HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey); 2103 return createLocalHRegion(hri, desc); 2104 } 2105 2106 /** 2107 * Create an HRegion that writes to the local tmp dirs. Creates the WAL for you. Be sure to call 2108 * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when you're finished with it. 2109 */ 2110 public HRegion createLocalHRegion(RegionInfo info, TableDescriptor desc) throws IOException { 2111 return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), desc); 2112 } 2113 2114 /** 2115 * Create an HRegion that writes to the local tmp dirs with specified wal 2116 * @param info regioninfo 2117 * @param desc table descriptor 2118 * @param wal wal for this region. 2119 * @return created hregion 2120 * @throws IOException 2121 */ 2122 public HRegion createLocalHRegion(RegionInfo info, TableDescriptor desc, WAL wal) 2123 throws IOException { 2124 return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, wal); 2125 } 2126 2127 /** 2128 * @param tableName the name of the table 2129 * @param startKey the start key of the region 2130 * @param stopKey the stop key of the region 2131 * @param callingMethod the name of the calling method probably a test method 2132 * @param conf the configuration to use 2133 * @param isReadOnly {@code true} if the table is read only, {@code false} otherwise 2134 * @param families the column families to use 2135 * @throws IOException if an IO problem is encountered 2136 * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} 2137 * when done. 2138 * @deprecated since 2.0.0 and will be removed in 3.0.0. Use 2139 * {@link #createLocalHRegion(TableName, byte[], byte[], boolean, Durability, WAL, byte[]...)} 2140 * instead. 2141 * @see #createLocalHRegion(TableName, byte[], byte[], boolean, Durability, WAL, byte[]...) 2142 * @see <a href="https://issues.apache.org/jira/browse/HBASE-13893">HBASE-13893</a> 2143 */ 2144 @Deprecated 2145 public HRegion createLocalHRegion(byte[] tableName, byte[] startKey, byte[] stopKey, 2146 String callingMethod, Configuration conf, boolean isReadOnly, Durability durability, 2147 WAL wal, byte[]... families) throws IOException { 2148 return this 2149 .createLocalHRegion(TableName.valueOf(tableName), startKey, stopKey, isReadOnly, durability, 2150 wal, families); 2151 } 2152 2153 /** 2154 * @param tableName 2155 * @param startKey 2156 * @param stopKey 2157 * @param isReadOnly 2158 * @param families 2159 * @return A region on which you must call 2160 * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. 2161 * @throws IOException 2162 */ 2163 public HRegion createLocalHRegion(TableName tableName, byte[] startKey, byte[] stopKey, 2164 boolean isReadOnly, Durability durability, WAL wal, byte[]... families) throws IOException { 2165 return createLocalHRegionWithInMemoryFlags(tableName,startKey, stopKey, isReadOnly, 2166 durability, wal, null, families); 2167 } 2168 2169 public HRegion createLocalHRegionWithInMemoryFlags(TableName tableName, byte[] startKey, 2170 byte[] stopKey, 2171 boolean isReadOnly, Durability durability, WAL wal, boolean[] compactedMemStore, 2172 byte[]... families) 2173 throws IOException { 2174 HTableDescriptor htd = new HTableDescriptor(tableName); 2175 htd.setReadOnly(isReadOnly); 2176 int i=0; 2177 for (byte[] family : families) { 2178 HColumnDescriptor hcd = new HColumnDescriptor(family); 2179 if(compactedMemStore != null && i < compactedMemStore.length) { 2180 hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC); 2181 } else { 2182 hcd.setInMemoryCompaction(MemoryCompactionPolicy.NONE); 2183 2184 } 2185 i++; 2186 // Set default to be three versions. 2187 hcd.setMaxVersions(Integer.MAX_VALUE); 2188 htd.addFamily(hcd); 2189 } 2190 htd.setDurability(durability); 2191 HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false); 2192 return createLocalHRegion(info, htd, wal); 2193 } 2194 2195 // 2196 // ========================================================================== 2197 2198 /** 2199 * Provide an existing table name to truncate. 2200 * Scans the table and issues a delete for each row read. 2201 * @param tableName existing table 2202 * @return HTable to that new table 2203 * @throws IOException 2204 */ 2205 public Table deleteTableData(TableName tableName) throws IOException { 2206 Table table = getConnection().getTable(tableName); 2207 Scan scan = new Scan(); 2208 ResultScanner resScan = table.getScanner(scan); 2209 for(Result res : resScan) { 2210 Delete del = new Delete(res.getRow()); 2211 table.delete(del); 2212 } 2213 resScan = table.getScanner(scan); 2214 resScan.close(); 2215 return table; 2216 } 2217 2218 /** 2219 * Truncate a table using the admin command. 2220 * Effectively disables, deletes, and recreates the table. 2221 * @param tableName table which must exist. 2222 * @param preserveRegions keep the existing split points 2223 * @return HTable for the new table 2224 */ 2225 public Table truncateTable(final TableName tableName, final boolean preserveRegions) throws 2226 IOException { 2227 Admin admin = getAdmin(); 2228 if (!admin.isTableDisabled(tableName)) { 2229 admin.disableTable(tableName); 2230 } 2231 admin.truncateTable(tableName, preserveRegions); 2232 return getConnection().getTable(tableName); 2233 } 2234 2235 /** 2236 * Truncate a table using the admin command. 2237 * Effectively disables, deletes, and recreates the table. 2238 * For previous behavior of issuing row deletes, see 2239 * deleteTableData. 2240 * Expressly does not preserve regions of existing table. 2241 * @param tableName table which must exist. 2242 * @return HTable for the new table 2243 */ 2244 public Table truncateTable(final TableName tableName) throws IOException { 2245 return truncateTable(tableName, false); 2246 } 2247 2248 /** 2249 * Load table with rows from 'aaa' to 'zzz'. 2250 * @param t Table 2251 * @param f Family 2252 * @return Count of rows loaded. 2253 * @throws IOException 2254 */ 2255 public int loadTable(final Table t, final byte[] f) throws IOException { 2256 return loadTable(t, new byte[][] {f}); 2257 } 2258 2259 /** 2260 * Load table with rows from 'aaa' to 'zzz'. 2261 * @param t Table 2262 * @param f Family 2263 * @return Count of rows loaded. 2264 * @throws IOException 2265 */ 2266 public int loadTable(final Table t, final byte[] f, boolean writeToWAL) throws IOException { 2267 return loadTable(t, new byte[][] {f}, null, writeToWAL); 2268 } 2269 2270 /** 2271 * Load table of multiple column families with rows from 'aaa' to 'zzz'. 2272 * @param t Table 2273 * @param f Array of Families to load 2274 * @return Count of rows loaded. 2275 * @throws IOException 2276 */ 2277 public int loadTable(final Table t, final byte[][] f) throws IOException { 2278 return loadTable(t, f, null); 2279 } 2280 2281 /** 2282 * Load table of multiple column families with rows from 'aaa' to 'zzz'. 2283 * @param t Table 2284 * @param f Array of Families to load 2285 * @param value the values of the cells. If null is passed, the row key is used as value 2286 * @return Count of rows loaded. 2287 * @throws IOException 2288 */ 2289 public int loadTable(final Table t, final byte[][] f, byte[] value) throws IOException { 2290 return loadTable(t, f, value, true); 2291 } 2292 2293 /** 2294 * Load table of multiple column families with rows from 'aaa' to 'zzz'. 2295 * @param t Table 2296 * @param f Array of Families to load 2297 * @param value the values of the cells. If null is passed, the row key is used as value 2298 * @return Count of rows loaded. 2299 * @throws IOException 2300 */ 2301 public int loadTable(final Table t, final byte[][] f, byte[] value, 2302 boolean writeToWAL) throws IOException { 2303 List<Put> puts = new ArrayList<>(); 2304 for (byte[] row : HBaseTestingUtility.ROWS) { 2305 Put put = new Put(row); 2306 put.setDurability(writeToWAL ? Durability.USE_DEFAULT : Durability.SKIP_WAL); 2307 for (int i = 0; i < f.length; i++) { 2308 byte[] value1 = value != null ? value : row; 2309 put.addColumn(f[i], f[i], value1); 2310 } 2311 puts.add(put); 2312 } 2313 t.put(puts); 2314 return puts.size(); 2315 } 2316 2317 /** A tracker for tracking and validating table rows 2318 * generated with {@link HBaseTestingUtility#loadTable(Table, byte[])} 2319 */ 2320 public static class SeenRowTracker { 2321 int dim = 'z' - 'a' + 1; 2322 int[][][] seenRows = new int[dim][dim][dim]; //count of how many times the row is seen 2323 byte[] startRow; 2324 byte[] stopRow; 2325 2326 public SeenRowTracker(byte[] startRow, byte[] stopRow) { 2327 this.startRow = startRow; 2328 this.stopRow = stopRow; 2329 } 2330 2331 void reset() { 2332 for (byte[] row : ROWS) { 2333 seenRows[i(row[0])][i(row[1])][i(row[2])] = 0; 2334 } 2335 } 2336 2337 int i(byte b) { 2338 return b - 'a'; 2339 } 2340 2341 public void addRow(byte[] row) { 2342 seenRows[i(row[0])][i(row[1])][i(row[2])]++; 2343 } 2344 2345 /** Validate that all the rows between startRow and stopRow are seen exactly once, and 2346 * all other rows none 2347 */ 2348 public void validate() { 2349 for (byte b1 = 'a'; b1 <= 'z'; b1++) { 2350 for (byte b2 = 'a'; b2 <= 'z'; b2++) { 2351 for (byte b3 = 'a'; b3 <= 'z'; b3++) { 2352 int count = seenRows[i(b1)][i(b2)][i(b3)]; 2353 int expectedCount = 0; 2354 if (Bytes.compareTo(new byte[] {b1,b2,b3}, startRow) >= 0 2355 && Bytes.compareTo(new byte[] {b1,b2,b3}, stopRow) < 0) { 2356 expectedCount = 1; 2357 } 2358 if (count != expectedCount) { 2359 String row = new String(new byte[] {b1,b2,b3}, StandardCharsets.UTF_8); 2360 throw new RuntimeException("Row:" + row + " has a seen count of " + count + " " + 2361 "instead of " + expectedCount); 2362 } 2363 } 2364 } 2365 } 2366 } 2367 } 2368 2369 public int loadRegion(final HRegion r, final byte[] f) throws IOException { 2370 return loadRegion(r, f, false); 2371 } 2372 2373 public int loadRegion(final Region r, final byte[] f) throws IOException { 2374 return loadRegion((HRegion)r, f); 2375 } 2376 2377 /** 2378 * Load region with rows from 'aaa' to 'zzz'. 2379 * @param r Region 2380 * @param f Family 2381 * @param flush flush the cache if true 2382 * @return Count of rows loaded. 2383 * @throws IOException 2384 */ 2385 public int loadRegion(final HRegion r, final byte[] f, final boolean flush) 2386 throws IOException { 2387 byte[] k = new byte[3]; 2388 int rowCount = 0; 2389 for (byte b1 = 'a'; b1 <= 'z'; b1++) { 2390 for (byte b2 = 'a'; b2 <= 'z'; b2++) { 2391 for (byte b3 = 'a'; b3 <= 'z'; b3++) { 2392 k[0] = b1; 2393 k[1] = b2; 2394 k[2] = b3; 2395 Put put = new Put(k); 2396 put.setDurability(Durability.SKIP_WAL); 2397 put.addColumn(f, null, k); 2398 if (r.getWAL() == null) { 2399 put.setDurability(Durability.SKIP_WAL); 2400 } 2401 int preRowCount = rowCount; 2402 int pause = 10; 2403 int maxPause = 1000; 2404 while (rowCount == preRowCount) { 2405 try { 2406 r.put(put); 2407 rowCount++; 2408 } catch (RegionTooBusyException e) { 2409 pause = (pause * 2 >= maxPause) ? maxPause : pause * 2; 2410 Threads.sleep(pause); 2411 } 2412 } 2413 } 2414 } 2415 if (flush) { 2416 r.flush(true); 2417 } 2418 } 2419 return rowCount; 2420 } 2421 2422 public void loadNumericRows(final Table t, final byte[] f, int startRow, int endRow) 2423 throws IOException { 2424 for (int i = startRow; i < endRow; i++) { 2425 byte[] data = Bytes.toBytes(String.valueOf(i)); 2426 Put put = new Put(data); 2427 put.addColumn(f, null, data); 2428 t.put(put); 2429 } 2430 } 2431 2432 public void loadRandomRows(final Table t, final byte[] f, int rowSize, int totalRows) 2433 throws IOException { 2434 Random r = new Random(); 2435 byte[] row = new byte[rowSize]; 2436 for (int i = 0; i < totalRows; i++) { 2437 r.nextBytes(row); 2438 Put put = new Put(row); 2439 put.addColumn(f, new byte[]{0}, new byte[]{0}); 2440 t.put(put); 2441 } 2442 } 2443 2444 public void verifyNumericRows(Table table, final byte[] f, int startRow, int endRow, 2445 int replicaId) 2446 throws IOException { 2447 for (int i = startRow; i < endRow; i++) { 2448 String failMsg = "Failed verification of row :" + i; 2449 byte[] data = Bytes.toBytes(String.valueOf(i)); 2450 Get get = new Get(data); 2451 get.setReplicaId(replicaId); 2452 get.setConsistency(Consistency.TIMELINE); 2453 Result result = table.get(get); 2454 assertTrue(failMsg, result.containsColumn(f, null)); 2455 assertEquals(failMsg, 1, result.getColumnCells(f, null).size()); 2456 Cell cell = result.getColumnLatestCell(f, null); 2457 assertTrue(failMsg, 2458 Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(), 2459 cell.getValueLength())); 2460 } 2461 } 2462 2463 public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow) 2464 throws IOException { 2465 verifyNumericRows((HRegion)region, f, startRow, endRow); 2466 } 2467 2468 public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow) 2469 throws IOException { 2470 verifyNumericRows(region, f, startRow, endRow, true); 2471 } 2472 2473 public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow, 2474 final boolean present) throws IOException { 2475 verifyNumericRows((HRegion)region, f, startRow, endRow, present); 2476 } 2477 2478 public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow, 2479 final boolean present) throws IOException { 2480 for (int i = startRow; i < endRow; i++) { 2481 String failMsg = "Failed verification of row :" + i; 2482 byte[] data = Bytes.toBytes(String.valueOf(i)); 2483 Result result = region.get(new Get(data)); 2484 2485 boolean hasResult = result != null && !result.isEmpty(); 2486 assertEquals(failMsg + result, present, hasResult); 2487 if (!present) continue; 2488 2489 assertTrue(failMsg, result.containsColumn(f, null)); 2490 assertEquals(failMsg, 1, result.getColumnCells(f, null).size()); 2491 Cell cell = result.getColumnLatestCell(f, null); 2492 assertTrue(failMsg, 2493 Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(), 2494 cell.getValueLength())); 2495 } 2496 } 2497 2498 public void deleteNumericRows(final Table t, final byte[] f, int startRow, int endRow) 2499 throws IOException { 2500 for (int i = startRow; i < endRow; i++) { 2501 byte[] data = Bytes.toBytes(String.valueOf(i)); 2502 Delete delete = new Delete(data); 2503 delete.addFamily(f); 2504 t.delete(delete); 2505 } 2506 } 2507 2508 /** 2509 * Return the number of rows in the given table. 2510 * @param table to count rows 2511 * @return count of rows 2512 */ 2513 public int countRows(final Table table) throws IOException { 2514 return countRows(table, new Scan()); 2515 } 2516 2517 public int countRows(final Table table, final Scan scan) throws IOException { 2518 try (ResultScanner results = table.getScanner(scan)) { 2519 int count = 0; 2520 while (results.next() != null) { 2521 count++; 2522 } 2523 return count; 2524 } 2525 } 2526 2527 public int countRows(final Table table, final byte[]... families) throws IOException { 2528 Scan scan = new Scan(); 2529 for (byte[] family: families) { 2530 scan.addFamily(family); 2531 } 2532 return countRows(table, scan); 2533 } 2534 2535 /** 2536 * Return the number of rows in the given table. 2537 */ 2538 public int countRows(final TableName tableName) throws IOException { 2539 Table table = getConnection().getTable(tableName); 2540 try { 2541 return countRows(table); 2542 } finally { 2543 table.close(); 2544 } 2545 } 2546 2547 public int countRows(final Region region) throws IOException { 2548 return countRows(region, new Scan()); 2549 } 2550 2551 public int countRows(final Region region, final Scan scan) throws IOException { 2552 InternalScanner scanner = region.getScanner(scan); 2553 try { 2554 return countRows(scanner); 2555 } finally { 2556 scanner.close(); 2557 } 2558 } 2559 2560 public int countRows(final InternalScanner scanner) throws IOException { 2561 int scannedCount = 0; 2562 List<Cell> results = new ArrayList<>(); 2563 boolean hasMore = true; 2564 while (hasMore) { 2565 hasMore = scanner.next(results); 2566 scannedCount += results.size(); 2567 results.clear(); 2568 } 2569 return scannedCount; 2570 } 2571 2572 /** 2573 * Return an md5 digest of the entire contents of a table. 2574 */ 2575 public String checksumRows(final Table table) throws Exception { 2576 2577 Scan scan = new Scan(); 2578 ResultScanner results = table.getScanner(scan); 2579 MessageDigest digest = MessageDigest.getInstance("MD5"); 2580 for (Result res : results) { 2581 digest.update(res.getRow()); 2582 } 2583 results.close(); 2584 return digest.toString(); 2585 } 2586 2587 /** All the row values for the data loaded by {@link #loadTable(Table, byte[])} */ 2588 public static final byte[][] ROWS = new byte[(int) Math.pow('z' - 'a' + 1, 3)][3]; // ~52KB 2589 static { 2590 int i = 0; 2591 for (byte b1 = 'a'; b1 <= 'z'; b1++) { 2592 for (byte b2 = 'a'; b2 <= 'z'; b2++) { 2593 for (byte b3 = 'a'; b3 <= 'z'; b3++) { 2594 ROWS[i][0] = b1; 2595 ROWS[i][1] = b2; 2596 ROWS[i][2] = b3; 2597 i++; 2598 } 2599 } 2600 } 2601 } 2602 2603 public static final byte[][] KEYS = { 2604 HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"), 2605 Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"), 2606 Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"), 2607 Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"), 2608 Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"), 2609 Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"), 2610 Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"), 2611 Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"), 2612 Bytes.toBytes("xxx"), Bytes.toBytes("yyy") 2613 }; 2614 2615 public static final byte[][] KEYS_FOR_HBA_CREATE_TABLE = { 2616 Bytes.toBytes("bbb"), 2617 Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"), 2618 Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"), 2619 Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"), 2620 Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"), 2621 Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"), 2622 Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"), 2623 Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"), 2624 Bytes.toBytes("xxx"), Bytes.toBytes("yyy"), Bytes.toBytes("zzz") 2625 }; 2626 2627 /** 2628 * Create rows in hbase:meta for regions of the specified table with the specified 2629 * start keys. The first startKey should be a 0 length byte array if you 2630 * want to form a proper range of regions. 2631 * @param conf 2632 * @param htd 2633 * @param startKeys 2634 * @return list of region info for regions added to meta 2635 * @throws IOException 2636 * @deprecated since 2.0 version and will be removed in 3.0 version. 2637 * use {@link #createMultiRegionsInMeta(Configuration, TableDescriptor, byte[][])} 2638 */ 2639 @Deprecated 2640 public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf, 2641 final HTableDescriptor htd, byte [][] startKeys) throws IOException { 2642 return createMultiRegionsInMeta(conf, (TableDescriptor) htd, startKeys) 2643 .stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList()); 2644 } 2645 /** 2646 * Create rows in hbase:meta for regions of the specified table with the specified 2647 * start keys. The first startKey should be a 0 length byte array if you 2648 * want to form a proper range of regions. 2649 * @param conf 2650 * @param htd 2651 * @param startKeys 2652 * @return list of region info for regions added to meta 2653 * @throws IOException 2654 */ 2655 public List<RegionInfo> createMultiRegionsInMeta(final Configuration conf, 2656 final TableDescriptor htd, byte [][] startKeys) 2657 throws IOException { 2658 Table meta = getConnection().getTable(TableName.META_TABLE_NAME); 2659 Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); 2660 List<RegionInfo> newRegions = new ArrayList<>(startKeys.length); 2661 MetaTableAccessor 2662 .updateTableState(getConnection(), htd.getTableName(), TableState.State.ENABLED); 2663 // add custom ones 2664 for (int i = 0; i < startKeys.length; i++) { 2665 int j = (i + 1) % startKeys.length; 2666 RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()) 2667 .setStartKey(startKeys[i]) 2668 .setEndKey(startKeys[j]) 2669 .build(); 2670 MetaTableAccessor.addRegionToMeta(getConnection(), hri); 2671 newRegions.add(hri); 2672 } 2673 2674 meta.close(); 2675 return newRegions; 2676 } 2677 2678 /** 2679 * Create an unmanaged WAL. Be sure to close it when you're through. 2680 */ 2681 public static WAL createWal(final Configuration conf, final Path rootDir, final RegionInfo hri) 2682 throws IOException { 2683 // The WAL subsystem will use the default rootDir rather than the passed in rootDir 2684 // unless I pass along via the conf. 2685 Configuration confForWAL = new Configuration(conf); 2686 confForWAL.set(HConstants.HBASE_DIR, rootDir.toString()); 2687 return new WALFactory(confForWAL, "hregion-" + RandomStringUtils.randomNumeric(8)).getWAL(hri); 2688 } 2689 2690 2691 /** 2692 * Create a region with it's own WAL. Be sure to call 2693 * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources. 2694 */ 2695 public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir, 2696 final Configuration conf, final TableDescriptor htd) throws IOException { 2697 return createRegionAndWAL(info, rootDir, conf, htd, true); 2698 } 2699 2700 /** 2701 * Create a region with it's own WAL. Be sure to call 2702 * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources. 2703 */ 2704 public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir, 2705 final Configuration conf, final TableDescriptor htd, BlockCache blockCache) 2706 throws IOException { 2707 HRegion region = createRegionAndWAL(info, rootDir, conf, htd, false); 2708 region.setBlockCache(blockCache); 2709 region.initialize(); 2710 return region; 2711 } 2712 /** 2713 * Create a region with it's own WAL. Be sure to call 2714 * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources. 2715 */ 2716 public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir, 2717 final Configuration conf, final TableDescriptor htd, MobFileCache mobFileCache) 2718 throws IOException { 2719 HRegion region = createRegionAndWAL(info, rootDir, conf, htd, false); 2720 region.setMobFileCache(mobFileCache); 2721 region.initialize(); 2722 return region; 2723 } 2724 2725 /** 2726 * Create a region with it's own WAL. Be sure to call 2727 * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources. 2728 */ 2729 public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir, 2730 final Configuration conf, final TableDescriptor htd, boolean initialize) 2731 throws IOException { 2732 ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 2733 0, null, MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT); 2734 WAL wal = createWal(conf, rootDir, info); 2735 return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize); 2736 } 2737 2738 /** 2739 * Returns all rows from the hbase:meta table. 2740 * 2741 * @throws IOException When reading the rows fails. 2742 */ 2743 public List<byte[]> getMetaTableRows() throws IOException { 2744 // TODO: Redo using MetaTableAccessor class 2745 Table t = getConnection().getTable(TableName.META_TABLE_NAME); 2746 List<byte[]> rows = new ArrayList<>(); 2747 ResultScanner s = t.getScanner(new Scan()); 2748 for (Result result : s) { 2749 LOG.info("getMetaTableRows: row -> " + 2750 Bytes.toStringBinary(result.getRow())); 2751 rows.add(result.getRow()); 2752 } 2753 s.close(); 2754 t.close(); 2755 return rows; 2756 } 2757 2758 /** 2759 * Returns all rows from the hbase:meta table for a given user table 2760 * 2761 * @throws IOException When reading the rows fails. 2762 */ 2763 public List<byte[]> getMetaTableRows(TableName tableName) throws IOException { 2764 // TODO: Redo using MetaTableAccessor. 2765 Table t = getConnection().getTable(TableName.META_TABLE_NAME); 2766 List<byte[]> rows = new ArrayList<>(); 2767 ResultScanner s = t.getScanner(new Scan()); 2768 for (Result result : s) { 2769 RegionInfo info = MetaTableAccessor.getRegionInfo(result); 2770 if (info == null) { 2771 LOG.error("No region info for row " + Bytes.toString(result.getRow())); 2772 // TODO figure out what to do for this new hosed case. 2773 continue; 2774 } 2775 2776 if (info.getTable().equals(tableName)) { 2777 LOG.info("getMetaTableRows: row -> " + 2778 Bytes.toStringBinary(result.getRow()) + info); 2779 rows.add(result.getRow()); 2780 } 2781 } 2782 s.close(); 2783 t.close(); 2784 return rows; 2785 } 2786 2787 /** 2788 * Returns all regions of the specified table 2789 * 2790 * @param tableName the table name 2791 * @return all regions of the specified table 2792 * @throws IOException when getting the regions fails. 2793 */ 2794 private List<RegionInfo> getRegions(TableName tableName) throws IOException { 2795 try (Admin admin = getConnection().getAdmin()) { 2796 return admin.getRegions(tableName); 2797 } 2798 } 2799 2800 /* 2801 * Find any other region server which is different from the one identified by parameter 2802 * @param rs 2803 * @return another region server 2804 */ 2805 public HRegionServer getOtherRegionServer(HRegionServer rs) { 2806 for (JVMClusterUtil.RegionServerThread rst : 2807 getMiniHBaseCluster().getRegionServerThreads()) { 2808 if (!(rst.getRegionServer() == rs)) { 2809 return rst.getRegionServer(); 2810 } 2811 } 2812 return null; 2813 } 2814 2815 /** 2816 * Tool to get the reference to the region server object that holds the 2817 * region of the specified user table. 2818 * @param tableName user table to lookup in hbase:meta 2819 * @return region server that holds it, null if the row doesn't exist 2820 * @throws IOException 2821 * @throws InterruptedException 2822 */ 2823 public HRegionServer getRSForFirstRegionInTable(TableName tableName) 2824 throws IOException, InterruptedException { 2825 List<RegionInfo> regions = getRegions(tableName); 2826 if (regions == null || regions.isEmpty()) { 2827 return null; 2828 } 2829 LOG.debug("Found " + regions.size() + " regions for table " + 2830 tableName); 2831 2832 byte[] firstRegionName = regions.stream() 2833 .filter(r -> !r.isOffline()) 2834 .map(RegionInfo::getRegionName) 2835 .findFirst() 2836 .orElseThrow(() -> new IOException("online regions not found in table " + tableName)); 2837 2838 LOG.debug("firstRegionName=" + Bytes.toString(firstRegionName)); 2839 long pause = getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE, 2840 HConstants.DEFAULT_HBASE_CLIENT_PAUSE); 2841 int numRetries = getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2842 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); 2843 RetryCounter retrier = new RetryCounter(numRetries+1, (int)pause, TimeUnit.MICROSECONDS); 2844 while(retrier.shouldRetry()) { 2845 int index = getMiniHBaseCluster().getServerWith(firstRegionName); 2846 if (index != -1) { 2847 return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer(); 2848 } 2849 // Came back -1. Region may not be online yet. Sleep a while. 2850 retrier.sleepUntilNextRetry(); 2851 } 2852 return null; 2853 } 2854 2855 /** 2856 * Starts a <code>MiniMRCluster</code> with a default number of 2857 * <code>TaskTracker</code>'s. 2858 * 2859 * @throws IOException When starting the cluster fails. 2860 */ 2861 public MiniMRCluster startMiniMapReduceCluster() throws IOException { 2862 // Set a very high max-disk-utilization percentage to avoid the NodeManagers from failing. 2863 conf.setIfUnset( 2864 "yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage", 2865 "99.0"); 2866 startMiniMapReduceCluster(2); 2867 return mrCluster; 2868 } 2869 2870 /** 2871 * Tasktracker has a bug where changing the hadoop.log.dir system property 2872 * will not change its internal static LOG_DIR variable. 2873 */ 2874 private void forceChangeTaskLogDir() { 2875 Field logDirField; 2876 try { 2877 logDirField = TaskLog.class.getDeclaredField("LOG_DIR"); 2878 logDirField.setAccessible(true); 2879 2880 Field modifiersField = Field.class.getDeclaredField("modifiers"); 2881 modifiersField.setAccessible(true); 2882 modifiersField.setInt(logDirField, logDirField.getModifiers() & ~Modifier.FINAL); 2883 2884 logDirField.set(null, new File(hadoopLogDir, "userlogs")); 2885 } catch (SecurityException e) { 2886 throw new RuntimeException(e); 2887 } catch (NoSuchFieldException e) { 2888 // TODO Auto-generated catch block 2889 throw new RuntimeException(e); 2890 } catch (IllegalArgumentException e) { 2891 throw new RuntimeException(e); 2892 } catch (IllegalAccessException e) { 2893 throw new RuntimeException(e); 2894 } 2895 } 2896 2897 /** 2898 * Starts a <code>MiniMRCluster</code>. Call {@link #setFileSystemURI(String)} to use a different 2899 * filesystem. 2900 * @param servers The number of <code>TaskTracker</code>'s to start. 2901 * @throws IOException When starting the cluster fails. 2902 */ 2903 private void startMiniMapReduceCluster(final int servers) throws IOException { 2904 if (mrCluster != null) { 2905 throw new IllegalStateException("MiniMRCluster is already running"); 2906 } 2907 LOG.info("Starting mini mapreduce cluster..."); 2908 setupClusterTestDir(); 2909 createDirsAndSetProperties(); 2910 2911 forceChangeTaskLogDir(); 2912 2913 //// hadoop2 specific settings 2914 // Tests were failing because this process used 6GB of virtual memory and was getting killed. 2915 // we up the VM usable so that processes don't get killed. 2916 conf.setFloat("yarn.nodemanager.vmem-pmem-ratio", 8.0f); 2917 2918 // Tests were failing due to MAPREDUCE-4880 / MAPREDUCE-4607 against hadoop 2.0.2-alpha and 2919 // this avoids the problem by disabling speculative task execution in tests. 2920 conf.setBoolean("mapreduce.map.speculative", false); 2921 conf.setBoolean("mapreduce.reduce.speculative", false); 2922 //// 2923 2924 // Allow the user to override FS URI for this map-reduce cluster to use. 2925 mrCluster = new MiniMRCluster(servers, 2926 FS_URI != null ? FS_URI : FileSystem.get(conf).getUri().toString(), 1, 2927 null, null, new JobConf(this.conf)); 2928 JobConf jobConf = MapreduceTestingShim.getJobConf(mrCluster); 2929 if (jobConf == null) { 2930 jobConf = mrCluster.createJobConf(); 2931 } 2932 2933 jobConf.set("mapreduce.cluster.local.dir", 2934 conf.get("mapreduce.cluster.local.dir")); //Hadoop MiniMR overwrites this while it should not 2935 LOG.info("Mini mapreduce cluster started"); 2936 2937 // In hadoop2, YARN/MR2 starts a mini cluster with its own conf instance and updates settings. 2938 // Our HBase MR jobs need several of these settings in order to properly run. So we copy the 2939 // necessary config properties here. YARN-129 required adding a few properties. 2940 conf.set("mapreduce.jobtracker.address", jobConf.get("mapreduce.jobtracker.address")); 2941 // this for mrv2 support; mr1 ignores this 2942 conf.set("mapreduce.framework.name", "yarn"); 2943 conf.setBoolean("yarn.is.minicluster", true); 2944 String rmAddress = jobConf.get("yarn.resourcemanager.address"); 2945 if (rmAddress != null) { 2946 conf.set("yarn.resourcemanager.address", rmAddress); 2947 } 2948 String historyAddress = jobConf.get("mapreduce.jobhistory.address"); 2949 if (historyAddress != null) { 2950 conf.set("mapreduce.jobhistory.address", historyAddress); 2951 } 2952 String schedulerAddress = 2953 jobConf.get("yarn.resourcemanager.scheduler.address"); 2954 if (schedulerAddress != null) { 2955 conf.set("yarn.resourcemanager.scheduler.address", schedulerAddress); 2956 } 2957 String mrJobHistoryWebappAddress = 2958 jobConf.get("mapreduce.jobhistory.webapp.address"); 2959 if (mrJobHistoryWebappAddress != null) { 2960 conf.set("mapreduce.jobhistory.webapp.address", mrJobHistoryWebappAddress); 2961 } 2962 String yarnRMWebappAddress = 2963 jobConf.get("yarn.resourcemanager.webapp.address"); 2964 if (yarnRMWebappAddress != null) { 2965 conf.set("yarn.resourcemanager.webapp.address", yarnRMWebappAddress); 2966 } 2967 } 2968 2969 /** 2970 * Stops the previously started <code>MiniMRCluster</code>. 2971 */ 2972 public void shutdownMiniMapReduceCluster() { 2973 if (mrCluster != null) { 2974 LOG.info("Stopping mini mapreduce cluster..."); 2975 mrCluster.shutdown(); 2976 mrCluster = null; 2977 LOG.info("Mini mapreduce cluster stopped"); 2978 } 2979 // Restore configuration to point to local jobtracker 2980 conf.set("mapreduce.jobtracker.address", "local"); 2981 } 2982 2983 /** 2984 * Create a stubbed out RegionServerService, mainly for getting FS. 2985 */ 2986 public RegionServerServices createMockRegionServerService() throws IOException { 2987 return createMockRegionServerService((ServerName)null); 2988 } 2989 2990 /** 2991 * Create a stubbed out RegionServerService, mainly for getting FS. 2992 * This version is used by TestTokenAuthentication 2993 */ 2994 public RegionServerServices createMockRegionServerService(RpcServerInterface rpc) throws 2995 IOException { 2996 final MockRegionServerServices rss = new MockRegionServerServices(getZooKeeperWatcher()); 2997 rss.setFileSystem(getTestFileSystem()); 2998 rss.setRpcServer(rpc); 2999 return rss; 3000 } 3001 3002 /** 3003 * Create a stubbed out RegionServerService, mainly for getting FS. 3004 * This version is used by TestOpenRegionHandler 3005 */ 3006 public RegionServerServices createMockRegionServerService(ServerName name) throws IOException { 3007 final MockRegionServerServices rss = new MockRegionServerServices(getZooKeeperWatcher(), name); 3008 rss.setFileSystem(getTestFileSystem()); 3009 return rss; 3010 } 3011 3012 /** 3013 * Switches the logger for the given class to DEBUG level. 3014 * @param clazz The class for which to switch to debug logging. 3015 * @deprecated In 2.3.0, will be removed in 4.0.0. Only support changing log level on log4j now as 3016 * HBase only uses log4j. You should do this by your own as it you know which log 3017 * framework you are using then set the log level to debug is very easy. 3018 */ 3019 @Deprecated 3020 public void enableDebug(Class<?> clazz) { 3021 Log4jUtils.enableDebug(clazz); 3022 } 3023 3024 /** 3025 * Expire the Master's session 3026 * @throws Exception 3027 */ 3028 public void expireMasterSession() throws Exception { 3029 HMaster master = getMiniHBaseCluster().getMaster(); 3030 expireSession(master.getZooKeeper(), false); 3031 } 3032 3033 /** 3034 * Expire a region server's session 3035 * @param index which RS 3036 */ 3037 public void expireRegionServerSession(int index) throws Exception { 3038 HRegionServer rs = getMiniHBaseCluster().getRegionServer(index); 3039 expireSession(rs.getZooKeeper(), false); 3040 decrementMinRegionServerCount(); 3041 } 3042 3043 private void decrementMinRegionServerCount() { 3044 // decrement the count for this.conf, for newly spwaned master 3045 // this.hbaseCluster shares this configuration too 3046 decrementMinRegionServerCount(getConfiguration()); 3047 3048 // each master thread keeps a copy of configuration 3049 for (MasterThread master : getHBaseCluster().getMasterThreads()) { 3050 decrementMinRegionServerCount(master.getMaster().getConfiguration()); 3051 } 3052 } 3053 3054 private void decrementMinRegionServerCount(Configuration conf) { 3055 int currentCount = conf.getInt( 3056 ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1); 3057 if (currentCount != -1) { 3058 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 3059 Math.max(currentCount - 1, 1)); 3060 } 3061 } 3062 3063 public void expireSession(ZKWatcher nodeZK) throws Exception { 3064 expireSession(nodeZK, false); 3065 } 3066 3067 /** 3068 * Expire a ZooKeeper session as recommended in ZooKeeper documentation 3069 * http://hbase.apache.org/book.html#trouble.zookeeper 3070 * There are issues when doing this: 3071 * [1] http://www.mail-archive.com/dev@zookeeper.apache.org/msg01942.html 3072 * [2] https://issues.apache.org/jira/browse/ZOOKEEPER-1105 3073 * 3074 * @param nodeZK - the ZK watcher to expire 3075 * @param checkStatus - true to check if we can create a Table with the 3076 * current configuration. 3077 */ 3078 public void expireSession(ZKWatcher nodeZK, boolean checkStatus) 3079 throws Exception { 3080 Configuration c = new Configuration(this.conf); 3081 String quorumServers = ZKConfig.getZKQuorumServersString(c); 3082 ZooKeeper zk = nodeZK.getRecoverableZooKeeper().getZooKeeper(); 3083 byte[] password = zk.getSessionPasswd(); 3084 long sessionID = zk.getSessionId(); 3085 3086 // Expiry seems to be asynchronous (see comment from P. Hunt in [1]), 3087 // so we create a first watcher to be sure that the 3088 // event was sent. We expect that if our watcher receives the event 3089 // other watchers on the same machine will get is as well. 3090 // When we ask to close the connection, ZK does not close it before 3091 // we receive all the events, so don't have to capture the event, just 3092 // closing the connection should be enough. 3093 ZooKeeper monitor = new ZooKeeper(quorumServers, 3094 1000, new org.apache.zookeeper.Watcher(){ 3095 @Override 3096 public void process(WatchedEvent watchedEvent) { 3097 LOG.info("Monitor ZKW received event="+watchedEvent); 3098 } 3099 } , sessionID, password); 3100 3101 // Making it expire 3102 ZooKeeper newZK = new ZooKeeper(quorumServers, 3103 1000, EmptyWatcher.instance, sessionID, password); 3104 3105 //ensure that we have connection to the server before closing down, otherwise 3106 //the close session event will be eaten out before we start CONNECTING state 3107 long start = System.currentTimeMillis(); 3108 while (newZK.getState() != States.CONNECTED 3109 && System.currentTimeMillis() - start < 1000) { 3110 Thread.sleep(1); 3111 } 3112 newZK.close(); 3113 LOG.info("ZK Closed Session 0x" + Long.toHexString(sessionID)); 3114 3115 // Now closing & waiting to be sure that the clients get it. 3116 monitor.close(); 3117 3118 if (checkStatus) { 3119 getConnection().getTable(TableName.META_TABLE_NAME).close(); 3120 } 3121 } 3122 3123 /** 3124 * Get the Mini HBase cluster. 3125 * 3126 * @return hbase cluster 3127 * @see #getHBaseClusterInterface() 3128 */ 3129 public MiniHBaseCluster getHBaseCluster() { 3130 return getMiniHBaseCluster(); 3131 } 3132 3133 /** 3134 * Returns the HBaseCluster instance. 3135 * <p>Returned object can be any of the subclasses of HBaseCluster, and the 3136 * tests referring this should not assume that the cluster is a mini cluster or a 3137 * distributed one. If the test only works on a mini cluster, then specific 3138 * method {@link #getMiniHBaseCluster()} can be used instead w/o the 3139 * need to type-cast. 3140 */ 3141 public HBaseCluster getHBaseClusterInterface() { 3142 //implementation note: we should rename this method as #getHBaseCluster(), 3143 //but this would require refactoring 90+ calls. 3144 return hbaseCluster; 3145 } 3146 3147 public void closeConnection() throws IOException { 3148 Closeables.close(hbaseAdmin, true); 3149 Closeables.close(connection, true); 3150 this.hbaseAdmin = null; 3151 this.connection = null; 3152 } 3153 3154 /** 3155 * Resets the connections so that the next time getConnection() is called, a new connection is 3156 * created. This is needed in cases where the entire cluster / all the masters are shutdown and 3157 * the connection is not valid anymore. 3158 * TODO: There should be a more coherent way of doing this. Unfortunately the way tests are 3159 * written, not all start() stop() calls go through this class. Most tests directly operate on 3160 * the underlying mini/local hbase cluster. That makes it difficult for this wrapper class to 3161 * maintain the connection state automatically. Cleaning this is a much bigger refactor. 3162 */ 3163 public void invalidateConnection() throws IOException { 3164 closeConnection(); 3165 // Update the master addresses if they changed. 3166 final String masterConfigBefore = conf.get(HConstants.MASTER_ADDRS_KEY); 3167 final String masterConfAfter = getMiniHBaseCluster().conf.get(HConstants.MASTER_ADDRS_KEY); 3168 LOG.info("Invalidated connection. Updating master addresses before: {} after: {}", 3169 masterConfigBefore, masterConfAfter); 3170 conf.set(HConstants.MASTER_ADDRS_KEY, 3171 getMiniHBaseCluster().conf.get(HConstants.MASTER_ADDRS_KEY)); 3172 } 3173 3174 /** 3175 * @return A Connection that can be shared. Don't close. Will be closed on shutdown of cluster. 3176 * @throws IOException 3177 */ 3178 public Connection getConnection() throws IOException { 3179 if (this.connection == null) { 3180 this.connection = ConnectionFactory.createConnection(this.conf); 3181 } 3182 return this.connection; 3183 } 3184 3185 /** 3186 * Returns a Admin instance. 3187 * This instance is shared between HBaseTestingUtility instance users. Closing it has no effect, 3188 * it will be closed automatically when the cluster shutdowns 3189 * 3190 * @return HBaseAdmin instance which is guaranteed to support only {@link Admin} interface. 3191 * Functions in HBaseAdmin not provided by {@link Admin} interface can be changed/deleted 3192 * anytime. 3193 * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #getAdmin()} instead. 3194 */ 3195 @Deprecated 3196 public synchronized HBaseAdmin getHBaseAdmin() 3197 throws IOException { 3198 if (hbaseAdmin == null){ 3199 this.hbaseAdmin = (HBaseAdmin) getConnection().getAdmin(); 3200 } 3201 return hbaseAdmin; 3202 } 3203 3204 /** 3205 * Returns an Admin instance which is shared between HBaseTestingUtility instance users. 3206 * Closing it has no effect, it will be closed automatically when the cluster shutdowns 3207 */ 3208 public synchronized Admin getAdmin() throws IOException { 3209 if (hbaseAdmin == null){ 3210 this.hbaseAdmin = (HBaseAdmin) getConnection().getAdmin(); 3211 } 3212 return hbaseAdmin; 3213 } 3214 3215 private HBaseAdmin hbaseAdmin = null; 3216 3217 /** 3218 * Returns an {@link Hbck} instance. Needs be closed when done. 3219 */ 3220 public Hbck getHbck() throws IOException { 3221 return getConnection().getHbck(); 3222 } 3223 3224 /** 3225 * Unassign the named region. 3226 * 3227 * @param regionName The region to unassign. 3228 */ 3229 public void unassignRegion(String regionName) throws IOException { 3230 unassignRegion(Bytes.toBytes(regionName)); 3231 } 3232 3233 /** 3234 * Unassign the named region. 3235 * 3236 * @param regionName The region to unassign. 3237 */ 3238 public void unassignRegion(byte[] regionName) throws IOException { 3239 getAdmin().unassign(regionName, true); 3240 } 3241 3242 /** 3243 * Closes the region containing the given row. 3244 * 3245 * @param row The row to find the containing region. 3246 * @param table The table to find the region. 3247 */ 3248 public void unassignRegionByRow(String row, RegionLocator table) throws IOException { 3249 unassignRegionByRow(Bytes.toBytes(row), table); 3250 } 3251 3252 /** 3253 * Closes the region containing the given row. 3254 * 3255 * @param row The row to find the containing region. 3256 * @param table The table to find the region. 3257 * @throws IOException 3258 */ 3259 public void unassignRegionByRow(byte[] row, RegionLocator table) throws IOException { 3260 HRegionLocation hrl = table.getRegionLocation(row); 3261 unassignRegion(hrl.getRegionInfo().getRegionName()); 3262 } 3263 3264 /* 3265 * Retrieves a splittable region randomly from tableName 3266 * 3267 * @param tableName name of table 3268 * @param maxAttempts maximum number of attempts, unlimited for value of -1 3269 * @return the HRegion chosen, null if none was found within limit of maxAttempts 3270 */ 3271 public HRegion getSplittableRegion(TableName tableName, int maxAttempts) { 3272 List<HRegion> regions = getHBaseCluster().getRegions(tableName); 3273 int regCount = regions.size(); 3274 Set<Integer> attempted = new HashSet<>(); 3275 int idx; 3276 int attempts = 0; 3277 do { 3278 regions = getHBaseCluster().getRegions(tableName); 3279 if (regCount != regions.size()) { 3280 // if there was region movement, clear attempted Set 3281 attempted.clear(); 3282 } 3283 regCount = regions.size(); 3284 // There are chances that before we get the region for the table from an RS the region may 3285 // be going for CLOSE. This may be because online schema change is enabled 3286 if (regCount > 0) { 3287 idx = random.nextInt(regCount); 3288 // if we have just tried this region, there is no need to try again 3289 if (attempted.contains(idx)) 3290 continue; 3291 try { 3292 regions.get(idx).checkSplit(); 3293 return regions.get(idx); 3294 } catch (Exception ex) { 3295 LOG.warn("Caught exception", ex); 3296 attempted.add(idx); 3297 } 3298 } 3299 attempts++; 3300 } while (maxAttempts == -1 || attempts < maxAttempts); 3301 return null; 3302 } 3303 3304 public MiniDFSCluster getDFSCluster() { 3305 return dfsCluster; 3306 } 3307 3308 public void setDFSCluster(MiniDFSCluster cluster) throws IllegalStateException, IOException { 3309 setDFSCluster(cluster, true); 3310 } 3311 3312 /** 3313 * Set the MiniDFSCluster 3314 * @param cluster cluster to use 3315 * @param requireDown require the that cluster not be "up" (MiniDFSCluster#isClusterUp) before 3316 * it is set. 3317 * @throws IllegalStateException if the passed cluster is up when it is required to be down 3318 * @throws IOException if the FileSystem could not be set from the passed dfs cluster 3319 */ 3320 public void setDFSCluster(MiniDFSCluster cluster, boolean requireDown) 3321 throws IllegalStateException, IOException { 3322 if (dfsCluster != null && requireDown && dfsCluster.isClusterUp()) { 3323 throw new IllegalStateException("DFSCluster is already running! Shut it down first."); 3324 } 3325 this.dfsCluster = cluster; 3326 this.setFs(); 3327 } 3328 3329 public FileSystem getTestFileSystem() throws IOException { 3330 return HFileSystem.get(conf); 3331 } 3332 3333 /** 3334 * Wait until all regions in a table have been assigned. Waits default timeout before giving up 3335 * (30 seconds). 3336 * @param table Table to wait on. 3337 * @throws InterruptedException 3338 * @throws IOException 3339 */ 3340 public void waitTableAvailable(TableName table) 3341 throws InterruptedException, IOException { 3342 waitTableAvailable(table.getName(), 30000); 3343 } 3344 3345 public void waitTableAvailable(TableName table, long timeoutMillis) 3346 throws InterruptedException, IOException { 3347 waitFor(timeoutMillis, predicateTableAvailable(table)); 3348 } 3349 3350 /** 3351 * Wait until all regions in a table have been assigned 3352 * @param table Table to wait on. 3353 * @param timeoutMillis Timeout. 3354 * @throws InterruptedException 3355 * @throws IOException 3356 */ 3357 public void waitTableAvailable(byte[] table, long timeoutMillis) 3358 throws InterruptedException, IOException { 3359 waitFor(timeoutMillis, predicateTableAvailable(TableName.valueOf(table))); 3360 } 3361 3362 public String explainTableAvailability(TableName tableName) throws IOException { 3363 String msg = explainTableState(tableName, TableState.State.ENABLED) + ", "; 3364 if (getHBaseCluster().getMaster().isAlive()) { 3365 Map<RegionInfo, ServerName> assignments = 3366 getHBaseCluster().getMaster().getAssignmentManager().getRegionStates() 3367 .getRegionAssignments(); 3368 final List<Pair<RegionInfo, ServerName>> metaLocations = 3369 MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); 3370 for (Pair<RegionInfo, ServerName> metaLocation : metaLocations) { 3371 RegionInfo hri = metaLocation.getFirst(); 3372 ServerName sn = metaLocation.getSecond(); 3373 if (!assignments.containsKey(hri)) { 3374 msg += ", region " + hri 3375 + " not assigned, but found in meta, it expected to be on " + sn; 3376 3377 } else if (sn == null) { 3378 msg += ", region " + hri 3379 + " assigned, but has no server in meta"; 3380 } else if (!sn.equals(assignments.get(hri))) { 3381 msg += ", region " + hri 3382 + " assigned, but has different servers in meta and AM ( " + 3383 sn + " <> " + assignments.get(hri); 3384 } 3385 } 3386 } 3387 return msg; 3388 } 3389 3390 public String explainTableState(final TableName table, TableState.State state) 3391 throws IOException { 3392 TableState tableState = MetaTableAccessor.getTableState(connection, table); 3393 if (tableState == null) { 3394 return "TableState in META: No table state in META for table " + table 3395 + " last state in meta (including deleted is " + findLastTableState(table) + ")"; 3396 } else if (!tableState.inStates(state)) { 3397 return "TableState in META: Not " + state + " state, but " + tableState; 3398 } else { 3399 return "TableState in META: OK"; 3400 } 3401 } 3402 3403 @Nullable 3404 public TableState findLastTableState(final TableName table) throws IOException { 3405 final AtomicReference<TableState> lastTableState = new AtomicReference<>(null); 3406 MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { 3407 @Override 3408 public boolean visit(Result r) throws IOException { 3409 if (!Arrays.equals(r.getRow(), table.getName())) 3410 return false; 3411 TableState state = MetaTableAccessor.getTableState(r); 3412 if (state != null) 3413 lastTableState.set(state); 3414 return true; 3415 } 3416 }; 3417 MetaTableAccessor 3418 .scanMeta(connection, null, null, 3419 MetaTableAccessor.QueryType.TABLE, 3420 Integer.MAX_VALUE, visitor); 3421 return lastTableState.get(); 3422 } 3423 3424 /** 3425 * Waits for a table to be 'enabled'. Enabled means that table is set as 'enabled' and the 3426 * regions have been all assigned. Will timeout after default period (30 seconds) 3427 * Tolerates nonexistent table. 3428 * @param table the table to wait on. 3429 * @throws InterruptedException if interrupted while waiting 3430 * @throws IOException if an IO problem is encountered 3431 */ 3432 public void waitTableEnabled(TableName table) 3433 throws InterruptedException, IOException { 3434 waitTableEnabled(table, 30000); 3435 } 3436 3437 /** 3438 * Waits for a table to be 'enabled'. Enabled means that table is set as 'enabled' and the 3439 * regions have been all assigned. 3440 * @see #waitTableEnabled(TableName, long) 3441 * @param table Table to wait on. 3442 * @param timeoutMillis Time to wait on it being marked enabled. 3443 * @throws InterruptedException 3444 * @throws IOException 3445 */ 3446 public void waitTableEnabled(byte[] table, long timeoutMillis) 3447 throws InterruptedException, IOException { 3448 waitTableEnabled(TableName.valueOf(table), timeoutMillis); 3449 } 3450 3451 public void waitTableEnabled(TableName table, long timeoutMillis) 3452 throws IOException { 3453 waitFor(timeoutMillis, predicateTableEnabled(table)); 3454 } 3455 3456 /** 3457 * Waits for a table to be 'disabled'. Disabled means that table is set as 'disabled' 3458 * Will timeout after default period (30 seconds) 3459 * @param table Table to wait on. 3460 * @throws InterruptedException 3461 * @throws IOException 3462 */ 3463 public void waitTableDisabled(byte[] table) 3464 throws InterruptedException, IOException { 3465 waitTableDisabled(table, 30000); 3466 } 3467 3468 public void waitTableDisabled(TableName table, long millisTimeout) 3469 throws InterruptedException, IOException { 3470 waitFor(millisTimeout, predicateTableDisabled(table)); 3471 } 3472 3473 /** 3474 * Waits for a table to be 'disabled'. Disabled means that table is set as 'disabled' 3475 * @param table Table to wait on. 3476 * @param timeoutMillis Time to wait on it being marked disabled. 3477 * @throws InterruptedException 3478 * @throws IOException 3479 */ 3480 public void waitTableDisabled(byte[] table, long timeoutMillis) 3481 throws InterruptedException, IOException { 3482 waitTableDisabled(TableName.valueOf(table), timeoutMillis); 3483 } 3484 3485 /** 3486 * Make sure that at least the specified number of region servers 3487 * are running 3488 * @param num minimum number of region servers that should be running 3489 * @return true if we started some servers 3490 * @throws IOException 3491 */ 3492 public boolean ensureSomeRegionServersAvailable(final int num) 3493 throws IOException { 3494 boolean startedServer = false; 3495 MiniHBaseCluster hbaseCluster = getMiniHBaseCluster(); 3496 for (int i=hbaseCluster.getLiveRegionServerThreads().size(); i<num; ++i) { 3497 LOG.info("Started new server=" + hbaseCluster.startRegionServer()); 3498 startedServer = true; 3499 } 3500 3501 return startedServer; 3502 } 3503 3504 3505 /** 3506 * Make sure that at least the specified number of region servers 3507 * are running. We don't count the ones that are currently stopping or are 3508 * stopped. 3509 * @param num minimum number of region servers that should be running 3510 * @return true if we started some servers 3511 * @throws IOException 3512 */ 3513 public boolean ensureSomeNonStoppedRegionServersAvailable(final int num) 3514 throws IOException { 3515 boolean startedServer = ensureSomeRegionServersAvailable(num); 3516 3517 int nonStoppedServers = 0; 3518 for (JVMClusterUtil.RegionServerThread rst : 3519 getMiniHBaseCluster().getRegionServerThreads()) { 3520 3521 HRegionServer hrs = rst.getRegionServer(); 3522 if (hrs.isStopping() || hrs.isStopped()) { 3523 LOG.info("A region server is stopped or stopping:"+hrs); 3524 } else { 3525 nonStoppedServers++; 3526 } 3527 } 3528 for (int i=nonStoppedServers; i<num; ++i) { 3529 LOG.info("Started new server=" + getMiniHBaseCluster().startRegionServer()); 3530 startedServer = true; 3531 } 3532 return startedServer; 3533 } 3534 3535 3536 /** 3537 * This method clones the passed <code>c</code> configuration setting a new 3538 * user into the clone. Use it getting new instances of FileSystem. Only 3539 * works for DistributedFileSystem w/o Kerberos. 3540 * @param c Initial configuration 3541 * @param differentiatingSuffix Suffix to differentiate this user from others. 3542 * @return A new configuration instance with a different user set into it. 3543 * @throws IOException 3544 */ 3545 public static User getDifferentUser(final Configuration c, 3546 final String differentiatingSuffix) 3547 throws IOException { 3548 FileSystem currentfs = FileSystem.get(c); 3549 if (!(currentfs instanceof DistributedFileSystem) || User.isHBaseSecurityEnabled(c)) { 3550 return User.getCurrent(); 3551 } 3552 // Else distributed filesystem. Make a new instance per daemon. Below 3553 // code is taken from the AppendTestUtil over in hdfs. 3554 String username = User.getCurrent().getName() + 3555 differentiatingSuffix; 3556 User user = User.createUserForTesting(c, username, 3557 new String[]{"supergroup"}); 3558 return user; 3559 } 3560 3561 public static NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster) 3562 throws IOException { 3563 NavigableSet<String> online = new TreeSet<>(); 3564 for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) { 3565 try { 3566 for (RegionInfo region : 3567 ProtobufUtil.getOnlineRegions(rst.getRegionServer().getRSRpcServices())) { 3568 online.add(region.getRegionNameAsString()); 3569 } 3570 } catch (RegionServerStoppedException e) { 3571 // That's fine. 3572 } 3573 } 3574 for (MasterThread mt : cluster.getLiveMasterThreads()) { 3575 try { 3576 for (RegionInfo region : 3577 ProtobufUtil.getOnlineRegions(mt.getMaster().getRSRpcServices())) { 3578 online.add(region.getRegionNameAsString()); 3579 } 3580 } catch (RegionServerStoppedException e) { 3581 // That's fine. 3582 } catch (ServerNotRunningYetException e) { 3583 // That's fine. 3584 } 3585 } 3586 return online; 3587 } 3588 3589 /** 3590 * Set maxRecoveryErrorCount in DFSClient. In 0.20 pre-append its hard-coded to 5 and 3591 * makes tests linger. Here is the exception you'll see: 3592 * <pre> 3593 * 2010-06-15 11:52:28,511 WARN [DataStreamer for file /hbase/.logs/wal.1276627923013 block 3594 * blk_928005470262850423_1021] hdfs.DFSClient$DFSOutputStream(2657): Error Recovery for block 3595 * blk_928005470262850423_1021 failed because recovery from primary datanode 127.0.0.1:53683 3596 * failed 4 times. Pipeline was 127.0.0.1:53687, 127.0.0.1:53683. Will retry... 3597 * </pre> 3598 * @param stream A DFSClient.DFSOutputStream. 3599 * @param max 3600 * @throws NoSuchFieldException 3601 * @throws SecurityException 3602 * @throws IllegalAccessException 3603 * @throws IllegalArgumentException 3604 */ 3605 public static void setMaxRecoveryErrorCount(final OutputStream stream, 3606 final int max) { 3607 try { 3608 Class<?> [] clazzes = DFSClient.class.getDeclaredClasses(); 3609 for (Class<?> clazz: clazzes) { 3610 String className = clazz.getSimpleName(); 3611 if (className.equals("DFSOutputStream")) { 3612 if (clazz.isInstance(stream)) { 3613 Field maxRecoveryErrorCountField = 3614 stream.getClass().getDeclaredField("maxRecoveryErrorCount"); 3615 maxRecoveryErrorCountField.setAccessible(true); 3616 maxRecoveryErrorCountField.setInt(stream, max); 3617 break; 3618 } 3619 } 3620 } 3621 } catch (Exception e) { 3622 LOG.info("Could not set max recovery field", e); 3623 } 3624 } 3625 3626 /** 3627 * Uses directly the assignment manager to assign the region. and waits until the specified region 3628 * has completed assignment. 3629 * @return true if the region is assigned false otherwise. 3630 */ 3631 public boolean assignRegion(final RegionInfo regionInfo) 3632 throws IOException, InterruptedException { 3633 final AssignmentManager am = getHBaseCluster().getMaster().getAssignmentManager(); 3634 am.assign(regionInfo); 3635 return AssignmentTestingUtil.waitForAssignment(am, regionInfo); 3636 } 3637 3638 /** 3639 * Move region to destination server and wait till region is completely moved and online 3640 * 3641 * @param destRegion region to move 3642 * @param destServer destination server of the region 3643 * @throws InterruptedException 3644 * @throws IOException 3645 */ 3646 public void moveRegionAndWait(RegionInfo destRegion, ServerName destServer) 3647 throws InterruptedException, IOException { 3648 HMaster master = getMiniHBaseCluster().getMaster(); 3649 // TODO: Here we start the move. The move can take a while. 3650 getAdmin().move(destRegion.getEncodedNameAsBytes(), destServer); 3651 while (true) { 3652 ServerName serverName = master.getAssignmentManager().getRegionStates() 3653 .getRegionServerOfRegion(destRegion); 3654 if (serverName != null && serverName.equals(destServer)) { 3655 assertRegionOnServer(destRegion, serverName, 2000); 3656 break; 3657 } 3658 Thread.sleep(10); 3659 } 3660 } 3661 3662 /** 3663 * Wait until all regions for a table in hbase:meta have a non-empty 3664 * info:server, up to a configuable timeout value (default is 60 seconds) 3665 * This means all regions have been deployed, 3666 * master has been informed and updated hbase:meta with the regions deployed 3667 * server. 3668 * @param tableName the table name 3669 * @throws IOException 3670 */ 3671 public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOException { 3672 waitUntilAllRegionsAssigned(tableName, 3673 this.conf.getLong("hbase.client.sync.wait.timeout.msec", 60000)); 3674 } 3675 3676 /** 3677 * Waith until all system table's regions get assigned 3678 * @throws IOException 3679 */ 3680 public void waitUntilAllSystemRegionsAssigned() throws IOException { 3681 waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); 3682 waitUntilAllRegionsAssigned(TableName.NAMESPACE_TABLE_NAME); 3683 } 3684 3685 /** 3686 * Wait until all regions for a table in hbase:meta have a non-empty 3687 * info:server, or until timeout. This means all regions have been deployed, 3688 * master has been informed and updated hbase:meta with the regions deployed 3689 * server. 3690 * @param tableName the table name 3691 * @param timeout timeout, in milliseconds 3692 * @throws IOException 3693 */ 3694 public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) 3695 throws IOException { 3696 if (!TableName.isMetaTableName(tableName)) { 3697 try (final Table meta = getConnection().getTable(TableName.META_TABLE_NAME)) { 3698 LOG.debug("Waiting until all regions of table " + tableName + " get assigned. Timeout = " + 3699 timeout + "ms"); 3700 waitFor(timeout, 200, true, new ExplainingPredicate<IOException>() { 3701 @Override 3702 public String explainFailure() throws IOException { 3703 return explainTableAvailability(tableName); 3704 } 3705 3706 @Override 3707 public boolean evaluate() throws IOException { 3708 Scan scan = new Scan(); 3709 scan.addFamily(HConstants.CATALOG_FAMILY); 3710 boolean tableFound = false; 3711 try (ResultScanner s = meta.getScanner(scan)) { 3712 for (Result r; (r = s.next()) != null;) { 3713 byte[] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); 3714 HRegionInfo info = HRegionInfo.parseFromOrNull(b); 3715 if (info != null && info.getTable().equals(tableName)) { 3716 // Get server hosting this region from catalog family. Return false if no server 3717 // hosting this region, or if the server hosting this region was recently killed 3718 // (for fault tolerance testing). 3719 tableFound = true; 3720 byte[] server = 3721 r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER); 3722 if (server == null) { 3723 return false; 3724 } else { 3725 byte[] startCode = 3726 r.getValue(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER); 3727 ServerName serverName = 3728 ServerName.valueOf(Bytes.toString(server).replaceFirst(":", ",") + "," + 3729 Bytes.toLong(startCode)); 3730 if (!getHBaseClusterInterface().isDistributedCluster() && 3731 getHBaseCluster().isKilledRS(serverName)) { 3732 return false; 3733 } 3734 } 3735 if (RegionStateStore.getRegionState(r, info) != RegionState.State.OPEN) { 3736 return false; 3737 } 3738 } 3739 } 3740 } 3741 if (!tableFound) { 3742 LOG.warn("Didn't find the entries for table " + tableName + " in meta, already deleted?"); 3743 } 3744 return tableFound; 3745 } 3746 }); 3747 } 3748 } 3749 LOG.info("All regions for table " + tableName + " assigned to meta. Checking AM states."); 3750 // check from the master state if we are using a mini cluster 3751 if (!getHBaseClusterInterface().isDistributedCluster()) { 3752 // So, all regions are in the meta table but make sure master knows of the assignments before 3753 // returning -- sometimes this can lag. 3754 HMaster master = getHBaseCluster().getMaster(); 3755 final RegionStates states = master.getAssignmentManager().getRegionStates(); 3756 waitFor(timeout, 200, new ExplainingPredicate<IOException>() { 3757 @Override 3758 public String explainFailure() throws IOException { 3759 return explainTableAvailability(tableName); 3760 } 3761 3762 @Override 3763 public boolean evaluate() throws IOException { 3764 List<RegionInfo> hris = states.getRegionsOfTable(tableName); 3765 return hris != null && !hris.isEmpty(); 3766 } 3767 }); 3768 } 3769 LOG.info("All regions for table " + tableName + " assigned."); 3770 } 3771 3772 /** 3773 * Do a small get/scan against one store. This is required because store 3774 * has no actual methods of querying itself, and relies on StoreScanner. 3775 */ 3776 public static List<Cell> getFromStoreFile(HStore store, 3777 Get get) throws IOException { 3778 Scan scan = new Scan(get); 3779 InternalScanner scanner = (InternalScanner) store.getScanner(scan, 3780 scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()), 3781 // originally MultiVersionConcurrencyControl.resetThreadReadPoint() was called to set 3782 // readpoint 0. 3783 0); 3784 3785 List<Cell> result = new ArrayList<>(); 3786 scanner.next(result); 3787 if (!result.isEmpty()) { 3788 // verify that we are on the row we want: 3789 Cell kv = result.get(0); 3790 if (!CellUtil.matchingRows(kv, get.getRow())) { 3791 result.clear(); 3792 } 3793 } 3794 scanner.close(); 3795 return result; 3796 } 3797 3798 /** 3799 * Create region split keys between startkey and endKey 3800 * 3801 * @param startKey 3802 * @param endKey 3803 * @param numRegions the number of regions to be created. it has to be greater than 3. 3804 * @return resulting split keys 3805 */ 3806 public byte[][] getRegionSplitStartKeys(byte[] startKey, byte[] endKey, int numRegions){ 3807 assertTrue(numRegions>3); 3808 byte [][] tmpSplitKeys = Bytes.split(startKey, endKey, numRegions - 3); 3809 byte [][] result = new byte[tmpSplitKeys.length+1][]; 3810 System.arraycopy(tmpSplitKeys, 0, result, 1, tmpSplitKeys.length); 3811 result[0] = HConstants.EMPTY_BYTE_ARRAY; 3812 return result; 3813 } 3814 3815 /** 3816 * Do a small get/scan against one store. This is required because store 3817 * has no actual methods of querying itself, and relies on StoreScanner. 3818 */ 3819 public static List<Cell> getFromStoreFile(HStore store, 3820 byte [] row, 3821 NavigableSet<byte[]> columns 3822 ) throws IOException { 3823 Get get = new Get(row); 3824 Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap(); 3825 s.put(store.getColumnFamilyDescriptor().getName(), columns); 3826 3827 return getFromStoreFile(store,get); 3828 } 3829 3830 public static void assertKVListsEqual(String additionalMsg, 3831 final List<? extends Cell> expected, 3832 final List<? extends Cell> actual) { 3833 final int eLen = expected.size(); 3834 final int aLen = actual.size(); 3835 final int minLen = Math.min(eLen, aLen); 3836 3837 int i; 3838 for (i = 0; i < minLen 3839 && CellComparator.getInstance().compare(expected.get(i), actual.get(i)) == 0; 3840 ++i) {} 3841 3842 if (additionalMsg == null) { 3843 additionalMsg = ""; 3844 } 3845 if (!additionalMsg.isEmpty()) { 3846 additionalMsg = ". " + additionalMsg; 3847 } 3848 3849 if (eLen != aLen || i != minLen) { 3850 throw new AssertionError( 3851 "Expected and actual KV arrays differ at position " + i + ": " + 3852 safeGetAsStr(expected, i) + " (length " + eLen +") vs. " + 3853 safeGetAsStr(actual, i) + " (length " + aLen + ")" + additionalMsg); 3854 } 3855 } 3856 3857 public static <T> String safeGetAsStr(List<T> lst, int i) { 3858 if (0 <= i && i < lst.size()) { 3859 return lst.get(i).toString(); 3860 } else { 3861 return "<out_of_range>"; 3862 } 3863 } 3864 3865 public String getClusterKey() { 3866 return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" 3867 + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":" 3868 + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, 3869 HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); 3870 } 3871 3872 /** Creates a random table with the given parameters */ 3873 public Table createRandomTable(TableName tableName, 3874 final Collection<String> families, 3875 final int maxVersions, 3876 final int numColsPerRow, 3877 final int numFlushes, 3878 final int numRegions, 3879 final int numRowsPerFlush) 3880 throws IOException, InterruptedException { 3881 3882 LOG.info("\n\nCreating random table " + tableName + " with " + numRegions + 3883 " regions, " + numFlushes + " storefiles per region, " + 3884 numRowsPerFlush + " rows per flush, maxVersions=" + maxVersions + 3885 "\n"); 3886 3887 final Random rand = new Random(tableName.hashCode() * 17L + 12938197137L); 3888 final int numCF = families.size(); 3889 final byte[][] cfBytes = new byte[numCF][]; 3890 { 3891 int cfIndex = 0; 3892 for (String cf : families) { 3893 cfBytes[cfIndex++] = Bytes.toBytes(cf); 3894 } 3895 } 3896 3897 final int actualStartKey = 0; 3898 final int actualEndKey = Integer.MAX_VALUE; 3899 final int keysPerRegion = (actualEndKey - actualStartKey) / numRegions; 3900 final int splitStartKey = actualStartKey + keysPerRegion; 3901 final int splitEndKey = actualEndKey - keysPerRegion; 3902 final String keyFormat = "%08x"; 3903 final Table table = createTable(tableName, cfBytes, 3904 maxVersions, 3905 Bytes.toBytes(String.format(keyFormat, splitStartKey)), 3906 Bytes.toBytes(String.format(keyFormat, splitEndKey)), 3907 numRegions); 3908 3909 if (hbaseCluster != null) { 3910 getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME); 3911 } 3912 3913 BufferedMutator mutator = getConnection().getBufferedMutator(tableName); 3914 3915 for (int iFlush = 0; iFlush < numFlushes; ++iFlush) { 3916 for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) { 3917 final byte[] row = Bytes.toBytes(String.format(keyFormat, 3918 actualStartKey + rand.nextInt(actualEndKey - actualStartKey))); 3919 3920 Put put = new Put(row); 3921 Delete del = new Delete(row); 3922 for (int iCol = 0; iCol < numColsPerRow; ++iCol) { 3923 final byte[] cf = cfBytes[rand.nextInt(numCF)]; 3924 final long ts = rand.nextInt(); 3925 final byte[] qual = Bytes.toBytes("col" + iCol); 3926 if (rand.nextBoolean()) { 3927 final byte[] value = Bytes.toBytes("value_for_row_" + iRow + 3928 "_cf_" + Bytes.toStringBinary(cf) + "_col_" + iCol + "_ts_" + 3929 ts + "_random_" + rand.nextLong()); 3930 put.addColumn(cf, qual, ts, value); 3931 } else if (rand.nextDouble() < 0.8) { 3932 del.addColumn(cf, qual, ts); 3933 } else { 3934 del.addColumns(cf, qual, ts); 3935 } 3936 } 3937 3938 if (!put.isEmpty()) { 3939 mutator.mutate(put); 3940 } 3941 3942 if (!del.isEmpty()) { 3943 mutator.mutate(del); 3944 } 3945 } 3946 LOG.info("Initiating flush #" + iFlush + " for table " + tableName); 3947 mutator.flush(); 3948 if (hbaseCluster != null) { 3949 getMiniHBaseCluster().flushcache(table.getName()); 3950 } 3951 } 3952 mutator.close(); 3953 3954 return table; 3955 } 3956 3957 public static int randomFreePort() { 3958 return HBaseCommonTestingUtility.randomFreePort(); 3959 } 3960 public static String randomMultiCastAddress() { 3961 return "226.1.1." + random.nextInt(254); 3962 } 3963 3964 public static void waitForHostPort(String host, int port) 3965 throws IOException { 3966 final int maxTimeMs = 10000; 3967 final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS; 3968 IOException savedException = null; 3969 LOG.info("Waiting for server at " + host + ":" + port); 3970 for (int attempt = 0; attempt < maxNumAttempts; ++attempt) { 3971 try { 3972 Socket sock = new Socket(InetAddress.getByName(host), port); 3973 sock.close(); 3974 savedException = null; 3975 LOG.info("Server at " + host + ":" + port + " is available"); 3976 break; 3977 } catch (UnknownHostException e) { 3978 throw new IOException("Failed to look up " + host, e); 3979 } catch (IOException e) { 3980 savedException = e; 3981 } 3982 Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS); 3983 } 3984 3985 if (savedException != null) { 3986 throw savedException; 3987 } 3988 } 3989 3990 /** 3991 * Creates a pre-split table for load testing. If the table already exists, 3992 * logs a warning and continues. 3993 * @return the number of regions the table was split into 3994 */ 3995 public static int createPreSplitLoadTestTable(Configuration conf, 3996 TableName tableName, byte[] columnFamily, Algorithm compression, 3997 DataBlockEncoding dataBlockEncoding) throws IOException { 3998 return createPreSplitLoadTestTable(conf, tableName, 3999 columnFamily, compression, dataBlockEncoding, DEFAULT_REGIONS_PER_SERVER, 1, 4000 Durability.USE_DEFAULT); 4001 } 4002 /** 4003 * Creates a pre-split table for load testing. If the table already exists, 4004 * logs a warning and continues. 4005 * @return the number of regions the table was split into 4006 */ 4007 public static int createPreSplitLoadTestTable(Configuration conf, 4008 TableName tableName, byte[] columnFamily, Algorithm compression, 4009 DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication, 4010 Durability durability) 4011 throws IOException { 4012 HTableDescriptor desc = new HTableDescriptor(tableName); 4013 desc.setDurability(durability); 4014 desc.setRegionReplication(regionReplication); 4015 HColumnDescriptor hcd = new HColumnDescriptor(columnFamily); 4016 hcd.setDataBlockEncoding(dataBlockEncoding); 4017 hcd.setCompressionType(compression); 4018 return createPreSplitLoadTestTable(conf, desc, hcd, numRegionsPerServer); 4019 } 4020 4021 /** 4022 * Creates a pre-split table for load testing. If the table already exists, 4023 * logs a warning and continues. 4024 * @return the number of regions the table was split into 4025 */ 4026 public static int createPreSplitLoadTestTable(Configuration conf, 4027 TableName tableName, byte[][] columnFamilies, Algorithm compression, 4028 DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication, 4029 Durability durability) 4030 throws IOException { 4031 HTableDescriptor desc = new HTableDescriptor(tableName); 4032 desc.setDurability(durability); 4033 desc.setRegionReplication(regionReplication); 4034 HColumnDescriptor[] hcds = new HColumnDescriptor[columnFamilies.length]; 4035 for (int i = 0; i < columnFamilies.length; i++) { 4036 HColumnDescriptor hcd = new HColumnDescriptor(columnFamilies[i]); 4037 hcd.setDataBlockEncoding(dataBlockEncoding); 4038 hcd.setCompressionType(compression); 4039 hcds[i] = hcd; 4040 } 4041 return createPreSplitLoadTestTable(conf, desc, hcds, numRegionsPerServer); 4042 } 4043 4044 /** 4045 * Creates a pre-split table for load testing. If the table already exists, 4046 * logs a warning and continues. 4047 * @return the number of regions the table was split into 4048 */ 4049 public static int createPreSplitLoadTestTable(Configuration conf, 4050 TableDescriptor desc, ColumnFamilyDescriptor hcd) throws IOException { 4051 return createPreSplitLoadTestTable(conf, desc, hcd, DEFAULT_REGIONS_PER_SERVER); 4052 } 4053 4054 /** 4055 * Creates a pre-split table for load testing. If the table already exists, 4056 * logs a warning and continues. 4057 * @return the number of regions the table was split into 4058 */ 4059 public static int createPreSplitLoadTestTable(Configuration conf, 4060 TableDescriptor desc, ColumnFamilyDescriptor hcd, int numRegionsPerServer) throws IOException { 4061 return createPreSplitLoadTestTable(conf, desc, new ColumnFamilyDescriptor[] {hcd}, 4062 numRegionsPerServer); 4063 } 4064 4065 /** 4066 * Creates a pre-split table for load testing. If the table already exists, 4067 * logs a warning and continues. 4068 * @return the number of regions the table was split into 4069 */ 4070 public static int createPreSplitLoadTestTable(Configuration conf, 4071 TableDescriptor desc, ColumnFamilyDescriptor[] hcds, 4072 int numRegionsPerServer) throws IOException { 4073 return createPreSplitLoadTestTable(conf, desc, hcds, 4074 new RegionSplitter.HexStringSplit(), numRegionsPerServer); 4075 } 4076 4077 /** 4078 * Creates a pre-split table for load testing. If the table already exists, 4079 * logs a warning and continues. 4080 * @return the number of regions the table was split into 4081 */ 4082 public static int createPreSplitLoadTestTable(Configuration conf, 4083 TableDescriptor td, ColumnFamilyDescriptor[] cds, 4084 SplitAlgorithm splitter, int numRegionsPerServer) throws IOException { 4085 TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(td); 4086 for (ColumnFamilyDescriptor cd : cds) { 4087 if (!td.hasColumnFamily(cd.getName())) { 4088 builder.setColumnFamily(cd); 4089 } 4090 } 4091 td = builder.build(); 4092 int totalNumberOfRegions = 0; 4093 Connection unmanagedConnection = ConnectionFactory.createConnection(conf); 4094 Admin admin = unmanagedConnection.getAdmin(); 4095 4096 try { 4097 // create a table a pre-splits regions. 4098 // The number of splits is set as: 4099 // region servers * regions per region server). 4100 int numberOfServers = admin.getRegionServers().size(); 4101 if (numberOfServers == 0) { 4102 throw new IllegalStateException("No live regionservers"); 4103 } 4104 4105 totalNumberOfRegions = numberOfServers * numRegionsPerServer; 4106 LOG.info("Number of live regionservers: " + numberOfServers + ", " + 4107 "pre-splitting table into " + totalNumberOfRegions + " regions " + 4108 "(regions per server: " + numRegionsPerServer + ")"); 4109 4110 byte[][] splits = splitter.split( 4111 totalNumberOfRegions); 4112 4113 admin.createTable(td, splits); 4114 } catch (MasterNotRunningException e) { 4115 LOG.error("Master not running", e); 4116 throw new IOException(e); 4117 } catch (TableExistsException e) { 4118 LOG.warn("Table " + td.getTableName() + 4119 " already exists, continuing"); 4120 } finally { 4121 admin.close(); 4122 unmanagedConnection.close(); 4123 } 4124 return totalNumberOfRegions; 4125 } 4126 4127 public static int getMetaRSPort(Connection connection) throws IOException { 4128 try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) { 4129 return locator.getRegionLocation(Bytes.toBytes("")).getPort(); 4130 } 4131 } 4132 4133 /** 4134 * Due to async racing issue, a region may not be in 4135 * the online region list of a region server yet, after 4136 * the assignment znode is deleted and the new assignment 4137 * is recorded in master. 4138 */ 4139 public void assertRegionOnServer( 4140 final RegionInfo hri, final ServerName server, 4141 final long timeout) throws IOException, InterruptedException { 4142 long timeoutTime = System.currentTimeMillis() + timeout; 4143 while (true) { 4144 List<RegionInfo> regions = getAdmin().getRegions(server); 4145 if (regions.stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0)) return; 4146 long now = System.currentTimeMillis(); 4147 if (now > timeoutTime) break; 4148 Thread.sleep(10); 4149 } 4150 fail("Could not find region " + hri.getRegionNameAsString() 4151 + " on server " + server); 4152 } 4153 4154 /** 4155 * Check to make sure the region is open on the specified 4156 * region server, but not on any other one. 4157 */ 4158 public void assertRegionOnlyOnServer( 4159 final RegionInfo hri, final ServerName server, 4160 final long timeout) throws IOException, InterruptedException { 4161 long timeoutTime = System.currentTimeMillis() + timeout; 4162 while (true) { 4163 List<RegionInfo> regions = getAdmin().getRegions(server); 4164 if (regions.stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0)) { 4165 List<JVMClusterUtil.RegionServerThread> rsThreads = 4166 getHBaseCluster().getLiveRegionServerThreads(); 4167 for (JVMClusterUtil.RegionServerThread rsThread: rsThreads) { 4168 HRegionServer rs = rsThread.getRegionServer(); 4169 if (server.equals(rs.getServerName())) { 4170 continue; 4171 } 4172 Collection<HRegion> hrs = rs.getOnlineRegionsLocalContext(); 4173 for (HRegion r: hrs) { 4174 assertTrue("Region should not be double assigned", 4175 r.getRegionInfo().getRegionId() != hri.getRegionId()); 4176 } 4177 } 4178 return; // good, we are happy 4179 } 4180 long now = System.currentTimeMillis(); 4181 if (now > timeoutTime) break; 4182 Thread.sleep(10); 4183 } 4184 fail("Could not find region " + hri.getRegionNameAsString() 4185 + " on server " + server); 4186 } 4187 4188 public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd) throws IOException { 4189 TableDescriptor td = 4190 TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).setColumnFamily(cd).build(); 4191 RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build(); 4192 return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), td); 4193 } 4194 4195 public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd, 4196 BlockCache blockCache) throws IOException { 4197 TableDescriptor td = 4198 TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).setColumnFamily(cd).build(); 4199 RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build(); 4200 return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), td, blockCache); 4201 } 4202 4203 public void setFileSystemURI(String fsURI) { 4204 FS_URI = fsURI; 4205 } 4206 4207 /** 4208 * Returns a {@link Predicate} for checking that there are no regions in transition in master 4209 */ 4210 public ExplainingPredicate<IOException> predicateNoRegionsInTransition() { 4211 return new ExplainingPredicate<IOException>() { 4212 @Override 4213 public String explainFailure() throws IOException { 4214 final RegionStates regionStates = getMiniHBaseCluster().getMaster() 4215 .getAssignmentManager().getRegionStates(); 4216 return "found in transition: " + regionStates.getRegionsInTransition().toString(); 4217 } 4218 4219 @Override 4220 public boolean evaluate() throws IOException { 4221 HMaster master = getMiniHBaseCluster().getMaster(); 4222 if (master == null) return false; 4223 AssignmentManager am = master.getAssignmentManager(); 4224 if (am == null) return false; 4225 return !am.hasRegionsInTransition(); 4226 } 4227 }; 4228 } 4229 4230 /** 4231 * Returns a {@link Predicate} for checking that table is enabled 4232 */ 4233 public Waiter.Predicate<IOException> predicateTableEnabled(final TableName tableName) { 4234 return new ExplainingPredicate<IOException>() { 4235 @Override 4236 public String explainFailure() throws IOException { 4237 return explainTableState(tableName, TableState.State.ENABLED); 4238 } 4239 4240 @Override 4241 public boolean evaluate() throws IOException { 4242 return getAdmin().tableExists(tableName) && getAdmin().isTableEnabled(tableName); 4243 } 4244 }; 4245 } 4246 4247 /** 4248 * Returns a {@link Predicate} for checking that table is enabled 4249 */ 4250 public Waiter.Predicate<IOException> predicateTableDisabled(final TableName tableName) { 4251 return new ExplainingPredicate<IOException>() { 4252 @Override 4253 public String explainFailure() throws IOException { 4254 return explainTableState(tableName, TableState.State.DISABLED); 4255 } 4256 4257 @Override 4258 public boolean evaluate() throws IOException { 4259 return getAdmin().isTableDisabled(tableName); 4260 } 4261 }; 4262 } 4263 4264 /** 4265 * Returns a {@link Predicate} for checking that table is enabled 4266 */ 4267 public Waiter.Predicate<IOException> predicateTableAvailable(final TableName tableName) { 4268 return new ExplainingPredicate<IOException>() { 4269 @Override 4270 public String explainFailure() throws IOException { 4271 return explainTableAvailability(tableName); 4272 } 4273 4274 @Override 4275 public boolean evaluate() throws IOException { 4276 boolean tableAvailable = getAdmin().isTableAvailable(tableName); 4277 if (tableAvailable) { 4278 try (Table table = getConnection().getTable(tableName)) { 4279 TableDescriptor htd = table.getDescriptor(); 4280 for (HRegionLocation loc : getConnection().getRegionLocator(tableName) 4281 .getAllRegionLocations()) { 4282 Scan scan = new Scan().withStartRow(loc.getRegionInfo().getStartKey()) 4283 .withStopRow(loc.getRegionInfo().getEndKey()).setOneRowLimit() 4284 .setMaxResultsPerColumnFamily(1).setCacheBlocks(false); 4285 for (byte[] family : htd.getColumnFamilyNames()) { 4286 scan.addFamily(family); 4287 } 4288 try (ResultScanner scanner = table.getScanner(scan)) { 4289 scanner.next(); 4290 } 4291 } 4292 } 4293 } 4294 return tableAvailable; 4295 } 4296 }; 4297 } 4298 4299 /** 4300 * Wait until no regions in transition. 4301 * @param timeout How long to wait. 4302 * @throws IOException 4303 */ 4304 public void waitUntilNoRegionsInTransition(final long timeout) throws IOException { 4305 waitFor(timeout, predicateNoRegionsInTransition()); 4306 } 4307 4308 /** 4309 * Wait until no regions in transition. (time limit 15min) 4310 * @throws IOException 4311 */ 4312 public void waitUntilNoRegionsInTransition() throws IOException { 4313 waitUntilNoRegionsInTransition(15 * 60000); 4314 } 4315 4316 /** 4317 * Wait until labels is ready in VisibilityLabelsCache. 4318 * @param timeoutMillis 4319 * @param labels 4320 */ 4321 public void waitLabelAvailable(long timeoutMillis, final String... labels) { 4322 final VisibilityLabelsCache labelsCache = VisibilityLabelsCache.get(); 4323 waitFor(timeoutMillis, new Waiter.ExplainingPredicate<RuntimeException>() { 4324 4325 @Override 4326 public boolean evaluate() { 4327 for (String label : labels) { 4328 if (labelsCache.getLabelOrdinal(label) == 0) { 4329 return false; 4330 } 4331 } 4332 return true; 4333 } 4334 4335 @Override 4336 public String explainFailure() { 4337 for (String label : labels) { 4338 if (labelsCache.getLabelOrdinal(label) == 0) { 4339 return label + " is not available yet"; 4340 } 4341 } 4342 return ""; 4343 } 4344 }); 4345 } 4346 4347 /** 4348 * Create a set of column descriptors with the combination of compression, 4349 * encoding, bloom codecs available. 4350 * @return the list of column descriptors 4351 */ 4352 public static List<HColumnDescriptor> generateColumnDescriptors() { 4353 return generateColumnDescriptors(""); 4354 } 4355 4356 /** 4357 * Create a set of column descriptors with the combination of compression, 4358 * encoding, bloom codecs available. 4359 * @param prefix family names prefix 4360 * @return the list of column descriptors 4361 */ 4362 public static List<HColumnDescriptor> generateColumnDescriptors(final String prefix) { 4363 List<HColumnDescriptor> htds = new ArrayList<>(); 4364 long familyId = 0; 4365 for (Compression.Algorithm compressionType: getSupportedCompressionAlgorithms()) { 4366 for (DataBlockEncoding encodingType: DataBlockEncoding.values()) { 4367 for (BloomType bloomType: BloomType.values()) { 4368 String name = String.format("%s-cf-!@#&-%d!@#", prefix, familyId); 4369 HColumnDescriptor htd = new HColumnDescriptor(name); 4370 htd.setCompressionType(compressionType); 4371 htd.setDataBlockEncoding(encodingType); 4372 htd.setBloomFilterType(bloomType); 4373 htds.add(htd); 4374 familyId++; 4375 } 4376 } 4377 } 4378 return htds; 4379 } 4380 4381 /** 4382 * Get supported compression algorithms. 4383 * @return supported compression algorithms. 4384 */ 4385 public static Compression.Algorithm[] getSupportedCompressionAlgorithms() { 4386 String[] allAlgos = HFile.getSupportedCompressionAlgorithms(); 4387 List<Compression.Algorithm> supportedAlgos = new ArrayList<>(); 4388 for (String algoName : allAlgos) { 4389 try { 4390 Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName); 4391 algo.getCompressor(); 4392 supportedAlgos.add(algo); 4393 } catch (Throwable t) { 4394 // this algo is not available 4395 } 4396 } 4397 return supportedAlgos.toArray(new Algorithm[supportedAlgos.size()]); 4398 } 4399 4400 public Result getClosestRowBefore(Region r, byte[] row, byte[] family) throws IOException { 4401 Scan scan = new Scan(row); 4402 scan.setSmall(true); 4403 scan.setCaching(1); 4404 scan.setReversed(true); 4405 scan.addFamily(family); 4406 try (RegionScanner scanner = r.getScanner(scan)) { 4407 List<Cell> cells = new ArrayList<>(1); 4408 scanner.next(cells); 4409 if (r.getRegionInfo().isMetaRegion() && !isTargetTable(row, cells.get(0))) { 4410 return null; 4411 } 4412 return Result.create(cells); 4413 } 4414 } 4415 4416 private boolean isTargetTable(final byte[] inRow, Cell c) { 4417 String inputRowString = Bytes.toString(inRow); 4418 int i = inputRowString.indexOf(HConstants.DELIMITER); 4419 String outputRowString = Bytes.toString(c.getRowArray(), c.getRowOffset(), c.getRowLength()); 4420 int o = outputRowString.indexOf(HConstants.DELIMITER); 4421 return inputRowString.substring(0, i).equals(outputRowString.substring(0, o)); 4422 } 4423 4424 /** 4425 * Sets up {@link MiniKdc} for testing security. 4426 * Uses {@link HBaseKerberosUtils} to set the given keytab file as 4427 * {@link HBaseKerberosUtils#KRB_KEYTAB_FILE}. 4428 * FYI, there is also the easier-to-use kerby KDC server and utility for using it, 4429 * {@link org.apache.hadoop.hbase.util.SimpleKdcServerUtil}. The kerby KDC server is preferred; 4430 * less baggage. It came in in HBASE-5291. 4431 */ 4432 public MiniKdc setupMiniKdc(File keytabFile) throws Exception { 4433 Properties conf = MiniKdc.createConf(); 4434 conf.put(MiniKdc.DEBUG, true); 4435 MiniKdc kdc = null; 4436 File dir = null; 4437 // There is time lag between selecting a port and trying to bind with it. It's possible that 4438 // another service captures the port in between which'll result in BindException. 4439 boolean bindException; 4440 int numTries = 0; 4441 do { 4442 try { 4443 bindException = false; 4444 dir = new File(getDataTestDir("kdc").toUri().getPath()); 4445 kdc = new MiniKdc(conf, dir); 4446 kdc.start(); 4447 } catch (BindException e) { 4448 FileUtils.deleteDirectory(dir); // clean directory 4449 numTries++; 4450 if (numTries == 3) { 4451 LOG.error("Failed setting up MiniKDC. Tried " + numTries + " times."); 4452 throw e; 4453 } 4454 LOG.error("BindException encountered when setting up MiniKdc. Trying again."); 4455 bindException = true; 4456 } 4457 } while (bindException); 4458 HBaseKerberosUtils.setKeytabFileForTesting(keytabFile.getAbsolutePath()); 4459 return kdc; 4460 } 4461 4462 public int getNumHFiles(final TableName tableName, final byte[] family) { 4463 int numHFiles = 0; 4464 for (RegionServerThread regionServerThread : getMiniHBaseCluster().getRegionServerThreads()) { 4465 numHFiles+= getNumHFilesForRS(regionServerThread.getRegionServer(), tableName, 4466 family); 4467 } 4468 return numHFiles; 4469 } 4470 4471 public int getNumHFilesForRS(final HRegionServer rs, final TableName tableName, 4472 final byte[] family) { 4473 int numHFiles = 0; 4474 for (Region region : rs.getRegions(tableName)) { 4475 numHFiles += region.getStore(family).getStorefilesCount(); 4476 } 4477 return numHFiles; 4478 } 4479 4480 public void verifyTableDescriptorIgnoreTableName(TableDescriptor ltd, TableDescriptor rtd) { 4481 assertEquals(ltd.getValues().hashCode(), rtd.getValues().hashCode()); 4482 Collection<ColumnFamilyDescriptor> ltdFamilies = Arrays.asList(ltd.getColumnFamilies()); 4483 Collection<ColumnFamilyDescriptor> rtdFamilies = Arrays.asList(rtd.getColumnFamilies()); 4484 assertEquals(ltdFamilies.size(), rtdFamilies.size()); 4485 for (Iterator<ColumnFamilyDescriptor> it = ltdFamilies.iterator(), it2 = 4486 rtdFamilies.iterator(); it.hasNext();) { 4487 assertEquals(0, 4488 ColumnFamilyDescriptor.COMPARATOR.compare(it.next(), it2.next())); 4489 } 4490 } 4491 4492 /** 4493 * Await the successful return of {@code condition}, sleeping {@code sleepMillis} between 4494 * invocations. 4495 */ 4496 public static void await(final long sleepMillis, final BooleanSupplier condition) 4497 throws InterruptedException { 4498 try { 4499 while (!condition.getAsBoolean()) { 4500 Thread.sleep(sleepMillis); 4501 } 4502 } catch (RuntimeException e) { 4503 if (e.getCause() instanceof AssertionError) { 4504 throw (AssertionError) e.getCause(); 4505 } 4506 throw e; 4507 } 4508 } 4509}