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