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.net.InetSocketAddress;
29 import java.nio.ByteBuffer;
30 import java.util.ArrayList;
31 import java.util.Collection;
32 import java.util.Comparator;
33 import java.util.List;
34 import java.util.Map;
35 import java.util.Set;
36 import java.util.SortedMap;
37 import java.util.TreeMap;
38 import java.util.concurrent.ArrayBlockingQueue;
39 import java.util.concurrent.BlockingQueue;
40 import java.util.concurrent.atomic.AtomicInteger;
41 import java.util.concurrent.atomic.AtomicLong;
42
43 import org.apache.commons.logging.Log;
44 import org.apache.commons.logging.LogFactory;
45 import org.apache.hadoop.classification.InterfaceAudience;
46 import org.apache.hadoop.conf.Configuration;
47 import org.apache.hadoop.fs.FSDataInputStream;
48 import org.apache.hadoop.fs.FSDataOutputStream;
49 import org.apache.hadoop.fs.FileStatus;
50 import org.apache.hadoop.fs.FileSystem;
51 import org.apache.hadoop.fs.Path;
52 import org.apache.hadoop.fs.PathFilter;
53 import org.apache.hadoop.hbase.HColumnDescriptor;
54 import org.apache.hadoop.hbase.HConstants;
55 import org.apache.hadoop.hbase.KeyValue;
56 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
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.io.FSDataInputStreamWrapper;
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 = 2;
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 InetSocketAddress[] favoredNodes;
341 protected ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
342 protected int bytesPerChecksum = DEFAULT_BYTES_PER_CHECKSUM;
343 protected boolean includeMVCCReadpoint = true;
344
345 WriterFactory(Configuration conf, CacheConfig cacheConf) {
346 this.conf = conf;
347 this.cacheConf = cacheConf;
348 }
349
350 public WriterFactory withPath(FileSystem fs, Path path) {
351 Preconditions.checkNotNull(fs);
352 Preconditions.checkNotNull(path);
353 this.fs = fs;
354 this.path = path;
355 return this;
356 }
357
358 public WriterFactory withOutputStream(FSDataOutputStream ostream) {
359 Preconditions.checkNotNull(ostream);
360 this.ostream = ostream;
361 return this;
362 }
363
364 public WriterFactory withBlockSize(int blockSize) {
365 this.blockSize = blockSize;
366 return this;
367 }
368
369 public WriterFactory withCompression(Compression.Algorithm compression) {
370 Preconditions.checkNotNull(compression);
371 this.compression = compression;
372 return this;
373 }
374
375 public WriterFactory withCompression(String compressAlgo) {
376 Preconditions.checkNotNull(compression);
377 this.compression = AbstractHFileWriter.compressionByName(compressAlgo);
378 return this;
379 }
380
381 public WriterFactory withDataBlockEncoder(HFileDataBlockEncoder encoder) {
382 Preconditions.checkNotNull(encoder);
383 this.encoder = encoder;
384 return this;
385 }
386
387 public WriterFactory withComparator(KeyComparator comparator) {
388 Preconditions.checkNotNull(comparator);
389 this.comparator = comparator;
390 return this;
391 }
392
393 public WriterFactory withFavoredNodes(InetSocketAddress[] favoredNodes) {
394
395 this.favoredNodes = favoredNodes;
396 return this;
397 }
398
399 public WriterFactory withChecksumType(ChecksumType checksumType) {
400 Preconditions.checkNotNull(checksumType);
401 this.checksumType = checksumType;
402 return this;
403 }
404
405 public WriterFactory withBytesPerChecksum(int bytesPerChecksum) {
406 this.bytesPerChecksum = bytesPerChecksum;
407 return this;
408 }
409
410
411
412
413
414 public WriterFactory includeMVCCReadpoint(boolean includeMVCCReadpoint) {
415 this.includeMVCCReadpoint = includeMVCCReadpoint;
416 return this;
417 }
418
419 public Writer create() throws IOException {
420 if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) {
421 throw new AssertionError("Please specify exactly one of " +
422 "filesystem/path or path");
423 }
424 if (path != null) {
425 ostream = AbstractHFileWriter.createOutputStream(conf, fs, path, favoredNodes);
426 }
427 return createWriter(fs, path, ostream, blockSize,
428 compression, encoder, comparator, checksumType, bytesPerChecksum, includeMVCCReadpoint);
429 }
430
431 protected abstract Writer createWriter(FileSystem fs, Path path,
432 FSDataOutputStream ostream, int blockSize,
433 Compression.Algorithm compress,
434 HFileDataBlockEncoder dataBlockEncoder,
435 KeyComparator comparator, ChecksumType checksumType,
436 int bytesPerChecksum, boolean includeMVCCReadpoint) throws IOException;
437 }
438
439
440 public static final String FORMAT_VERSION_KEY = "hfile.format.version";
441
442 public static int getFormatVersion(Configuration conf) {
443 int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION);
444 checkFormatVersion(version);
445 return version;
446 }
447
448
449
450
451
452
453 public static final WriterFactory getWriterFactoryNoCache(Configuration
454 conf) {
455 Configuration tempConf = new Configuration(conf);
456 tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
457 return HFile.getWriterFactory(conf, new CacheConfig(tempConf));
458 }
459
460
461
462
463 public static final WriterFactory getWriterFactory(Configuration conf,
464 CacheConfig cacheConf) {
465 int version = getFormatVersion(conf);
466 switch (version) {
467 case 2:
468 return new HFileWriterV2.WriterFactoryV2(conf, cacheConf);
469 default:
470 throw new IllegalArgumentException("Cannot create writer for HFile " +
471 "format version " + version);
472 }
473 }
474
475
476 public interface CachingBlockReader {
477 HFileBlock readBlock(long offset, long onDiskBlockSize,
478 boolean cacheBlock, final boolean pread, final boolean isCompaction,
479 BlockType expectedBlockType)
480 throws IOException;
481 }
482
483
484 public interface Reader extends Closeable, CachingBlockReader {
485
486
487
488
489
490 String getName();
491
492 RawComparator<byte []> getComparator();
493
494 HFileScanner getScanner(boolean cacheBlocks,
495 final boolean pread, final boolean isCompaction);
496
497 ByteBuffer getMetaBlock(String metaBlockName,
498 boolean cacheBlock) throws IOException;
499
500 Map<byte[], byte[]> loadFileInfo() throws IOException;
501
502 byte[] getLastKey();
503
504 byte[] midkey() throws IOException;
505
506 long length();
507
508 long getEntries();
509
510 byte[] getFirstKey();
511
512 long indexSize();
513
514 byte[] getFirstRowKey();
515
516 byte[] getLastRowKey();
517
518 FixedFileTrailer getTrailer();
519
520 HFileBlockIndex.BlockIndexReader getDataBlockIndexReader();
521
522 HFileScanner getScanner(boolean cacheBlocks, boolean pread);
523
524 Compression.Algorithm getCompressionAlgorithm();
525
526
527
528
529
530
531 DataInput getGeneralBloomFilterMetadata() throws IOException;
532
533
534
535
536
537
538 DataInput getDeleteBloomFilterMetadata() throws IOException;
539
540 Path getPath();
541
542
543 void close(boolean evictOnClose) throws IOException;
544
545 DataBlockEncoding getEncodingOnDisk();
546 }
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561 private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis,
562 long size, CacheConfig cacheConf, DataBlockEncoding preferredEncodingInCache,
563 HFileSystem hfs) throws IOException {
564 FixedFileTrailer trailer = null;
565 try {
566 boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum();
567 assert !isHBaseChecksum;
568 trailer = FixedFileTrailer.readFromStream(fsdis.getStream(isHBaseChecksum), size);
569 } catch (IllegalArgumentException iae) {
570 throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, iae);
571 }
572 switch (trailer.getMajorVersion()) {
573 case 2:
574 return new HFileReaderV2(
575 path, trailer, fsdis, size, cacheConf, preferredEncodingInCache, hfs);
576 default:
577 throw new CorruptHFileException("Invalid HFile version " + trailer.getMajorVersion());
578 }
579 }
580
581
582
583
584
585
586
587
588
589 public static Reader createReaderWithEncoding(
590 FileSystem fs, Path path, CacheConfig cacheConf,
591 DataBlockEncoding preferredEncodingInCache) throws IOException {
592 final boolean closeIStream = true;
593 FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fs, path);
594 return pickReaderVersion(path, stream, fs.getFileStatus(path).getLen(),
595 cacheConf, preferredEncodingInCache, stream.getHfs());
596 }
597
598
599
600
601
602
603
604
605
606
607
608 public static Reader createReaderWithEncoding(FileSystem fs, Path path,
609 FSDataInputStreamWrapper fsdis, long size, CacheConfig cacheConf,
610 DataBlockEncoding preferredEncodingInCache) throws IOException {
611 HFileSystem hfs = null;
612
613
614
615
616
617 if (!(fs instanceof HFileSystem)) {
618 hfs = new HFileSystem(fs);
619 } else {
620 hfs = (HFileSystem)fs;
621 }
622 return pickReaderVersion(path, fsdis, size, cacheConf, preferredEncodingInCache, hfs);
623 }
624
625
626
627
628
629
630
631
632
633 public static Reader createReader(
634 FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
635 Preconditions.checkNotNull(cacheConf, "Cannot create Reader with null CacheConf");
636 return createReaderWithEncoding(fs, path, cacheConf,
637 DataBlockEncoding.NONE);
638 }
639
640
641
642
643 static Reader createReaderFromStream(Path path,
644 FSDataInputStream fsdis, long size, CacheConfig cacheConf)
645 throws IOException {
646 FSDataInputStreamWrapper wrapper = new FSDataInputStreamWrapper(fsdis);
647 return pickReaderVersion(path, wrapper, size, cacheConf, DataBlockEncoding.NONE, null);
648 }
649
650
651
652
653 static class FileInfo implements SortedMap<byte [], byte []> {
654 static final String RESERVED_PREFIX = "hfile.";
655 static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
656 static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
657 static final byte [] AVG_KEY_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
658 static final byte [] AVG_VALUE_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
659 static final byte [] COMPARATOR = Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
660 private final SortedMap<byte [], byte []> map = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
661
662 public FileInfo() {
663 super();
664 }
665
666
667
668
669
670
671
672
673
674
675
676
677 public FileInfo append(final byte[] k, final byte[] v,
678 final boolean checkPrefix) throws IOException {
679 if (k == null || v == null) {
680 throw new NullPointerException("Key nor value may be null");
681 }
682 if (checkPrefix && isReservedFileInfoKey(k)) {
683 throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
684 + " are reserved");
685 }
686 put(k, v);
687 return this;
688 }
689
690 public void clear() {
691 this.map.clear();
692 }
693
694 public Comparator<? super byte[]> comparator() {
695 return map.comparator();
696 }
697
698 public boolean containsKey(Object key) {
699 return map.containsKey(key);
700 }
701
702 public boolean containsValue(Object value) {
703 return map.containsValue(value);
704 }
705
706 public Set<java.util.Map.Entry<byte[], byte[]>> entrySet() {
707 return map.entrySet();
708 }
709
710 public boolean equals(Object o) {
711 return map.equals(o);
712 }
713
714 public byte[] firstKey() {
715 return map.firstKey();
716 }
717
718 public byte[] get(Object key) {
719 return map.get(key);
720 }
721
722 public int hashCode() {
723 return map.hashCode();
724 }
725
726 public SortedMap<byte[], byte[]> headMap(byte[] toKey) {
727 return this.map.headMap(toKey);
728 }
729
730 public boolean isEmpty() {
731 return map.isEmpty();
732 }
733
734 public Set<byte[]> keySet() {
735 return map.keySet();
736 }
737
738 public byte[] lastKey() {
739 return map.lastKey();
740 }
741
742 public byte[] put(byte[] key, byte[] value) {
743 return this.map.put(key, value);
744 }
745
746 public void putAll(Map<? extends byte[], ? extends byte[]> m) {
747 this.map.putAll(m);
748 }
749
750 public byte[] remove(Object key) {
751 return this.map.remove(key);
752 }
753
754 public int size() {
755 return map.size();
756 }
757
758 public SortedMap<byte[], byte[]> subMap(byte[] fromKey, byte[] toKey) {
759 return this.map.subMap(fromKey, toKey);
760 }
761
762 public SortedMap<byte[], byte[]> tailMap(byte[] fromKey) {
763 return this.map.tailMap(fromKey);
764 }
765
766 public Collection<byte[]> values() {
767 return map.values();
768 }
769
770
771
772
773
774
775
776
777 void write(final DataOutputStream out) throws IOException {
778 HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
779 for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
780 HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
781 bbpBuilder.setFirst(ByteString.copyFrom(e.getKey()));
782 bbpBuilder.setSecond(ByteString.copyFrom(e.getValue()));
783 builder.addMapEntry(bbpBuilder.build());
784 }
785 out.write(ProtobufUtil.PB_MAGIC);
786 builder.build().writeDelimitedTo(out);
787 }
788
789
790
791
792
793
794
795
796 void read(final DataInputStream in) throws IOException {
797
798 int pblen = ProtobufUtil.lengthOfPBMagic();
799 byte [] pbuf = new byte[pblen];
800 if (in.markSupported()) in.mark(pblen);
801 int read = in.read(pbuf);
802 if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
803 if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
804 parsePB(HFileProtos.FileInfoProto.parseDelimitedFrom(in));
805 } else {
806 if (in.markSupported()) {
807 in.reset();
808 parseWritable(in);
809 } else {
810
811 ByteArrayInputStream bais = new ByteArrayInputStream(pbuf);
812 SequenceInputStream sis = new SequenceInputStream(bais, in);
813
814
815
816 parseWritable(new DataInputStream(sis));
817 }
818 }
819 }
820
821
822
823
824
825
826 void parseWritable(final DataInputStream in) throws IOException {
827
828 this.map.clear();
829
830 int entries = in.readInt();
831
832 for (int i = 0; i < entries; i++) {
833 byte [] key = Bytes.readByteArray(in);
834
835 in.readByte();
836 byte [] value = Bytes.readByteArray(in);
837 this.map.put(key, value);
838 }
839 }
840
841
842
843
844
845 void parsePB(final HFileProtos.FileInfoProto fip) {
846 this.map.clear();
847 for (BytesBytesPair pair: fip.getMapEntryList()) {
848 this.map.put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
849 }
850 }
851 }
852
853
854 public static boolean isReservedFileInfoKey(byte[] key) {
855 return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
856 }
857
858
859
860
861
862
863
864
865
866
867
868
869
870 public static String[] getSupportedCompressionAlgorithms() {
871 return Compression.getSupportedAlgorithms();
872 }
873
874
875
876
877
878
879 static int longToInt(final long l) {
880
881
882 return (int)(l & 0x00000000ffffffffL);
883 }
884
885
886
887
888
889
890
891
892
893
894 static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
895 throws IOException {
896 List<Path> res = new ArrayList<Path>();
897 PathFilter dirFilter = new FSUtils.DirFilter(fs);
898 FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
899 for(FileStatus dir : familyDirs) {
900 FileStatus[] files = fs.listStatus(dir.getPath());
901 for (FileStatus file : files) {
902 if (!file.isDir()) {
903 res.add(file.getPath());
904 }
905 }
906 }
907 return res;
908 }
909
910 public static void main(String[] args) throws IOException {
911 HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter();
912 System.exit(prettyPrinter.run(args));
913 }
914
915
916
917
918
919
920
921
922
923
924 public static void checkFormatVersion(int version)
925 throws IllegalArgumentException {
926 if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
927 throw new IllegalArgumentException("Invalid HFile version: " + version
928 + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
929 + MAX_FORMAT_VERSION + ")");
930 }
931 }
932 }