001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.io.hfile;
020
021
022import java.io.ByteArrayInputStream;
023import java.io.ByteArrayOutputStream;
024import java.io.DataInput;
025import java.io.DataInputStream;
026import java.io.DataOutputStream;
027import java.io.IOException;
028import java.nio.ByteBuffer;
029
030import org.apache.hadoop.fs.FSDataInputStream;
031import org.apache.hadoop.hbase.CellComparator;
032import org.apache.hadoop.hbase.CellComparatorImpl;
033import org.apache.hadoop.hbase.CellComparatorImpl.MetaCellComparator;
034import org.apache.hadoop.hbase.KeyValue;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.apache.hadoop.hbase.io.compress.Compression;
037import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
038import org.apache.hadoop.hbase.shaded.protobuf.generated.HFileProtos;
039import org.apache.hadoop.hbase.util.Bytes;
040
041import org.slf4j.Logger;
042import org.slf4j.LoggerFactory;
043
044/**
045 * The {@link HFile} has a fixed trailer which contains offsets to other
046 * variable parts of the file. Also includes basic metadata on this file. The
047 * trailer size is fixed within a given {@link HFile} format version only, but
048 * we always store the version number as the last four-byte integer of the file.
049 * The version number itself is split into two portions, a major 
050 * version and a minor version. The last three bytes of a file are the major
051 * version and a single preceding byte is the minor number. The major version
052 * determines which readers/writers to use to read/write a hfile while a minor
053 * version determines smaller changes in hfile format that do not need a new
054 * reader/writer type.
055 */
056@InterfaceAudience.Private
057public class FixedFileTrailer {
058  private static final Logger LOG = LoggerFactory.getLogger(FixedFileTrailer.class);
059
060  /**
061   * We store the comparator class name as a fixed-length field in the trailer.
062   */
063  private static final int MAX_COMPARATOR_NAME_LENGTH = 128;
064
065  /**
066   * Offset to the fileinfo data, a small block of vitals. Necessary in v1 but
067   * only potentially useful for pretty-printing in v2.
068   */
069  private long fileInfoOffset;
070
071  /**
072   * In version 1, the offset to the data block index. Starting from version 2,
073   * the meaning of this field is the offset to the section of the file that
074   * should be loaded at the time the file is being opened: i.e. on open we load
075   * the root index, file info, etc. See http://hbase.apache.org/book.html#_hfile_format_2
076   * in the reference guide.
077   */
078  private long loadOnOpenDataOffset;
079
080  /** The number of entries in the root data index. */
081  private int dataIndexCount;
082
083  /** Total uncompressed size of all blocks of the data index */
084  private long uncompressedDataIndexSize;
085
086  /** The number of entries in the meta index */
087  private int metaIndexCount;
088
089  /** The total uncompressed size of keys/values stored in the file. */
090  private long totalUncompressedBytes;
091
092  /**
093   * The number of key/value pairs in the file. This field was int in version 1,
094   * but is now long.
095   */
096  private long entryCount;
097
098  /** The compression codec used for all blocks. */
099  private Compression.Algorithm compressionCodec = Compression.Algorithm.NONE;
100
101  /**
102   * The number of levels in the potentially multi-level data index. Used from
103   * version 2 onwards.
104   */
105  private int numDataIndexLevels;
106
107  /** The offset of the first data block. */
108  private long firstDataBlockOffset;
109
110  /**
111   * It is guaranteed that no key/value data blocks start after this offset in
112   * the file.
113   */
114  private long lastDataBlockOffset;
115
116  /** Raw key comparator class name in version 3 */
117  // We could write the actual class name from 2.0 onwards and handle BC
118  private String comparatorClassName = CellComparator.getInstance().getClass().getName();
119
120  /** The encryption key */
121  private byte[] encryptionKey;
122
123  /** The {@link HFile} format major version. */
124  private final int majorVersion;
125
126  /** The {@link HFile} format minor version. */
127  private final int minorVersion;
128
129  FixedFileTrailer(int majorVersion, int minorVersion) {
130    this.majorVersion = majorVersion;
131    this.minorVersion = minorVersion;
132    HFile.checkFormatVersion(majorVersion);
133  }
134
135  private static int[] computeTrailerSizeByVersion() {
136    int versionToSize[] = new int[HFile.MAX_FORMAT_VERSION + 1];
137    // We support only 2 major versions now. ie. V2, V3
138    versionToSize[2] = 212;
139    for (int version = 3; version <= HFile.MAX_FORMAT_VERSION; version++) {
140      // Max FFT size for V3 and above is taken as 4KB for future enhancements
141      // if any.
142      // Unless the trailer size exceeds 4K this can continue
143      versionToSize[version] = 1024 * 4;
144    }
145    return versionToSize;
146  }
147
148  private static int getMaxTrailerSize() {
149    int maxSize = 0;
150    for (int version = HFile.MIN_FORMAT_VERSION;
151         version <= HFile.MAX_FORMAT_VERSION;
152         ++version)
153      maxSize = Math.max(getTrailerSize(version), maxSize);
154    return maxSize;
155  }
156
157  private static final int TRAILER_SIZE[] = computeTrailerSizeByVersion();
158  private static final int MAX_TRAILER_SIZE = getMaxTrailerSize();
159
160  private static final int NOT_PB_SIZE = BlockType.MAGIC_LENGTH + Bytes.SIZEOF_INT;
161
162  static int getTrailerSize(int version) {
163    return TRAILER_SIZE[version];
164  }
165
166  public int getTrailerSize() {
167    return getTrailerSize(majorVersion);
168  }
169
170  /**
171   * Write the trailer to a data stream. We support writing version 1 for
172   * testing and for determining version 1 trailer size. It is also easy to see
173   * what fields changed in version 2.
174   *
175   * @param outputStream
176   * @throws IOException
177   */
178  void serialize(DataOutputStream outputStream) throws IOException {
179    HFile.checkFormatVersion(majorVersion);
180
181    ByteArrayOutputStream baos = new ByteArrayOutputStream();
182    DataOutputStream baosDos = new DataOutputStream(baos);
183
184    BlockType.TRAILER.write(baosDos);
185    serializeAsPB(baosDos);
186
187    // The last 4 bytes of the file encode the major and minor version universally
188    baosDos.writeInt(materializeVersion(majorVersion, minorVersion));
189
190    baos.writeTo(outputStream);
191  }
192
193  @org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting
194  HFileProtos.FileTrailerProto toProtobuf() {
195    HFileProtos.FileTrailerProto.Builder builder = HFileProtos.FileTrailerProto.newBuilder()
196        .setFileInfoOffset(fileInfoOffset)
197        .setLoadOnOpenDataOffset(loadOnOpenDataOffset)
198        .setUncompressedDataIndexSize(uncompressedDataIndexSize)
199        .setTotalUncompressedBytes(totalUncompressedBytes)
200        .setDataIndexCount(dataIndexCount)
201        .setMetaIndexCount(metaIndexCount)
202        .setEntryCount(entryCount)
203        .setNumDataIndexLevels(numDataIndexLevels)
204        .setFirstDataBlockOffset(firstDataBlockOffset)
205        .setLastDataBlockOffset(lastDataBlockOffset)
206        .setComparatorClassName(getHBase1CompatibleName(comparatorClassName))
207        .setCompressionCodec(compressionCodec.ordinal());
208    if (encryptionKey != null) {
209      builder.setEncryptionKey(UnsafeByteOperations.unsafeWrap(encryptionKey));
210    }
211    return builder.build();
212  }
213
214  /**
215   * Write trailer data as protobuf.
216   * NOTE: we run a translation on the comparator name and will serialize the old hbase-1.x where
217   * it makes sense. See {@link #getHBase1CompatibleName(String)}.
218   */
219  void serializeAsPB(DataOutputStream output) throws IOException {
220    ByteArrayOutputStream baos = new ByteArrayOutputStream();
221    // We need this extra copy unfortunately to determine the final size of the
222    // delimited output, see use of baos.size() below.
223    toProtobuf().writeDelimitedTo(baos);
224    baos.writeTo(output);
225    // Pad to make up the difference between variable PB encoding length and the
226    // length when encoded as writable under earlier V2 formats. Failure to pad
227    // properly or if the PB encoding is too big would mean the trailer wont be read
228    // in properly by HFile.
229    int padding = getTrailerSize() - NOT_PB_SIZE - baos.size();
230    if (padding < 0) {
231      throw new IOException("Pbuf encoding size exceeded fixed trailer size limit");
232    }
233    for (int i = 0; i < padding; i++) {
234      output.write(0);
235    }
236  }
237
238  /**
239   * Deserialize the fixed file trailer from the given stream. The version needs
240   * to already be specified. Make sure this is consistent with
241   * {@link #serialize(DataOutputStream)}.
242   *
243   * @param inputStream
244   * @throws IOException
245   */
246  void deserialize(DataInputStream inputStream) throws IOException {
247    HFile.checkFormatVersion(majorVersion);
248
249    BlockType.TRAILER.readAndCheck(inputStream);
250
251    if (majorVersion > 2
252        || (majorVersion == 2 && minorVersion >= HFileReaderImpl.PBUF_TRAILER_MINOR_VERSION)) {
253      deserializeFromPB(inputStream);
254    } else {
255      deserializeFromWritable(inputStream);
256    }
257
258    // The last 4 bytes of the file encode the major and minor version universally
259    int version = inputStream.readInt();
260    expectMajorVersion(extractMajorVersion(version));
261    expectMinorVersion(extractMinorVersion(version));
262  }
263
264  /**
265   * Deserialize the file trailer as protobuf
266   * @param inputStream
267   * @throws IOException
268   */
269  void deserializeFromPB(DataInputStream inputStream) throws IOException {
270    // read PB and skip padding
271    int start = inputStream.available();
272    HFileProtos.FileTrailerProto trailerProto =
273        HFileProtos.FileTrailerProto.PARSER.parseDelimitedFrom(inputStream);
274    int size = start - inputStream.available();
275    inputStream.skip(getTrailerSize() - NOT_PB_SIZE - size);
276
277    // process the PB
278    if (trailerProto.hasFileInfoOffset()) {
279      fileInfoOffset = trailerProto.getFileInfoOffset();
280    }
281    if (trailerProto.hasLoadOnOpenDataOffset()) {
282      loadOnOpenDataOffset = trailerProto.getLoadOnOpenDataOffset();
283    }
284    if (trailerProto.hasUncompressedDataIndexSize()) {
285      uncompressedDataIndexSize = trailerProto.getUncompressedDataIndexSize();
286    }
287    if (trailerProto.hasTotalUncompressedBytes()) {
288      totalUncompressedBytes = trailerProto.getTotalUncompressedBytes();
289    }
290    if (trailerProto.hasDataIndexCount()) {
291      dataIndexCount = trailerProto.getDataIndexCount();
292    }
293    if (trailerProto.hasMetaIndexCount()) {
294      metaIndexCount = trailerProto.getMetaIndexCount();
295    }
296    if (trailerProto.hasEntryCount()) {
297      entryCount = trailerProto.getEntryCount();
298    }
299    if (trailerProto.hasNumDataIndexLevels()) {
300      numDataIndexLevels = trailerProto.getNumDataIndexLevels();
301    }
302    if (trailerProto.hasFirstDataBlockOffset()) {
303      firstDataBlockOffset = trailerProto.getFirstDataBlockOffset();
304    }
305    if (trailerProto.hasLastDataBlockOffset()) {
306      lastDataBlockOffset = trailerProto.getLastDataBlockOffset();
307    }
308    if (trailerProto.hasComparatorClassName()) {
309      setComparatorClass(getComparatorClass(trailerProto.getComparatorClassName()));
310    }
311    if (trailerProto.hasCompressionCodec()) {
312      compressionCodec = Compression.Algorithm.values()[trailerProto.getCompressionCodec()];
313    } else {
314      compressionCodec = Compression.Algorithm.NONE;
315    }
316    if (trailerProto.hasEncryptionKey()) {
317      encryptionKey = trailerProto.getEncryptionKey().toByteArray();
318    }
319  }
320
321  /**
322   * Deserialize the file trailer as writable data
323   * @param input
324   * @throws IOException
325   */
326  void deserializeFromWritable(DataInput input) throws IOException {
327    fileInfoOffset = input.readLong();
328    loadOnOpenDataOffset = input.readLong();
329    dataIndexCount = input.readInt();
330    uncompressedDataIndexSize = input.readLong();
331    metaIndexCount = input.readInt();
332
333    totalUncompressedBytes = input.readLong();
334    entryCount = input.readLong();
335    compressionCodec = Compression.Algorithm.values()[input.readInt()];
336    numDataIndexLevels = input.readInt();
337    firstDataBlockOffset = input.readLong();
338    lastDataBlockOffset = input.readLong();
339    // TODO this is a classname encoded into an  HFile's trailer. We are going to need to have 
340    // some compat code here.
341    setComparatorClass(getComparatorClass(Bytes.readStringFixedSize(input,
342        MAX_COMPARATOR_NAME_LENGTH)));
343  }
344  
345  private void append(StringBuilder sb, String s) {
346    if (sb.length() > 0)
347      sb.append(", ");
348    sb.append(s);
349  }
350
351  @Override
352  public String toString() {
353    StringBuilder sb = new StringBuilder();
354    append(sb, "fileinfoOffset=" + fileInfoOffset);
355    append(sb, "loadOnOpenDataOffset=" + loadOnOpenDataOffset);
356    append(sb, "dataIndexCount=" + dataIndexCount);
357    append(sb, "metaIndexCount=" + metaIndexCount);
358    append(sb, "totalUncomressedBytes=" + totalUncompressedBytes);
359    append(sb, "entryCount=" + entryCount);
360    append(sb, "compressionCodec=" + compressionCodec);
361    append(sb, "uncompressedDataIndexSize=" + uncompressedDataIndexSize);
362    append(sb, "numDataIndexLevels=" + numDataIndexLevels);
363    append(sb, "firstDataBlockOffset=" + firstDataBlockOffset);
364    append(sb, "lastDataBlockOffset=" + lastDataBlockOffset);
365    append(sb, "comparatorClassName=" + comparatorClassName);
366    if (majorVersion >= 3) {
367      append(sb, "encryptionKey=" + (encryptionKey != null ? "PRESENT" : "NONE"));
368    }
369    append(sb, "majorVersion=" + majorVersion);
370    append(sb, "minorVersion=" + minorVersion);
371
372    return sb.toString();
373  }
374
375  /**
376   * Reads a file trailer from the given file.
377   *
378   * @param istream the input stream with the ability to seek. Does not have to
379   *          be buffered, as only one read operation is made.
380   * @param fileSize the file size. Can be obtained using
381   *          {@link org.apache.hadoop.fs.FileSystem#getFileStatus(
382   *          org.apache.hadoop.fs.Path)}.
383   * @return the fixed file trailer read
384   * @throws IOException if failed to read from the underlying stream, or the
385   *           trailer is corrupted, or the version of the trailer is
386   *           unsupported
387   */
388  public static FixedFileTrailer readFromStream(FSDataInputStream istream,
389      long fileSize) throws IOException {
390    int bufferSize = MAX_TRAILER_SIZE;
391    long seekPoint = fileSize - bufferSize;
392    if (seekPoint < 0) {
393      // It is hard to imagine such a small HFile.
394      seekPoint = 0;
395      bufferSize = (int) fileSize;
396    }
397
398    HFileUtil.seekOnMultipleSources(istream, seekPoint);
399
400    ByteBuffer buf = ByteBuffer.allocate(bufferSize);
401    istream.readFully(buf.array(), buf.arrayOffset(),
402        buf.arrayOffset() + buf.limit());
403
404    // Read the version from the last int of the file.
405    buf.position(buf.limit() - Bytes.SIZEOF_INT);
406    int version = buf.getInt();
407
408    // Extract the major and minor versions.
409    int majorVersion = extractMajorVersion(version);
410    int minorVersion = extractMinorVersion(version);
411
412    HFile.checkFormatVersion(majorVersion); // throws IAE if invalid
413
414    int trailerSize = getTrailerSize(majorVersion);
415
416    FixedFileTrailer fft = new FixedFileTrailer(majorVersion, minorVersion);
417    fft.deserialize(new DataInputStream(new ByteArrayInputStream(buf.array(),
418        buf.arrayOffset() + bufferSize - trailerSize, trailerSize)));
419    return fft;
420  }
421
422  public void expectMajorVersion(int expected) {
423    if (majorVersion != expected) {
424      throw new IllegalArgumentException("Invalid HFile major version: "
425          + majorVersion 
426          + " (expected: " + expected + ")");
427    }
428  }
429
430  public void expectMinorVersion(int expected) {
431    if (minorVersion != expected) {
432      throw new IllegalArgumentException("Invalid HFile minor version: "
433          + minorVersion + " (expected: " + expected + ")");
434    }
435  }
436
437  public void expectAtLeastMajorVersion(int lowerBound) {
438    if (majorVersion < lowerBound) {
439      throw new IllegalArgumentException("Invalid HFile major version: "
440          + majorVersion
441          + " (expected: " + lowerBound + " or higher).");
442    }
443  }
444
445  public long getFileInfoOffset() {
446    return fileInfoOffset;
447  }
448
449  public void setFileInfoOffset(long fileInfoOffset) {
450    this.fileInfoOffset = fileInfoOffset;
451  }
452
453  public long getLoadOnOpenDataOffset() {
454    return loadOnOpenDataOffset;
455  }
456
457  public void setLoadOnOpenOffset(long loadOnOpenDataOffset) {
458    this.loadOnOpenDataOffset = loadOnOpenDataOffset;
459  }
460
461  public int getDataIndexCount() {
462    return dataIndexCount;
463  }
464
465  public void setDataIndexCount(int dataIndexCount) {
466    this.dataIndexCount = dataIndexCount;
467  }
468
469  public int getMetaIndexCount() {
470    return metaIndexCount;
471  }
472
473  public void setMetaIndexCount(int metaIndexCount) {
474    this.metaIndexCount = metaIndexCount;
475  }
476
477  public long getTotalUncompressedBytes() {
478    return totalUncompressedBytes;
479  }
480
481  public void setTotalUncompressedBytes(long totalUncompressedBytes) {
482    this.totalUncompressedBytes = totalUncompressedBytes;
483  }
484
485  public long getEntryCount() {
486    return entryCount;
487  }
488
489  public void setEntryCount(long newEntryCount) {
490    entryCount = newEntryCount;
491  }
492
493  public Compression.Algorithm getCompressionCodec() {
494    return compressionCodec;
495  }
496
497  public void setCompressionCodec(Compression.Algorithm compressionCodec) {
498    this.compressionCodec = compressionCodec;
499  }
500
501  public int getNumDataIndexLevels() {
502    expectAtLeastMajorVersion(2);
503    return numDataIndexLevels;
504  }
505
506  public void setNumDataIndexLevels(int numDataIndexLevels) {
507    expectAtLeastMajorVersion(2);
508    this.numDataIndexLevels = numDataIndexLevels;
509  }
510
511  public long getLastDataBlockOffset() {
512    expectAtLeastMajorVersion(2);
513    return lastDataBlockOffset;
514  }
515
516  public void setLastDataBlockOffset(long lastDataBlockOffset) {
517    expectAtLeastMajorVersion(2);
518    this.lastDataBlockOffset = lastDataBlockOffset;
519  }
520
521  public long getFirstDataBlockOffset() {
522    expectAtLeastMajorVersion(2);
523    return firstDataBlockOffset;
524  }
525
526  public void setFirstDataBlockOffset(long firstDataBlockOffset) {
527    expectAtLeastMajorVersion(2);
528    this.firstDataBlockOffset = firstDataBlockOffset;
529  }
530
531  public String getComparatorClassName() {
532    return comparatorClassName;
533  }
534
535  /**
536   * Returns the major version of this HFile format
537   */
538  public int getMajorVersion() {
539    return majorVersion;
540  }
541
542  /**
543   * Returns the minor version of this HFile format
544   */
545  public int getMinorVersion() {
546    return minorVersion;
547  }
548
549  public void setComparatorClass(Class<? extends CellComparator> klass) {
550    // Is the comparator instantiable?
551    try {
552      // If null, it should be the Bytes.BYTES_RAWCOMPARATOR
553      if (klass != null) {
554        CellComparator comp = klass.getDeclaredConstructor().newInstance();
555        // if the name wasn't one of the legacy names, maybe its a legit new
556        // kind of comparator.
557        this.comparatorClassName = klass.getName();
558      }
559    } catch (Exception e) {
560      throw new RuntimeException("Comparator class " + klass.getName() + " is not instantiable", e);
561    }
562  }
563
564  /**
565   * If a 'standard' Comparator, write the old name for the Comparator when we serialize rather
566   * than the new name; writing the new name will make it so newly-written hfiles are not parseable
567   * by hbase-1.x, a facility we'd like to preserve across rolling upgrade and hbase-1.x clusters
568   * reading hbase-2.x produce.
569   *
570   * The Comparators in hbase-2.x work the same as they did in hbase-1.x; they compare
571   * KeyValues. In hbase-2.x they were renamed making use of the more generic 'Cell'
572   * nomenclature to indicate that we intend to move away from KeyValues post hbase-2. A naming
573   * change is not reason enough to make it so hbase-1.x cannot read hbase-2.x files given the
574   * structure goes unchanged (hfile v3). So, lets write the old names for Comparators into the
575   * hfile tails in hbase-2. Here is where we do the translation.
576   * {@link #getComparatorClass(String)} does translation going the other way.
577   *
578   * <p>The translation is done on the serialized Protobuf only.</p>
579   *
580   * @param comparator String class name of the Comparator used in this hfile.
581   * @return What to store in the trailer as our comparator name.
582   * @since hbase-2.0.0.
583   * @deprecated Since hbase-2.0.0. Will be removed in hbase-3.0.0.
584   * @see #getComparatorClass(String)
585   */
586  @Deprecated
587  private String getHBase1CompatibleName(final String comparator) {
588    if (comparator.equals(CellComparatorImpl.class.getName())) {
589      return KeyValue.COMPARATOR.getClass().getName();
590    }
591    if (comparator.equals(MetaCellComparator.class.getName())) {
592      return KeyValue.META_COMPARATOR.getClass().getName();
593    }
594    return comparator;
595  }
596
597  @SuppressWarnings("unchecked")
598  private static Class<? extends CellComparator> getComparatorClass(String comparatorClassName)
599      throws IOException {
600    Class<? extends CellComparator> comparatorKlass;
601    // for BC
602    if (comparatorClassName.equals(KeyValue.COMPARATOR.getLegacyKeyComparatorName())
603        || comparatorClassName.equals(KeyValue.COMPARATOR.getClass().getName())
604        || (comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator"))) {
605      comparatorKlass = CellComparatorImpl.class;
606    } else if (comparatorClassName.equals(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName())
607        || comparatorClassName.equals(KeyValue.META_COMPARATOR.getClass().getName())
608        || (comparatorClassName
609            .equals("org.apache.hadoop.hbase.CellComparator$MetaCellComparator"))) {
610      comparatorKlass = MetaCellComparator.class;
611    } else if (comparatorClassName.equals("org.apache.hadoop.hbase.KeyValue$RawBytesComparator")
612        || comparatorClassName.equals("org.apache.hadoop.hbase.util.Bytes$ByteArrayComparator")) {
613      // When the comparator to be used is Bytes.BYTES_RAWCOMPARATOR, we just return null from here
614      // Bytes.BYTES_RAWCOMPARATOR is not a CellComparator
615      comparatorKlass = null;
616    } else {
617      // if the name wasn't one of the legacy names, maybe its a legit new kind of comparator.
618      try {
619        comparatorKlass = (Class<? extends CellComparator>) Class.forName(comparatorClassName);
620      } catch (ClassNotFoundException e) {
621        throw new IOException(e);
622      }
623    }
624    return comparatorKlass;
625  }
626
627  public static CellComparator createComparator(
628      String comparatorClassName) throws IOException {
629    try {
630
631      Class<? extends CellComparator> comparatorClass = getComparatorClass(comparatorClassName);
632      if(comparatorClass != null){
633        return comparatorClass.getDeclaredConstructor().newInstance();
634      }
635      LOG.warn("No Comparator class for " + comparatorClassName + ". Returning Null.");
636      return null;
637    } catch (Exception e) {
638      throw new IOException("Comparator class " + comparatorClassName +
639        " is not instantiable", e);
640    }
641  }
642
643  CellComparator createComparator() throws IOException {
644    expectAtLeastMajorVersion(2);
645    return createComparator(comparatorClassName);
646  }
647
648  public long getUncompressedDataIndexSize() {
649    return uncompressedDataIndexSize;
650  }
651
652  public void setUncompressedDataIndexSize(
653      long uncompressedDataIndexSize) {
654    expectAtLeastMajorVersion(2);
655    this.uncompressedDataIndexSize = uncompressedDataIndexSize;
656  }
657
658  public byte[] getEncryptionKey() {
659    // This is a v3 feature but if reading a v2 file the encryptionKey will just be null which
660    // if fine for this feature.
661    expectAtLeastMajorVersion(2);
662    return encryptionKey;
663  }
664
665  public void setEncryptionKey(byte[] keyBytes) {
666    this.encryptionKey = keyBytes;
667  }
668
669  /**
670   * Extracts the major version for a 4-byte serialized version data.
671   * The major version is the 3 least significant bytes
672   */
673  private static int extractMajorVersion(int serializedVersion) {
674    return (serializedVersion & 0x00ffffff);
675  }
676
677  /**
678   * Extracts the minor version for a 4-byte serialized version data.
679   * The major version are the 3 the most significant bytes
680   */
681  private static int extractMinorVersion(int serializedVersion) {
682    return (serializedVersion >>> 24);
683  }
684
685  /**
686   * Create a 4 byte serialized version number by combining the
687   * minor and major version numbers.
688   */
689  static int materializeVersion(int majorVersion, int minorVersion) {
690    return ((majorVersion & 0x00ffffff) | (minorVersion << 24));
691  }
692}