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