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