View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements. See the NOTICE file distributed with this
4    * work for additional information regarding copyright ownership. The ASF
5    * licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License.
7    * You may obtain a copy of the License at
8    *
9    * http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14   * License for the specific language governing permissions and limitations
15   * under the License.
16   */
17  package org.apache.hadoop.hbase.io.encoding;
18  
19  import java.io.IOException;
20  import java.io.OutputStream;
21  import java.util.HashMap;
22  import java.util.Map;
23  
24  import org.apache.hadoop.hbase.classification.InterfaceAudience;
25  import org.apache.hadoop.hbase.classification.InterfaceStability;
26  import org.apache.hadoop.hbase.util.Bytes;
27  
28  /**
29   * Provide access to all data block encoding algorithms. All of the algorithms
30   * are required to have unique id which should <b>NEVER</b> be changed. If you
31   * want to add a new algorithm/version, assign it a new id. Announce the new id
32   * in the HBase mailing list to prevent collisions.
33   */
34  @InterfaceAudience.Public
35  @InterfaceStability.Evolving
36  public enum DataBlockEncoding {
37  
38    /** Disable data block encoding. */
39    NONE(0, null),
40    // id 1 is reserved for the BITSET algorithm to be added later
41    PREFIX(2, "org.apache.hadoop.hbase.io.encoding.PrefixKeyDeltaEncoder"),
42    DIFF(3, "org.apache.hadoop.hbase.io.encoding.DiffKeyDeltaEncoder"),
43    FAST_DIFF(4, "org.apache.hadoop.hbase.io.encoding.FastDiffDeltaEncoder"),
44    // id 5 is reserved for the COPY_KEY algorithm for benchmarking
45    // COPY_KEY(5, "org.apache.hadoop.hbase.io.encoding.CopyKeyDataBlockEncoder"),
46    PREFIX_TREE(6, "org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec");
47  
48    private final short id;
49    private final byte[] idInBytes;
50    private DataBlockEncoder encoder;
51    private final String encoderCls;
52  
53    public static final int ID_SIZE = Bytes.SIZEOF_SHORT;
54  
55    /** Maps data block encoding ids to enum instances. */
56    private static Map<Short, DataBlockEncoding> idToEncoding =
57        new HashMap<Short, DataBlockEncoding>();
58  
59    static {
60      for (DataBlockEncoding algo : values()) {
61        if (idToEncoding.containsKey(algo.id)) {
62          throw new RuntimeException(String.format(
63              "Two data block encoder algorithms '%s' and '%s' have " +
64              "the same id %d",
65              idToEncoding.get(algo.id).toString(), algo.toString(),
66              (int) algo.id));
67        }
68        idToEncoding.put(algo.id, algo);
69      }
70    }
71  
72    private DataBlockEncoding(int id, String encoderClsName) {
73      if (id < Short.MIN_VALUE || id > Short.MAX_VALUE) {
74        throw new AssertionError(
75            "Data block encoding algorithm id is out of range: " + id);
76      }
77      this.id = (short) id;
78      this.idInBytes = Bytes.toBytes(this.id);
79      if (idInBytes.length != ID_SIZE) {
80        // White this may seem redundant, if we accidentally serialize
81        // the id as e.g. an int instead of a short, all encoders will break.
82        throw new RuntimeException("Unexpected length of encoder ID byte " +
83            "representation: " + Bytes.toStringBinary(idInBytes));
84      }
85      this.encoderCls = encoderClsName;
86    }
87  
88    /**
89     * @return name converted to bytes.
90     */
91    public byte[] getNameInBytes() {
92      return Bytes.toBytes(toString());
93    }
94  
95    /**
96     * @return The id of a data block encoder.
97     */
98    public short getId() {
99      return id;
100   }
101 
102   /**
103    * Writes id in bytes.
104    * @param stream where the id should be written.
105    */
106   public void writeIdInBytes(OutputStream stream) throws IOException {
107     stream.write(idInBytes);
108   }
109 
110 
111   /**
112    * Writes id bytes to the given array starting from offset.
113    *
114    * @param dest output array
115    * @param offset starting offset of the output array
116    * @throws IOException
117    */
118   public void writeIdInBytes(byte[] dest, int offset) throws IOException {
119     System.arraycopy(idInBytes, 0, dest, offset, ID_SIZE);
120   }
121 
122   /**
123    * Return new data block encoder for given algorithm type.
124    * @return data block encoder if algorithm is specified, null if none is
125    *         selected.
126    */
127   public DataBlockEncoder getEncoder() {
128     if (encoder == null && id != 0) {
129       // lazily create the encoder
130       encoder = createEncoder(encoderCls);
131     }
132     return encoder;
133   }
134 
135   /**
136    * Find and create data block encoder for given id;
137    * @param encoderId id of data block encoder.
138    * @return Newly created data block encoder.
139    */
140   public static DataBlockEncoder getDataBlockEncoderById(short encoderId) {
141     if (!idToEncoding.containsKey(encoderId)) {
142       throw new IllegalArgumentException(String.format(
143           "There is no data block encoder for given id '%d'",
144           (int) encoderId));
145     }
146 
147     return idToEncoding.get(encoderId).getEncoder();
148   }
149 
150   /**
151    * Find and return the name of data block encoder for the given id.
152    * @param encoderId id of data block encoder
153    * @return name, same as used in options in column family
154    */
155   public static String getNameFromId(short encoderId) {
156     return idToEncoding.get(encoderId).toString();
157   }
158 
159   /**
160    * Check if given encoder has this id.
161    * @param encoder encoder which id will be checked
162    * @param encoderId id which we except
163    * @return true if id is right for given encoder, false otherwise
164    * @exception IllegalArgumentException
165    *            thrown when there is no matching data block encoder
166    */
167   public static boolean isCorrectEncoder(DataBlockEncoder encoder,
168       short encoderId) {
169     if (!idToEncoding.containsKey(encoderId)) {
170       throw new IllegalArgumentException(String.format(
171           "There is no data block encoder for given id '%d'",
172           (int) encoderId));
173     }
174 
175     DataBlockEncoding algorithm = idToEncoding.get(encoderId);
176     String encoderCls = encoder.getClass().getName();
177     return encoderCls.equals(algorithm.encoderCls);
178   }
179 
180   public static DataBlockEncoding getEncodingById(short dataBlockEncodingId) {
181     return idToEncoding.get(dataBlockEncodingId);
182   }
183 
184   protected static DataBlockEncoder createEncoder(String fullyQualifiedClassName){
185       try {
186         return (DataBlockEncoder)Class.forName(fullyQualifiedClassName).newInstance();
187       } catch (InstantiationException e) {
188         throw new RuntimeException(e);
189       } catch (IllegalAccessException e) {
190         throw new RuntimeException(e);
191       } catch (ClassNotFoundException e) {
192         throw new IllegalArgumentException(e);
193       }
194   }
195 
196 }