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