View Javadoc

1   /*
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.io.hfile;
21  
22  import java.io.DataInputStream;
23  import java.io.DataOutput;
24  import java.io.IOException;
25  import java.io.OutputStream;
26  import java.nio.ByteBuffer;
27  
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.hbase.util.Bytes;
30  
31  /**
32   * Various types of HFile blocks. Ordinal values of these enum constants must not be relied upon.
33   * The values in the enum appear in the order they appear in a version 2 HFile.
34   */
35  @InterfaceAudience.Private
36  public enum BlockType {
37  
38    // Scanned block section
39  
40    /** Data block, both versions */
41    DATA("DATABLK*", BlockCategory.DATA),
42  
43    /** An encoded data block (e.g. with prefix compression), version 2 */
44    ENCODED_DATA("DATABLKE", BlockCategory.DATA) {
45      @Override
46      public int getId() {
47        return DATA.ordinal();
48      }
49    },
50  
51    /** Version 2 leaf index block. Appears in the data block section */
52    LEAF_INDEX("IDXLEAF2", BlockCategory.INDEX),
53  
54    /** Bloom filter block, version 2 */
55    BLOOM_CHUNK("BLMFBLK2", BlockCategory.BLOOM),
56  
57    // Non-scanned block section
58  
59    /** Meta blocks */
60    META("METABLKc", BlockCategory.META),
61  
62    /** Intermediate-level version 2 index in the non-data block section */
63    INTERMEDIATE_INDEX("IDXINTE2", BlockCategory.INDEX),
64  
65    // Load-on-open section.
66  
67    /** Root index block, also used for the single-level meta index, version 2 */
68    ROOT_INDEX("IDXROOT2", BlockCategory.INDEX),
69  
70    /** File info, version 2 */
71    FILE_INFO("FILEINF2", BlockCategory.META),
72  
73    /** General Bloom filter metadata, version 2 */
74    GENERAL_BLOOM_META("BLMFMET2", BlockCategory.BLOOM),
75  
76    /** Delete Family Bloom filter metadata, version 2 */
77    DELETE_FAMILY_BLOOM_META("DFBLMET2", BlockCategory.BLOOM),
78  
79    // Trailer
80  
81    /** Fixed file trailer, both versions (always just a magic string) */
82    TRAILER("TRABLK\"$", BlockCategory.META),
83  
84    // Legacy blocks
85  
86    /** Block index magic string in version 1 */
87    INDEX_V1("IDXBLK)+", BlockCategory.INDEX);
88  
89    public enum BlockCategory {
90      DATA, META, INDEX, BLOOM, ALL_CATEGORIES, UNKNOWN;
91  
92      /**
93       * Throws an exception if the block category passed is the special category
94       * meaning "all categories".
95       */
96      public void expectSpecific() {
97        if (this == ALL_CATEGORIES) {
98          throw new IllegalArgumentException("Expected a specific block " +
99              "category but got " + this);
100       }
101     }
102   }
103 
104   public static final int MAGIC_LENGTH = 8;
105 
106   private final byte[] magic;
107   private final BlockCategory metricCat;
108 
109   private BlockType(String magicStr, BlockCategory metricCat) {
110     magic = Bytes.toBytes(magicStr);
111     this.metricCat = metricCat;
112     assert magic.length == MAGIC_LENGTH;
113   }
114 
115   /**
116    * Use this instead of {@link #ordinal()}. They work exactly the same, except
117    * DATA and ENCODED_DATA get the same id using this method (overridden for
118    * {@link #ENCODED_DATA}).
119    * @return block type id from 0 to the number of block types - 1
120    */
121   public int getId() {
122     // Default implementation, can be overridden for individual enum members.
123     return ordinal();
124   }
125 
126   public void writeToStream(OutputStream out) throws IOException {
127     out.write(magic);
128   }
129 
130   public void write(DataOutput out) throws IOException {
131     out.write(magic);
132   }
133 
134   public void write(ByteBuffer buf) {
135     buf.put(magic);
136   }
137 
138   public BlockCategory getCategory() {
139     return metricCat;
140   }
141 
142   public static BlockType parse(byte[] buf, int offset, int length)
143       throws IOException {
144     if (length != MAGIC_LENGTH) {
145       throw new IOException("Magic record of invalid length: "
146           + Bytes.toStringBinary(buf, offset, length));
147     }
148 
149     for (BlockType blockType : values())
150       if (Bytes.compareTo(blockType.magic, 0, MAGIC_LENGTH, buf, offset,
151           MAGIC_LENGTH) == 0)
152         return blockType;
153 
154     throw new IOException("Invalid HFile block magic: "
155         + Bytes.toStringBinary(buf, offset, MAGIC_LENGTH));
156   }
157 
158   public static BlockType read(DataInputStream in) throws IOException {
159     byte[] buf = new byte[MAGIC_LENGTH];
160     in.readFully(buf);
161     return parse(buf, 0, buf.length);
162   }
163 
164   public static BlockType read(ByteBuffer buf) throws IOException {
165     byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), MAGIC_LENGTH)];
166     buf.get(magicBuf);
167     BlockType blockType = parse(magicBuf, 0, magicBuf.length);
168     // If we got here, we have read exactly MAGIC_LENGTH bytes.
169     return blockType;
170   }
171 
172   /**
173    * Put the magic record out to the specified byte array position.
174    *
175    * @param bytes the byte array
176    * @param offset position in the array
177    * @return incremented offset
178    */
179   public int put(byte[] bytes, int offset) {
180     System.arraycopy(magic, 0, bytes, offset, MAGIC_LENGTH);
181     return offset + MAGIC_LENGTH;
182   }
183 
184   /**
185    * Reads a magic record of the length {@link #MAGIC_LENGTH} from the given
186    * stream and expects it to match this block type.
187    */
188   public void readAndCheck(DataInputStream in) throws IOException {
189     byte[] buf = new byte[MAGIC_LENGTH];
190     in.readFully(buf);
191     if (Bytes.compareTo(buf, magic) != 0) {
192       throw new IOException("Invalid magic: expected "
193           + Bytes.toStringBinary(magic) + ", got " + Bytes.toStringBinary(buf));
194     }
195   }
196 
197   /**
198    * Reads a magic record of the length {@link #MAGIC_LENGTH} from the given
199    * byte buffer and expects it to match this block type.
200    */
201   public void readAndCheck(ByteBuffer in) throws IOException {
202     byte[] buf = new byte[MAGIC_LENGTH];
203     in.get(buf);
204     if (Bytes.compareTo(buf, magic) != 0) {
205       throw new IOException("Invalid magic: expected "
206           + Bytes.toStringBinary(magic) + ", got " + Bytes.toStringBinary(buf));
207     }
208   }
209 
210   /**
211    * @return whether this block type is encoded or unencoded data block
212    */
213   public final boolean isData() {
214     return this == DATA || this == ENCODED_DATA;
215   }
216 
217 }