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