001/*
002 * Licensed to the Apache Software Foundation (ASF) under one or more
003 * contributor license agreements. See the NOTICE file distributed with this
004 * work for additional information regarding copyright ownership. The ASF
005 * licenses this file to you under the Apache License, Version 2.0 (the
006 * "License"); you may not use this file except in compliance with the License.
007 * You may obtain a copy of the License at
008 *
009 * http://www.apache.org/licenses/LICENSE-2.0
010 *
011 * Unless required by applicable law or agreed to in writing, software
012 * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
013 * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
014 * License for the specific language governing permissions and limitations
015 * under the License.
016 */
017package org.apache.hadoop.hbase.io.encoding;
018
019import java.io.IOException;
020import java.io.OutputStream;
021
022import org.apache.hadoop.hbase.util.Bytes;
023import org.apache.yetus.audience.InterfaceAudience;
024
025/**
026 * Provide access to all data block encoding algorithms. All of the algorithms
027 * are required to have unique id which should <b>NEVER</b> be changed. If you
028 * want to add a new algorithm/version, assign it a new id. Announce the new id
029 * in the HBase mailing list to prevent collisions.
030 */
031@InterfaceAudience.Public
032public enum DataBlockEncoding {
033
034  /** Disable data block encoding. */
035  NONE(0, null),
036  // id 1 is reserved for the BITSET algorithm to be added later
037  PREFIX(2, "org.apache.hadoop.hbase.io.encoding.PrefixKeyDeltaEncoder"),
038  DIFF(3, "org.apache.hadoop.hbase.io.encoding.DiffKeyDeltaEncoder"),
039  FAST_DIFF(4, "org.apache.hadoop.hbase.io.encoding.FastDiffDeltaEncoder"),
040  // id 5 is reserved for the COPY_KEY algorithm for benchmarking
041  // COPY_KEY(5, "org.apache.hadoop.hbase.io.encoding.CopyKeyDataBlockEncoder"),
042  // PREFIX_TREE(6, "org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec"),
043  ROW_INDEX_V1(7, "org.apache.hadoop.hbase.io.encoding.RowIndexCodecV1");
044
045  private final short id;
046  private final byte[] idInBytes;
047  private DataBlockEncoder encoder;
048  private final String encoderCls;
049
050  public static final int ID_SIZE = Bytes.SIZEOF_SHORT;
051
052  /** Maps data block encoding ids to enum instances. */
053  private static DataBlockEncoding[] idArray = new DataBlockEncoding[Byte.MAX_VALUE + 1];
054
055  static {
056    for (DataBlockEncoding algo : values()) {
057      if (idArray[algo.id] != null) {
058        throw new RuntimeException(String.format(
059          "Two data block encoder algorithms '%s' and '%s' have " + "the same id %d",
060          idArray[algo.id].toString(), algo.toString(), (int) algo.id));
061      }
062      idArray[algo.id] = algo;
063    }
064  }
065
066  private DataBlockEncoding(int id, String encoderClsName) {
067    if (id < 0 || id > Byte.MAX_VALUE) {
068      throw new AssertionError(
069          "Data block encoding algorithm id is out of range: " + id);
070    }
071    this.id = (short) id;
072    this.idInBytes = Bytes.toBytes(this.id);
073    if (idInBytes.length != ID_SIZE) {
074      // White this may seem redundant, if we accidentally serialize
075      // the id as e.g. an int instead of a short, all encoders will break.
076      throw new RuntimeException("Unexpected length of encoder ID byte " +
077          "representation: " + Bytes.toStringBinary(idInBytes));
078    }
079    this.encoderCls = encoderClsName;
080  }
081
082  /**
083   * @return name converted to bytes.
084   */
085  public byte[] getNameInBytes() {
086    return Bytes.toBytes(toString());
087  }
088
089  /**
090   * @return The id of a data block encoder.
091   */
092  public short getId() {
093    return id;
094  }
095
096  /**
097   * Writes id in bytes.
098   * @param stream where the id should be written.
099   */
100  public void writeIdInBytes(OutputStream stream) throws IOException {
101    stream.write(idInBytes);
102  }
103
104
105  /**
106   * Writes id bytes to the given array starting from offset.
107   *
108   * @param dest output array
109   * @param offset starting offset of the output array
110   * @throws IOException
111   */
112  public void writeIdInBytes(byte[] dest, int offset) throws IOException {
113    System.arraycopy(idInBytes, 0, dest, offset, ID_SIZE);
114  }
115
116  /**
117   * Return new data block encoder for given algorithm type.
118   * @return data block encoder if algorithm is specified, null if none is
119   *         selected.
120   */
121  public DataBlockEncoder getEncoder() {
122    if (encoder == null && id != 0) {
123      // lazily create the encoder
124      encoder = createEncoder(encoderCls);
125    }
126    return encoder;
127  }
128
129  /**
130   * Find and create data block encoder for given id;
131   * @param encoderId id of data block encoder.
132   * @return Newly created data block encoder.
133   */
134  public static DataBlockEncoder getDataBlockEncoderById(short encoderId) {
135    return getEncodingById(encoderId).getEncoder();
136  }
137
138  /**
139   * Find and return the name of data block encoder for the given id.
140   * @param encoderId id of data block encoder
141   * @return name, same as used in options in column family
142   */
143  public static String getNameFromId(short encoderId) {
144    return getEncodingById(encoderId).toString();
145  }
146
147  /**
148   * Check if given encoder has this id.
149   * @param encoder encoder which id will be checked
150   * @param encoderId id which we except
151   * @return true if id is right for given encoder, false otherwise
152   * @exception IllegalArgumentException
153   *            thrown when there is no matching data block encoder
154   */
155  public static boolean isCorrectEncoder(DataBlockEncoder encoder,
156      short encoderId) {
157    DataBlockEncoding algorithm = getEncodingById(encoderId);
158    String encoderCls = encoder.getClass().getName();
159    return encoderCls.equals(algorithm.encoderCls);
160  }
161
162  public static DataBlockEncoding getEncodingById(short dataBlockEncodingId) {
163    DataBlockEncoding algorithm = null;
164    if (dataBlockEncodingId >= 0 && dataBlockEncodingId <= Byte.MAX_VALUE) {
165      algorithm = idArray[dataBlockEncodingId];
166    }
167    if (algorithm == null) {
168      throw new IllegalArgumentException(String.format(
169          "There is no data block encoder for given id '%d'",
170          (int) dataBlockEncodingId));
171    }
172    return algorithm;
173  }
174
175  protected static DataBlockEncoder createEncoder(String fullyQualifiedClassName) {
176    try {
177      return (DataBlockEncoder) Class.forName(fullyQualifiedClassName).getDeclaredConstructor()
178          .newInstance();
179    } catch (Exception e) {
180      throw new RuntimeException(e);
181    }
182  }
183
184}