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