1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.io.hfile;
21
22 import java.io.Closeable;
23 import java.io.DataInput;
24 import java.io.IOException;
25 import java.nio.ByteBuffer;
26 import java.util.ArrayList;
27 import java.util.Collection;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.concurrent.ArrayBlockingQueue;
31 import java.util.concurrent.BlockingQueue;
32 import java.util.concurrent.atomic.AtomicInteger;
33 import java.util.concurrent.atomic.AtomicLong;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.FSDataInputStream;
39 import org.apache.hadoop.fs.FSDataOutputStream;
40 import org.apache.hadoop.fs.FileStatus;
41 import org.apache.hadoop.fs.FileSystem;
42 import org.apache.hadoop.fs.Path;
43 import org.apache.hadoop.fs.PathFilter;
44 import org.apache.hadoop.hbase.HColumnDescriptor;
45 import org.apache.hadoop.hbase.HConstants;
46 import org.apache.hadoop.hbase.KeyValue;
47 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
48 import org.apache.hadoop.hbase.fs.HFileSystem;
49 import org.apache.hadoop.hbase.io.HbaseMapWritable;
50 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
51 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
52 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware;
53 import org.apache.hadoop.hbase.util.BloomFilterWriter;
54 import org.apache.hadoop.hbase.util.Bytes;
55 import org.apache.hadoop.hbase.util.ChecksumType;
56 import org.apache.hadoop.hbase.util.FSUtils;
57 import org.apache.hadoop.io.RawComparator;
58 import org.apache.hadoop.io.Writable;
59
60 import com.google.common.base.Preconditions;
61 import com.google.common.collect.Lists;
62
63
64
65
66
67
68
69
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 public class HFile {
127 static final Log LOG = LogFactory.getLog(HFile.class);
128
129
130
131
132 public final static int MAXIMUM_KEY_LENGTH = Integer.MAX_VALUE;
133
134
135
136
137 public final static int DEFAULT_BLOCKSIZE = 64 * 1024;
138
139
140
141
142 public final static Compression.Algorithm DEFAULT_COMPRESSION_ALGORITHM =
143 Compression.Algorithm.NONE;
144
145
146 public static final int MIN_FORMAT_VERSION = 1;
147
148
149 public static final int MAX_FORMAT_VERSION = 2;
150
151
152 public final static String DEFAULT_COMPRESSION =
153 DEFAULT_COMPRESSION_ALGORITHM.getName();
154
155
156
157
158
159
160
161 public final static int MIN_NUM_HFILE_PATH_LEVELS = 5;
162
163
164
165
166 public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
167 public static final ChecksumType DEFAULT_CHECKSUM_TYPE = ChecksumType.CRC32;
168
169
170 private static final AtomicInteger readOps = new AtomicInteger();
171 private static final AtomicLong readTimeNano = new AtomicLong();
172 private static final AtomicInteger writeOps = new AtomicInteger();
173 private static final AtomicLong writeTimeNano = new AtomicLong();
174
175
176 private static final AtomicInteger preadOps = new AtomicInteger();
177 private static final AtomicLong preadTimeNano = new AtomicLong();
178
179
180 static final AtomicLong checksumFailures = new AtomicLong();
181
182
183
184
185
186
187 private static final int LATENCY_BUFFER_SIZE = 5000;
188 private static final BlockingQueue<Long> fsReadLatenciesNanos =
189 new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
190 private static final BlockingQueue<Long> fsWriteLatenciesNanos =
191 new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
192 private static final BlockingQueue<Long> fsPreadLatenciesNanos =
193 new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
194
195 public static final void offerReadLatency(long latencyNanos, boolean pread) {
196 if (pread) {
197 fsPreadLatenciesNanos.offer(latencyNanos);
198 preadOps.incrementAndGet();
199 preadTimeNano.addAndGet(latencyNanos);
200 } else {
201 fsReadLatenciesNanos.offer(latencyNanos);
202 readTimeNano.addAndGet(latencyNanos);
203 readOps.incrementAndGet();
204 }
205 }
206
207 public static final void offerWriteLatency(long latencyNanos) {
208 fsWriteLatenciesNanos.offer(latencyNanos);
209
210 writeTimeNano.addAndGet(latencyNanos);
211 writeOps.incrementAndGet();
212 }
213
214 public static final Collection<Long> getReadLatenciesNanos() {
215 final List<Long> latencies =
216 Lists.newArrayListWithCapacity(fsReadLatenciesNanos.size());
217 fsReadLatenciesNanos.drainTo(latencies);
218 return latencies;
219 }
220
221 public static final Collection<Long> getPreadLatenciesNanos() {
222 final List<Long> latencies =
223 Lists.newArrayListWithCapacity(fsPreadLatenciesNanos.size());
224 fsPreadLatenciesNanos.drainTo(latencies);
225 return latencies;
226 }
227
228 public static final Collection<Long> getWriteLatenciesNanos() {
229 final List<Long> latencies =
230 Lists.newArrayListWithCapacity(fsWriteLatenciesNanos.size());
231 fsWriteLatenciesNanos.drainTo(latencies);
232 return latencies;
233 }
234
235
236 public static volatile AtomicLong dataBlockReadCnt = new AtomicLong(0);
237
238
239 public static final int getReadOps() {
240 return readOps.getAndSet(0);
241 }
242
243 public static final long getReadTimeMs() {
244 return readTimeNano.getAndSet(0) / 1000000;
245 }
246
247
248 public static final int getPreadOps() {
249 return preadOps.getAndSet(0);
250 }
251
252 public static final long getPreadTimeMs() {
253 return preadTimeNano.getAndSet(0) / 1000000;
254 }
255
256 public static final int getWriteOps() {
257 return writeOps.getAndSet(0);
258 }
259
260 public static final long getWriteTimeMs() {
261 return writeTimeNano.getAndSet(0) / 1000000;
262 }
263
264
265
266
267
268 public static final long getChecksumFailuresCount() {
269 return checksumFailures.getAndSet(0);
270 }
271
272
273 public interface Writer extends Closeable {
274
275
276 void appendFileInfo(byte[] key, byte[] value) throws IOException;
277
278 void append(KeyValue kv) throws IOException;
279
280 void append(byte[] key, byte[] value) throws IOException;
281
282
283 Path getPath();
284
285 String getColumnFamilyName();
286
287 void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
288
289
290
291
292
293 void addInlineBlockWriter(InlineBlockWriter bloomWriter);
294
295
296
297
298
299
300 void addGeneralBloomFilter(BloomFilterWriter bfw);
301
302
303
304
305
306 void addDeleteFamilyBloomFilter(BloomFilterWriter bfw) throws IOException;
307 }
308
309
310
311
312
313 public static abstract class WriterFactory {
314 protected final Configuration conf;
315 protected final CacheConfig cacheConf;
316 protected FileSystem fs;
317 protected Path path;
318 protected FSDataOutputStream ostream;
319 protected int blockSize = HColumnDescriptor.DEFAULT_BLOCKSIZE;
320 protected Compression.Algorithm compression =
321 HFile.DEFAULT_COMPRESSION_ALGORITHM;
322 protected HFileDataBlockEncoder encoder = NoOpDataBlockEncoder.INSTANCE;
323 protected KeyComparator comparator;
324 protected ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
325 protected int bytesPerChecksum = DEFAULT_BYTES_PER_CHECKSUM;
326 protected boolean includeMVCCReadpoint = true;
327
328 WriterFactory(Configuration conf, CacheConfig cacheConf) {
329 this.conf = conf;
330 this.cacheConf = cacheConf;
331 }
332
333 public WriterFactory withPath(FileSystem fs, Path path) {
334 Preconditions.checkNotNull(fs);
335 Preconditions.checkNotNull(path);
336 this.fs = fs;
337 this.path = path;
338 return this;
339 }
340
341 public WriterFactory withOutputStream(FSDataOutputStream ostream) {
342 Preconditions.checkNotNull(ostream);
343 this.ostream = ostream;
344 return this;
345 }
346
347 public WriterFactory withBlockSize(int blockSize) {
348 this.blockSize = blockSize;
349 return this;
350 }
351
352 public WriterFactory withCompression(Compression.Algorithm compression) {
353 Preconditions.checkNotNull(compression);
354 this.compression = compression;
355 return this;
356 }
357
358 public WriterFactory withCompression(String compressAlgo) {
359 Preconditions.checkNotNull(compression);
360 this.compression = AbstractHFileWriter.compressionByName(compressAlgo);
361 return this;
362 }
363
364 public WriterFactory withDataBlockEncoder(HFileDataBlockEncoder encoder) {
365 Preconditions.checkNotNull(encoder);
366 this.encoder = encoder;
367 return this;
368 }
369
370 public WriterFactory withComparator(KeyComparator comparator) {
371 Preconditions.checkNotNull(comparator);
372 this.comparator = comparator;
373 return this;
374 }
375
376 public WriterFactory withChecksumType(ChecksumType checksumType) {
377 Preconditions.checkNotNull(checksumType);
378 this.checksumType = checksumType;
379 return this;
380 }
381
382 public WriterFactory withBytesPerChecksum(int bytesPerChecksum) {
383 this.bytesPerChecksum = bytesPerChecksum;
384 return this;
385 }
386
387 public WriterFactory includeMVCCReadpoint(boolean includeMVCCReadpoint) {
388 this.includeMVCCReadpoint = includeMVCCReadpoint;
389 return this;
390 }
391
392 public Writer create() throws IOException {
393 if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) {
394 throw new AssertionError("Please specify exactly one of " +
395 "filesystem/path or path");
396 }
397 if (path != null) {
398 ostream = AbstractHFileWriter.createOutputStream(conf, fs, path);
399 }
400 return createWriter(fs, path, ostream, blockSize, compression, encoder, comparator,
401 checksumType, bytesPerChecksum, includeMVCCReadpoint);
402 }
403
404 protected abstract Writer createWriter(FileSystem fs, Path path,
405 FSDataOutputStream ostream, int blockSize,
406 Compression.Algorithm compress,
407 HFileDataBlockEncoder dataBlockEncoder,
408 KeyComparator comparator, ChecksumType checksumType,
409 int bytesPerChecksum, boolean includeMVCCReadpoint) throws IOException;
410 }
411
412
413 public static final String FORMAT_VERSION_KEY = "hfile.format.version";
414
415 public static int getFormatVersion(Configuration conf) {
416 int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION);
417 checkFormatVersion(version);
418 return version;
419 }
420
421
422
423
424
425
426 public static final WriterFactory getWriterFactoryNoCache(Configuration
427 conf) {
428 Configuration tempConf = new Configuration(conf);
429 tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
430 return HFile.getWriterFactory(conf, new CacheConfig(tempConf));
431 }
432
433
434
435
436 public static final WriterFactory getWriterFactory(Configuration conf,
437 CacheConfig cacheConf) {
438 SchemaMetrics.configureGlobally(conf);
439 int version = getFormatVersion(conf);
440 switch (version) {
441 case 1:
442 return new HFileWriterV1.WriterFactoryV1(conf, cacheConf);
443 case 2:
444 return new HFileWriterV2.WriterFactoryV2(conf, cacheConf);
445 default:
446 throw new IllegalArgumentException("Cannot create writer for HFile " +
447 "format version " + version);
448 }
449 }
450
451
452 public interface CachingBlockReader {
453 HFileBlock readBlock(long offset, long onDiskBlockSize,
454 boolean cacheBlock, final boolean pread, final boolean isCompaction,
455 BlockType expectedBlockType)
456 throws IOException;
457 }
458
459
460 public interface Reader extends Closeable, CachingBlockReader,
461 SchemaAware {
462
463
464
465
466
467 String getName();
468
469 String getColumnFamilyName();
470
471 RawComparator<byte []> getComparator();
472
473 HFileScanner getScanner(boolean cacheBlocks,
474 final boolean pread, final boolean isCompaction);
475
476 ByteBuffer getMetaBlock(String metaBlockName,
477 boolean cacheBlock) throws IOException;
478
479 Map<byte[], byte[]> loadFileInfo() throws IOException;
480
481 byte[] getLastKey();
482
483 byte[] midkey() throws IOException;
484
485 long length();
486
487 long getEntries();
488
489 byte[] getFirstKey();
490
491 long indexSize();
492
493 byte[] getFirstRowKey();
494
495 byte[] getLastRowKey();
496
497 FixedFileTrailer getTrailer();
498
499 HFileBlockIndex.BlockIndexReader getDataBlockIndexReader();
500
501 HFileScanner getScanner(boolean cacheBlocks, boolean pread);
502
503 Compression.Algorithm getCompressionAlgorithm();
504
505
506
507
508
509
510 DataInput getGeneralBloomFilterMetadata() throws IOException;
511
512
513
514
515
516
517 DataInput getDeleteBloomFilterMetadata() throws IOException;
518
519 Path getPath();
520
521
522 void close(boolean evictOnClose) throws IOException;
523
524 DataBlockEncoding getEncodingOnDisk();
525 }
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542 private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis,
543 FSDataInputStream fsdisNoFsChecksum,
544 long size, boolean closeIStream, CacheConfig cacheConf,
545 DataBlockEncoding preferredEncodingInCache, HFileSystem hfs)
546 throws IOException {
547 FixedFileTrailer trailer = null;
548 try {
549 trailer = FixedFileTrailer.readFromStream(fsdis, size);
550 } catch (IllegalArgumentException iae) {
551 throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, iae);
552 }
553 switch (trailer.getMajorVersion()) {
554 case 1:
555 return new HFileReaderV1(path, trailer, fsdis, size, closeIStream,
556 cacheConf);
557 case 2:
558 return new HFileReaderV2(path, trailer, fsdis, fsdisNoFsChecksum,
559 size, closeIStream,
560 cacheConf, preferredEncodingInCache, hfs);
561 default:
562 throw new CorruptHFileException("Invalid HFile version " + trailer.getMajorVersion());
563 }
564 }
565
566
567
568
569
570
571
572
573
574 public static Reader createReaderWithEncoding(
575 FileSystem fs, Path path, CacheConfig cacheConf,
576 DataBlockEncoding preferredEncodingInCache) throws IOException {
577 final boolean closeIStream = true;
578 HFileSystem hfs = null;
579 FSDataInputStream fsdis = fs.open(path);
580 FSDataInputStream fsdisNoFsChecksum = fsdis;
581
582
583
584
585 if (!(fs instanceof HFileSystem)) {
586 hfs = new HFileSystem(fs);
587 } else {
588 hfs = (HFileSystem)fs;
589
590
591 if (hfs != null) {
592 fsdisNoFsChecksum = hfs.getNoChecksumFs().open(path);
593 }
594 }
595 return pickReaderVersion(path, fsdis, fsdisNoFsChecksum,
596 fs.getFileStatus(path).getLen(), closeIStream, cacheConf,
597 preferredEncodingInCache, hfs);
598 }
599
600
601
602
603
604
605
606
607
608
609
610
611
612 public static Reader createReaderWithEncoding(
613 FileSystem fs, Path path, FSDataInputStream fsdis,
614 FSDataInputStream fsdisNoFsChecksum, long size, CacheConfig cacheConf,
615 DataBlockEncoding preferredEncodingInCache, boolean closeIStream)
616 throws IOException {
617 HFileSystem hfs = null;
618
619
620
621
622
623 if (!(fs instanceof HFileSystem)) {
624 hfs = new HFileSystem(fs);
625 } else {
626 hfs = (HFileSystem)fs;
627 }
628 return pickReaderVersion(path, fsdis, fsdisNoFsChecksum, size,
629 closeIStream, cacheConf,
630 preferredEncodingInCache, hfs);
631 }
632
633
634
635
636
637
638
639
640 public static Reader createReader(
641 FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
642 return createReaderWithEncoding(fs, path, cacheConf,
643 DataBlockEncoding.NONE);
644 }
645
646
647
648
649 static Reader createReaderFromStream(Path path,
650 FSDataInputStream fsdis, long size, CacheConfig cacheConf)
651 throws IOException {
652 final boolean closeIStream = false;
653 return pickReaderVersion(path, fsdis, fsdis, size, closeIStream, cacheConf,
654 DataBlockEncoding.NONE, null);
655 }
656
657
658
659
660 static class FileInfo extends HbaseMapWritable<byte [], byte []> {
661 static final String RESERVED_PREFIX = "hfile.";
662 static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
663 static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
664 static final byte [] AVG_KEY_LEN =
665 Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
666 static final byte [] AVG_VALUE_LEN =
667 Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
668 static final byte [] COMPARATOR =
669 Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
670
671
672
673
674
675
676
677
678
679
680
681
682 public FileInfo append(final byte[] k, final byte[] v,
683 final boolean checkPrefix) throws IOException {
684 if (k == null || v == null) {
685 throw new NullPointerException("Key nor value may be null");
686 }
687 if (checkPrefix && isReservedFileInfoKey(k)) {
688 throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
689 + " are reserved");
690 }
691 put(k, v);
692 return this;
693 }
694
695 }
696
697
698 public static boolean isReservedFileInfoKey(byte[] key) {
699 return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
700 }
701
702
703
704
705
706
707
708
709
710
711
712
713
714 public static String[] getSupportedCompressionAlgorithms() {
715 return Compression.getSupportedAlgorithms();
716 }
717
718
719
720
721
722
723 static int longToInt(final long l) {
724
725
726 return (int)(l & 0x00000000ffffffffL);
727 }
728
729
730
731
732
733
734
735
736
737
738 static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
739 throws IOException {
740 List<Path> res = new ArrayList<Path>();
741 PathFilter dirFilter = new FSUtils.DirFilter(fs);
742 FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
743 for(FileStatus dir : familyDirs) {
744 FileStatus[] files = fs.listStatus(dir.getPath());
745 for (FileStatus file : files) {
746 if (!file.isDir()) {
747 res.add(file.getPath());
748 }
749 }
750 }
751 return res;
752 }
753
754 public static void main(String[] args) throws IOException {
755 HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter();
756 System.exit(prettyPrinter.run(args));
757 }
758
759
760
761
762
763
764
765
766
767
768 public static void checkFormatVersion(int version)
769 throws IllegalArgumentException {
770 if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
771 throw new IllegalArgumentException("Invalid HFile version: " + version
772 + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
773 + MAX_FORMAT_VERSION + ")");
774 }
775 }
776
777 }