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