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.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 public static class FileInfo implements SortedMap<byte[], byte[]> {
524 static final String RESERVED_PREFIX = "hfile.";
525 static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
526 static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
527 static final byte [] AVG_KEY_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
528 static final byte [] AVG_VALUE_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
529 static final byte [] COMPARATOR = Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
530 static final byte [] TAGS_COMPRESSED = Bytes.toBytes(RESERVED_PREFIX + "TAGS_COMPRESSED");
531 public static final byte [] MAX_TAGS_LEN = Bytes.toBytes(RESERVED_PREFIX + "MAX_TAGS_LEN");
532 private final SortedMap<byte [], byte []> map = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
533
534 public FileInfo() {
535 super();
536 }
537
538
539
540
541
542
543
544
545
546
547
548
549 public FileInfo append(final byte[] k, final byte[] v,
550 final boolean checkPrefix) throws IOException {
551 if (k == null || v == null) {
552 throw new NullPointerException("Key nor value may be null");
553 }
554 if (checkPrefix && isReservedFileInfoKey(k)) {
555 throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
556 + " are reserved");
557 }
558 put(k, v);
559 return this;
560 }
561
562 public void clear() {
563 this.map.clear();
564 }
565
566 public Comparator<? super byte[]> comparator() {
567 return map.comparator();
568 }
569
570 public boolean containsKey(Object key) {
571 return map.containsKey(key);
572 }
573
574 public boolean containsValue(Object value) {
575 return map.containsValue(value);
576 }
577
578 public Set<java.util.Map.Entry<byte[], byte[]>> entrySet() {
579 return map.entrySet();
580 }
581
582 public boolean equals(Object o) {
583 return map.equals(o);
584 }
585
586 public byte[] firstKey() {
587 return map.firstKey();
588 }
589
590 public byte[] get(Object key) {
591 return map.get(key);
592 }
593
594 public int hashCode() {
595 return map.hashCode();
596 }
597
598 public SortedMap<byte[], byte[]> headMap(byte[] toKey) {
599 return this.map.headMap(toKey);
600 }
601
602 public boolean isEmpty() {
603 return map.isEmpty();
604 }
605
606 public Set<byte[]> keySet() {
607 return map.keySet();
608 }
609
610 public byte[] lastKey() {
611 return map.lastKey();
612 }
613
614 public byte[] put(byte[] key, byte[] value) {
615 return this.map.put(key, value);
616 }
617
618 public void putAll(Map<? extends byte[], ? extends byte[]> m) {
619 this.map.putAll(m);
620 }
621
622 public byte[] remove(Object key) {
623 return this.map.remove(key);
624 }
625
626 public int size() {
627 return map.size();
628 }
629
630 public SortedMap<byte[], byte[]> subMap(byte[] fromKey, byte[] toKey) {
631 return this.map.subMap(fromKey, toKey);
632 }
633
634 public SortedMap<byte[], byte[]> tailMap(byte[] fromKey) {
635 return this.map.tailMap(fromKey);
636 }
637
638 public Collection<byte[]> values() {
639 return map.values();
640 }
641
642
643
644
645
646
647
648
649 void write(final DataOutputStream out) throws IOException {
650 HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
651 for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
652 HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
653 bbpBuilder.setFirst(ByteStringer.wrap(e.getKey()));
654 bbpBuilder.setSecond(ByteStringer.wrap(e.getValue()));
655 builder.addMapEntry(bbpBuilder.build());
656 }
657 out.write(ProtobufUtil.PB_MAGIC);
658 builder.build().writeDelimitedTo(out);
659 }
660
661
662
663
664
665
666
667
668 void read(final DataInputStream in) throws IOException {
669
670 int pblen = ProtobufUtil.lengthOfPBMagic();
671 byte [] pbuf = new byte[pblen];
672 if (in.markSupported()) in.mark(pblen);
673 int read = in.read(pbuf);
674 if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
675 if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
676 parsePB(HFileProtos.FileInfoProto.parseDelimitedFrom(in));
677 } else {
678 if (in.markSupported()) {
679 in.reset();
680 parseWritable(in);
681 } else {
682
683 ByteArrayInputStream bais = new ByteArrayInputStream(pbuf);
684 SequenceInputStream sis = new SequenceInputStream(bais, in);
685
686
687
688 parseWritable(new DataInputStream(sis));
689 }
690 }
691 }
692
693
694
695
696
697
698 void parseWritable(final DataInputStream in) throws IOException {
699
700 this.map.clear();
701
702 int entries = in.readInt();
703
704 for (int i = 0; i < entries; i++) {
705 byte [] key = Bytes.readByteArray(in);
706
707 in.readByte();
708 byte [] value = Bytes.readByteArray(in);
709 this.map.put(key, value);
710 }
711 }
712
713
714
715
716
717 void parsePB(final HFileProtos.FileInfoProto fip) {
718 this.map.clear();
719 for (BytesBytesPair pair: fip.getMapEntryList()) {
720 this.map.put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
721 }
722 }
723 }
724
725
726 public static boolean isReservedFileInfoKey(byte[] key) {
727 return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
728 }
729
730
731
732
733
734
735
736
737
738
739
740
741
742 public static String[] getSupportedCompressionAlgorithms() {
743 return Compression.getSupportedAlgorithms();
744 }
745
746
747
748
749
750
751 static int longToInt(final long l) {
752
753
754 return (int)(l & 0x00000000ffffffffL);
755 }
756
757
758
759
760
761
762
763
764
765
766 static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
767 throws IOException {
768 List<Path> res = new ArrayList<Path>();
769 PathFilter dirFilter = new FSUtils.DirFilter(fs);
770 FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
771 for(FileStatus dir : familyDirs) {
772 FileStatus[] files = fs.listStatus(dir.getPath());
773 for (FileStatus file : files) {
774 if (!file.isDir()) {
775 res.add(file.getPath());
776 }
777 }
778 }
779 return res;
780 }
781
782 public static void main(String[] args) throws IOException {
783 HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter();
784 System.exit(prettyPrinter.run(args));
785 }
786
787
788
789
790
791
792
793
794
795
796 public static void checkFormatVersion(int version)
797 throws IllegalArgumentException {
798 if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
799 throw new IllegalArgumentException("Invalid HFile version: " + version
800 + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
801 + MAX_FORMAT_VERSION + ")");
802 }
803 }
804 }