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