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