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