1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.io.hfile;
20
21 import java.io.ByteArrayInputStream;
22 import java.io.Closeable;
23 import java.io.DataInput;
24 import java.io.DataInputStream;
25 import java.io.DataOutputStream;
26 import java.io.IOException;
27 import java.io.SequenceInputStream;
28 import java.nio.ByteBuffer;
29 import java.util.ArrayList;
30 import java.util.Collection;
31 import java.util.Comparator;
32 import java.util.List;
33 import java.util.Map;
34 import java.util.Set;
35 import java.util.SortedMap;
36 import java.util.TreeMap;
37 import java.util.concurrent.ArrayBlockingQueue;
38 import java.util.concurrent.BlockingQueue;
39 import java.util.concurrent.atomic.AtomicInteger;
40 import java.util.concurrent.atomic.AtomicLong;
41
42 import org.apache.commons.logging.Log;
43 import org.apache.commons.logging.LogFactory;
44 import org.apache.hadoop.classification.InterfaceAudience;
45 import org.apache.hadoop.conf.Configuration;
46 import org.apache.hadoop.fs.FSDataInputStream;
47 import org.apache.hadoop.fs.FSDataOutputStream;
48 import org.apache.hadoop.fs.FileStatus;
49 import org.apache.hadoop.fs.FileSystem;
50 import org.apache.hadoop.fs.Path;
51 import org.apache.hadoop.fs.PathFilter;
52 import org.apache.hadoop.hbase.HColumnDescriptor;
53 import org.apache.hadoop.hbase.HConstants;
54 import org.apache.hadoop.hbase.KeyValue;
55 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
56 import org.apache.hadoop.hbase.exceptions.CorruptHFileException;
57 import org.apache.hadoop.hbase.fs.HFileSystem;
58 import org.apache.hadoop.hbase.io.compress.Compression;
59 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
60 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
61 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
62 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
63 import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
64 import org.apache.hadoop.hbase.regionserver.StoreFile.WriterBuilder;
65 import org.apache.hadoop.hbase.util.BloomFilterWriter;
66 import org.apache.hadoop.hbase.util.Bytes;
67 import org.apache.hadoop.hbase.util.ChecksumType;
68 import org.apache.hadoop.hbase.util.FSUtils;
69 import org.apache.hadoop.io.RawComparator;
70 import org.apache.hadoop.io.Writable;
71
72 import com.google.common.base.Preconditions;
73 import com.google.common.collect.Lists;
74 import com.google.protobuf.ByteString;
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139 @InterfaceAudience.Private
140 public class HFile {
141 static final Log LOG = LogFactory.getLog(HFile.class);
142
143
144
145
146 public final static int MAXIMUM_KEY_LENGTH = Integer.MAX_VALUE;
147
148
149
150
151 public final static Compression.Algorithm DEFAULT_COMPRESSION_ALGORITHM =
152 Compression.Algorithm.NONE;
153
154
155 public static final int MIN_FORMAT_VERSION = 1;
156
157
158
159 public static final int MAX_FORMAT_VERSION = 2;
160
161
162 public final static String DEFAULT_COMPRESSION =
163 DEFAULT_COMPRESSION_ALGORITHM.getName();
164
165
166 public static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
167
168
169
170
171
172
173
174 public final static int MIN_NUM_HFILE_PATH_LEVELS = 5;
175
176
177
178
179 public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
180 public static final ChecksumType DEFAULT_CHECKSUM_TYPE = ChecksumType.CRC32;
181
182
183 private static final AtomicInteger readOps = new AtomicInteger();
184 private static final AtomicLong readTimeNano = new AtomicLong();
185 private static final AtomicInteger writeOps = new AtomicInteger();
186 private static final AtomicLong writeTimeNano = new AtomicLong();
187
188
189 private static final AtomicInteger preadOps = new AtomicInteger();
190 private static final AtomicLong preadTimeNano = new AtomicLong();
191
192
193 static final AtomicLong checksumFailures = new AtomicLong();
194
195
196
197
198
199
200 private static final int LATENCY_BUFFER_SIZE = 5000;
201 private static final BlockingQueue<Long> fsReadLatenciesNanos =
202 new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
203 private static final BlockingQueue<Long> fsWriteLatenciesNanos =
204 new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
205 private static final BlockingQueue<Long> fsPreadLatenciesNanos =
206 new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
207
208 public static final void offerReadLatency(long latencyNanos, boolean pread) {
209 if (pread) {
210 fsPreadLatenciesNanos.offer(latencyNanos);
211 preadOps.incrementAndGet();
212 preadTimeNano.addAndGet(latencyNanos);
213 } else {
214 fsReadLatenciesNanos.offer(latencyNanos);
215 readTimeNano.addAndGet(latencyNanos);
216 readOps.incrementAndGet();
217 }
218 }
219
220 public static final void offerWriteLatency(long latencyNanos) {
221 fsWriteLatenciesNanos.offer(latencyNanos);
222
223 writeTimeNano.addAndGet(latencyNanos);
224 writeOps.incrementAndGet();
225 }
226
227 public static final Collection<Long> getReadLatenciesNanos() {
228 final List<Long> latencies =
229 Lists.newArrayListWithCapacity(fsReadLatenciesNanos.size());
230 fsReadLatenciesNanos.drainTo(latencies);
231 return latencies;
232 }
233
234 public static final Collection<Long> getPreadLatenciesNanos() {
235 final List<Long> latencies =
236 Lists.newArrayListWithCapacity(fsPreadLatenciesNanos.size());
237 fsPreadLatenciesNanos.drainTo(latencies);
238 return latencies;
239 }
240
241 public static final Collection<Long> getWriteLatenciesNanos() {
242 final List<Long> latencies =
243 Lists.newArrayListWithCapacity(fsWriteLatenciesNanos.size());
244 fsWriteLatenciesNanos.drainTo(latencies);
245 return latencies;
246 }
247
248
249 public static volatile AtomicLong dataBlockReadCnt = new AtomicLong(0);
250
251
252 public static final int getReadOps() {
253 return readOps.getAndSet(0);
254 }
255
256 public static final long getReadTimeMs() {
257 return readTimeNano.getAndSet(0) / 1000000;
258 }
259
260
261 public static final int getPreadOps() {
262 return preadOps.getAndSet(0);
263 }
264
265 public static final long getPreadTimeMs() {
266 return preadTimeNano.getAndSet(0) / 1000000;
267 }
268
269 public static final int getWriteOps() {
270 return writeOps.getAndSet(0);
271 }
272
273 public static final long getWriteTimeMs() {
274 return writeTimeNano.getAndSet(0) / 1000000;
275 }
276
277
278
279
280
281 public static final long getChecksumFailuresCount() {
282 return checksumFailures.getAndSet(0);
283 }
284
285
286 public interface Writer extends Closeable {
287
288
289 void appendFileInfo(byte[] key, byte[] value) throws IOException;
290
291 void append(KeyValue kv) throws IOException;
292
293 void append(byte[] key, byte[] value) throws IOException;
294
295
296 Path getPath();
297
298
299
300
301
302 void addInlineBlockWriter(InlineBlockWriter bloomWriter);
303
304
305
306
307
308
309 void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
310
311
312
313
314
315
316 void addGeneralBloomFilter(BloomFilterWriter bfw);
317
318
319
320
321
322 void addDeleteFamilyBloomFilter(BloomFilterWriter bfw) throws IOException;
323 }
324
325
326
327
328
329 public static abstract class WriterFactory {
330 protected final Configuration conf;
331 protected final CacheConfig cacheConf;
332 protected FileSystem fs;
333 protected Path path;
334 protected FSDataOutputStream ostream;
335 protected int blockSize = HColumnDescriptor.DEFAULT_BLOCKSIZE;
336 protected Compression.Algorithm compression =
337 HFile.DEFAULT_COMPRESSION_ALGORITHM;
338 protected HFileDataBlockEncoder encoder = NoOpDataBlockEncoder.INSTANCE;
339 protected KeyComparator comparator;
340 protected ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
341 protected int bytesPerChecksum = DEFAULT_BYTES_PER_CHECKSUM;
342 protected boolean includeMVCCReadpoint = true;
343
344 WriterFactory(Configuration conf, CacheConfig cacheConf) {
345 this.conf = conf;
346 this.cacheConf = cacheConf;
347 }
348
349 public WriterFactory withPath(FileSystem fs, Path path) {
350 Preconditions.checkNotNull(fs);
351 Preconditions.checkNotNull(path);
352 this.fs = fs;
353 this.path = path;
354 return this;
355 }
356
357 public WriterFactory withOutputStream(FSDataOutputStream ostream) {
358 Preconditions.checkNotNull(ostream);
359 this.ostream = ostream;
360 return this;
361 }
362
363 public WriterFactory withBlockSize(int blockSize) {
364 this.blockSize = blockSize;
365 return this;
366 }
367
368 public WriterFactory withCompression(Compression.Algorithm compression) {
369 Preconditions.checkNotNull(compression);
370 this.compression = compression;
371 return this;
372 }
373
374 public WriterFactory withCompression(String compressAlgo) {
375 Preconditions.checkNotNull(compression);
376 this.compression = AbstractHFileWriter.compressionByName(compressAlgo);
377 return this;
378 }
379
380 public WriterFactory withDataBlockEncoder(HFileDataBlockEncoder encoder) {
381 Preconditions.checkNotNull(encoder);
382 this.encoder = encoder;
383 return this;
384 }
385
386 public WriterFactory withComparator(KeyComparator comparator) {
387 Preconditions.checkNotNull(comparator);
388 this.comparator = comparator;
389 return this;
390 }
391
392 public WriterFactory withChecksumType(ChecksumType checksumType) {
393 Preconditions.checkNotNull(checksumType);
394 this.checksumType = checksumType;
395 return this;
396 }
397
398 public WriterFactory withBytesPerChecksum(int bytesPerChecksum) {
399 this.bytesPerChecksum = bytesPerChecksum;
400 return this;
401 }
402
403
404
405
406
407 public WriterFactory includeMVCCReadpoint(boolean includeMVCCReadpoint) {
408 this.includeMVCCReadpoint = includeMVCCReadpoint;
409 return this;
410 }
411
412 public Writer create() throws IOException {
413 if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) {
414 throw new AssertionError("Please specify exactly one of " +
415 "filesystem/path or path");
416 }
417 if (path != null) {
418 ostream = AbstractHFileWriter.createOutputStream(conf, fs, path);
419 }
420 return createWriter(fs, path, ostream, blockSize,
421 compression, encoder, comparator, checksumType, bytesPerChecksum, includeMVCCReadpoint);
422 }
423
424 protected abstract Writer createWriter(FileSystem fs, Path path,
425 FSDataOutputStream ostream, int blockSize,
426 Compression.Algorithm compress,
427 HFileDataBlockEncoder dataBlockEncoder,
428 KeyComparator comparator, ChecksumType checksumType,
429 int bytesPerChecksum, boolean includeMVCCReadpoint) throws IOException;
430 }
431
432
433 public static final String FORMAT_VERSION_KEY = "hfile.format.version";
434
435 public static int getFormatVersion(Configuration conf) {
436 int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION);
437 checkFormatVersion(version);
438 return version;
439 }
440
441
442
443
444
445
446 public static final WriterFactory getWriterFactoryNoCache(Configuration
447 conf) {
448 Configuration tempConf = new Configuration(conf);
449 tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
450 return HFile.getWriterFactory(conf, new CacheConfig(tempConf));
451 }
452
453
454
455
456 public static final WriterFactory getWriterFactory(Configuration conf,
457 CacheConfig cacheConf) {
458 int version = getFormatVersion(conf);
459 switch (version) {
460 case 2:
461 return new HFileWriterV2.WriterFactoryV2(conf, cacheConf);
462 default:
463 throw new IllegalArgumentException("Cannot create writer for HFile " +
464 "format version " + version);
465 }
466 }
467
468
469 public interface CachingBlockReader {
470 HFileBlock readBlock(long offset, long onDiskBlockSize,
471 boolean cacheBlock, final boolean pread, final boolean isCompaction,
472 BlockType expectedBlockType)
473 throws IOException;
474 }
475
476
477 public interface Reader extends Closeable, CachingBlockReader {
478
479
480
481
482
483 String getName();
484
485 RawComparator<byte []> getComparator();
486
487 HFileScanner getScanner(boolean cacheBlocks,
488 final boolean pread, final boolean isCompaction);
489
490 ByteBuffer getMetaBlock(String metaBlockName,
491 boolean cacheBlock) throws IOException;
492
493 Map<byte[], byte[]> loadFileInfo() throws IOException;
494
495 byte[] getLastKey();
496
497 byte[] midkey() throws IOException;
498
499 long length();
500
501 long getEntries();
502
503 byte[] getFirstKey();
504
505 long indexSize();
506
507 byte[] getFirstRowKey();
508
509 byte[] getLastRowKey();
510
511 FixedFileTrailer getTrailer();
512
513 HFileBlockIndex.BlockIndexReader getDataBlockIndexReader();
514
515 HFileScanner getScanner(boolean cacheBlocks, boolean pread);
516
517 Compression.Algorithm getCompressionAlgorithm();
518
519
520
521
522
523
524 DataInput getGeneralBloomFilterMetadata() throws IOException;
525
526
527
528
529
530
531 DataInput getDeleteBloomFilterMetadata() throws IOException;
532
533 Path getPath();
534
535
536 void close(boolean evictOnClose) throws IOException;
537
538 DataBlockEncoding getEncodingOnDisk();
539 }
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556 private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis,
557 FSDataInputStream fsdisNoFsChecksum,
558 long size, boolean closeIStream, CacheConfig cacheConf,
559 DataBlockEncoding preferredEncodingInCache, HFileSystem hfs)
560 throws IOException {
561 FixedFileTrailer trailer = null;
562 try {
563 trailer = FixedFileTrailer.readFromStream(fsdis, size);
564 } catch (IllegalArgumentException iae) {
565 throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, iae);
566 }
567 switch (trailer.getMajorVersion()) {
568 case 2:
569 return new HFileReaderV2(path, trailer, fsdis, fsdisNoFsChecksum,
570 size, closeIStream,
571 cacheConf, preferredEncodingInCache, hfs);
572 default:
573 throw new CorruptHFileException("Invalid HFile version " + trailer.getMajorVersion());
574 }
575 }
576
577
578
579
580
581
582
583
584
585 public static Reader createReaderWithEncoding(
586 FileSystem fs, Path path, CacheConfig cacheConf,
587 DataBlockEncoding preferredEncodingInCache) throws IOException {
588 final boolean closeIStream = true;
589 HFileSystem hfs = null;
590 FSDataInputStream fsdis = fs.open(path);
591 FSDataInputStream fsdisNoFsChecksum = fsdis;
592
593
594
595
596 if (!(fs instanceof HFileSystem)) {
597 hfs = new HFileSystem(fs);
598 } else {
599 hfs = (HFileSystem)fs;
600
601
602 fsdisNoFsChecksum = hfs.getNoChecksumFs().open(path);
603 }
604 return pickReaderVersion(path, fsdis, fsdisNoFsChecksum,
605 fs.getFileStatus(path).getLen(), closeIStream, cacheConf,
606 preferredEncodingInCache, hfs);
607 }
608
609
610
611
612
613
614
615
616
617
618
619
620
621 public static Reader createReaderWithEncoding(
622 FileSystem fs, Path path, FSDataInputStream fsdis,
623 FSDataInputStream fsdisNoFsChecksum, long size, CacheConfig cacheConf,
624 DataBlockEncoding preferredEncodingInCache, boolean closeIStream)
625 throws IOException {
626 HFileSystem hfs = null;
627
628
629
630
631
632 if (!(fs instanceof HFileSystem)) {
633 hfs = new HFileSystem(fs);
634 } else {
635 hfs = (HFileSystem)fs;
636 }
637 return pickReaderVersion(path, fsdis, fsdisNoFsChecksum, size,
638 closeIStream, cacheConf,
639 preferredEncodingInCache, hfs);
640 }
641
642
643
644
645
646
647
648
649
650 public static Reader createReader(
651 FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
652 Preconditions.checkNotNull(cacheConf, "Cannot create Reader with null CacheConf");
653 return createReaderWithEncoding(fs, path, cacheConf,
654 DataBlockEncoding.NONE);
655 }
656
657
658
659
660 static Reader createReaderFromStream(Path path,
661 FSDataInputStream fsdis, long size, CacheConfig cacheConf)
662 throws IOException {
663 final boolean closeIStream = false;
664 return pickReaderVersion(path, fsdis, fsdis, size, closeIStream, cacheConf,
665 DataBlockEncoding.NONE, null);
666 }
667
668
669
670
671 static class FileInfo implements SortedMap<byte [], byte []> {
672 static final String RESERVED_PREFIX = "hfile.";
673 static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
674 static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
675 static final byte [] AVG_KEY_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
676 static final byte [] AVG_VALUE_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
677 static final byte [] COMPARATOR = Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
678 private final SortedMap<byte [], byte []> map = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
679
680 public FileInfo() {
681 super();
682 }
683
684
685
686
687
688
689
690
691
692
693
694
695 public FileInfo append(final byte[] k, final byte[] v,
696 final boolean checkPrefix) throws IOException {
697 if (k == null || v == null) {
698 throw new NullPointerException("Key nor value may be null");
699 }
700 if (checkPrefix && isReservedFileInfoKey(k)) {
701 throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
702 + " are reserved");
703 }
704 put(k, v);
705 return this;
706 }
707
708 public void clear() {
709 this.map.clear();
710 }
711
712 public Comparator<? super byte[]> comparator() {
713 return map.comparator();
714 }
715
716 public boolean containsKey(Object key) {
717 return map.containsKey(key);
718 }
719
720 public boolean containsValue(Object value) {
721 return map.containsValue(value);
722 }
723
724 public Set<java.util.Map.Entry<byte[], byte[]>> entrySet() {
725 return map.entrySet();
726 }
727
728 public boolean equals(Object o) {
729 return map.equals(o);
730 }
731
732 public byte[] firstKey() {
733 return map.firstKey();
734 }
735
736 public byte[] get(Object key) {
737 return map.get(key);
738 }
739
740 public int hashCode() {
741 return map.hashCode();
742 }
743
744 public SortedMap<byte[], byte[]> headMap(byte[] toKey) {
745 return this.map.headMap(toKey);
746 }
747
748 public boolean isEmpty() {
749 return map.isEmpty();
750 }
751
752 public Set<byte[]> keySet() {
753 return map.keySet();
754 }
755
756 public byte[] lastKey() {
757 return map.lastKey();
758 }
759
760 public byte[] put(byte[] key, byte[] value) {
761 return this.map.put(key, value);
762 }
763
764 public void putAll(Map<? extends byte[], ? extends byte[]> m) {
765 this.map.putAll(m);
766 }
767
768 public byte[] remove(Object key) {
769 return this.map.remove(key);
770 }
771
772 public int size() {
773 return map.size();
774 }
775
776 public SortedMap<byte[], byte[]> subMap(byte[] fromKey, byte[] toKey) {
777 return this.map.subMap(fromKey, toKey);
778 }
779
780 public SortedMap<byte[], byte[]> tailMap(byte[] fromKey) {
781 return this.map.tailMap(fromKey);
782 }
783
784 public Collection<byte[]> values() {
785 return map.values();
786 }
787
788
789
790
791
792
793
794
795 void write(final DataOutputStream out) throws IOException {
796 HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
797 for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
798 HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
799 bbpBuilder.setFirst(ByteString.copyFrom(e.getKey()));
800 bbpBuilder.setSecond(ByteString.copyFrom(e.getValue()));
801 builder.addMapEntry(bbpBuilder.build());
802 }
803 out.write(ProtobufUtil.PB_MAGIC);
804 builder.build().writeDelimitedTo(out);
805 }
806
807
808
809
810
811
812
813
814 void read(final DataInputStream in) throws IOException {
815
816 int pblen = ProtobufUtil.lengthOfPBMagic();
817 byte [] pbuf = new byte[pblen];
818 if (in.markSupported()) in.mark(pblen);
819 int read = in.read(pbuf);
820 if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
821 if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
822 parsePB(HFileProtos.FileInfoProto.parseDelimitedFrom(in));
823 } else {
824 if (in.markSupported()) {
825 in.reset();
826 parseWritable(in);
827 } else {
828
829 ByteArrayInputStream bais = new ByteArrayInputStream(pbuf);
830 SequenceInputStream sis = new SequenceInputStream(bais, in);
831
832
833
834 parseWritable(new DataInputStream(sis));
835 }
836 }
837 }
838
839
840
841
842
843
844 void parseWritable(final DataInputStream in) throws IOException {
845
846 this.map.clear();
847
848 int entries = in.readInt();
849
850 for (int i = 0; i < entries; i++) {
851 byte [] key = Bytes.readByteArray(in);
852
853 in.readByte();
854 byte [] value = Bytes.readByteArray(in);
855 this.map.put(key, value);
856 }
857 }
858
859
860
861
862
863 void parsePB(final HFileProtos.FileInfoProto fip) {
864 this.map.clear();
865 for (BytesBytesPair pair: fip.getMapEntryList()) {
866 this.map.put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
867 }
868 }
869 }
870
871
872 public static boolean isReservedFileInfoKey(byte[] key) {
873 return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
874 }
875
876
877
878
879
880
881
882
883
884
885
886
887
888 public static String[] getSupportedCompressionAlgorithms() {
889 return Compression.getSupportedAlgorithms();
890 }
891
892
893
894
895
896
897 static int longToInt(final long l) {
898
899
900 return (int)(l & 0x00000000ffffffffL);
901 }
902
903
904
905
906
907
908
909
910
911
912 static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
913 throws IOException {
914 List<Path> res = new ArrayList<Path>();
915 PathFilter dirFilter = new FSUtils.DirFilter(fs);
916 FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
917 for(FileStatus dir : familyDirs) {
918 FileStatus[] files = fs.listStatus(dir.getPath());
919 for (FileStatus file : files) {
920 if (!file.isDir()) {
921 res.add(file.getPath());
922 }
923 }
924 }
925 return res;
926 }
927
928 public static void main(String[] args) throws IOException {
929 HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter();
930 System.exit(prettyPrinter.run(args));
931 }
932
933
934
935
936
937
938
939
940
941
942 public static void checkFormatVersion(int version)
943 throws IllegalArgumentException {
944 if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
945 throw new IllegalArgumentException("Invalid HFile version: " + version
946 + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
947 + MAX_FORMAT_VERSION + ")");
948 }
949 }
950 }