1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase;
19
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertTrue;
22 import static org.junit.Assert.fail;
23
24 import java.io.File;
25 import java.io.IOException;
26 import java.io.OutputStream;
27 import java.lang.reflect.Field;
28 import java.lang.reflect.Modifier;
29 import java.net.BindException;
30 import java.net.DatagramSocket;
31 import java.net.InetAddress;
32 import java.net.ServerSocket;
33 import java.net.Socket;
34 import java.net.UnknownHostException;
35 import java.security.MessageDigest;
36 import java.util.ArrayList;
37 import java.util.Arrays;
38 import java.util.Collection;
39 import java.util.Collections;
40 import java.util.HashSet;
41 import java.util.List;
42 import java.util.Map;
43 import java.util.NavigableSet;
44 import java.util.Properties;
45 import java.util.Random;
46 import java.util.Set;
47 import java.util.TreeSet;
48 import java.util.UUID;
49 import java.util.concurrent.atomic.AtomicReference;
50 import java.util.concurrent.TimeUnit;
51
52 import org.apache.commons.io.FileUtils;
53 import org.apache.commons.lang.RandomStringUtils;
54 import org.apache.commons.logging.Log;
55 import org.apache.commons.logging.LogFactory;
56 import org.apache.commons.logging.impl.Jdk14Logger;
57 import org.apache.commons.logging.impl.Log4JLogger;
58 import org.apache.hadoop.conf.Configuration;
59 import org.apache.hadoop.fs.FileSystem;
60 import org.apache.hadoop.fs.Path;
61 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
62 import org.apache.hadoop.hbase.Waiter.Predicate;
63 import org.apache.hadoop.hbase.classification.InterfaceAudience;
64 import org.apache.hadoop.hbase.classification.InterfaceStability;
65 import org.apache.hadoop.hbase.client.Admin;
66 import org.apache.hadoop.hbase.client.Connection;
67 import org.apache.hadoop.hbase.client.ConnectionFactory;
68 import org.apache.hadoop.hbase.client.Consistency;
69 import org.apache.hadoop.hbase.client.Delete;
70 import org.apache.hadoop.hbase.client.Durability;
71 import org.apache.hadoop.hbase.client.Get;
72 import org.apache.hadoop.hbase.client.HBaseAdmin;
73 import org.apache.hadoop.hbase.client.HConnection;
74 import org.apache.hadoop.hbase.client.HTable;
75 import org.apache.hadoop.hbase.client.Put;
76 import org.apache.hadoop.hbase.client.RegionLocator;
77 import org.apache.hadoop.hbase.client.Result;
78 import org.apache.hadoop.hbase.client.ResultScanner;
79 import org.apache.hadoop.hbase.client.Scan;
80 import org.apache.hadoop.hbase.client.Table;
81 import org.apache.hadoop.hbase.fs.HFileSystem;
82 import org.apache.hadoop.hbase.io.compress.Compression;
83 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
84 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
85 import org.apache.hadoop.hbase.io.hfile.ChecksumUtil;
86 import org.apache.hadoop.hbase.io.hfile.HFile;
87 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
88 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
89 import org.apache.hadoop.hbase.mapreduce.MapreduceTestingShim;
90 import org.apache.hadoop.hbase.master.AssignmentManager;
91 import org.apache.hadoop.hbase.master.HMaster;
92 import org.apache.hadoop.hbase.master.RegionStates;
93 import org.apache.hadoop.hbase.master.ServerManager;
94 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
95 import org.apache.hadoop.hbase.regionserver.BloomType;
96 import org.apache.hadoop.hbase.regionserver.HRegion;
97 import org.apache.hadoop.hbase.regionserver.HRegionServer;
98 import org.apache.hadoop.hbase.regionserver.HStore;
99 import org.apache.hadoop.hbase.regionserver.InternalScanner;
100 import org.apache.hadoop.hbase.regionserver.Region;
101 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
102 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
103 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
104 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
105 import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
106 import org.apache.hadoop.hbase.security.User;
107 import org.apache.hadoop.hbase.security.visibility.VisibilityLabelsCache;
108 import org.apache.hadoop.hbase.util.Bytes;
109 import org.apache.hadoop.hbase.util.CommonFSUtils;
110 import org.apache.hadoop.hbase.util.FSTableDescriptors;
111 import org.apache.hadoop.hbase.util.FSUtils;
112 import org.apache.hadoop.hbase.util.JVMClusterUtil;
113 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
114 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
115 import org.apache.hadoop.hbase.util.Pair;
116 import org.apache.hadoop.hbase.util.RegionSplitter;
117 import org.apache.hadoop.hbase.util.RetryCounter;
118 import org.apache.hadoop.hbase.util.Threads;
119 import org.apache.hadoop.hbase.wal.WAL;
120 import org.apache.hadoop.hbase.wal.WALFactory;
121 import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
122 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
123 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
124 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
125 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
126 import org.apache.hadoop.hdfs.DFSClient;
127 import org.apache.hadoop.hdfs.DistributedFileSystem;
128 import org.apache.hadoop.hdfs.MiniDFSCluster;
129 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
130 import org.apache.hadoop.mapred.JobConf;
131 import org.apache.hadoop.mapred.MiniMRCluster;
132 import org.apache.hadoop.mapred.TaskLog;
133 import org.apache.hadoop.minikdc.MiniKdc;
134 import org.apache.zookeeper.KeeperException;
135 import org.apache.zookeeper.KeeperException.NodeExistsException;
136 import org.apache.zookeeper.WatchedEvent;
137 import org.apache.zookeeper.ZooKeeper;
138 import org.apache.zookeeper.ZooKeeper.States;
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154 @InterfaceAudience.Public
155 @InterfaceStability.Evolving
156 @SuppressWarnings("deprecation")
157 public class HBaseTestingUtility extends HBaseCommonTestingUtility {
158 private MiniZooKeeperCluster zkCluster = null;
159
160 public static final String REGIONS_PER_SERVER_KEY = "hbase.test.regions-per-server";
161
162
163
164
165 public static final int DEFAULT_REGIONS_PER_SERVER = 3;
166
167
168 public static final String PRESPLIT_TEST_TABLE_KEY = "hbase.test.pre-split-table";
169 public static final boolean PRESPLIT_TEST_TABLE = true;
170
171 public static final String USE_LOCAL_FILESYSTEM = "hbase.test.local.fileSystem";
172
173
174
175
176 private boolean passedZkCluster = false;
177 private MiniDFSCluster dfsCluster = null;
178
179 private volatile HBaseCluster hbaseCluster = null;
180 private MiniMRCluster mrCluster = null;
181
182
183 private volatile boolean miniClusterRunning;
184
185 private String hadoopLogDir;
186
187
188 private File clusterTestDir = null;
189
190
191
192 private Path dataTestDirOnTestFS = null;
193
194 private final AtomicReference<Connection> connectionRef = new AtomicReference<>();
195
196
197
198
199
200
201
202
203 @Deprecated
204 private static final String TEST_DIRECTORY_KEY = "test.build.data";
205
206
207 private static String FS_URI;
208
209
210 private static final Set<Integer> takenRandomPorts = new HashSet<Integer>();
211
212
213 public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
214 Arrays.asList(new Object[][] {
215 { Compression.Algorithm.NONE },
216 { Compression.Algorithm.GZ }
217 });
218
219
220 public static final List<Object[]> BOOLEAN_PARAMETERIZED =
221 Arrays.asList(new Object[][] {
222 { new Boolean(false) },
223 { new Boolean(true) }
224 });
225
226
227 public static final List<Object[]> MEMSTORETS_TAGS_PARAMETRIZED = memStoreTSAndTagsCombination() ;
228
229 public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS ={
230 Compression.Algorithm.NONE, Compression.Algorithm.GZ
231 };
232
233
234
235
236
237
238
239 public static boolean available(int port) {
240 ServerSocket ss = null;
241 DatagramSocket ds = null;
242 try {
243 ss = new ServerSocket(port);
244 ss.setReuseAddress(true);
245 ds = new DatagramSocket(port);
246 ds.setReuseAddress(true);
247 return true;
248 } catch (IOException e) {
249
250 } finally {
251 if (ds != null) {
252 ds.close();
253 }
254
255 if (ss != null) {
256 try {
257 ss.close();
258 } catch (IOException e) {
259
260 }
261 }
262 }
263
264 return false;
265 }
266
267
268
269
270
271 private static List<Object[]> bloomAndCompressionCombinations() {
272 List<Object[]> configurations = new ArrayList<Object[]>();
273 for (Compression.Algorithm comprAlgo :
274 HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
275 for (BloomType bloomType : BloomType.values()) {
276 configurations.add(new Object[] { comprAlgo, bloomType });
277 }
278 }
279 return Collections.unmodifiableList(configurations);
280 }
281
282
283
284
285 private static List<Object[]> memStoreTSAndTagsCombination() {
286 List<Object[]> configurations = new ArrayList<Object[]>();
287 configurations.add(new Object[] { false, false });
288 configurations.add(new Object[] { false, true });
289 configurations.add(new Object[] { true, false });
290 configurations.add(new Object[] { true, true });
291 return Collections.unmodifiableList(configurations);
292 }
293
294 public static final Collection<Object[]> BLOOM_AND_COMPRESSION_COMBINATIONS =
295 bloomAndCompressionCombinations();
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310 public HBaseTestingUtility() {
311 this(HBaseConfiguration.create());
312 }
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328 public HBaseTestingUtility(Configuration conf) {
329 super(conf);
330
331
332 ChecksumUtil.generateExceptionForChecksumFailureForTest(true);
333
334
335 if (this.conf.get("fs.defaultFS") != null) {
336 this.conf.set("original.defaultFS", this.conf.get("fs.defaultFS"));
337 }
338 if (this.conf.get(HConstants.HBASE_DIR) != null) {
339 this.conf.set("original.hbase.dir", this.conf.get(HConstants.HBASE_DIR));
340 }
341
342
343 String dataTestDir = getDataTestDir().toString();
344 this.conf.set("fs.defaultFS","file:///");
345 this.conf.set(HConstants.HBASE_DIR, "file://" + dataTestDir);
346 LOG.debug("Setting " + HConstants.HBASE_DIR + " to " + dataTestDir);
347 this.conf.setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, false);
348 }
349
350
351
352
353
354 @Deprecated
355 public static HBaseTestingUtility createLocalHTU() {
356 return new HBaseTestingUtility();
357 }
358
359
360
361
362
363 @Deprecated
364 public static HBaseTestingUtility createLocalHTU(Configuration c) {
365 return new HBaseTestingUtility(c);
366 }
367
368
369
370
371 public static void closeRegion(final Region r) throws IOException {
372 if (r != null) {
373 ((HRegion)r).close();
374 }
375 }
376
377
378
379
380
381
382
383
384
385
386
387
388 @Override
389 public Configuration getConfiguration() {
390 return super.getConfiguration();
391 }
392
393 public void setHBaseCluster(HBaseCluster hbaseCluster) {
394 this.hbaseCluster = hbaseCluster;
395 }
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413 @Override
414 protected Path setupDataTestDir() {
415 Path testPath = super.setupDataTestDir();
416 if (null == testPath) {
417 return null;
418 }
419
420 createSubDirAndSystemProperty(
421 "hadoop.log.dir",
422 testPath, "hadoop-log-dir");
423
424
425
426 createSubDirAndSystemProperty(
427 "hadoop.tmp.dir",
428 testPath, "hadoop-tmp-dir");
429
430
431 createSubDir(
432 "mapreduce.cluster.local.dir",
433 testPath, "mapred-local-dir");
434
435 return testPath;
436 }
437
438 public void setJobWithoutMRCluster() throws IOException {
439 conf.set("hbase.fs.tmp.dir", getDataTestDirOnTestFS("hbase-staging").toString());
440 conf.setBoolean(HBaseTestingUtility.USE_LOCAL_FILESYSTEM, true);
441 }
442
443 private void createSubDirAndSystemProperty(
444 String propertyName, Path parent, String subDirName){
445
446 String sysValue = System.getProperty(propertyName);
447
448 if (sysValue != null) {
449
450
451 LOG.info("System.getProperty(\""+propertyName+"\") already set to: "+
452 sysValue + " so I do NOT create it in " + parent);
453 String confValue = conf.get(propertyName);
454 if (confValue != null && !confValue.endsWith(sysValue)){
455 LOG.warn(
456 propertyName + " property value differs in configuration and system: "+
457 "Configuration="+confValue+" while System="+sysValue+
458 " Erasing configuration value by system value."
459 );
460 }
461 conf.set(propertyName, sysValue);
462 } else {
463
464 createSubDir(propertyName, parent, subDirName);
465 System.setProperty(propertyName, conf.get(propertyName));
466 }
467 }
468
469
470
471
472
473
474
475 private Path getBaseTestDirOnTestFS() throws IOException {
476 FileSystem fs = getTestFileSystem();
477 return new Path(fs.getWorkingDirectory(), "test-data");
478 }
479
480
481
482
483 public HTableDescriptor getMetaTableDescriptor() {
484 try {
485 return new FSTableDescriptors(conf).get(TableName.META_TABLE_NAME);
486 } catch (IOException e) {
487 throw new RuntimeException("Unable to create META table descriptor", e);
488 }
489 }
490
491
492
493
494
495
496 Path getClusterTestDir() {
497 if (clusterTestDir == null){
498 setupClusterTestDir();
499 }
500 return new Path(clusterTestDir.getAbsolutePath());
501 }
502
503
504
505
506 private void setupClusterTestDir() {
507 if (clusterTestDir != null) {
508 return;
509 }
510
511
512
513 Path testDir = getDataTestDir("dfscluster_" + UUID.randomUUID().toString());
514 clusterTestDir = new File(testDir.toString()).getAbsoluteFile();
515
516 boolean b = deleteOnExit();
517 if (b) clusterTestDir.deleteOnExit();
518 conf.set(TEST_DIRECTORY_KEY, clusterTestDir.getPath());
519 LOG.info("Created new mini-cluster data directory: " + clusterTestDir + ", deleteOnExit=" + b);
520 }
521
522
523
524
525
526
527
528 public Path getDataTestDirOnTestFS() throws IOException {
529 if (dataTestDirOnTestFS == null) {
530 setupDataTestDirOnTestFS();
531 }
532
533 return dataTestDirOnTestFS;
534 }
535
536
537
538
539
540
541
542
543 public Path getDataTestDirOnTestFS(final String subdirName) throws IOException {
544 return new Path(getDataTestDirOnTestFS(), subdirName);
545 }
546
547
548
549
550
551 private void setupDataTestDirOnTestFS() throws IOException {
552 if (dataTestDirOnTestFS != null) {
553 LOG.warn("Data test on test fs dir already setup in "
554 + dataTestDirOnTestFS.toString());
555 return;
556 }
557 dataTestDirOnTestFS = getNewDataTestDirOnTestFS();
558 }
559
560
561
562
563 private Path getNewDataTestDirOnTestFS() throws IOException {
564
565
566
567
568 FileSystem fs = getTestFileSystem();
569 Path newDataTestDir = null;
570 String randomStr = UUID.randomUUID().toString();
571 if (fs.getUri().getScheme().equals(FileSystem.getLocal(conf).getUri().getScheme())) {
572 newDataTestDir = new Path(getDataTestDir(), randomStr);
573 File dataTestDir = new File(newDataTestDir.toString());
574 if (deleteOnExit()) dataTestDir.deleteOnExit();
575 } else {
576 Path base = getBaseTestDirOnTestFS();
577 newDataTestDir = new Path(base, randomStr);
578 if (deleteOnExit()) fs.deleteOnExit(newDataTestDir);
579 }
580 return newDataTestDir;
581 }
582
583
584
585
586
587
588 public boolean cleanupDataTestDirOnTestFS() throws IOException {
589 boolean ret = getTestFileSystem().delete(dataTestDirOnTestFS, true);
590 if (ret)
591 dataTestDirOnTestFS = null;
592 return ret;
593 }
594
595
596
597
598
599
600 public boolean cleanupDataTestDirOnTestFS(String subdirName) throws IOException {
601 Path cpath = getDataTestDirOnTestFS(subdirName);
602 return getTestFileSystem().delete(cpath, true);
603 }
604
605
606
607
608
609
610
611
612 public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception {
613 return startMiniDFSCluster(servers, null);
614 }
615
616
617
618
619
620
621
622
623
624
625
626
627 public MiniDFSCluster startMiniDFSCluster(final String hosts[])
628 throws Exception {
629 if ( hosts != null && hosts.length != 0) {
630 return startMiniDFSCluster(hosts.length, hosts);
631 } else {
632 return startMiniDFSCluster(1, null);
633 }
634 }
635
636
637
638
639
640
641
642
643
644
645 public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[])
646 throws Exception {
647 return startMiniDFSCluster(servers, null, hosts);
648 }
649
650 private void setFs() throws IOException {
651 if(this.dfsCluster == null){
652 LOG.info("Skipping setting fs because dfsCluster is null");
653 return;
654 }
655 FileSystem fs = this.dfsCluster.getFileSystem();
656 FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
657
658
659 conf.unset(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE);
660 }
661
662 public MiniDFSCluster startMiniDFSCluster(int servers, final String racks[], String hosts[])
663 throws Exception {
664 createDirsAndSetProperties();
665 EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
666
667
668 org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.util.MBeans.class).
669 setLevel(org.apache.log4j.Level.ERROR);
670 org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class).
671 setLevel(org.apache.log4j.Level.ERROR);
672
673 this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
674 true, null, racks, hosts, null);
675
676
677 setFs();
678
679
680 this.dfsCluster.waitClusterUp();
681
682
683 dataTestDirOnTestFS = null;
684 String dataTestDir = getDataTestDir().toString();
685 conf.set(HConstants.HBASE_DIR, dataTestDir);
686 LOG.debug("Setting " + HConstants.HBASE_DIR + " to " + dataTestDir);
687
688 return this.dfsCluster;
689 }
690
691 public MiniDFSCluster startMiniDFSClusterForTestWAL(int namenodePort) throws IOException {
692 createDirsAndSetProperties();
693 dfsCluster = new MiniDFSCluster(namenodePort, conf, 5, false, true, true, null,
694 null, null, null);
695 return dfsCluster;
696 }
697
698
699 private void createDirsAndSetProperties() throws IOException {
700 setupClusterTestDir();
701 System.setProperty(TEST_DIRECTORY_KEY, clusterTestDir.getPath());
702 createDirAndSetProperty("cache_data", "test.cache.data");
703 createDirAndSetProperty("hadoop_tmp", "hadoop.tmp.dir");
704 hadoopLogDir = createDirAndSetProperty("hadoop_logs", "hadoop.log.dir");
705 createDirAndSetProperty("mapred_local", "mapreduce.cluster.local.dir");
706 createDirAndSetProperty("mapred_temp", "mapreduce.cluster.temp.dir");
707 enableShortCircuit();
708
709 Path root = getDataTestDirOnTestFS("hadoop");
710 conf.set(MapreduceTestingShim.getMROutputDirProp(),
711 new Path(root, "mapred-output-dir").toString());
712 conf.set("mapreduce.jobtracker.system.dir", new Path(root, "mapred-system-dir").toString());
713 conf.set("mapreduce.jobtracker.staging.root.dir",
714 new Path(root, "mapreduce-jobtracker-staging-root-dir").toString());
715 conf.set("mapreduce.job.working.dir", new Path(root, "mapred-working-dir").toString());
716 }
717
718
719
720
721
722
723
724 public boolean isReadShortCircuitOn(){
725 final String propName = "hbase.tests.use.shortcircuit.reads";
726 String readOnProp = System.getProperty(propName);
727 if (readOnProp != null){
728 return Boolean.parseBoolean(readOnProp);
729 } else {
730 return conf.getBoolean(propName, false);
731 }
732 }
733
734
735
736
737 private void enableShortCircuit() {
738 if (isReadShortCircuitOn()) {
739 String curUser = System.getProperty("user.name");
740 LOG.info("read short circuit is ON for user " + curUser);
741
742 conf.set("dfs.block.local-path-access.user", curUser);
743
744 conf.setBoolean("dfs.client.read.shortcircuit", true);
745
746 conf.setBoolean("dfs.client.read.shortcircuit.skip.checksum", true);
747 } else {
748 LOG.info("read short circuit is OFF");
749 }
750 }
751
752 private String createDirAndSetProperty(final String relPath, String property) {
753 String path = getDataTestDir(relPath).toString();
754 System.setProperty(property, path);
755 conf.set(property, path);
756 new File(path).mkdirs();
757 LOG.info("Setting " + property + " to " + path + " in system properties and HBase conf");
758 return path;
759 }
760
761
762
763
764
765
766 public void shutdownMiniDFSCluster() throws IOException {
767 if (this.dfsCluster != null) {
768
769 this.dfsCluster.shutdown();
770 dfsCluster = null;
771 dataTestDirOnTestFS = null;
772 FSUtils.setFsDefault(this.conf, new Path("file:///"));
773 }
774 }
775
776
777
778
779
780
781
782
783 public MiniZooKeeperCluster startMiniZKCluster() throws Exception {
784 return startMiniZKCluster(1);
785 }
786
787
788
789
790
791
792
793
794
795 public MiniZooKeeperCluster startMiniZKCluster(
796 final int zooKeeperServerNum,
797 final int ... clientPortList)
798 throws Exception {
799 setupClusterTestDir();
800 return startMiniZKCluster(clusterTestDir, zooKeeperServerNum, clientPortList);
801 }
802
803 private MiniZooKeeperCluster startMiniZKCluster(final File dir)
804 throws Exception {
805 return startMiniZKCluster(dir, 1, null);
806 }
807
808
809
810
811
812 private MiniZooKeeperCluster startMiniZKCluster(final File dir,
813 final int zooKeeperServerNum,
814 final int [] clientPortList)
815 throws Exception {
816 if (this.zkCluster != null) {
817 throw new IOException("Cluster already running at " + dir);
818 }
819 this.passedZkCluster = false;
820 this.zkCluster = new MiniZooKeeperCluster(this.getConfiguration());
821 final int defPort = this.conf.getInt("test.hbase.zookeeper.property.clientPort", 0);
822 if (defPort > 0){
823
824 this.zkCluster.setDefaultClientPort(defPort);
825 }
826
827 if (clientPortList != null) {
828
829 int clientPortListSize = (clientPortList.length <= zooKeeperServerNum) ?
830 clientPortList.length : zooKeeperServerNum;
831 for (int i=0; i < clientPortListSize; i++) {
832 this.zkCluster.addClientPort(clientPortList[i]);
833 }
834 }
835 int clientPort = this.zkCluster.startup(dir,zooKeeperServerNum);
836 this.conf.set(HConstants.ZOOKEEPER_CLIENT_PORT,
837 Integer.toString(clientPort));
838 return this.zkCluster;
839 }
840
841
842
843
844
845
846
847 public void shutdownMiniZKCluster() throws IOException {
848 if (this.zkCluster != null) {
849 this.zkCluster.shutdown();
850 this.zkCluster = null;
851 }
852 }
853
854
855
856
857
858
859
860 public MiniHBaseCluster startMiniCluster() throws Exception {
861 return startMiniCluster(1, 1);
862 }
863
864
865
866
867
868
869
870 public MiniHBaseCluster startMiniCluster(boolean withWALDir) throws Exception {
871 return startMiniCluster(1, 1, 1, null, null, null, false, withWALDir);
872 }
873
874
875
876
877
878
879
880
881
882 public MiniHBaseCluster startMiniCluster(final int numSlaves, boolean create)
883 throws Exception {
884 return startMiniCluster(1, numSlaves, create);
885 }
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900 public MiniHBaseCluster startMiniCluster(final int numSlaves)
901 throws Exception {
902 return startMiniCluster(1, numSlaves, false);
903 }
904
905 public MiniHBaseCluster startMiniCluster(final int numSlaves, boolean create, boolean withWALDir)
906 throws Exception {
907 return startMiniCluster(1, numSlaves, numSlaves, null, null, null, create, withWALDir);
908 }
909
910
911
912
913
914
915
916
917 public MiniHBaseCluster startMiniCluster(final int numMasters,
918 final int numSlaves, boolean create)
919 throws Exception {
920 return startMiniCluster(numMasters, numSlaves, null, create);
921 }
922
923
924
925
926
927
928
929 public MiniHBaseCluster startMiniCluster(final int numMasters,
930 final int numSlaves)
931 throws Exception {
932 return startMiniCluster(numMasters, numSlaves, null, false);
933 }
934
935 public MiniHBaseCluster startMiniCluster(final int numMasters,
936 final int numSlaves, final String[] dataNodeHosts, boolean create)
937 throws Exception {
938 return startMiniCluster(numMasters, numSlaves, numSlaves, dataNodeHosts,
939 null, null, create, false);
940 }
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966 public MiniHBaseCluster startMiniCluster(final int numMasters,
967 final int numSlaves, final String[] dataNodeHosts) throws Exception {
968 return startMiniCluster(numMasters, numSlaves, numSlaves, dataNodeHosts,
969 null, null);
970 }
971
972
973
974
975
976 public MiniHBaseCluster startMiniCluster(final int numMasters,
977 final int numSlaves, final int numDataNodes) throws Exception {
978 return startMiniCluster(numMasters, numSlaves, numDataNodes, null, null, null);
979 }
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008 public MiniHBaseCluster startMiniCluster(final int numMasters,
1009 final int numSlaves, final String[] dataNodeHosts, Class<? extends HMaster> masterClass,
1010 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
1011 throws Exception {
1012 return startMiniCluster(
1013 numMasters, numSlaves, numSlaves, dataNodeHosts, masterClass, regionserverClass);
1014 }
1015
1016 public MiniHBaseCluster startMiniCluster(final int numMasters,
1017 final int numSlaves, int numDataNodes, final String[] dataNodeHosts,
1018 Class<? extends HMaster> masterClass,
1019 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass)
1020 throws Exception {
1021 return startMiniCluster(numMasters, numSlaves, numDataNodes, dataNodeHosts,
1022 masterClass, regionserverClass, false, false);
1023 }
1024
1025
1026
1027
1028
1029
1030
1031
1032 public MiniHBaseCluster startMiniCluster(final int numMasters,
1033 final int numSlaves, int numDataNodes, final String[] dataNodeHosts,
1034 Class<? extends HMaster> masterClass,
1035 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass,
1036 boolean create, boolean withWALDir)
1037 throws Exception {
1038 if (dataNodeHosts != null && dataNodeHosts.length != 0) {
1039 numDataNodes = dataNodeHosts.length;
1040 }
1041
1042 LOG.info("Starting up minicluster with " + numMasters + " master(s) and " +
1043 numSlaves + " regionserver(s) and " + numDataNodes + " datanode(s)");
1044
1045
1046 if (miniClusterRunning) {
1047 throw new IllegalStateException("A mini-cluster is already running");
1048 }
1049 miniClusterRunning = true;
1050
1051 setupClusterTestDir();
1052 System.setProperty(TEST_DIRECTORY_KEY, this.clusterTestDir.getPath());
1053
1054
1055
1056 if (this.dfsCluster == null) {
1057 LOG.info("STARTING DFS");
1058 dfsCluster = startMiniDFSCluster(numDataNodes, dataNodeHosts);
1059 }
1060
1061
1062 if (this.zkCluster == null) {
1063 startMiniZKCluster(clusterTestDir);
1064 }
1065
1066
1067 return startMiniHBaseCluster(numMasters, numSlaves, masterClass,
1068 regionserverClass, create, withWALDir);
1069 }
1070
1071 public MiniHBaseCluster startMiniHBaseCluster(final int numMasters, final int numSlaves)
1072 throws IOException, InterruptedException{
1073 return startMiniHBaseCluster(numMasters, numSlaves, null, null, false, false);
1074 }
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089 public MiniHBaseCluster startMiniHBaseCluster(final int numMasters,
1090 final int numSlaves, Class<? extends HMaster> masterClass,
1091 Class<? extends MiniHBaseCluster.MiniHBaseClusterRegionServer> regionserverClass,
1092 boolean create, boolean withWALDir)
1093 throws IOException, InterruptedException {
1094
1095 createRootDir(create);
1096
1097 if (withWALDir) {
1098 createWALRootDir();
1099 }
1100
1101
1102
1103 if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1) == -1) {
1104 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, numSlaves);
1105 }
1106 if (conf.getInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1) == -1) {
1107 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, numSlaves);
1108 }
1109
1110 Configuration c = new Configuration(this.conf);
1111 this.hbaseCluster =
1112 new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass);
1113
1114 conf.set(HConstants.MASTER_ADDRS_KEY,
1115 c.get(HConstants.MASTER_ADDRS_KEY, HConstants.MASTER_ADDRS_DEFAULT));
1116
1117 Table t = new HTable(c, TableName.META_TABLE_NAME);
1118 ResultScanner s = t.getScanner(new Scan());
1119 while (s.next() != null) {
1120 continue;
1121 }
1122 s.close();
1123 t.close();
1124
1125 getHBaseAdmin();
1126 LOG.info("Minicluster is up");
1127
1128
1129
1130 setHBaseFsTmpDir();
1131
1132 return (MiniHBaseCluster)this.hbaseCluster;
1133 }
1134
1135
1136
1137
1138
1139
1140
1141 public void restartHBaseCluster(int servers) throws IOException, InterruptedException {
1142 this.hbaseCluster = new MiniHBaseCluster(this.conf, servers);
1143
1144 Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
1145 ResultScanner s = t.getScanner(new Scan());
1146 while (s.next() != null) {
1147
1148 }
1149 LOG.info("HBase has been restarted");
1150 s.close();
1151 t.close();
1152 }
1153
1154
1155
1156
1157
1158
1159 public MiniHBaseCluster getMiniHBaseCluster() {
1160 if (this.hbaseCluster == null || this.hbaseCluster instanceof MiniHBaseCluster) {
1161 return (MiniHBaseCluster)this.hbaseCluster;
1162 }
1163 throw new RuntimeException(hbaseCluster + " not an instance of " +
1164 MiniHBaseCluster.class.getName());
1165 }
1166
1167
1168
1169
1170
1171
1172 public void shutdownMiniCluster() throws Exception {
1173 LOG.info("Shutting down minicluster");
1174 shutdownMiniHBaseCluster();
1175 if (!this.passedZkCluster){
1176 shutdownMiniZKCluster();
1177 }
1178 shutdownMiniDFSCluster();
1179
1180 cleanupTestDir();
1181 miniClusterRunning = false;
1182 LOG.info("Minicluster is down");
1183 }
1184
1185
1186
1187
1188 @Override
1189 public boolean cleanupTestDir() {
1190 boolean ret = super.cleanupTestDir();
1191 if (deleteDir(this.clusterTestDir)) {
1192 this.clusterTestDir = null;
1193 return ret & true;
1194 }
1195 return false;
1196 }
1197
1198
1199
1200
1201
1202 public void shutdownMiniHBaseCluster() throws IOException {
1203 closeConnection();
1204
1205
1206 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
1207 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, -1);
1208 if (this.hbaseCluster != null) {
1209 this.hbaseCluster.shutdown();
1210
1211 this.hbaseCluster.waitUntilShutDown();
1212 this.hbaseCluster = null;
1213 }
1214
1215 if (zooKeeperWatcher != null) {
1216 zooKeeperWatcher.close();
1217 zooKeeperWatcher = null;
1218 }
1219 }
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229 public Path getDefaultRootDirPath(boolean create) throws IOException {
1230 if (!create) {
1231 return getDataTestDirOnTestFS();
1232 } else {
1233 return getNewDataTestDirOnTestFS();
1234 }
1235 }
1236
1237
1238
1239
1240
1241
1242
1243
1244 public Path getDefaultRootDirPath() throws IOException {
1245 return getDefaultRootDirPath(false);
1246 }
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260 public Path createRootDir(boolean create) throws IOException {
1261 FileSystem fs = FileSystem.get(this.conf);
1262 Path hbaseRootdir = getDefaultRootDirPath(create);
1263 FSUtils.setRootDir(this.conf, hbaseRootdir);
1264 fs.mkdirs(hbaseRootdir);
1265 FSUtils.setVersion(fs, hbaseRootdir);
1266 return hbaseRootdir;
1267 }
1268
1269
1270
1271
1272
1273
1274
1275 public Path createRootDir() throws IOException {
1276 return createRootDir(false);
1277 }
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288 public Path createWALRootDir() throws IOException {
1289 FileSystem fs = FileSystem.get(this.conf);
1290 Path walDir = getNewDataTestDirOnTestFS();
1291 FSUtils.setWALRootDir(this.conf, walDir);
1292 fs.mkdirs(walDir);
1293 return walDir;
1294 }
1295
1296 private void setHBaseFsTmpDir() throws IOException {
1297 String hbaseFsTmpDirInString = this.conf.get("hbase.fs.tmp.dir");
1298 if (hbaseFsTmpDirInString == null) {
1299 this.conf.set("hbase.fs.tmp.dir", getDataTestDirOnTestFS("hbase-staging").toString());
1300 LOG.info("Setting hbase.fs.tmp.dir to " + this.conf.get("hbase.fs.tmp.dir"));
1301 } else {
1302 LOG.info("The hbase.fs.tmp.dir is set to " + hbaseFsTmpDirInString);
1303 }
1304 }
1305
1306
1307
1308
1309
1310 public void flush() throws IOException {
1311 getMiniHBaseCluster().flushcache();
1312 }
1313
1314
1315
1316
1317
1318 public void flush(TableName tableName) throws IOException {
1319 getMiniHBaseCluster().flushcache(tableName);
1320 }
1321
1322
1323
1324
1325
1326 public void compact(boolean major) throws IOException {
1327 getMiniHBaseCluster().compact(major);
1328 }
1329
1330
1331
1332
1333
1334 public void compact(TableName tableName, boolean major) throws IOException {
1335 getMiniHBaseCluster().compact(tableName, major);
1336 }
1337
1338
1339
1340
1341
1342
1343
1344
1345 public Table createTable(TableName tableName, String family)
1346 throws IOException{
1347 return createTable(tableName, new String[]{family});
1348 }
1349
1350
1351
1352
1353
1354
1355
1356
1357 public HTable createTable(byte[] tableName, byte[] family)
1358 throws IOException{
1359 return createTable(TableName.valueOf(tableName), new byte[][]{family});
1360 }
1361
1362
1363
1364
1365
1366
1367
1368
1369 public Table createTable(TableName tableName, String[] families)
1370 throws IOException {
1371 List<byte[]> fams = new ArrayList<byte[]>(families.length);
1372 for (String family : families) {
1373 fams.add(Bytes.toBytes(family));
1374 }
1375 return createTable(tableName, fams.toArray(new byte[0][]));
1376 }
1377
1378
1379
1380
1381
1382
1383
1384
1385 public HTable createTable(TableName tableName, byte[] family)
1386 throws IOException{
1387 return createTable(tableName, new byte[][]{family});
1388 }
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398 public HTable createMultiRegionTable(TableName tableName, byte[] family, int numRegions)
1399 throws IOException {
1400 if (numRegions < 3) throw new IOException("Must create at least 3 regions");
1401 byte[] startKey = Bytes.toBytes("aaaaa");
1402 byte[] endKey = Bytes.toBytes("zzzzz");
1403 byte[][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
1404
1405 return createTable(tableName, new byte[][] { family }, splitKeys);
1406 }
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416 public HTable createTable(byte[] tableName, byte[][] families)
1417 throws IOException {
1418 return createTable(tableName, families,
1419 new Configuration(getConfiguration()));
1420 }
1421
1422
1423
1424
1425
1426
1427
1428
1429 public HTable createTable(TableName tableName, byte[][] families)
1430 throws IOException {
1431 return createTable(tableName, families, (byte[][]) null);
1432 }
1433
1434
1435
1436
1437
1438
1439
1440
1441 public HTable createMultiRegionTable(TableName tableName, byte[][] families) throws IOException {
1442 return createTable(tableName, families, KEYS_FOR_HBA_CREATE_TABLE);
1443 }
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453 public HTable createTable(TableName tableName, byte[][] families, byte[][] splitKeys)
1454 throws IOException {
1455 return createTable(tableName, families, splitKeys, new Configuration(getConfiguration()));
1456 }
1457
1458 public HTable createTable(byte[] tableName, byte[][] families,
1459 int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
1460 return createTable(TableName.valueOf(tableName), families, numVersions,
1461 startKey, endKey, numRegions);
1462 }
1463
1464 public HTable createTable(String tableName, byte[][] families,
1465 int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
1466 return createTable(TableName.valueOf(tableName), families, numVersions,
1467 startKey, endKey, numRegions);
1468 }
1469
1470 public HTable createTable(TableName tableName, byte[][] families,
1471 int numVersions, byte[] startKey, byte[] endKey, int numRegions)
1472 throws IOException{
1473 HTableDescriptor desc = new HTableDescriptor(tableName);
1474 for (byte[] family : families) {
1475 HColumnDescriptor hcd = new HColumnDescriptor(family)
1476 .setMaxVersions(numVersions);
1477 desc.addFamily(hcd);
1478 }
1479 getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
1480
1481 waitUntilAllRegionsAssigned(tableName);
1482 return new HTable(getConfiguration(), tableName);
1483 }
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493 public HTable createTable(HTableDescriptor htd, byte[][] families, Configuration c)
1494 throws IOException {
1495 return createTable(htd, families, (byte[][]) null, c);
1496 }
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507 public HTable createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
1508 Configuration c) throws IOException {
1509
1510
1511
1512 return createTable(htd, families, splitKeys, BloomType.NONE, HConstants.DEFAULT_BLOCKSIZE, c);
1513 }
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527 public HTable createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
1528 BloomType type, int blockSize, Configuration c) throws IOException {
1529 for (byte[] family : families) {
1530 HColumnDescriptor hcd = new HColumnDescriptor(family);
1531
1532
1533
1534 hcd.setBloomFilterType(type);
1535 hcd.setBlocksize(blockSize);
1536 htd.addFamily(hcd);
1537 }
1538 getHBaseAdmin().createTable(htd, splitKeys);
1539
1540
1541 waitUntilAllRegionsAssigned(htd.getTableName());
1542 return (HTable) getConnection().getTable(htd.getTableName());
1543 }
1544
1545
1546
1547
1548
1549
1550
1551
1552 public HTable createTable(HTableDescriptor htd, byte[][] splitRows)
1553 throws IOException {
1554 getHBaseAdmin().createTable(htd, splitRows);
1555
1556 waitUntilAllRegionsAssigned(htd.getTableName());
1557 return new HTable(getConfiguration(), htd.getTableName());
1558 }
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568 public HTable createTable(TableName tableName, byte[][] families,
1569 final Configuration c)
1570 throws IOException {
1571 return createTable(tableName, families, (byte[][]) null, c);
1572 }
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583 public HTable createTable(TableName tableName, byte[][] families, byte[][] splitKeys,
1584 final Configuration c) throws IOException {
1585 return createTable(new HTableDescriptor(tableName), families, splitKeys, c);
1586 }
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596 public HTable createTable(byte[] tableName, byte[][] families,
1597 final Configuration c)
1598 throws IOException {
1599 HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
1600 for(byte[] family : families) {
1601 HColumnDescriptor hcd = new HColumnDescriptor(family);
1602
1603
1604
1605 hcd.setBloomFilterType(BloomType.NONE);
1606 desc.addFamily(hcd);
1607 }
1608 getHBaseAdmin().createTable(desc);
1609 return new HTable(c, desc.getTableName());
1610 }
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621 public HTable createTable(TableName tableName, byte[][] families,
1622 final Configuration c, int numVersions)
1623 throws IOException {
1624 HTableDescriptor desc = new HTableDescriptor(tableName);
1625 for(byte[] family : families) {
1626 HColumnDescriptor hcd = new HColumnDescriptor(family)
1627 .setMaxVersions(numVersions);
1628 desc.addFamily(hcd);
1629 }
1630 getHBaseAdmin().createTable(desc);
1631
1632 waitUntilAllRegionsAssigned(tableName);
1633 return new HTable(c, tableName);
1634 }
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645 public HTable createTable(byte[] tableName, byte[][] families,
1646 final Configuration c, int numVersions)
1647 throws IOException {
1648 HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
1649 for(byte[] family : families) {
1650 HColumnDescriptor hcd = new HColumnDescriptor(family)
1651 .setMaxVersions(numVersions);
1652 desc.addFamily(hcd);
1653 }
1654 getHBaseAdmin().createTable(desc);
1655 return new HTable(c, desc.getTableName());
1656 }
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666 public HTable createTable(byte[] tableName, byte[] family, int numVersions)
1667 throws IOException {
1668 return createTable(tableName, new byte[][]{family}, numVersions);
1669 }
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679 public HTable createTable(TableName tableName, byte[] family, int numVersions)
1680 throws IOException {
1681 return createTable(tableName, new byte[][]{family}, numVersions);
1682 }
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692 public HTable createTable(byte[] tableName, byte[][] families,
1693 int numVersions)
1694 throws IOException {
1695 return createTable(TableName.valueOf(tableName), families, numVersions);
1696 }
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706 public HTable createTable(TableName tableName, byte[][] families,
1707 int numVersions)
1708 throws IOException {
1709 return createTable(tableName, families, numVersions, (byte[][]) null);
1710 }
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721 public HTable createTable(TableName tableName, byte[][] families, int numVersions,
1722 byte[][] splitKeys) throws IOException {
1723 HTableDescriptor desc = new HTableDescriptor(tableName);
1724 for (byte[] family : families) {
1725 HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions);
1726 desc.addFamily(hcd);
1727 }
1728 getHBaseAdmin().createTable(desc, splitKeys);
1729
1730 waitUntilAllRegionsAssigned(tableName);
1731 return new HTable(new Configuration(getConfiguration()), tableName);
1732 }
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742 public HTable createMultiRegionTable(TableName tableName, byte[][] families, int numVersions)
1743 throws IOException {
1744 return createTable(tableName, families, numVersions, KEYS_FOR_HBA_CREATE_TABLE);
1745 }
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756 public HTable createTable(byte[] tableName, byte[][] families,
1757 int numVersions, int blockSize) throws IOException {
1758 return createTable(TableName.valueOf(tableName),
1759 families, numVersions, blockSize);
1760 }
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771 public HTable createTable(TableName tableName, byte[][] families,
1772 int numVersions, int blockSize) throws IOException {
1773 HTableDescriptor desc = new HTableDescriptor(tableName);
1774 for (byte[] family : families) {
1775 HColumnDescriptor hcd = new HColumnDescriptor(family)
1776 .setMaxVersions(numVersions)
1777 .setBlocksize(blockSize);
1778 desc.addFamily(hcd);
1779 }
1780 getHBaseAdmin().createTable(desc);
1781
1782 waitUntilAllRegionsAssigned(tableName);
1783 return new HTable(new Configuration(getConfiguration()), tableName);
1784 }
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794 public HTable createTable(byte[] tableName, byte[][] families,
1795 int[] numVersions)
1796 throws IOException {
1797 return createTable(TableName.valueOf(tableName), families, numVersions);
1798 }
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808 public HTable createTable(TableName tableName, byte[][] families,
1809 int[] numVersions)
1810 throws IOException {
1811 HTableDescriptor desc = new HTableDescriptor(tableName);
1812 int i = 0;
1813 for (byte[] family : families) {
1814 HColumnDescriptor hcd = new HColumnDescriptor(family)
1815 .setMaxVersions(numVersions[i]);
1816 desc.addFamily(hcd);
1817 i++;
1818 }
1819 getHBaseAdmin().createTable(desc);
1820
1821 waitUntilAllRegionsAssigned(tableName);
1822 return new HTable(new Configuration(getConfiguration()), tableName);
1823 }
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833 public HTable createTable(byte[] tableName, byte[] family, byte[][] splitRows)
1834 throws IOException{
1835 return createTable(TableName.valueOf(tableName), family, splitRows);
1836 }
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846 public HTable createTable(TableName tableName, byte[] family, byte[][] splitRows)
1847 throws IOException {
1848 HTableDescriptor desc = new HTableDescriptor(tableName);
1849 HColumnDescriptor hcd = new HColumnDescriptor(family);
1850 desc.addFamily(hcd);
1851 getHBaseAdmin().createTable(desc, splitRows);
1852
1853 waitUntilAllRegionsAssigned(tableName);
1854 return new HTable(getConfiguration(), tableName);
1855 }
1856
1857
1858
1859
1860
1861
1862
1863
1864 public HTable createMultiRegionTable(TableName tableName, byte[] family) throws IOException {
1865 return createTable(tableName, family, KEYS_FOR_HBA_CREATE_TABLE);
1866 }
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876 public HTable createTable(byte[] tableName, byte[][] families, byte[][] splitRows)
1877 throws IOException {
1878 HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
1879 for(byte[] family:families) {
1880 HColumnDescriptor hcd = new HColumnDescriptor(family);
1881 desc.addFamily(hcd);
1882 }
1883 getHBaseAdmin().createTable(desc, splitRows);
1884
1885 waitUntilAllRegionsAssigned(desc.getTableName());
1886 return new HTable(getConfiguration(), desc.getTableName());
1887 }
1888
1889
1890
1891
1892 public static WAL createWal(final Configuration conf, final Path rootDir, final Path walRootDir, final HRegionInfo hri)
1893 throws IOException {
1894
1895
1896 Configuration confForWAL = new Configuration(conf);
1897 confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
1898 FSUtils.setWALRootDir(confForWAL, walRootDir);
1899 return (new WALFactory(confForWAL,
1900 Collections.<WALActionsListener>singletonList(new MetricsWAL()),
1901 "hregion-" + RandomStringUtils.randomNumeric(8))).
1902 getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
1903 }
1904
1905
1906
1907
1908
1909 public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
1910 final Path walRootDir, final Configuration conf, final HTableDescriptor htd) throws IOException {
1911 return createRegionAndWAL(info, rootDir, walRootDir, conf, htd, true);
1912 }
1913
1914
1915
1916
1917
1918 public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
1919 final Path walRootDir, final Configuration conf, final HTableDescriptor htd, boolean initialize)
1920 throws IOException {
1921 WAL wal = createWal(conf, rootDir, walRootDir, info);
1922 return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize);
1923 }
1924
1925
1926
1927
1928 public static void closeRegionAndWAL(final Region r) throws IOException {
1929 closeRegionAndWAL((HRegion)r);
1930 }
1931
1932
1933
1934
1935 public static void closeRegionAndWAL(final HRegion r) throws IOException {
1936 if (r == null) return;
1937 r.close();
1938 if (r.getWAL() == null) return;
1939 r.getWAL().close();
1940 }
1941
1942
1943
1944
1945 @SuppressWarnings("serial")
1946 public static void modifyTableSync(Admin admin, HTableDescriptor desc)
1947 throws IOException, InterruptedException {
1948 admin.modifyTable(desc.getTableName(), desc);
1949 Pair<Integer, Integer> status = new Pair<Integer, Integer>() {{
1950 setFirst(0);
1951 setSecond(0);
1952 }};
1953 int i = 0;
1954 do {
1955 status = admin.getAlterStatus(desc.getTableName());
1956 if (status.getSecond() != 0) {
1957 LOG.debug(status.getSecond() - status.getFirst() + "/" + status.getSecond()
1958 + " regions updated.");
1959 Thread.sleep(1 * 1000l);
1960 } else {
1961 LOG.debug("All regions updated.");
1962 break;
1963 }
1964 } while (status.getFirst() != 0 && i++ < 500);
1965 if (status.getFirst() != 0) {
1966 throw new IOException("Failed to update all regions even after 500 seconds.");
1967 }
1968 }
1969
1970
1971
1972
1973 public static void setReplicas(Admin admin, TableName table, int replicaCount)
1974 throws IOException, InterruptedException {
1975 admin.disableTable(table);
1976 HTableDescriptor desc = admin.getTableDescriptor(table);
1977 desc.setRegionReplication(replicaCount);
1978 admin.modifyTable(desc.getTableName(), desc);
1979 admin.enableTable(table);
1980 }
1981
1982
1983
1984
1985
1986 public void deleteTable(String tableName) throws IOException {
1987 deleteTable(TableName.valueOf(tableName));
1988 }
1989
1990
1991
1992
1993
1994 public void deleteTable(byte[] tableName) throws IOException {
1995 deleteTable(TableName.valueOf(tableName));
1996 }
1997
1998
1999
2000
2001
2002 public void deleteTable(TableName tableName) throws IOException {
2003 try {
2004 getHBaseAdmin().disableTable(tableName);
2005 } catch (TableNotEnabledException e) {
2006 LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
2007 }
2008 getHBaseAdmin().deleteTable(tableName);
2009 }
2010
2011
2012
2013
2014
2015 public void deleteTableIfAny(TableName tableName) throws IOException {
2016 try {
2017 deleteTable(tableName);
2018 } catch (TableNotFoundException e) {
2019
2020 }
2021 }
2022
2023
2024
2025
2026
2027 public final static byte [] fam1 = Bytes.toBytes("colfamily11");
2028 public final static byte [] fam2 = Bytes.toBytes("colfamily21");
2029 public final static byte [] fam3 = Bytes.toBytes("colfamily31");
2030 public static final byte[][] COLUMNS = {fam1, fam2, fam3};
2031 private static final int MAXVERSIONS = 3;
2032
2033 public static final char FIRST_CHAR = 'a';
2034 public static final char LAST_CHAR = 'z';
2035 public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
2036 public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET);
2037
2038
2039
2040
2041
2042
2043
2044
2045 public HTableDescriptor createTableDescriptor(final String name,
2046 final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
2047 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
2048 for (byte[] cfName : new byte[][]{ fam1, fam2, fam3 }) {
2049 htd.addFamily(new HColumnDescriptor(cfName)
2050 .setMinVersions(minVersions)
2051 .setMaxVersions(versions)
2052 .setKeepDeletedCells(keepDeleted)
2053 .setBlockCacheEnabled(false)
2054 .setTimeToLive(ttl)
2055 );
2056 }
2057 return htd;
2058 }
2059
2060
2061
2062
2063
2064
2065
2066 public HTableDescriptor createTableDescriptor(final String name) {
2067 return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS,
2068 MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
2069 }
2070
2071
2072
2073
2074
2075 public HRegion createHRegion(
2076 final HRegionInfo info,
2077 final Path rootDir,
2078 final Configuration conf,
2079 final HTableDescriptor htd) throws IOException {
2080 return HRegion.createHRegion(info, rootDir, conf, htd);
2081 }
2082
2083 public HTableDescriptor createTableDescriptor(final TableName tableName,
2084 byte[] family) {
2085 return createTableDescriptor(tableName, new byte[][] {family}, 1);
2086 }
2087
2088 public HTableDescriptor createTableDescriptor(final TableName tableName,
2089 byte[][] families, int maxVersions) {
2090 HTableDescriptor desc = new HTableDescriptor(tableName);
2091 for (byte[] family : families) {
2092 HColumnDescriptor hcd = new HColumnDescriptor(family)
2093 .setMaxVersions(maxVersions);
2094 desc.addFamily(hcd);
2095 }
2096 return desc;
2097 }
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107 public HRegion createLocalHRegion(HTableDescriptor desc, byte [] startKey,
2108 byte [] endKey)
2109 throws IOException {
2110 HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey);
2111 return createLocalHRegion(hri, desc);
2112 }
2113
2114
2115
2116
2117
2118
2119
2120
2121 public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc) throws IOException {
2122 return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc);
2123 }
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135 public HRegion createLocalHRegion(HRegionInfo info, Configuration conf, HTableDescriptor desc,
2136 WAL wal) throws IOException {
2137 return HRegion.createHRegion(info, getDataTestDir(), conf, desc, wal);
2138 }
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152 public HRegion createLocalHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
2153 String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
2154 WAL wal, byte[]... families) throws IOException {
2155 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
2156 htd.setReadOnly(isReadOnly);
2157 for (byte[] family : families) {
2158 HColumnDescriptor hcd = new HColumnDescriptor(family);
2159
2160 hcd.setMaxVersions(Integer.MAX_VALUE);
2161 htd.addFamily(hcd);
2162 }
2163 htd.setDurability(durability);
2164 HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false);
2165 return createLocalHRegion(info, conf, htd, wal);
2166 }
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177 public HTable deleteTableData(byte[] tableName) throws IOException {
2178 return deleteTableData(TableName.valueOf(tableName));
2179 }
2180
2181
2182
2183
2184
2185
2186
2187
2188 public HTable deleteTableData(TableName tableName) throws IOException {
2189 HTable table = new HTable(getConfiguration(), tableName);
2190 Scan scan = new Scan();
2191 ResultScanner resScan = table.getScanner(scan);
2192 for(Result res : resScan) {
2193 Delete del = new Delete(res.getRow());
2194 table.delete(del);
2195 }
2196 resScan = table.getScanner(scan);
2197 resScan.close();
2198 return table;
2199 }
2200
2201
2202
2203
2204
2205
2206
2207
2208 public HTable truncateTable(final TableName tableName, final boolean preserveRegions)
2209 throws IOException {
2210 Admin admin = getHBaseAdmin();
2211 if (!admin.isTableDisabled(tableName)) {
2212 admin.disableTable(tableName);
2213 }
2214 admin.truncateTable(tableName, preserveRegions);
2215 return new HTable(getConfiguration(), tableName);
2216 }
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227 public HTable truncateTable(final TableName tableName) throws IOException {
2228 return truncateTable(tableName, false);
2229 }
2230
2231
2232
2233
2234
2235
2236
2237
2238 public HTable truncateTable(final byte[] tableName, final boolean preserveRegions)
2239 throws IOException {
2240 return truncateTable(TableName.valueOf(tableName), preserveRegions);
2241 }
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252 public HTable truncateTable(final byte[] tableName) throws IOException {
2253 return truncateTable(tableName, false);
2254 }
2255
2256
2257
2258
2259
2260
2261
2262
2263 public int loadTable(final Table t, final byte[] f) throws IOException {
2264 return loadTable(t, new byte[][] {f});
2265 }
2266
2267
2268
2269
2270
2271
2272
2273
2274 public int loadTable(final Table t, final byte[] f, boolean writeToWAL) throws IOException {
2275 return loadTable(t, new byte[][] {f}, null, writeToWAL);
2276 }
2277
2278
2279
2280
2281
2282
2283
2284
2285 public int loadTable(final Table t, final byte[][] f) throws IOException {
2286 return loadTable(t, f, null);
2287 }
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297 public int loadTable(final Table t, final byte[][] f, byte[] value) throws IOException {
2298 return loadTable(t, f, value, true);
2299 }
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309 public int loadTable(final Table t, final byte[][] f, byte[] value, boolean writeToWAL) throws IOException {
2310 List<Put> puts = new ArrayList<>();
2311 for (byte[] row : HBaseTestingUtility.ROWS) {
2312 Put put = new Put(row);
2313 put.setDurability(writeToWAL ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
2314 for (int i = 0; i < f.length; i++) {
2315 put.add(f[i], f[i], value != null ? value : row);
2316 }
2317 puts.add(put);
2318 }
2319 t.put(puts);
2320 return puts.size();
2321 }
2322
2323
2324
2325
2326 public static class SeenRowTracker {
2327 int dim = 'z' - 'a' + 1;
2328 int[][][] seenRows = new int[dim][dim][dim];
2329 byte[] startRow;
2330 byte[] stopRow;
2331
2332 public SeenRowTracker(byte[] startRow, byte[] stopRow) {
2333 this.startRow = startRow;
2334 this.stopRow = stopRow;
2335 }
2336
2337 void reset() {
2338 for (byte[] row : ROWS) {
2339 seenRows[i(row[0])][i(row[1])][i(row[2])] = 0;
2340 }
2341 }
2342
2343 int i(byte b) {
2344 return b - 'a';
2345 }
2346
2347 public void addRow(byte[] row) {
2348 seenRows[i(row[0])][i(row[1])][i(row[2])]++;
2349 }
2350
2351
2352
2353
2354 public void validate() {
2355 for (byte b1 = 'a'; b1 <= 'z'; b1++) {
2356 for (byte b2 = 'a'; b2 <= 'z'; b2++) {
2357 for (byte b3 = 'a'; b3 <= 'z'; b3++) {
2358 int count = seenRows[i(b1)][i(b2)][i(b3)];
2359 int expectedCount = 0;
2360 if (Bytes.compareTo(new byte[] {b1,b2,b3}, startRow) >= 0
2361 && Bytes.compareTo(new byte[] {b1,b2,b3}, stopRow) < 0) {
2362 expectedCount = 1;
2363 }
2364 if (count != expectedCount) {
2365 String row = new String(new byte[] {b1,b2,b3});
2366 throw new RuntimeException("Row:" + row + " has a seen count of " + count + " instead of " + expectedCount);
2367 }
2368 }
2369 }
2370 }
2371 }
2372 }
2373
2374 public int loadRegion(final HRegion r, final byte[] f) throws IOException {
2375 return loadRegion(r, f, false);
2376 }
2377
2378 public int loadRegion(final Region r, final byte[] f) throws IOException {
2379 return loadRegion((HRegion)r, f);
2380 }
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390 public int loadRegion(final HRegion r, final byte[] f, final boolean flush)
2391 throws IOException {
2392 byte[] k = new byte[3];
2393 int rowCount = 0;
2394 for (byte b1 = 'a'; b1 <= 'z'; b1++) {
2395 for (byte b2 = 'a'; b2 <= 'z'; b2++) {
2396 for (byte b3 = 'a'; b3 <= 'z'; b3++) {
2397 k[0] = b1;
2398 k[1] = b2;
2399 k[2] = b3;
2400 Put put = new Put(k);
2401 put.setDurability(Durability.SKIP_WAL);
2402 put.add(f, null, k);
2403 if (r.getWAL() == null) {
2404 put.setDurability(Durability.SKIP_WAL);
2405 }
2406 int preRowCount = rowCount;
2407 int pause = 10;
2408 int maxPause = 1000;
2409 while (rowCount == preRowCount) {
2410 try {
2411 r.put(put);
2412 rowCount++;
2413 } catch (RegionTooBusyException e) {
2414 pause = (pause * 2 >= maxPause) ? maxPause : pause * 2;
2415 Threads.sleep(pause);
2416 }
2417 }
2418 }
2419 }
2420 if (flush) {
2421 r.flush(true);
2422 }
2423 }
2424 return rowCount;
2425 }
2426
2427 public void loadNumericRows(final Table t, final byte[] f, int startRow, int endRow)
2428 throws IOException {
2429 for (int i = startRow; i < endRow; i++) {
2430 byte[] data = Bytes.toBytes(String.valueOf(i));
2431 Put put = new Put(data);
2432 put.add(f, null, data);
2433 t.put(put);
2434 }
2435 }
2436
2437 public void verifyNumericRows(Table table, final byte[] f, int startRow, int endRow,
2438 int replicaId)
2439 throws IOException {
2440 for (int i = startRow; i < endRow; i++) {
2441 String failMsg = "Failed verification of row :" + i;
2442 byte[] data = Bytes.toBytes(String.valueOf(i));
2443 Get get = new Get(data);
2444 get.setReplicaId(replicaId);
2445 get.setConsistency(Consistency.TIMELINE);
2446 Result result = table.get(get);
2447 assertTrue(failMsg, result.containsColumn(f, null));
2448 assertEquals(failMsg, result.getColumnCells(f, null).size(), 1);
2449 Cell cell = result.getColumnLatestCell(f, null);
2450 assertTrue(failMsg,
2451 Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(),
2452 cell.getValueLength()));
2453 }
2454 }
2455
2456 public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow)
2457 throws IOException {
2458 verifyNumericRows((HRegion)region, f, startRow, endRow);
2459 }
2460
2461 public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow)
2462 throws IOException {
2463 verifyNumericRows(region, f, startRow, endRow, true);
2464 }
2465
2466 public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow,
2467 final boolean present) throws IOException {
2468 verifyNumericRows((HRegion)region, f, startRow, endRow, present);
2469 }
2470
2471 public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow,
2472 final boolean present) throws IOException {
2473 for (int i = startRow; i < endRow; i++) {
2474 String failMsg = "Failed verification of row :" + i;
2475 byte[] data = Bytes.toBytes(String.valueOf(i));
2476 Result result = region.get(new Get(data));
2477
2478 boolean hasResult = result != null && !result.isEmpty();
2479 assertEquals(failMsg + result, present, hasResult);
2480 if (!present) continue;
2481
2482 assertTrue(failMsg, result.containsColumn(f, null));
2483 assertEquals(failMsg, result.getColumnCells(f, null).size(), 1);
2484 Cell cell = result.getColumnLatestCell(f, null);
2485 assertTrue(failMsg,
2486 Bytes.equals(data, 0, data.length, cell.getValueArray(), cell.getValueOffset(),
2487 cell.getValueLength()));
2488 }
2489 }
2490
2491 public void deleteNumericRows(final HTable t, final byte[] f, int startRow, int endRow)
2492 throws IOException {
2493 for (int i = startRow; i < endRow; i++) {
2494 byte[] data = Bytes.toBytes(String.valueOf(i));
2495 Delete delete = new Delete(data);
2496 delete.deleteFamily(f);
2497 t.delete(delete);
2498 }
2499 }
2500
2501
2502
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
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
2539
2540
2541 public String checksumRows(final Table table) throws Exception {
2542 Scan scan = new Scan();
2543 ResultScanner results = table.getScanner(scan);
2544 MessageDigest digest = MessageDigest.getInstance("MD5");
2545 for (Result res : results) {
2546 digest.update(res.getRow());
2547 }
2548 results.close();
2549 return digest.toString();
2550 }
2551
2552
2553 public static final byte[][] ROWS = new byte[(int) Math.pow('z' - 'a' + 1, 3)][3];
2554 static {
2555 int i = 0;
2556 for (byte b1 = 'a'; b1 <= 'z'; b1++) {
2557 for (byte b2 = 'a'; b2 <= 'z'; b2++) {
2558 for (byte b3 = 'a'; b3 <= 'z'; b3++) {
2559 ROWS[i][0] = b1;
2560 ROWS[i][1] = b2;
2561 ROWS[i][2] = b3;
2562 i++;
2563 }
2564 }
2565 }
2566 }
2567
2568 public static final byte[][] KEYS = {
2569 HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("bbb"),
2570 Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
2571 Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
2572 Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
2573 Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
2574 Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
2575 Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
2576 Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
2577 Bytes.toBytes("xxx"), Bytes.toBytes("yyy")
2578 };
2579
2580 public static final byte[][] KEYS_FOR_HBA_CREATE_TABLE = {
2581 Bytes.toBytes("bbb"),
2582 Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
2583 Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
2584 Bytes.toBytes("iii"), Bytes.toBytes("jjj"), Bytes.toBytes("kkk"),
2585 Bytes.toBytes("lll"), Bytes.toBytes("mmm"), Bytes.toBytes("nnn"),
2586 Bytes.toBytes("ooo"), Bytes.toBytes("ppp"), Bytes.toBytes("qqq"),
2587 Bytes.toBytes("rrr"), Bytes.toBytes("sss"), Bytes.toBytes("ttt"),
2588 Bytes.toBytes("uuu"), Bytes.toBytes("vvv"), Bytes.toBytes("www"),
2589 Bytes.toBytes("xxx"), Bytes.toBytes("yyy"), Bytes.toBytes("zzz")
2590 };
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602 public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
2603 final HTableDescriptor htd, byte [][] startKeys)
2604 throws IOException {
2605 Table meta = new HTable(conf, TableName.META_TABLE_NAME);
2606 Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
2607 List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
2608
2609 for (int i = 0; i < startKeys.length; i++) {
2610 int j = (i + 1) % startKeys.length;
2611 HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKeys[i],
2612 startKeys[j]);
2613 MetaTableAccessor.addRegionToMeta(meta, hri);
2614 newRegions.add(hri);
2615 }
2616
2617 meta.close();
2618 return newRegions;
2619 }
2620
2621
2622
2623
2624
2625
2626 public List<byte[]> getMetaTableRows() throws IOException {
2627
2628 Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
2629 List<byte[]> rows = new ArrayList<byte[]>();
2630 ResultScanner s = t.getScanner(new Scan());
2631 for (Result result : s) {
2632 LOG.info("getMetaTableRows: row -> " +
2633 Bytes.toStringBinary(result.getRow()));
2634 rows.add(result.getRow());
2635 }
2636 s.close();
2637 t.close();
2638 return rows;
2639 }
2640
2641
2642
2643
2644
2645
2646 public List<byte[]> getMetaTableRows(TableName tableName) throws IOException {
2647
2648 Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME);
2649 List<byte[]> rows = new ArrayList<byte[]>();
2650 ResultScanner s = t.getScanner(new Scan());
2651 for (Result result : s) {
2652 HRegionInfo info = HRegionInfo.getHRegionInfo(result);
2653 if (info == null) {
2654 LOG.error("No region info for row " + Bytes.toString(result.getRow()));
2655
2656 continue;
2657 }
2658
2659 if (info.getTable().equals(tableName)) {
2660 LOG.info("getMetaTableRows: row -> " +
2661 Bytes.toStringBinary(result.getRow()) + info);
2662 rows.add(result.getRow());
2663 }
2664 }
2665 s.close();
2666 t.close();
2667 return rows;
2668 }
2669
2670
2671
2672
2673
2674
2675 public HRegionServer getOtherRegionServer(HRegionServer rs) {
2676 for (JVMClusterUtil.RegionServerThread rst :
2677 getMiniHBaseCluster().getRegionServerThreads()) {
2678 if (!(rst.getRegionServer() == rs)) {
2679 return rst.getRegionServer();
2680 }
2681 }
2682 return null;
2683 }
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696 public HRegionServer getRSForFirstRegionInTable(TableName tableName)
2697 throws IOException, InterruptedException {
2698 List<byte[]> metaRows = getMetaTableRows(tableName);
2699 if (metaRows == null || metaRows.isEmpty()) {
2700 return null;
2701 }
2702 LOG.debug("Found " + metaRows.size() + " rows for table " +
2703 tableName);
2704 byte [] firstrow = metaRows.get(0);
2705 LOG.debug("FirstRow=" + Bytes.toString(firstrow));
2706 long pause = getConfiguration().getLong(HConstants.HBASE_CLIENT_PAUSE,
2707 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
2708 int numRetries = getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
2709 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
2710 RetryCounter retrier = new RetryCounter(numRetries+1, (int)pause, TimeUnit.MICROSECONDS);
2711 while(retrier.shouldRetry()) {
2712 int index = getMiniHBaseCluster().getServerWith(firstrow);
2713 if (index != -1) {
2714 return getMiniHBaseCluster().getRegionServerThreads().get(index).getRegionServer();
2715 }
2716
2717 retrier.sleepUntilNextRetry();
2718 }
2719 return null;
2720 }
2721
2722
2723
2724
2725
2726
2727
2728 public MiniMRCluster startMiniMapReduceCluster() throws IOException {
2729 startMiniMapReduceCluster(2);
2730 return mrCluster;
2731 }
2732
2733
2734
2735
2736
2737 private void forceChangeTaskLogDir() {
2738 Field logDirField;
2739 try {
2740 logDirField = TaskLog.class.getDeclaredField("LOG_DIR");
2741 logDirField.setAccessible(true);
2742
2743 Field modifiersField = Field.class.getDeclaredField("modifiers");
2744 modifiersField.setAccessible(true);
2745 modifiersField.setInt(logDirField, logDirField.getModifiers() & ~Modifier.FINAL);
2746
2747 logDirField.set(null, new File(hadoopLogDir, "userlogs"));
2748 } catch (SecurityException e) {
2749 throw new RuntimeException(e);
2750 } catch (NoSuchFieldException e) {
2751
2752 throw new RuntimeException(e);
2753 } catch (IllegalArgumentException e) {
2754 throw new RuntimeException(e);
2755 } catch (IllegalAccessException e) {
2756 throw new RuntimeException(e);
2757 }
2758 }
2759
2760
2761
2762
2763
2764
2765
2766 private void startMiniMapReduceCluster(final int servers) throws IOException {
2767 if (mrCluster != null) {
2768 throw new IllegalStateException("MiniMRCluster is already running");
2769 }
2770 LOG.info("Starting mini mapreduce cluster...");
2771 setupClusterTestDir();
2772 createDirsAndSetProperties();
2773
2774 forceChangeTaskLogDir();
2775
2776
2777
2778
2779 conf.setFloat("yarn.nodemanager.vmem-pmem-ratio", 8.0f);
2780
2781
2782
2783 conf.setBoolean("mapreduce.map.speculative", false);
2784 conf.setBoolean("mapreduce.reduce.speculative", false);
2785
2786
2787
2788 mrCluster = new MiniMRCluster(servers,
2789 FS_URI != null ? FS_URI : FileSystem.get(conf).getUri().toString(), 1,
2790 null, null, new JobConf(this.conf));
2791 JobConf jobConf = MapreduceTestingShim.getJobConf(mrCluster);
2792 if (jobConf == null) {
2793 jobConf = mrCluster.createJobConf();
2794 }
2795
2796 jobConf.set("mapreduce.cluster.local.dir",
2797 conf.get("mapreduce.cluster.local.dir"));
2798 LOG.info("Mini mapreduce cluster started");
2799
2800
2801
2802
2803 conf.set("mapreduce.jobtracker.address", jobConf.get("mapreduce.jobtracker.address"));
2804
2805 conf.set("mapreduce.framework.name", "yarn");
2806 conf.setBoolean("yarn.is.minicluster", true);
2807 String rmAddress = jobConf.get("yarn.resourcemanager.address");
2808 if (rmAddress != null) {
2809 conf.set("yarn.resourcemanager.address", rmAddress);
2810 }
2811 String historyAddress = jobConf.get("mapreduce.jobhistory.address");
2812 if (historyAddress != null) {
2813 conf.set("mapreduce.jobhistory.address", historyAddress);
2814 }
2815 String schedulerAddress =
2816 jobConf.get("yarn.resourcemanager.scheduler.address");
2817 if (schedulerAddress != null) {
2818 conf.set("yarn.resourcemanager.scheduler.address", schedulerAddress);
2819 }
2820 String mrJobHistoryWebappAddress =
2821 jobConf.get("mapreduce.jobhistory.webapp.address");
2822 if (mrJobHistoryWebappAddress != null) {
2823 conf.set("mapreduce.jobhistory.webapp.address", mrJobHistoryWebappAddress);
2824 }
2825 String yarnRMWebappAddress =
2826 jobConf.get("yarn.resourcemanager.webapp.address");
2827 if (yarnRMWebappAddress != null) {
2828 conf.set("yarn.resourcemanager.webapp.address", yarnRMWebappAddress);
2829 }
2830 }
2831
2832
2833
2834
2835 public void shutdownMiniMapReduceCluster() {
2836 if (mrCluster != null) {
2837 LOG.info("Stopping mini mapreduce cluster...");
2838 mrCluster.shutdown();
2839 mrCluster = null;
2840 LOG.info("Mini mapreduce cluster stopped");
2841 }
2842
2843 conf.set("mapreduce.jobtracker.address", "local");
2844 }
2845
2846
2847
2848
2849 public RegionServerServices createMockRegionServerService() throws IOException {
2850 return createMockRegionServerService((ServerName)null);
2851 }
2852
2853
2854
2855
2856
2857 public RegionServerServices createMockRegionServerService(RpcServerInterface rpc) throws IOException {
2858 final MockRegionServerServices rss = new MockRegionServerServices(getZooKeeperWatcher());
2859 rss.setFileSystem(getTestFileSystem());
2860 rss.setRpcServer(rpc);
2861 return rss;
2862 }
2863
2864
2865
2866
2867
2868 public RegionServerServices createMockRegionServerService(ServerName name) throws IOException {
2869 final MockRegionServerServices rss = new MockRegionServerServices(getZooKeeperWatcher(), name);
2870 rss.setFileSystem(getTestFileSystem());
2871 return rss;
2872 }
2873
2874
2875
2876
2877
2878
2879 public void enableDebug(Class<?> clazz) {
2880 Log l = LogFactory.getLog(clazz);
2881 if (l instanceof Log4JLogger) {
2882 ((Log4JLogger) l).getLogger().setLevel(org.apache.log4j.Level.DEBUG);
2883 } else if (l instanceof Jdk14Logger) {
2884 ((Jdk14Logger) l).getLogger().setLevel(java.util.logging.Level.ALL);
2885 }
2886 }
2887
2888
2889
2890
2891
2892 public void expireMasterSession() throws Exception {
2893 HMaster master = getMiniHBaseCluster().getMaster();
2894 expireSession(master.getZooKeeper(), false);
2895 }
2896
2897
2898
2899
2900
2901
2902 public void expireRegionServerSession(int index) throws Exception {
2903 HRegionServer rs = getMiniHBaseCluster().getRegionServer(index);
2904 expireSession(rs.getZooKeeper(), false);
2905 decrementMinRegionServerCount();
2906 }
2907
2908 private void decrementMinRegionServerCount() {
2909
2910
2911 decrementMinRegionServerCount(getConfiguration());
2912
2913
2914 for (MasterThread master : getHBaseCluster().getMasterThreads()) {
2915 decrementMinRegionServerCount(master.getMaster().getConfiguration());
2916 }
2917 }
2918
2919 private void decrementMinRegionServerCount(Configuration conf) {
2920 int currentCount = conf.getInt(
2921 ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
2922 if (currentCount != -1) {
2923 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
2924 Math.max(currentCount - 1, 1));
2925 }
2926 }
2927
2928 public void expireSession(ZooKeeperWatcher nodeZK) throws Exception {
2929 expireSession(nodeZK, false);
2930 }
2931
2932 @Deprecated
2933 public void expireSession(ZooKeeperWatcher nodeZK, Server server)
2934 throws Exception {
2935 expireSession(nodeZK, false);
2936 }
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949 public void expireSession(ZooKeeperWatcher nodeZK, boolean checkStatus)
2950 throws Exception {
2951 Configuration c = new Configuration(this.conf);
2952 String quorumServers = ZKConfig.getZKQuorumServersString(c);
2953 ZooKeeper zk = nodeZK.getRecoverableZooKeeper().getZooKeeper();
2954 byte[] password = zk.getSessionPasswd();
2955 long sessionID = zk.getSessionId();
2956
2957
2958
2959
2960
2961
2962
2963
2964 ZooKeeper monitor = new ZooKeeper(quorumServers,
2965 1000, new org.apache.zookeeper.Watcher(){
2966 @Override
2967 public void process(WatchedEvent watchedEvent) {
2968 LOG.info("Monitor ZKW received event="+watchedEvent);
2969 }
2970 } , sessionID, password);
2971
2972
2973 ZooKeeper newZK = new ZooKeeper(quorumServers,
2974 1000, EmptyWatcher.instance, sessionID, password);
2975
2976
2977
2978 long start = System.currentTimeMillis();
2979 while (newZK.getState() != States.CONNECTED
2980 && System.currentTimeMillis() - start < 1000) {
2981 Thread.sleep(1);
2982 }
2983 newZK.close();
2984 LOG.info("ZK Closed Session 0x" + Long.toHexString(sessionID));
2985
2986
2987 monitor.close();
2988
2989 if (checkStatus) {
2990 new HTable(new Configuration(conf), TableName.META_TABLE_NAME).close();
2991 }
2992 }
2993
2994
2995
2996
2997
2998
2999
3000 public MiniHBaseCluster getHBaseCluster() {
3001 return getMiniHBaseCluster();
3002 }
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012 public HBaseCluster getHBaseClusterInterface() {
3013
3014
3015 return hbaseCluster;
3016 }
3017
3018
3019
3020
3021
3022
3023
3024 public Connection getConnection() throws IOException {
3025 Connection connection = this.connectionRef.get();
3026 while (connection == null) {
3027 connection = ConnectionFactory.createConnection(this.conf);
3028 if (! this.connectionRef.compareAndSet(null, connection)) {
3029 try {
3030 connection.close();
3031 } catch (IOException exception) {
3032 LOG.debug("Ignored failure while closing connection on contended connection creation.",
3033 exception);
3034 }
3035 connection = this.connectionRef.get();
3036 }
3037 }
3038 return connection;
3039 }
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050 public synchronized HBaseAdmin getHBaseAdmin()
3051 throws IOException {
3052 if (hbaseAdmin == null){
3053 this.hbaseAdmin = new HBaseAdminForTests(getConnection());
3054 }
3055 return hbaseAdmin;
3056 }
3057
3058 private HBaseAdminForTests hbaseAdmin = null;
3059 private static class HBaseAdminForTests extends HBaseAdmin {
3060 public HBaseAdminForTests(Connection connection) throws MasterNotRunningException,
3061 ZooKeeperConnectionException, IOException {
3062 super(connection);
3063 }
3064
3065 @Override
3066 public synchronized void close() throws IOException {
3067 LOG.warn("close() called on HBaseAdmin instance returned from " +
3068 "HBaseTestingUtility.getHBaseAdmin()");
3069 }
3070
3071 private synchronized void close0() throws IOException {
3072 super.close();
3073 }
3074 }
3075
3076 public void closeConnection() throws IOException {
3077 if (hbaseAdmin != null) {
3078 try {
3079 hbaseAdmin.close0();
3080 } catch (IOException exception) {
3081 LOG.debug("Ignored failure while closing admin.", exception);
3082 }
3083 hbaseAdmin = null;
3084 }
3085 Connection connection = this.connectionRef.getAndSet(null);
3086 if (connection != null) {
3087 try {
3088 connection.close();
3089 } catch (IOException exception) {
3090 LOG.debug("Ignored failure while closing connection.", exception);
3091 }
3092 }
3093 }
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104 public synchronized ZooKeeperWatcher getZooKeeperWatcher()
3105 throws IOException {
3106 if (zooKeeperWatcher == null) {
3107 zooKeeperWatcher = new ZooKeeperWatcher(conf, "testing utility",
3108 new Abortable() {
3109 @Override public void abort(String why, Throwable e) {
3110 throw new RuntimeException("Unexpected abort in HBaseTestingUtility:"+why, e);
3111 }
3112 @Override public boolean isAborted() {return false;}
3113 });
3114 }
3115 return zooKeeperWatcher;
3116 }
3117 private ZooKeeperWatcher zooKeeperWatcher;
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127 public void closeRegion(String regionName) throws IOException {
3128 closeRegion(Bytes.toBytes(regionName));
3129 }
3130
3131
3132
3133
3134
3135
3136
3137 public void closeRegion(byte[] regionName) throws IOException {
3138 getHBaseAdmin().closeRegion(regionName, null);
3139 }
3140
3141
3142
3143
3144
3145
3146
3147
3148 public void closeRegionByRow(String row, RegionLocator table) throws IOException {
3149 closeRegionByRow(Bytes.toBytes(row), table);
3150 }
3151
3152
3153
3154
3155
3156
3157
3158
3159 public void closeRegionByRow(byte[] row, RegionLocator table) throws IOException {
3160 HRegionLocation hrl = table.getRegionLocation(row);
3161 closeRegion(hrl.getRegionInfo().getRegionName());
3162 }
3163
3164
3165
3166
3167
3168
3169
3170
3171 public HRegion getSplittableRegion(TableName tableName, int maxAttempts) {
3172 List<HRegion> regions = getHBaseCluster().getRegions(tableName);
3173 int regCount = regions.size();
3174 Set<Integer> attempted = new HashSet<Integer>();
3175 int idx;
3176 int attempts = 0;
3177 do {
3178 regions = getHBaseCluster().getRegions(tableName);
3179 if (regCount != regions.size()) {
3180
3181 attempted.clear();
3182 }
3183 regCount = regions.size();
3184
3185
3186 if (regCount > 0) {
3187 idx = random.nextInt(regCount);
3188
3189 if (attempted.contains(idx))
3190 continue;
3191 try {
3192 regions.get(idx).checkSplit();
3193 return regions.get(idx);
3194 } catch (Exception ex) {
3195 LOG.warn("Caught exception", ex);
3196 attempted.add(idx);
3197 }
3198 }
3199 attempts++;
3200 } while (maxAttempts == -1 || attempts < maxAttempts);
3201 return null;
3202 }
3203
3204 public MiniZooKeeperCluster getZkCluster() {
3205 return zkCluster;
3206 }
3207
3208 public void setZkCluster(MiniZooKeeperCluster zkCluster) {
3209 this.passedZkCluster = true;
3210 this.zkCluster = zkCluster;
3211 conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkCluster.getClientPort());
3212 }
3213
3214 public MiniDFSCluster getDFSCluster() {
3215 return dfsCluster;
3216 }
3217
3218 public void setDFSCluster(MiniDFSCluster cluster) throws IllegalStateException, IOException {
3219 setDFSCluster(cluster, true);
3220 }
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230 public void setDFSCluster(MiniDFSCluster cluster, boolean requireDown)
3231 throws IllegalStateException, IOException {
3232 if (dfsCluster != null && requireDown && dfsCluster.isClusterUp()) {
3233 throw new IllegalStateException("DFSCluster is already running! Shut it down first.");
3234 }
3235 this.dfsCluster = cluster;
3236 this.setFs();
3237 }
3238
3239 public FileSystem getTestFileSystem() throws IOException {
3240 return HFileSystem.get(conf);
3241 }
3242
3243
3244
3245
3246
3247
3248
3249
3250 public void waitTableAvailable(TableName table)
3251 throws InterruptedException, IOException {
3252 waitTableAvailable(table.getName(), 30000);
3253 }
3254
3255 public void waitTableAvailable(TableName table, long timeoutMillis)
3256 throws InterruptedException, IOException {
3257 waitFor(timeoutMillis, predicateTableAvailable(table));
3258 }
3259
3260 public String explainTableAvailability(TableName tableName) throws IOException {
3261 String msg = explainTableState(tableName) + ",";
3262 if (getHBaseCluster().getMaster().isAlive()) {
3263 Map<HRegionInfo, ServerName> assignments =
3264 getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
3265 .getRegionAssignments();
3266 final List<Pair<HRegionInfo, ServerName>> metaLocations =
3267 MetaTableAccessor
3268 .getTableRegionsAndLocations(getZooKeeperWatcher(), getConnection(), tableName);
3269 for (Pair<HRegionInfo, ServerName> metaLocation : metaLocations) {
3270 HRegionInfo hri = metaLocation.getFirst();
3271 ServerName sn = metaLocation.getSecond();
3272 if (!assignments.containsKey(hri)) {
3273 msg += ", region " + hri
3274 + " not assigned, but found in meta, it expected to be on " + sn;
3275
3276 } else if (sn == null) {
3277 msg += ", region " + hri
3278 + " assigned, but has no server in meta";
3279 } else if (!sn.equals(assignments.get(hri))) {
3280 msg += ", region " + hri
3281 + " assigned, but has different servers in meta and AM ( " +
3282 sn + " <> " + assignments.get(hri);
3283 }
3284 }
3285 }
3286 return msg;
3287 }
3288
3289 public String explainTableState(TableName tableName) throws IOException {
3290 try {
3291 if (getHBaseAdmin().isTableEnabled(tableName))
3292 return "table enabled in zk";
3293 else if (getHBaseAdmin().isTableDisabled(tableName))
3294 return "table disabled in zk";
3295 else
3296 return "table in uknown state";
3297 } catch (TableNotFoundException e) {
3298 return "table not exists";
3299 }
3300 }
3301
3302
3303
3304
3305
3306
3307
3308
3309 public void waitTableAvailable(byte[] table, long timeoutMillis)
3310 throws InterruptedException, IOException {
3311 waitTableAvailable(getHBaseAdmin(), table, timeoutMillis);
3312 }
3313
3314 public void waitTableAvailable(Admin admin, byte[] table, long timeoutMillis)
3315 throws InterruptedException, IOException {
3316 long startWait = System.currentTimeMillis();
3317 while (!admin.isTableAvailable(TableName.valueOf(table))) {
3318 assertTrue("Timed out waiting for table to become available " +
3319 Bytes.toStringBinary(table),
3320 System.currentTimeMillis() - startWait < timeoutMillis);
3321 Thread.sleep(200);
3322 }
3323 }
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333 public void waitTableEnabled(TableName table)
3334 throws InterruptedException, IOException {
3335 waitTableEnabled(table, 30000);
3336 }
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347 public void waitTableEnabled(byte[] table, long timeoutMillis)
3348 throws InterruptedException, IOException {
3349 waitTableEnabled(TableName.valueOf(table), timeoutMillis);
3350 }
3351
3352 public void waitTableEnabled(TableName table, long timeoutMillis)
3353 throws IOException {
3354 waitFor(timeoutMillis, predicateTableEnabled(table));
3355 }
3356
3357
3358
3359
3360
3361
3362
3363
3364 public void waitTableDisabled(byte[] table)
3365 throws InterruptedException, IOException {
3366 waitTableDisabled(getHBaseAdmin(), table, 30000);
3367 }
3368
3369 public void waitTableDisabled(Admin admin, byte[] table)
3370 throws InterruptedException, IOException {
3371 waitTableDisabled(admin, table, 30000);
3372 }
3373
3374
3375
3376
3377
3378
3379
3380
3381 public void waitTableDisabled(byte[] table, long timeoutMillis)
3382 throws InterruptedException, IOException {
3383 waitTableDisabled(getHBaseAdmin(), table, timeoutMillis);
3384 }
3385
3386 public void waitTableDisabled(Admin admin, byte[] table, long timeoutMillis)
3387 throws InterruptedException, IOException {
3388 TableName tableName = TableName.valueOf(table);
3389 long startWait = System.currentTimeMillis();
3390 while (!admin.isTableDisabled(tableName)) {
3391 assertTrue("Timed out waiting for table to become disabled " +
3392 Bytes.toStringBinary(table),
3393 System.currentTimeMillis() - startWait < timeoutMillis);
3394 Thread.sleep(200);
3395 }
3396 }
3397
3398
3399
3400
3401
3402
3403
3404
3405 public boolean ensureSomeRegionServersAvailable(final int num)
3406 throws IOException {
3407 boolean startedServer = false;
3408 MiniHBaseCluster hbaseCluster = getMiniHBaseCluster();
3409 for (int i=hbaseCluster.getLiveRegionServerThreads().size(); i<num; ++i) {
3410 LOG.info("Started new server=" + hbaseCluster.startRegionServer());
3411 startedServer = true;
3412 }
3413
3414 return startedServer;
3415 }
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426 public boolean ensureSomeNonStoppedRegionServersAvailable(final int num)
3427 throws IOException {
3428 boolean startedServer = ensureSomeRegionServersAvailable(num);
3429
3430 int nonStoppedServers = 0;
3431 for (JVMClusterUtil.RegionServerThread rst :
3432 getMiniHBaseCluster().getRegionServerThreads()) {
3433
3434 HRegionServer hrs = rst.getRegionServer();
3435 if (hrs.isStopping() || hrs.isStopped()) {
3436 LOG.info("A region server is stopped or stopping:"+hrs);
3437 } else {
3438 nonStoppedServers++;
3439 }
3440 }
3441 for (int i=nonStoppedServers; i<num; ++i) {
3442 LOG.info("Started new server=" + getMiniHBaseCluster().startRegionServer());
3443 startedServer = true;
3444 }
3445 return startedServer;
3446 }
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458 public static User getDifferentUser(final Configuration c,
3459 final String differentiatingSuffix)
3460 throws IOException {
3461 FileSystem currentfs = FileSystem.get(c);
3462 if (!(currentfs instanceof DistributedFileSystem) || User.isHBaseSecurityEnabled(c)) {
3463 return User.getCurrent();
3464 }
3465
3466
3467 String username = User.getCurrent().getName() +
3468 differentiatingSuffix;
3469 User user = User.createUserForTesting(c, username,
3470 new String[]{"supergroup"});
3471 return user;
3472 }
3473
3474 public static NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
3475 throws IOException {
3476 NavigableSet<String> online = new TreeSet<String>();
3477 for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
3478 try {
3479 for (HRegionInfo region :
3480 ProtobufUtil.getOnlineRegions(rst.getRegionServer().getRSRpcServices())) {
3481 online.add(region.getRegionNameAsString());
3482 }
3483 } catch (RegionServerStoppedException e) {
3484
3485 }
3486 }
3487 for (MasterThread mt : cluster.getLiveMasterThreads()) {
3488 try {
3489 for (HRegionInfo region :
3490 ProtobufUtil.getOnlineRegions(mt.getMaster().getRSRpcServices())) {
3491 online.add(region.getRegionNameAsString());
3492 }
3493 } catch (RegionServerStoppedException e) {
3494
3495 } catch (ServerNotRunningYetException e) {
3496
3497 }
3498 }
3499 return online;
3500 }
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515 public static void setMaxRecoveryErrorCount(final OutputStream stream,
3516 final int max) {
3517 try {
3518 Class<?> [] clazzes = DFSClient.class.getDeclaredClasses();
3519 for (Class<?> clazz: clazzes) {
3520 String className = clazz.getSimpleName();
3521 if (className.equals("DFSOutputStream")) {
3522 if (clazz.isInstance(stream)) {
3523 Field maxRecoveryErrorCountField =
3524 stream.getClass().getDeclaredField("maxRecoveryErrorCount");
3525 maxRecoveryErrorCountField.setAccessible(true);
3526 maxRecoveryErrorCountField.setInt(stream, max);
3527 break;
3528 }
3529 }
3530 }
3531 } catch (Exception e) {
3532 LOG.info("Could not set max recovery field", e);
3533 }
3534 }
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544 public boolean assignRegion(final HRegionInfo regionInfo)
3545 throws IOException, InterruptedException {
3546 final AssignmentManager am = getHBaseCluster().getMaster().getAssignmentManager();
3547 am.assign(regionInfo, true);
3548 return am.waitForAssignment(regionInfo);
3549 }
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559 public void moveRegionAndWait(HRegionInfo destRegion, ServerName destServer)
3560 throws InterruptedException, IOException {
3561 HMaster master = getMiniHBaseCluster().getMaster();
3562 getHBaseAdmin().move(destRegion.getEncodedNameAsBytes(),
3563 Bytes.toBytes(destServer.getServerName()));
3564 while (true) {
3565 ServerName serverName = master.getAssignmentManager().getRegionStates()
3566 .getRegionServerOfRegion(destRegion);
3567 if (serverName != null && serverName.equals(destServer)) {
3568 assertRegionOnServer(destRegion, serverName, 200);
3569 break;
3570 }
3571 Thread.sleep(10);
3572 }
3573 }
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584 public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOException {
3585 waitUntilAllRegionsAssigned(
3586 tableName,
3587 this.conf.getLong("hbase.client.sync.wait.timeout.msec", 60000));
3588 }
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599 public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout)
3600 throws IOException {
3601 final Table meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME);
3602 try {
3603 waitFor(timeout, 200, true, new Predicate<IOException>() {
3604 @Override
3605 public boolean evaluate() throws IOException {
3606 boolean allRegionsAssigned = true;
3607 Scan scan = new Scan();
3608 scan.addFamily(HConstants.CATALOG_FAMILY);
3609 ResultScanner s = meta.getScanner(scan);
3610 try {
3611 Result r;
3612 while ((r = s.next()) != null) {
3613 byte[] b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
3614 HRegionInfo info = HRegionInfo.parseFromOrNull(b);
3615 if (info != null && info.getTable().equals(tableName)) {
3616 b = r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
3617 allRegionsAssigned &= (b != null);
3618 }
3619 }
3620 } finally {
3621 s.close();
3622 }
3623 return allRegionsAssigned;
3624 }
3625 });
3626 } finally {
3627 meta.close();
3628 }
3629
3630
3631 if (!getHBaseClusterInterface().isDistributedCluster()) {
3632
3633
3634 HMaster master = getHBaseCluster().getMaster();
3635 final RegionStates states = master.getAssignmentManager().getRegionStates();
3636 waitFor(timeout, 200, new ExplainingPredicate<IOException>() {
3637 @Override
3638 public String explainFailure() throws IOException {
3639 return explainTableAvailability(tableName);
3640 }
3641
3642 @Override
3643 public boolean evaluate() throws IOException {
3644 List<HRegionInfo> hris = states.getRegionsOfTable(tableName);
3645 return hris != null && !hris.isEmpty();
3646 }
3647 });
3648 }
3649 }
3650
3651
3652
3653
3654
3655 public static List<Cell> getFromStoreFile(HStore store,
3656 Get get) throws IOException {
3657 Scan scan = new Scan(get);
3658 InternalScanner scanner = (InternalScanner) store.getScanner(scan,
3659 scan.getFamilyMap().get(store.getFamily().getName()),
3660
3661
3662 0);
3663
3664 List<Cell> result = new ArrayList<Cell>();
3665 scanner.next(result);
3666 if (!result.isEmpty()) {
3667
3668 Cell kv = result.get(0);
3669 if (!CellUtil.matchingRow(kv, get.getRow())) {
3670 result.clear();
3671 }
3672 }
3673 scanner.close();
3674 return result;
3675 }
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685 public byte[][] getRegionSplitStartKeys(byte[] startKey, byte[] endKey, int numRegions){
3686 assertTrue(numRegions>3);
3687 byte [][] tmpSplitKeys = Bytes.split(startKey, endKey, numRegions - 3);
3688 byte [][] result = new byte[tmpSplitKeys.length+1][];
3689 System.arraycopy(tmpSplitKeys, 0, result, 1, tmpSplitKeys.length);
3690 result[0] = HConstants.EMPTY_BYTE_ARRAY;
3691 return result;
3692 }
3693
3694
3695
3696
3697
3698 public static List<Cell> getFromStoreFile(HStore store,
3699 byte [] row,
3700 NavigableSet<byte[]> columns
3701 ) throws IOException {
3702 Get get = new Get(row);
3703 Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap();
3704 s.put(store.getFamily().getName(), columns);
3705
3706 return getFromStoreFile(store,get);
3707 }
3708
3709
3710
3711
3712
3713 public static ZooKeeperWatcher getZooKeeperWatcher(
3714 HBaseTestingUtility TEST_UTIL) throws ZooKeeperConnectionException,
3715 IOException {
3716 ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
3717 "unittest", new Abortable() {
3718 boolean aborted = false;
3719
3720 @Override
3721 public void abort(String why, Throwable e) {
3722 aborted = true;
3723 throw new RuntimeException("Fatal ZK error, why=" + why, e);
3724 }
3725
3726 @Override
3727 public boolean isAborted() {
3728 return aborted;
3729 }
3730 });
3731 return zkw;
3732 }
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745 public static ZooKeeperWatcher createAndForceNodeToOpenedState(
3746 HBaseTestingUtility TEST_UTIL, Region region,
3747 ServerName serverName) throws ZooKeeperConnectionException,
3748 IOException, KeeperException, NodeExistsException {
3749 return createAndForceNodeToOpenedState(TEST_UTIL, (HRegion)region, serverName);
3750 }
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763 public static ZooKeeperWatcher createAndForceNodeToOpenedState(
3764 HBaseTestingUtility TEST_UTIL, HRegion region,
3765 ServerName serverName) throws ZooKeeperConnectionException,
3766 IOException, KeeperException, NodeExistsException {
3767 ZooKeeperWatcher zkw = getZooKeeperWatcher(TEST_UTIL);
3768 ZKAssign.createNodeOffline(zkw, region.getRegionInfo(), serverName);
3769 int version = ZKAssign.transitionNodeOpening(zkw, region
3770 .getRegionInfo(), serverName);
3771 ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), serverName,
3772 version);
3773 return zkw;
3774 }
3775
3776 public static void assertKVListsEqual(String additionalMsg,
3777 final List<? extends Cell> expected,
3778 final List<? extends Cell> actual) {
3779 final int eLen = expected.size();
3780 final int aLen = actual.size();
3781 final int minLen = Math.min(eLen, aLen);
3782
3783 int i;
3784 for (i = 0; i < minLen
3785 && KeyValue.COMPARATOR.compare(expected.get(i), actual.get(i)) == 0;
3786 ++i) {}
3787
3788 if (additionalMsg == null) {
3789 additionalMsg = "";
3790 }
3791 if (!additionalMsg.isEmpty()) {
3792 additionalMsg = ". " + additionalMsg;
3793 }
3794
3795 if (eLen != aLen || i != minLen) {
3796 throw new AssertionError(
3797 "Expected and actual KV arrays differ at position " + i + ": " +
3798 safeGetAsStr(expected, i) + " (length " + eLen +") vs. " +
3799 safeGetAsStr(actual, i) + " (length " + aLen + ")" + additionalMsg);
3800 }
3801 }
3802
3803 public static <T> String safeGetAsStr(List<T> lst, int i) {
3804 if (0 <= i && i < lst.size()) {
3805 return lst.get(i).toString();
3806 } else {
3807 return "<out_of_range>";
3808 }
3809 }
3810
3811 public String getClusterKey() {
3812 return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":"
3813 + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":"
3814 + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
3815 HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
3816 }
3817
3818
3819 public HTable createRandomTable(String tableName,
3820 final Collection<String> families,
3821 final int maxVersions,
3822 final int numColsPerRow,
3823 final int numFlushes,
3824 final int numRegions,
3825 final int numRowsPerFlush)
3826 throws IOException, InterruptedException {
3827
3828 LOG.info("\n\nCreating random table " + tableName + " with " + numRegions +
3829 " regions, " + numFlushes + " storefiles per region, " +
3830 numRowsPerFlush + " rows per flush, maxVersions=" + maxVersions +
3831 "\n");
3832
3833 final Random rand = new Random(tableName.hashCode() * 17L + 12938197137L);
3834 final int numCF = families.size();
3835 final byte[][] cfBytes = new byte[numCF][];
3836 {
3837 int cfIndex = 0;
3838 for (String cf : families) {
3839 cfBytes[cfIndex++] = Bytes.toBytes(cf);
3840 }
3841 }
3842
3843 final int actualStartKey = 0;
3844 final int actualEndKey = Integer.MAX_VALUE;
3845 final int keysPerRegion = (actualEndKey - actualStartKey) / numRegions;
3846 final int splitStartKey = actualStartKey + keysPerRegion;
3847 final int splitEndKey = actualEndKey - keysPerRegion;
3848 final String keyFormat = "%08x";
3849 final HTable table = createTable(tableName, cfBytes,
3850 maxVersions,
3851 Bytes.toBytes(String.format(keyFormat, splitStartKey)),
3852 Bytes.toBytes(String.format(keyFormat, splitEndKey)),
3853 numRegions);
3854
3855 if (hbaseCluster != null) {
3856 getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME);
3857 }
3858
3859 for (int iFlush = 0; iFlush < numFlushes; ++iFlush) {
3860 for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) {
3861 final byte[] row = Bytes.toBytes(String.format(keyFormat,
3862 actualStartKey + rand.nextInt(actualEndKey - actualStartKey)));
3863
3864 Put put = new Put(row);
3865 Delete del = new Delete(row);
3866 for (int iCol = 0; iCol < numColsPerRow; ++iCol) {
3867 final byte[] cf = cfBytes[rand.nextInt(numCF)];
3868 final long ts = rand.nextInt();
3869 final byte[] qual = Bytes.toBytes("col" + iCol);
3870 if (rand.nextBoolean()) {
3871 final byte[] value = Bytes.toBytes("value_for_row_" + iRow +
3872 "_cf_" + Bytes.toStringBinary(cf) + "_col_" + iCol + "_ts_" +
3873 ts + "_random_" + rand.nextLong());
3874 put.add(cf, qual, ts, value);
3875 } else if (rand.nextDouble() < 0.8) {
3876 del.deleteColumn(cf, qual, ts);
3877 } else {
3878 del.deleteColumns(cf, qual, ts);
3879 }
3880 }
3881
3882 if (!put.isEmpty()) {
3883 table.put(put);
3884 }
3885
3886 if (!del.isEmpty()) {
3887 table.delete(del);
3888 }
3889 }
3890 LOG.info("Initiating flush #" + iFlush + " for table " + tableName);
3891 table.flushCommits();
3892 if (hbaseCluster != null) {
3893 getMiniHBaseCluster().flushcache(table.getName());
3894 }
3895 }
3896
3897 return table;
3898 }
3899
3900 private static final int MIN_RANDOM_PORT = 0xc000;
3901 private static final int MAX_RANDOM_PORT = 0xfffe;
3902 private static Random random = new Random();
3903
3904
3905
3906
3907
3908 public static int randomPort() {
3909 return MIN_RANDOM_PORT
3910 + random.nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT);
3911 }
3912
3913
3914
3915
3916
3917 public static int randomFreePort() {
3918 int port = 0;
3919 do {
3920 port = randomPort();
3921 if (takenRandomPorts.contains(port)) {
3922 port = 0;
3923 continue;
3924 }
3925 takenRandomPorts.add(port);
3926
3927 try {
3928 ServerSocket sock = new ServerSocket(port);
3929 sock.close();
3930 } catch (IOException ex) {
3931 port = 0;
3932 }
3933 } while (port == 0);
3934 return port;
3935 }
3936
3937
3938 public static String randomMultiCastAddress() {
3939 return "226.1.1." + random.nextInt(254);
3940 }
3941
3942
3943
3944 public static void waitForHostPort(String host, int port)
3945 throws IOException {
3946 final int maxTimeMs = 10000;
3947 final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
3948 IOException savedException = null;
3949 LOG.info("Waiting for server at " + host + ":" + port);
3950 for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
3951 try {
3952 Socket sock = new Socket(InetAddress.getByName(host), port);
3953 sock.close();
3954 savedException = null;
3955 LOG.info("Server at " + host + ":" + port + " is available");
3956 break;
3957 } catch (UnknownHostException e) {
3958 throw new IOException("Failed to look up " + host, e);
3959 } catch (IOException e) {
3960 savedException = e;
3961 }
3962 Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
3963 }
3964
3965 if (savedException != null) {
3966 throw savedException;
3967 }
3968 }
3969
3970
3971
3972
3973
3974
3975 public static int createPreSplitLoadTestTable(Configuration conf,
3976 TableName tableName, byte[] columnFamily, Algorithm compression,
3977 DataBlockEncoding dataBlockEncoding) throws IOException {
3978 return createPreSplitLoadTestTable(conf, tableName,
3979 columnFamily, compression, dataBlockEncoding, DEFAULT_REGIONS_PER_SERVER, 1,
3980 Durability.USE_DEFAULT);
3981 }
3982
3983
3984
3985
3986
3987 public static int createPreSplitLoadTestTable(Configuration conf,
3988 TableName tableName, byte[] columnFamily, Algorithm compression,
3989 DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication,
3990 Durability durability)
3991 throws IOException {
3992 HTableDescriptor desc = new HTableDescriptor(tableName);
3993 desc.setDurability(durability);
3994 desc.setRegionReplication(regionReplication);
3995 HColumnDescriptor hcd = new HColumnDescriptor(columnFamily);
3996 hcd.setDataBlockEncoding(dataBlockEncoding);
3997 hcd.setCompressionType(compression);
3998 return createPreSplitLoadTestTable(conf, desc, hcd, numRegionsPerServer);
3999 }
4000
4001
4002
4003
4004
4005
4006 public static int createPreSplitLoadTestTable(Configuration conf,
4007 TableName tableName, byte[][] columnFamilies, Algorithm compression,
4008 DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication,
4009 Durability durability)
4010 throws IOException {
4011 HTableDescriptor desc = new HTableDescriptor(tableName);
4012 desc.setDurability(durability);
4013 desc.setRegionReplication(regionReplication);
4014 HColumnDescriptor[] hcds = new HColumnDescriptor[columnFamilies.length];
4015 for (int i = 0; i < columnFamilies.length; i++) {
4016 HColumnDescriptor hcd = new HColumnDescriptor(columnFamilies[i]);
4017 hcd.setDataBlockEncoding(dataBlockEncoding);
4018 hcd.setCompressionType(compression);
4019 hcds[i] = hcd;
4020 }
4021 return createPreSplitLoadTestTable(conf, desc, hcds, numRegionsPerServer);
4022 }
4023
4024
4025
4026
4027
4028
4029 public static int createPreSplitLoadTestTable(Configuration conf,
4030 HTableDescriptor desc, HColumnDescriptor hcd) throws IOException {
4031 return createPreSplitLoadTestTable(conf, desc, hcd, DEFAULT_REGIONS_PER_SERVER);
4032 }
4033
4034
4035
4036
4037
4038
4039 public static int createPreSplitLoadTestTable(Configuration conf,
4040 HTableDescriptor desc, HColumnDescriptor hcd, int numRegionsPerServer) throws IOException {
4041 return createPreSplitLoadTestTable(conf, desc, new HColumnDescriptor[] {hcd},
4042 numRegionsPerServer);
4043 }
4044
4045
4046
4047
4048
4049
4050 public static int createPreSplitLoadTestTable(Configuration conf,
4051 HTableDescriptor desc, HColumnDescriptor[] hcds, int numRegionsPerServer) throws IOException {
4052 for (HColumnDescriptor hcd : hcds) {
4053 if (!desc.hasFamily(hcd.getName())) {
4054 desc.addFamily(hcd);
4055 }
4056 }
4057
4058 int totalNumberOfRegions = 0;
4059 Connection unmanagedConnection = ConnectionFactory.createConnection(conf);
4060 Admin admin = unmanagedConnection.getAdmin();
4061
4062 try {
4063
4064
4065
4066 int numberOfServers = admin.getClusterStatus().getServers().size();
4067 if (numberOfServers == 0) {
4068 throw new IllegalStateException("No live regionservers");
4069 }
4070
4071 totalNumberOfRegions = numberOfServers * numRegionsPerServer;
4072 LOG.info("Number of live regionservers: " + numberOfServers + ", " +
4073 "pre-splitting table into " + totalNumberOfRegions + " regions " +
4074 "(regions per server: " + numRegionsPerServer + ")");
4075
4076 byte[][] splits = new RegionSplitter.HexStringSplit().split(
4077 totalNumberOfRegions);
4078
4079 admin.createTable(desc, splits);
4080 } catch (MasterNotRunningException e) {
4081 LOG.error("Master not running", e);
4082 throw new IOException(e);
4083 } catch (TableExistsException e) {
4084 LOG.warn("Table " + desc.getTableName() +
4085 " already exists, continuing");
4086 } finally {
4087 admin.close();
4088 unmanagedConnection.close();
4089 }
4090 return totalNumberOfRegions;
4091 }
4092
4093 public static int getMetaRSPort(Configuration conf) throws IOException {
4094 try (Connection c = ConnectionFactory.createConnection();
4095 RegionLocator locator = c.getRegionLocator(TableName.META_TABLE_NAME)) {
4096 return locator.getRegionLocation(Bytes.toBytes("")).getPort();
4097 }
4098 }
4099
4100
4101
4102
4103
4104
4105
4106 public void assertRegionOnServer(
4107 final HRegionInfo hri, final ServerName server,
4108 final long timeout) throws IOException, InterruptedException {
4109 long timeoutTime = System.currentTimeMillis() + timeout;
4110 while (true) {
4111 List<HRegionInfo> regions = getHBaseAdmin().getOnlineRegions(server);
4112 if (regions.contains(hri)) return;
4113 long now = System.currentTimeMillis();
4114 if (now > timeoutTime) break;
4115 Thread.sleep(10);
4116 }
4117 fail("Could not find region " + hri.getRegionNameAsString()
4118 + " on server " + server);
4119 }
4120
4121
4122
4123
4124
4125 public void assertRegionOnlyOnServer(
4126 final HRegionInfo hri, final ServerName server,
4127 final long timeout) throws IOException, InterruptedException {
4128 long timeoutTime = System.currentTimeMillis() + timeout;
4129 while (true) {
4130 List<HRegionInfo> regions = getHBaseAdmin().getOnlineRegions(server);
4131 if (regions.contains(hri)) {
4132 List<JVMClusterUtil.RegionServerThread> rsThreads =
4133 getHBaseCluster().getLiveRegionServerThreads();
4134 for (JVMClusterUtil.RegionServerThread rsThread: rsThreads) {
4135 HRegionServer rs = rsThread.getRegionServer();
4136 if (server.equals(rs.getServerName())) {
4137 continue;
4138 }
4139 Collection<Region> hrs = rs.getOnlineRegionsLocalContext();
4140 for (Region r: hrs) {
4141 assertTrue("Region should not be double assigned",
4142 r.getRegionInfo().getRegionId() != hri.getRegionId());
4143 }
4144 }
4145 return;
4146 }
4147 long now = System.currentTimeMillis();
4148 if (now > timeoutTime) break;
4149 Thread.sleep(10);
4150 }
4151 fail("Could not find region " + hri.getRegionNameAsString()
4152 + " on server " + server);
4153 }
4154
4155 public HRegion createTestRegion(String tableName, HColumnDescriptor hcd)
4156 throws IOException {
4157 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
4158 htd.addFamily(hcd);
4159 HRegionInfo info =
4160 new HRegionInfo(TableName.valueOf(tableName), null, null, false);
4161 HRegion region =
4162 HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), htd);
4163 return region;
4164 }
4165
4166 public void setFileSystemURI(String fsURI) {
4167 FS_URI = fsURI;
4168 }
4169
4170
4171
4172
4173 public <E extends Exception> long waitFor(long timeout, Predicate<E> predicate)
4174 throws E {
4175 return Waiter.waitFor(this.conf, timeout, predicate);
4176 }
4177
4178
4179
4180
4181 public <E extends Exception> long waitFor(long timeout, long interval, Predicate<E> predicate)
4182 throws E {
4183 return Waiter.waitFor(this.conf, timeout, interval, predicate);
4184 }
4185
4186
4187
4188
4189 public <E extends Exception> long waitFor(long timeout, long interval,
4190 boolean failIfTimeout, Predicate<E> predicate) throws E {
4191 return Waiter.waitFor(this.conf, timeout, interval, failIfTimeout, predicate);
4192 }
4193
4194
4195
4196
4197 public ExplainingPredicate<IOException> predicateNoRegionsInTransition() {
4198 return new ExplainingPredicate<IOException>() {
4199 @Override
4200 public String explainFailure() throws IOException {
4201 final RegionStates regionStates = getMiniHBaseCluster().getMaster()
4202 .getAssignmentManager().getRegionStates();
4203 return "found in transition: " + regionStates.getRegionsInTransition().toString();
4204 }
4205
4206 @Override
4207 public boolean evaluate() throws IOException {
4208 HMaster master = getMiniHBaseCluster().getMaster();
4209 if (master == null) return false;
4210 AssignmentManager am = master.getAssignmentManager();
4211 if (am == null) return false;
4212 final RegionStates regionStates = am.getRegionStates();
4213 return !regionStates.isRegionsInTransition();
4214 }
4215 };
4216 }
4217
4218
4219
4220
4221 public Waiter.Predicate<IOException> predicateTableEnabled(final TableName tableName) {
4222 return new ExplainingPredicate<IOException>() {
4223 @Override
4224 public String explainFailure() throws IOException {
4225 return explainTableState(tableName);
4226 }
4227
4228 @Override
4229 public boolean evaluate() throws IOException {
4230 try {
4231 return getHBaseAdmin().tableExists(tableName)
4232 && getHBaseAdmin().isTableEnabled(tableName);
4233 } catch (TableNotFoundException tnfe) {
4234
4235 return false;
4236 }
4237 }
4238 };
4239 }
4240
4241
4242
4243
4244 public Waiter.Predicate<IOException> predicateTableDisabled(final TableName tableName) {
4245 return new ExplainingPredicate<IOException>() {
4246 @Override
4247 public String explainFailure() throws IOException {
4248 return explainTableState(tableName);
4249 }
4250
4251 @Override
4252 public boolean evaluate() throws IOException {
4253 return getHBaseAdmin().isTableDisabled(tableName);
4254 }
4255 };
4256 }
4257
4258
4259
4260
4261 public Waiter.Predicate<IOException> predicateTableAvailable(final TableName tableName) {
4262 return new ExplainingPredicate<IOException>() {
4263 @Override
4264 public String explainFailure() throws IOException {
4265 return explainTableAvailability(tableName);
4266 }
4267
4268 @Override
4269 public boolean evaluate() throws IOException {
4270 boolean tableAvailable = getHBaseAdmin().isTableAvailable(tableName);
4271 if (tableAvailable) {
4272 try (Table table = getConnection().getTable(tableName)) {
4273 HTableDescriptor htd = table.getTableDescriptor();
4274 for (HRegionLocation loc : getConnection().getRegionLocator(tableName)
4275 .getAllRegionLocations()) {
4276 Scan scan = new Scan().withStartRow(loc.getRegionInfo().getStartKey())
4277 .withStopRow(loc.getRegionInfo().getEndKey()).setOneRowLimit()
4278 .setMaxResultsPerColumnFamily(1).setCacheBlocks(false);
4279 for (byte[] family : htd.getFamiliesKeys()) {
4280 scan.addFamily(family);
4281 }
4282 try (ResultScanner scanner = table.getScanner(scan)) {
4283 scanner.next();
4284 }
4285 }
4286 }
4287 }
4288 return tableAvailable;
4289 }
4290 };
4291 }
4292
4293
4294
4295
4296
4297
4298 public void waitUntilNoRegionsInTransition(final long timeout) throws IOException {
4299 waitFor(timeout, predicateNoRegionsInTransition());
4300 }
4301
4302
4303
4304
4305
4306 public void waitUntilNoRegionsInTransition() throws IOException {
4307 waitUntilNoRegionsInTransition(15 * 60000);
4308 }
4309
4310
4311
4312
4313
4314
4315 public void waitLabelAvailable(long timeoutMillis, final String... labels) {
4316 final VisibilityLabelsCache labelsCache = VisibilityLabelsCache.get();
4317 waitFor(timeoutMillis, new Waiter.ExplainingPredicate<RuntimeException>() {
4318
4319 @Override
4320 public boolean evaluate() {
4321 for (String label : labels) {
4322 if (labelsCache.getLabelOrdinal(label) == 0) {
4323 return false;
4324 }
4325 }
4326 return true;
4327 }
4328
4329 @Override
4330 public String explainFailure() {
4331 for (String label : labels) {
4332 if (labelsCache.getLabelOrdinal(label) == 0) {
4333 return label + " is not available yet";
4334 }
4335 }
4336 return "";
4337 }
4338 });
4339 }
4340
4341
4342
4343
4344
4345
4346 public static List<HColumnDescriptor> generateColumnDescriptors() {
4347 return generateColumnDescriptors("");
4348 }
4349
4350
4351
4352
4353
4354
4355
4356 public static List<HColumnDescriptor> generateColumnDescriptors(final String prefix) {
4357 List<HColumnDescriptor> htds = new ArrayList<HColumnDescriptor>();
4358 long familyId = 0;
4359 for (Compression.Algorithm compressionType: getSupportedCompressionAlgorithms()) {
4360 for (DataBlockEncoding encodingType: DataBlockEncoding.values()) {
4361 for (BloomType bloomType: BloomType.values()) {
4362 String name = String.format("%s-cf-!@#&-%d!@#", prefix, familyId);
4363 HColumnDescriptor htd = new HColumnDescriptor(name);
4364 htd.setCompressionType(compressionType);
4365 htd.setDataBlockEncoding(encodingType);
4366 htd.setBloomFilterType(bloomType);
4367 htds.add(htd);
4368 familyId++;
4369 }
4370 }
4371 }
4372 return htds;
4373 }
4374
4375
4376
4377
4378
4379 public static Compression.Algorithm[] getSupportedCompressionAlgorithms() {
4380 String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
4381 List<Compression.Algorithm> supportedAlgos = new ArrayList<Compression.Algorithm>();
4382 for (String algoName : allAlgos) {
4383 try {
4384 Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
4385 algo.getCompressor();
4386 supportedAlgos.add(algo);
4387 } catch (Throwable t) {
4388
4389 }
4390 }
4391 return supportedAlgos.toArray(new Algorithm[supportedAlgos.size()]);
4392 }
4393
4394
4395
4396
4397
4398
4399
4400 public MiniKdc setupMiniKdc(File keytabFile) throws Exception {
4401 Properties conf = MiniKdc.createConf();
4402 conf.put(MiniKdc.DEBUG, true);
4403 MiniKdc kdc = null;
4404 File dir = null;
4405
4406
4407 boolean bindException;
4408 int numTries = 0;
4409 do {
4410 try {
4411 bindException = false;
4412 dir = new File(getDataTestDir("kdc").toUri().getPath());
4413 kdc = new MiniKdc(conf, dir);
4414 kdc.start();
4415 } catch (BindException e) {
4416 FileUtils.deleteDirectory(dir);
4417 numTries++;
4418 if (numTries == 3) {
4419 LOG.error("Failed setting up MiniKDC. Tried " + numTries + " times.");
4420 throw e;
4421 }
4422 LOG.error("BindException encountered when setting up MiniKdc. Trying again.");
4423 bindException = true;
4424 }
4425 } while (bindException);
4426 HBaseKerberosUtils.setKeytabFileForTesting(keytabFile.getAbsolutePath());
4427 return kdc;
4428 }
4429
4430 public int getNumHFiles(final TableName tableName, final byte[] family) {
4431 int numHFiles = 0;
4432 for (RegionServerThread regionServerThread : getMiniHBaseCluster().getRegionServerThreads()) {
4433 numHFiles+= getNumHFilesForRS(regionServerThread.getRegionServer(), tableName,
4434 family);
4435 }
4436 return numHFiles;
4437 }
4438
4439 public int getNumHFilesForRS(final HRegionServer rs, final TableName tableName,
4440 final byte[] family) {
4441 int numHFiles = 0;
4442 for (Region region : rs.getOnlineRegions(tableName)) {
4443 numHFiles += region.getStore(family).getStorefilesCount();
4444 }
4445 return numHFiles;
4446 }
4447 }