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