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