001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.hfile;
019
020import java.io.DataInputStream;
021import java.io.DataOutput;
022import java.io.IOException;
023import java.io.OutputStream;
024import java.nio.ByteBuffer;
025import org.apache.hadoop.hbase.nio.ByteBuff;
026import org.apache.hadoop.hbase.util.Bytes;
027import org.apache.yetus.audience.InterfaceAudience;
028
029/**
030 * Various types of HFile blocks. Ordinal values of these enum constants must not be relied upon.
031 * The values in the enum appear in the order they appear in a version 2 HFile.
032 */
033@InterfaceAudience.Private
034@SuppressWarnings("ImmutableEnumChecker")
035public enum BlockType {
036
037  // Scanned block section
038
039  /** Data block, both versions */
040  DATA("DATABLK*", BlockCategory.DATA),
041
042  /** An encoded data block (e.g. with prefix compression), version 2 */
043  ENCODED_DATA("DATABLKE", BlockCategory.DATA) {
044    @Override
045    public int getId() {
046      return DATA.ordinal();
047    }
048  },
049
050  /** Version 2 leaf index block. Appears in the data block section */
051  LEAF_INDEX("IDXLEAF2", BlockCategory.INDEX),
052
053  /** Bloom filter block, version 2 */
054  BLOOM_CHUNK("BLMFBLK2", BlockCategory.BLOOM),
055
056  // Non-scanned block section
057
058  /** Meta blocks */
059  META("METABLKc", BlockCategory.META),
060
061  /** Intermediate-level version 2 index in the non-data block section */
062  INTERMEDIATE_INDEX("IDXINTE2", BlockCategory.INDEX),
063
064  // Load-on-open section.
065
066  /** Root index block, also used for the single-level meta index, version 2 */
067  ROOT_INDEX("IDXROOT2", BlockCategory.INDEX),
068
069  /** File info, version 2 */
070  FILE_INFO("FILEINF2", BlockCategory.META),
071
072  /** General Bloom filter metadata, version 2 */
073  GENERAL_BLOOM_META("BLMFMET2", BlockCategory.BLOOM),
074
075  /** Delete Family Bloom filter metadata, version 2 */
076  DELETE_FAMILY_BLOOM_META("DFBLMET2", BlockCategory.BLOOM),
077
078  // Trailer
079
080  /** Fixed file trailer, both versions (always just a magic string) */
081  TRAILER("TRABLK\"$", BlockCategory.META),
082
083  // Legacy blocks
084
085  /** Block index magic string in version 1 */
086  INDEX_V1("IDXBLK)+", BlockCategory.INDEX);
087
088  public enum BlockCategory {
089    DATA,
090    META,
091    INDEX,
092    BLOOM,
093    ALL_CATEGORIES,
094    UNKNOWN;
095
096    /**
097     * Throws an exception if the block category passed is the special category meaning "all
098     * categories".
099     */
100    public void expectSpecific() {
101      if (this == ALL_CATEGORIES) {
102        throw new IllegalArgumentException(
103          "Expected a specific block " + "category but got " + this);
104      }
105    }
106  }
107
108  public static final int MAGIC_LENGTH = 8;
109
110  private final byte[] magic;
111  private final BlockCategory metricCat;
112
113  private BlockType(String magicStr, BlockCategory metricCat) {
114    magic = Bytes.toBytes(magicStr);
115    this.metricCat = metricCat;
116    assert magic.length == MAGIC_LENGTH;
117  }
118
119  /**
120   * Use this instead of {@link #ordinal()}. They work exactly the same, except DATA and
121   * ENCODED_DATA get the same id using this method (overridden for {@link #ENCODED_DATA}).
122   * @return block type id from 0 to the number of block types - 1
123   */
124  public int getId() {
125    // Default implementation, can be overridden for individual enum members.
126    return ordinal();
127  }
128
129  public void writeToStream(OutputStream out) throws IOException {
130    out.write(magic);
131  }
132
133  public void write(DataOutput out) throws IOException {
134    out.write(magic);
135  }
136
137  public void write(ByteBuffer buf) {
138    buf.put(magic);
139  }
140
141  public void write(ByteBuff buf) {
142    buf.put(magic);
143  }
144
145  public BlockCategory getCategory() {
146    return metricCat;
147  }
148
149  public static BlockType parse(byte[] buf, int offset, int length) throws IOException {
150    if (length != MAGIC_LENGTH) {
151      throw new IOException(
152        "Magic record of invalid length: " + Bytes.toStringBinary(buf, offset, length));
153    }
154
155    for (BlockType blockType : values())
156      if (Bytes.compareTo(blockType.magic, 0, MAGIC_LENGTH, buf, offset, MAGIC_LENGTH) == 0)
157        return blockType;
158
159    throw new IOException(
160      "Invalid HFile block magic: " + Bytes.toStringBinary(buf, offset, MAGIC_LENGTH));
161  }
162
163  public static BlockType read(DataInputStream in) throws IOException {
164    byte[] buf = new byte[MAGIC_LENGTH];
165    in.readFully(buf);
166    return parse(buf, 0, buf.length);
167  }
168
169  public static BlockType read(ByteBuff buf) throws IOException {
170    byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), MAGIC_LENGTH)];
171    buf.get(magicBuf);
172    BlockType blockType = parse(magicBuf, 0, magicBuf.length);
173    // If we got here, we have read exactly MAGIC_LENGTH bytes.
174    return blockType;
175  }
176
177  /**
178   * Put the magic record out to the specified byte array position.
179   * @param bytes  the byte array
180   * @param offset position in the array
181   * @return incremented offset
182   */
183  // System.arraycopy is static native. We can't do anything about this until minimum JDK is 9.
184  @SuppressWarnings("UnsafeFinalization")
185  public int put(byte[] bytes, int offset) {
186    System.arraycopy(magic, 0, bytes, offset, MAGIC_LENGTH);
187    return offset + MAGIC_LENGTH;
188  }
189
190  /**
191   * Reads a magic record of the length {@link #MAGIC_LENGTH} from the given stream and expects it
192   * to match this block type.
193   */
194  public void readAndCheck(DataInputStream in) throws IOException {
195    byte[] buf = new byte[MAGIC_LENGTH];
196    in.readFully(buf);
197    if (Bytes.compareTo(buf, magic) != 0) {
198      throw new IOException("Invalid magic: expected " + Bytes.toStringBinary(magic) + ", got "
199        + Bytes.toStringBinary(buf));
200    }
201  }
202
203  /**
204   * Reads a magic record of the length {@link #MAGIC_LENGTH} from the given byte buffer and expects
205   * it to match this block type.
206   */
207  public void readAndCheck(ByteBuffer in) throws IOException {
208    byte[] buf = new byte[MAGIC_LENGTH];
209    in.get(buf);
210    if (Bytes.compareTo(buf, magic) != 0) {
211      throw new IOException("Invalid magic: expected " + Bytes.toStringBinary(magic) + ", got "
212        + Bytes.toStringBinary(buf));
213    }
214  }
215
216  /** Returns whether this block type is encoded or unencoded data block */
217  public final boolean isData() {
218    return this == DATA || this == ENCODED_DATA;
219  }
220
221  /** Returns whether this block category is index */
222  public final boolean isIndex() {
223    return this.getCategory() == BlockCategory.INDEX;
224  }
225
226  /** Returns whether this block category is bloom filter */
227  public final boolean isBloom() {
228    return this.getCategory() == BlockCategory.BLOOM;
229  }
230}