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