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;
019
020import static org.apache.hadoop.hbase.HConstants.EMPTY_BYTE_ARRAY;
021import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
022
023import java.io.DataOutput;
024import java.io.DataOutputStream;
025import java.io.IOException;
026import java.io.OutputStream;
027import java.math.BigDecimal;
028import java.nio.ByteBuffer;
029import java.util.ArrayList;
030import java.util.Iterator;
031import java.util.List;
032import java.util.Optional;
033import org.apache.hadoop.hbase.filter.ByteArrayComparable;
034import org.apache.hadoop.hbase.io.TagCompressionContext;
035import org.apache.hadoop.hbase.io.util.Dictionary;
036import org.apache.hadoop.hbase.io.util.StreamUtils;
037import org.apache.hadoop.hbase.util.ByteBufferUtils;
038import org.apache.hadoop.hbase.util.ByteRange;
039import org.apache.hadoop.hbase.util.Bytes;
040import org.apache.hadoop.hbase.util.ClassSize;
041import org.apache.yetus.audience.InterfaceAudience;
042
043/**
044 * Utility methods helpful slinging {@link Cell} instances. It has more powerful and rich set of
045 * APIs than those in {@link CellUtil} for internal usage.
046 */
047@InterfaceAudience.Private
048public final class PrivateCellUtil {
049
050  /**
051   * Private constructor to keep this class from being instantiated.
052   */
053  private PrivateCellUtil() {
054  }
055
056  /******************* ByteRange *******************************/
057
058  public static ByteRange fillRowRange(Cell cell, ByteRange range) {
059    return range.set(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
060  }
061
062  public static ByteRange fillFamilyRange(Cell cell, ByteRange range) {
063    return range.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
064  }
065
066  public static ByteRange fillQualifierRange(Cell cell, ByteRange range) {
067    return range.set(cell.getQualifierArray(), cell.getQualifierOffset(),
068      cell.getQualifierLength());
069  }
070
071  public static ByteRange fillValueRange(Cell cell, ByteRange range) {
072    return range.set(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
073  }
074
075  public static ByteRange fillTagRange(Cell cell, ByteRange range) {
076    return range.set(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
077  }
078
079  /********************* misc *************************************/
080
081  public static byte getRowByte(Cell cell, int index) {
082    if (cell instanceof ByteBufferExtendedCell) {
083      return ((ByteBufferExtendedCell) cell).getRowByteBuffer()
084        .get(((ByteBufferExtendedCell) cell).getRowPosition() + index);
085    }
086    return cell.getRowArray()[cell.getRowOffset() + index];
087  }
088
089  public static byte getQualifierByte(Cell cell, int index) {
090    if (cell instanceof ByteBufferExtendedCell) {
091      return ((ByteBufferExtendedCell) cell).getQualifierByteBuffer()
092        .get(((ByteBufferExtendedCell) cell).getQualifierPosition() + index);
093    }
094    return cell.getQualifierArray()[cell.getQualifierOffset() + index];
095  }
096
097  public static ByteBuffer getValueBufferShallowCopy(Cell cell) {
098    ByteBuffer buffer =
099      ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
100    return buffer;
101  }
102
103  /** Returns A new cell which is having the extra tags also added to it. */
104  public static Cell createCell(Cell cell, List<Tag> tags) {
105    return createCell(cell, TagUtil.fromList(tags));
106  }
107
108  /** Returns A new cell which is having the extra tags also added to it. */
109  public static Cell createCell(Cell cell, byte[] tags) {
110    if (cell instanceof ByteBufferExtendedCell) {
111      return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) cell, tags);
112    }
113    return new TagRewriteCell(cell, tags);
114  }
115
116  public static Cell createCell(Cell cell, byte[] value, byte[] tags) {
117    if (cell instanceof ByteBufferExtendedCell) {
118      return new ValueAndTagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) cell, value,
119        tags);
120    }
121    return new ValueAndTagRewriteCell(cell, value, tags);
122  }
123
124  /**
125   * This can be used when a Cell has to change with addition/removal of one or more tags. This is
126   * an efficient way to do so in which only the tags bytes part need to recreated and copied. All
127   * other parts, refer to the original Cell.
128   */
129  static class TagRewriteCell implements ExtendedCell {
130    protected Cell cell;
131    protected byte[] tags;
132    private static final int HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
133
134    /**
135     * Construct a TagRewriteCell
136     * @param cell The original Cell which it rewrites
137     * @param tags the tags bytes. The array suppose to contain the tags bytes alone.
138     */
139    public TagRewriteCell(Cell cell, byte[] tags) {
140      assert cell instanceof ExtendedCell;
141      assert tags != null;
142      this.cell = cell;
143      this.tags = tags;
144      // tag offset will be treated as 0 and length this.tags.length
145      if (this.cell instanceof TagRewriteCell) {
146        // Cleaning the ref so that the byte[] can be GCed
147        ((TagRewriteCell) this.cell).tags = null;
148      }
149    }
150
151    @Override
152    public byte[] getRowArray() {
153      return cell.getRowArray();
154    }
155
156    @Override
157    public int getRowOffset() {
158      return cell.getRowOffset();
159    }
160
161    @Override
162    public short getRowLength() {
163      return cell.getRowLength();
164    }
165
166    @Override
167    public byte[] getFamilyArray() {
168      return cell.getFamilyArray();
169    }
170
171    @Override
172    public int getFamilyOffset() {
173      return cell.getFamilyOffset();
174    }
175
176    @Override
177    public byte getFamilyLength() {
178      return cell.getFamilyLength();
179    }
180
181    @Override
182    public byte[] getQualifierArray() {
183      return cell.getQualifierArray();
184    }
185
186    @Override
187    public int getQualifierOffset() {
188      return cell.getQualifierOffset();
189    }
190
191    @Override
192    public int getQualifierLength() {
193      return cell.getQualifierLength();
194    }
195
196    @Override
197    public long getTimestamp() {
198      return cell.getTimestamp();
199    }
200
201    @Override
202    public byte getTypeByte() {
203      return cell.getTypeByte();
204    }
205
206    @Override
207    public long getSequenceId() {
208      return cell.getSequenceId();
209    }
210
211    @Override
212    public byte[] getValueArray() {
213      return cell.getValueArray();
214    }
215
216    @Override
217    public int getValueOffset() {
218      return cell.getValueOffset();
219    }
220
221    @Override
222    public int getValueLength() {
223      return cell.getValueLength();
224    }
225
226    @Override
227    public byte[] getTagsArray() {
228      return this.tags;
229    }
230
231    @Override
232    public int getTagsOffset() {
233      return 0;
234    }
235
236    @Override
237    public int getTagsLength() {
238      if (null == this.tags) {
239        // Nulled out tags array optimization in constructor
240        return 0;
241      }
242      return this.tags.length;
243    }
244
245    @Override
246    public long heapSize() {
247      long sum = HEAP_SIZE_OVERHEAD + cell.heapSize();
248      if (this.tags != null) {
249        sum += ClassSize.sizeOf(this.tags);
250      }
251      return sum;
252    }
253
254    @Override
255    public void setTimestamp(long ts) throws IOException {
256      // The incoming cell is supposed to be ExtendedCell type.
257      PrivateCellUtil.setTimestamp(cell, ts);
258    }
259
260    @Override
261    public void setTimestamp(byte[] ts) throws IOException {
262      // The incoming cell is supposed to be ExtendedCell type.
263      PrivateCellUtil.setTimestamp(cell, ts);
264    }
265
266    @Override
267    public void setSequenceId(long seqId) throws IOException {
268      // The incoming cell is supposed to be ExtendedCell type.
269      PrivateCellUtil.setSequenceId(cell, seqId);
270    }
271
272    @Override
273    public int write(OutputStream out, boolean withTags) throws IOException {
274      int len = ((ExtendedCell) this.cell).write(out, false);
275      if (withTags && this.tags != null) {
276        // Write the tagsLength 2 bytes
277        out.write((byte) (0xff & (this.tags.length >> 8)));
278        out.write((byte) (0xff & this.tags.length));
279        out.write(this.tags);
280        len += KeyValue.TAGS_LENGTH_SIZE + this.tags.length;
281      }
282      return len;
283    }
284
285    @Override
286    public int getSerializedSize(boolean withTags) {
287      int len = ((ExtendedCell) this.cell).getSerializedSize(false);
288      if (withTags && this.tags != null) {
289        len += KeyValue.TAGS_LENGTH_SIZE + this.tags.length;
290      }
291      return len;
292    }
293
294    @Override
295    public void write(ByteBuffer buf, int offset) {
296      offset = KeyValueUtil.appendTo(this.cell, buf, offset, false);
297      int tagsLen = this.tags == null ? 0 : this.tags.length;
298      if (tagsLen > 0) {
299        offset = ByteBufferUtils.putAsShort(buf, offset, tagsLen);
300        ByteBufferUtils.copyFromArrayToBuffer(buf, offset, this.tags, 0, tagsLen);
301      }
302    }
303
304    @Override
305    public ExtendedCell deepClone() {
306      Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
307      return new TagRewriteCell(clonedBaseCell, this.tags);
308    }
309  }
310
311  static class TagRewriteByteBufferExtendedCell extends ByteBufferExtendedCell {
312
313    protected ByteBufferExtendedCell cell;
314    protected byte[] tags;
315    private static final int HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
316
317    /**
318     * @param cell The original ByteBufferExtendedCell which it rewrites
319     * @param tags the tags bytes. The array suppose to contain the tags bytes alone.
320     */
321    public TagRewriteByteBufferExtendedCell(ByteBufferExtendedCell cell, byte[] tags) {
322      assert tags != null;
323      this.cell = cell;
324      this.tags = tags;
325      // tag offset will be treated as 0 and length this.tags.length
326      if (this.cell instanceof TagRewriteByteBufferExtendedCell) {
327        // Cleaning the ref so that the byte[] can be GCed
328        ((TagRewriteByteBufferExtendedCell) this.cell).tags = null;
329      }
330    }
331
332    @Override
333    public byte[] getRowArray() {
334      return this.cell.getRowArray();
335    }
336
337    @Override
338    public int getRowOffset() {
339      return this.cell.getRowOffset();
340    }
341
342    @Override
343    public short getRowLength() {
344      return this.cell.getRowLength();
345    }
346
347    @Override
348    public byte[] getFamilyArray() {
349      return this.cell.getFamilyArray();
350    }
351
352    @Override
353    public int getFamilyOffset() {
354      return this.cell.getFamilyOffset();
355    }
356
357    @Override
358    public byte getFamilyLength() {
359      return this.cell.getFamilyLength();
360    }
361
362    @Override
363    public byte[] getQualifierArray() {
364      return this.cell.getQualifierArray();
365    }
366
367    @Override
368    public int getQualifierOffset() {
369      return this.cell.getQualifierOffset();
370    }
371
372    @Override
373    public int getQualifierLength() {
374      return this.cell.getQualifierLength();
375    }
376
377    @Override
378    public long getTimestamp() {
379      return this.cell.getTimestamp();
380    }
381
382    @Override
383    public byte getTypeByte() {
384      return this.cell.getTypeByte();
385    }
386
387    @Override
388    public long getSequenceId() {
389      return this.cell.getSequenceId();
390    }
391
392    @Override
393    public byte[] getValueArray() {
394      return this.cell.getValueArray();
395    }
396
397    @Override
398    public int getValueOffset() {
399      return this.cell.getValueOffset();
400    }
401
402    @Override
403    public int getValueLength() {
404      return this.cell.getValueLength();
405    }
406
407    @Override
408    public byte[] getTagsArray() {
409      return this.tags;
410    }
411
412    @Override
413    public int getTagsOffset() {
414      return 0;
415    }
416
417    @Override
418    public int getTagsLength() {
419      if (null == this.tags) {
420        // Nulled out tags array optimization in constructor
421        return 0;
422      }
423      return this.tags.length;
424    }
425
426    @Override
427    public void setSequenceId(long seqId) throws IOException {
428      PrivateCellUtil.setSequenceId(this.cell, seqId);
429    }
430
431    @Override
432    public void setTimestamp(long ts) throws IOException {
433      PrivateCellUtil.setTimestamp(this.cell, ts);
434    }
435
436    @Override
437    public void setTimestamp(byte[] ts) throws IOException {
438      PrivateCellUtil.setTimestamp(this.cell, ts);
439    }
440
441    @Override
442    public long heapSize() {
443      long sum = HEAP_SIZE_OVERHEAD + cell.heapSize();
444      // this.tags is on heap byte[]
445      if (this.tags != null) {
446        sum += ClassSize.sizeOf(this.tags);
447      }
448      return sum;
449    }
450
451    @Override
452    public int write(OutputStream out, boolean withTags) throws IOException {
453      int len = ((ExtendedCell) this.cell).write(out, false);
454      if (withTags && this.tags != null) {
455        // Write the tagsLength 2 bytes
456        out.write((byte) (0xff & (this.tags.length >> 8)));
457        out.write((byte) (0xff & this.tags.length));
458        out.write(this.tags);
459        len += KeyValue.TAGS_LENGTH_SIZE + this.tags.length;
460      }
461      return len;
462    }
463
464    @Override
465    public int getSerializedSize(boolean withTags) {
466      int len = ((ExtendedCell) this.cell).getSerializedSize(false);
467      if (withTags && this.tags != null) {
468        len += KeyValue.TAGS_LENGTH_SIZE + this.tags.length;
469      }
470      return len;
471    }
472
473    @Override
474    public void write(ByteBuffer buf, int offset) {
475      offset = KeyValueUtil.appendTo(this.cell, buf, offset, false);
476      int tagsLen = this.tags == null ? 0 : this.tags.length;
477      if (tagsLen > 0) {
478        offset = ByteBufferUtils.putAsShort(buf, offset, tagsLen);
479        ByteBufferUtils.copyFromArrayToBuffer(buf, offset, this.tags, 0, tagsLen);
480      }
481    }
482
483    @Override
484    public ExtendedCell deepClone() {
485      Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
486      if (clonedBaseCell instanceof ByteBufferExtendedCell) {
487        return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) clonedBaseCell,
488          this.tags);
489      }
490      return new TagRewriteCell(clonedBaseCell, this.tags);
491    }
492
493    @Override
494    public ByteBuffer getRowByteBuffer() {
495      return this.cell.getRowByteBuffer();
496    }
497
498    @Override
499    public int getRowPosition() {
500      return this.cell.getRowPosition();
501    }
502
503    @Override
504    public ByteBuffer getFamilyByteBuffer() {
505      return this.cell.getFamilyByteBuffer();
506    }
507
508    @Override
509    public int getFamilyPosition() {
510      return this.cell.getFamilyPosition();
511    }
512
513    @Override
514    public ByteBuffer getQualifierByteBuffer() {
515      return this.cell.getQualifierByteBuffer();
516    }
517
518    @Override
519    public int getQualifierPosition() {
520      return this.cell.getQualifierPosition();
521    }
522
523    @Override
524    public ByteBuffer getValueByteBuffer() {
525      return this.cell.getValueByteBuffer();
526    }
527
528    @Override
529    public int getValuePosition() {
530      return this.cell.getValuePosition();
531    }
532
533    @Override
534    public ByteBuffer getTagsByteBuffer() {
535      return this.tags == null ? HConstants.EMPTY_BYTE_BUFFER : ByteBuffer.wrap(this.tags);
536    }
537
538    @Override
539    public int getTagsPosition() {
540      return 0;
541    }
542  }
543
544  static class ValueAndTagRewriteCell extends TagRewriteCell {
545
546    protected byte[] value;
547
548    public ValueAndTagRewriteCell(Cell cell, byte[] value, byte[] tags) {
549      super(cell, tags);
550      this.value = value;
551    }
552
553    @Override
554    public byte[] getValueArray() {
555      return this.value;
556    }
557
558    @Override
559    public int getValueOffset() {
560      return 0;
561    }
562
563    @Override
564    public int getValueLength() {
565      return this.value == null ? 0 : this.value.length;
566    }
567
568    @Override
569    public long heapSize() {
570      long sum = ClassSize.REFERENCE + super.heapSize();
571      if (this.value != null) {
572        sum += ClassSize.sizeOf(this.value);
573      }
574      return sum;
575    }
576
577    @Override
578    public int write(OutputStream out, boolean withTags) throws IOException {
579      return write(out, withTags, this.cell, this.value, this.tags);
580    }
581
582    /**
583     * Made into a static method so as to reuse the logic within
584     * ValueAndTagRewriteByteBufferExtendedCell
585     */
586    static int write(OutputStream out, boolean withTags, Cell cell, byte[] value, byte[] tags)
587      throws IOException {
588      int valLen = value == null ? 0 : value.length;
589      ByteBufferUtils.putInt(out, KeyValueUtil.keyLength(cell));// Key length
590      ByteBufferUtils.putInt(out, valLen);// Value length
591      int len = 2 * Bytes.SIZEOF_INT;
592      len += writeFlatKey(cell, out);// Key
593      if (valLen > 0) {
594        out.write(value);// Value
595      }
596      len += valLen;
597      if (withTags && tags != null) {
598        // Write the tagsLength 2 bytes
599        out.write((byte) (0xff & (tags.length >> 8)));
600        out.write((byte) (0xff & tags.length));
601        out.write(tags);
602        len += KeyValue.TAGS_LENGTH_SIZE + tags.length;
603      }
604      return len;
605    }
606
607    @Override
608    public int getSerializedSize(boolean withTags) {
609      return super.getSerializedSize(withTags) - this.cell.getValueLength() + this.value.length;
610    }
611
612    @Override
613    public void write(ByteBuffer buf, int offset) {
614      write(buf, offset, this.cell, this.value, this.tags);
615    }
616
617    /**
618     * Made into a static method so as to reuse the logic within
619     * ValueAndTagRewriteByteBufferExtendedCell
620     */
621    static void write(ByteBuffer buf, int offset, Cell cell, byte[] value, byte[] tags) {
622      offset = ByteBufferUtils.putInt(buf, offset, KeyValueUtil.keyLength(cell));// Key length
623      offset = ByteBufferUtils.putInt(buf, offset, value.length);// Value length
624      offset = KeyValueUtil.appendKeyTo(cell, buf, offset);
625      ByteBufferUtils.copyFromArrayToBuffer(buf, offset, value, 0, value.length);
626      offset += value.length;
627      int tagsLen = tags == null ? 0 : tags.length;
628      if (tagsLen > 0) {
629        offset = ByteBufferUtils.putAsShort(buf, offset, tagsLen);
630        ByteBufferUtils.copyFromArrayToBuffer(buf, offset, tags, 0, tagsLen);
631      }
632    }
633
634    @Override
635    public ExtendedCell deepClone() {
636      Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
637      return new ValueAndTagRewriteCell(clonedBaseCell, this.value, this.tags);
638    }
639  }
640
641  static class ValueAndTagRewriteByteBufferExtendedCell extends TagRewriteByteBufferExtendedCell {
642
643    protected byte[] value;
644
645    public ValueAndTagRewriteByteBufferExtendedCell(ByteBufferExtendedCell cell, byte[] value,
646      byte[] tags) {
647      super(cell, tags);
648      this.value = value;
649    }
650
651    @Override
652    public byte[] getValueArray() {
653      return this.value;
654    }
655
656    @Override
657    public int getValueOffset() {
658      return 0;
659    }
660
661    @Override
662    public int getValueLength() {
663      return this.value == null ? 0 : this.value.length;
664    }
665
666    @Override
667    public ByteBuffer getValueByteBuffer() {
668      return ByteBuffer.wrap(this.value);
669    }
670
671    @Override
672    public int getValuePosition() {
673      return 0;
674    }
675
676    @Override
677    public long heapSize() {
678      long sum = ClassSize.REFERENCE + super.heapSize();
679      if (this.value != null) {
680        sum += ClassSize.sizeOf(this.value);
681      }
682      return sum;
683    }
684
685    @Override
686    public int write(OutputStream out, boolean withTags) throws IOException {
687      return ValueAndTagRewriteCell.write(out, withTags, this.cell, this.value, this.tags);
688    }
689
690    @Override
691    public int getSerializedSize(boolean withTags) {
692      return super.getSerializedSize(withTags) - this.cell.getValueLength() + this.value.length;
693    }
694
695    @Override
696    public void write(ByteBuffer buf, int offset) {
697      ValueAndTagRewriteCell.write(buf, offset, this.cell, this.value, this.tags);
698    }
699
700    @Override
701    public ExtendedCell deepClone() {
702      Cell clonedBaseCell = this.cell.deepClone();
703      if (clonedBaseCell instanceof ByteBufferExtendedCell) {
704        return new ValueAndTagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) clonedBaseCell,
705          this.value, this.tags);
706      }
707      return new ValueAndTagRewriteCell(clonedBaseCell, this.value, this.tags);
708    }
709  }
710
711  public static boolean matchingRows(final Cell left, final byte[] buf, final int offset,
712    final int length) {
713    if (left instanceof ByteBufferExtendedCell) {
714      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
715        ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), buf, offset, length);
716    }
717    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
718      length);
719  }
720
721  public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
722    final int length) {
723    if (left instanceof ByteBufferExtendedCell) {
724      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
725        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), buf, offset,
726        length);
727    }
728    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
729      offset, length);
730  }
731
732  /**
733   * Finds if the qualifier part of the cell and the KV serialized byte[] are equal
734   * @param left   the cell with which we need to match the qualifier
735   * @param buf    the serialized keyvalue format byte[]
736   * @param offset the offset of the qualifier in the byte[]
737   * @param length the length of the qualifier in the byte[]
738   * @return true if the qualifier matches, false otherwise
739   */
740  public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
741    final int length) {
742    if (buf == null) {
743      return left.getQualifierLength() == 0;
744    }
745    if (left instanceof ByteBufferExtendedCell) {
746      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
747        ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(), buf,
748        offset, length);
749    }
750    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
751      left.getQualifierLength(), buf, offset, length);
752  }
753
754  /**
755   * Finds if the start of the qualifier part of the Cell matches <code>buf</code>
756   * @param left       the cell with which we need to match the qualifier
757   * @param startsWith the serialized keyvalue format byte[]
758   * @return true if the qualifier have same staring characters, false otherwise
759   */
760  public static boolean qualifierStartsWith(final Cell left, final byte[] startsWith) {
761    if (startsWith == null || startsWith.length == 0) {
762      throw new IllegalArgumentException("Cannot pass an empty startsWith");
763    }
764    if (left.getQualifierLength() < startsWith.length) {
765      return false;
766    }
767    if (left instanceof ByteBufferExtendedCell) {
768      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
769        ((ByteBufferExtendedCell) left).getQualifierPosition(), startsWith.length, startsWith, 0,
770        startsWith.length);
771    }
772    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), startsWith.length,
773      startsWith, 0, startsWith.length);
774  }
775
776  public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
777    final int flength, final byte[] qual, final int qoffset, final int qlength) {
778    if (!matchingFamily(left, fam, foffset, flength)) {
779      return false;
780    }
781    return matchingQualifier(left, qual, qoffset, qlength);
782  }
783
784  public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
785    int rvlength) {
786    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
787      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
788        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
789        ((ByteBufferExtendedCell) right).getValueByteBuffer(),
790        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
791    }
792    if (left instanceof ByteBufferExtendedCell) {
793      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
794        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
795        right.getValueOffset(), rvlength);
796    }
797    if (right instanceof ByteBufferExtendedCell) {
798      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
799        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
800        left.getValueOffset(), lvlength);
801    }
802    return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
803      right.getValueArray(), right.getValueOffset(), rvlength);
804  }
805
806  public static boolean matchingType(Cell a, Cell b) {
807    return a.getTypeByte() == b.getTypeByte();
808  }
809
810  public static boolean matchingTags(final Cell left, final Cell right, int llength, int rlength) {
811    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
812      ByteBufferExtendedCell leftBBCell = (ByteBufferExtendedCell) left;
813      ByteBufferExtendedCell rightBBCell = (ByteBufferExtendedCell) right;
814      return ByteBufferUtils.equals(leftBBCell.getTagsByteBuffer(), leftBBCell.getTagsPosition(),
815        llength, rightBBCell.getTagsByteBuffer(), rightBBCell.getTagsPosition(), rlength);
816    }
817    if (left instanceof ByteBufferExtendedCell) {
818      ByteBufferExtendedCell leftBBCell = (ByteBufferExtendedCell) left;
819      return ByteBufferUtils.equals(leftBBCell.getTagsByteBuffer(), leftBBCell.getTagsPosition(),
820        llength, right.getTagsArray(), right.getTagsOffset(), rlength);
821    }
822    if (right instanceof ByteBufferExtendedCell) {
823      ByteBufferExtendedCell rightBBCell = (ByteBufferExtendedCell) right;
824      return ByteBufferUtils.equals(rightBBCell.getTagsByteBuffer(), rightBBCell.getTagsPosition(),
825        rlength, left.getTagsArray(), left.getTagsOffset(), llength);
826    }
827    return Bytes.equals(left.getTagsArray(), left.getTagsOffset(), llength, right.getTagsArray(),
828      right.getTagsOffset(), rlength);
829  }
830
831  /**
832   * Return true if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
833   * or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
834   */
835  public static boolean isDelete(final byte type) {
836    return KeyValue.Type.Delete.getCode() <= type && type <= KeyValue.Type.DeleteFamily.getCode();
837  }
838
839  /** Returns True if this cell is a {@link KeyValue.Type#Delete} type. */
840  public static boolean isDeleteType(Cell cell) {
841    return cell.getTypeByte() == KeyValue.Type.Delete.getCode();
842  }
843
844  public static boolean isDeleteFamily(final Cell cell) {
845    return cell.getTypeByte() == KeyValue.Type.DeleteFamily.getCode();
846  }
847
848  public static boolean isDeleteFamilyVersion(final Cell cell) {
849    return cell.getTypeByte() == KeyValue.Type.DeleteFamilyVersion.getCode();
850  }
851
852  public static boolean isDeleteColumns(final Cell cell) {
853    return cell.getTypeByte() == KeyValue.Type.DeleteColumn.getCode();
854  }
855
856  public static boolean isDeleteColumnVersion(final Cell cell) {
857    return cell.getTypeByte() == KeyValue.Type.Delete.getCode();
858  }
859
860  /** Returns True if this cell is a delete family or column type. */
861  public static boolean isDeleteColumnOrFamily(Cell cell) {
862    int t = cell.getTypeByte();
863    return t == KeyValue.Type.DeleteColumn.getCode() || t == KeyValue.Type.DeleteFamily.getCode();
864  }
865
866  public static byte[] cloneTags(Cell cell) {
867    byte[] output = new byte[cell.getTagsLength()];
868    copyTagsTo(cell, output, 0);
869    return output;
870  }
871
872  /** Copies the tags info into the tag portion of the cell */
873  public static int copyTagsTo(Cell cell, byte[] destination, int destinationOffset) {
874    int tlen = cell.getTagsLength();
875    if (cell instanceof ByteBufferExtendedCell) {
876      ByteBufferUtils.copyFromBufferToArray(destination,
877        ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
878        ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
879    } else {
880      System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
881        tlen);
882    }
883    return destinationOffset + tlen;
884  }
885
886  /** Copies the tags info into the tag portion of the cell */
887  public static int copyTagsTo(Cell cell, ByteBuffer destination, int destinationOffset) {
888    int tlen = cell.getTagsLength();
889    if (cell instanceof ByteBufferExtendedCell) {
890      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
891        destination, ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
892    } else {
893      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
894        cell.getTagsOffset(), tlen);
895    }
896    return destinationOffset + tlen;
897  }
898
899  /**
900   * Return tags in the given Cell as a List
901   * @param cell The Cell
902   * @return Tags in the given Cell as a List
903   */
904  public static List<Tag> getTags(Cell cell) {
905    List<Tag> tags = new ArrayList<>();
906    Iterator<Tag> tagsItr = tagsIterator(cell);
907    while (tagsItr.hasNext()) {
908      tags.add(tagsItr.next());
909    }
910    return tags;
911  }
912
913  /**
914   * Retrieve Cell's first tag, matching the passed in type
915   * @param cell The Cell
916   * @param type Type of the Tag to retrieve
917   * @return Optional, empty if there is no tag of the passed in tag type
918   */
919  public static Optional<Tag> getTag(Cell cell, byte type) {
920    boolean bufferBacked = cell instanceof ByteBufferExtendedCell;
921    int length = cell.getTagsLength();
922    int offset =
923      bufferBacked ? ((ByteBufferExtendedCell) cell).getTagsPosition() : cell.getTagsOffset();
924    int pos = offset;
925    while (pos < offset + length) {
926      int tagLen;
927      if (bufferBacked) {
928        ByteBuffer tagsBuffer = ((ByteBufferExtendedCell) cell).getTagsByteBuffer();
929        tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
930        if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
931          return Optional.of(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
932        }
933      } else {
934        tagLen = Bytes.readAsInt(cell.getTagsArray(), pos, TAG_LENGTH_SIZE);
935        if (cell.getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
936          return Optional
937            .of(new ArrayBackedTag(cell.getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
938        }
939      }
940      pos += TAG_LENGTH_SIZE + tagLen;
941    }
942    return Optional.empty();
943  }
944
945  /**
946   * Utility method to iterate through the tags in the given cell.
947   * @param cell The Cell over which tags iterator is needed.
948   * @return iterator for the tags
949   */
950  public static Iterator<Tag> tagsIterator(final Cell cell) {
951    final int tagsLength = cell.getTagsLength();
952    // Save an object allocation where we can
953    if (tagsLength == 0) {
954      return TagUtil.EMPTY_TAGS_ITR;
955    }
956    if (cell instanceof ByteBufferExtendedCell) {
957      return tagsIterator(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
958        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
959    }
960    return CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
961  }
962
963  public static Iterator<Tag> tagsIterator(final ByteBuffer tags, final int offset,
964    final int length) {
965    return new Iterator<Tag>() {
966      private int pos = offset;
967      private int endOffset = offset + length - 1;
968
969      @Override
970      public boolean hasNext() {
971        return this.pos < endOffset;
972      }
973
974      @Override
975      public Tag next() {
976        if (hasNext()) {
977          int curTagLen = ByteBufferUtils.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
978          Tag tag = new ByteBufferTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
979          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
980          return tag;
981        }
982        return null;
983      }
984
985      @Override
986      public void remove() {
987        throw new UnsupportedOperationException();
988      }
989    };
990  }
991
992  /**
993   * Returns true if the first range start1...end1 overlaps with the second range start2...end2,
994   * assuming the byte arrays represent row keys
995   */
996  public static boolean overlappingKeys(final byte[] start1, final byte[] end1, final byte[] start2,
997    final byte[] end2) {
998    return (end2.length == 0 || start1.length == 0 || Bytes.compareTo(start1, end2) < 0)
999      && (end1.length == 0 || start2.length == 0 || Bytes.compareTo(start2, end1) < 0);
1000  }
1001
1002  /** Write rowkey excluding the common part. */
1003  public static void writeRowKeyExcludingCommon(Cell cell, short rLen, int commonPrefix,
1004    DataOutputStream out) throws IOException {
1005    if (commonPrefix == 0) {
1006      out.writeShort(rLen);
1007    } else if (commonPrefix == 1) {
1008      out.writeByte((byte) rLen);
1009      commonPrefix--;
1010    } else {
1011      commonPrefix -= KeyValue.ROW_LENGTH_SIZE;
1012    }
1013    if (rLen > commonPrefix) {
1014      writeRowSkippingBytes(out, cell, rLen, commonPrefix);
1015    }
1016  }
1017
1018  /**
1019   * Writes the row from the given cell to the output stream excluding the common prefix
1020   * @param out     The dataoutputstream to which the data has to be written
1021   * @param cell    The cell whose contents has to be written
1022   * @param rlength the row length
1023   */
1024  public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
1025    int commonPrefix) throws IOException {
1026    if (cell instanceof ByteBufferExtendedCell) {
1027      ByteBufferUtils.copyBufferToStream((DataOutput) out,
1028        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1029        ((ByteBufferExtendedCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
1030    } else {
1031      out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
1032    }
1033  }
1034
1035  /**
1036   * Find length of common prefix in keys of the cells, considering key as byte[] if serialized in
1037   * {@link KeyValue}. The key format is &lt;2 bytes rk len&gt;&lt;rk&gt;&lt;1 byte cf
1038   * len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes timestamp&gt;&lt;1 byte type&gt;
1039   * @param c1                the cell
1040   * @param c2                the cell
1041   * @param bypassFamilyCheck when true assume the family bytes same in both cells. Pass it as true
1042   *                          when dealing with Cells in same CF so as to avoid some checks
1043   * @param withTsType        when true check timestamp and type bytes also.
1044   * @return length of common prefix
1045   */
1046  public static int findCommonPrefixInFlatKey(Cell c1, Cell c2, boolean bypassFamilyCheck,
1047    boolean withTsType) {
1048    // Compare the 2 bytes in RK length part
1049    short rLen1 = c1.getRowLength();
1050    short rLen2 = c2.getRowLength();
1051    int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
1052    if (rLen1 != rLen2) {
1053      // early out when the RK length itself is not matching
1054      return ByteBufferUtils.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE,
1055        Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
1056    }
1057    // Compare the RKs
1058    int rkCommonPrefix = 0;
1059    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1060      rkCommonPrefix =
1061        ByteBufferUtils.findCommonPrefix(((ByteBufferExtendedCell) c1).getRowByteBuffer(),
1062          ((ByteBufferExtendedCell) c1).getRowPosition(), rLen1,
1063          ((ByteBufferExtendedCell) c2).getRowByteBuffer(),
1064          ((ByteBufferExtendedCell) c2).getRowPosition(), rLen2);
1065    } else {
1066      // There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes
1067      // either
1068      // in flush or compactions. In flushes both cells are KV and in case of compaction it will be
1069      // either
1070      // KV or BBCell
1071      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1,
1072        c2.getRowArray(), c2.getRowOffset(), rLen2);
1073    }
1074    commonPrefix += rkCommonPrefix;
1075    if (rkCommonPrefix != rLen1) {
1076      // Early out when RK is not fully matching.
1077      return commonPrefix;
1078    }
1079    // Compare 1 byte CF length part
1080    byte fLen1 = c1.getFamilyLength();
1081    if (bypassFamilyCheck) {
1082      // This flag will be true when caller is sure that the family will be same for both the cells
1083      // Just make commonPrefix to increment by the family part
1084      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE + fLen1;
1085    } else {
1086      byte fLen2 = c2.getFamilyLength();
1087      if (fLen1 != fLen2) {
1088        // early out when the CF length itself is not matching
1089        return commonPrefix;
1090      }
1091      // CF lengths are same so there is one more byte common in key part
1092      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
1093      // Compare the CF names
1094      int fCommonPrefix;
1095      if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1096        fCommonPrefix =
1097          ByteBufferUtils.findCommonPrefix(((ByteBufferExtendedCell) c1).getFamilyByteBuffer(),
1098            ((ByteBufferExtendedCell) c1).getFamilyPosition(), fLen1,
1099            ((ByteBufferExtendedCell) c2).getFamilyByteBuffer(),
1100            ((ByteBufferExtendedCell) c2).getFamilyPosition(), fLen2);
1101      } else {
1102        fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(),
1103          fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
1104      }
1105      commonPrefix += fCommonPrefix;
1106      if (fCommonPrefix != fLen1) {
1107        return commonPrefix;
1108      }
1109    }
1110    // Compare the Qualifiers
1111    int qLen1 = c1.getQualifierLength();
1112    int qLen2 = c2.getQualifierLength();
1113    int qCommon;
1114    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1115      qCommon =
1116        ByteBufferUtils.findCommonPrefix(((ByteBufferExtendedCell) c1).getQualifierByteBuffer(),
1117          ((ByteBufferExtendedCell) c1).getQualifierPosition(), qLen1,
1118          ((ByteBufferExtendedCell) c2).getQualifierByteBuffer(),
1119          ((ByteBufferExtendedCell) c2).getQualifierPosition(), qLen2);
1120    } else {
1121      qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
1122        qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
1123    }
1124    commonPrefix += qCommon;
1125    if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
1126      return commonPrefix;
1127    }
1128    // Compare the timestamp parts
1129    int tsCommonPrefix = ByteBufferUtils.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0,
1130      KeyValue.TIMESTAMP_SIZE, Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
1131    commonPrefix += tsCommonPrefix;
1132    if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
1133      return commonPrefix;
1134    }
1135    // Compare the type
1136    if (c1.getTypeByte() == c2.getTypeByte()) {
1137      commonPrefix += KeyValue.TYPE_SIZE;
1138    }
1139    return commonPrefix;
1140  }
1141
1142  /**
1143   * Used to compare two cells based on the column hint provided. This is specifically used when we
1144   * need to optimize the seeks based on the next indexed key. This is an advanced usage API
1145   * specifically needed for some optimizations.
1146   * @param nextIndexedCell the next indexed cell
1147   * @param currentCell     the cell to be compared
1148   * @param foff            the family offset of the currentCell
1149   * @param flen            the family length of the currentCell
1150   * @param colHint         the column hint provided - could be null
1151   * @param coff            the offset of the column hint if provided, if not offset of the
1152   *                        currentCell's qualifier
1153   * @param clen            the length of the column hint if provided, if not length of the
1154   *                        currentCell's qualifier
1155   * @param ts              the timestamp to be seeked
1156   * @param type            the type to be seeked
1157   * @return an int based on the given column hint TODO : To be moved out of here because this is a
1158   *         special API used in scan optimization.
1159   */
1160  // compare a key against row/fam/qual/ts/type
1161  public static final int compareKeyBasedOnColHint(CellComparator comparator, Cell nextIndexedCell,
1162    Cell currentCell, int foff, int flen, byte[] colHint, int coff, int clen, long ts, byte type) {
1163    int compare = comparator.compareRows(nextIndexedCell, currentCell);
1164    if (compare != 0) {
1165      return compare;
1166    }
1167    // If the column is not specified, the "minimum" key type appears the
1168    // latest in the sorted order, regardless of the timestamp. This is used
1169    // for specifying the last key/value in a given row, because there is no
1170    // "lexicographically last column" (it would be infinitely long). The
1171    // "maximum" key type does not need this behavior.
1172    if (
1173      nextIndexedCell.getFamilyLength() + nextIndexedCell.getQualifierLength() == 0
1174        && nextIndexedCell.getTypeByte() == KeyValue.Type.Minimum.getCode()
1175    ) {
1176      // left is "bigger", i.e. it appears later in the sorted order
1177      return 1;
1178    }
1179    if (flen + clen == 0 && type == KeyValue.Type.Minimum.getCode()) {
1180      return -1;
1181    }
1182
1183    compare = comparator.compareFamilies(nextIndexedCell, currentCell);
1184    if (compare != 0) {
1185      return compare;
1186    }
1187    if (colHint == null) {
1188      compare = comparator.compareQualifiers(nextIndexedCell, currentCell);
1189    } else {
1190      compare = CellUtil.compareQualifiers(nextIndexedCell, colHint, coff, clen);
1191    }
1192    if (compare != 0) {
1193      return compare;
1194    }
1195    // Next compare timestamps.
1196    compare = comparator.compareTimestamps(nextIndexedCell.getTimestamp(), ts);
1197    if (compare != 0) {
1198      return compare;
1199    }
1200
1201    // Compare types. Let the delete types sort ahead of puts; i.e. types
1202    // of higher numbers sort before those of lesser numbers. Maximum (255)
1203    // appears ahead of everything, and minimum (0) appears after
1204    // everything.
1205    return (0xff & type) - (0xff & nextIndexedCell.getTypeByte());
1206  }
1207
1208  /**
1209   * Compares only the key portion of a cell. It does not include the sequence id/mvcc of the cell
1210   * @return an int greater than 0 if left &gt; than right lesser than 0 if left &lt; than right
1211   *         equal to 0 if left is equal to right
1212   */
1213  public static final int compareKeyIgnoresMvcc(CellComparator comparator, Cell left, Cell right) {
1214    return ((CellComparatorImpl) comparator).compare(left, right, true);
1215  }
1216
1217  /**
1218   * Compare cell's row against given comparator
1219   * @param cell       the cell to use for comparison
1220   * @param comparator the {@link CellComparator} to use for comparison
1221   * @return result comparing cell's row
1222   */
1223  public static int compareRow(Cell cell, ByteArrayComparable comparator) {
1224    if (cell instanceof ByteBufferExtendedCell) {
1225      return comparator.compareTo(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1226        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
1227    }
1228    return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
1229  }
1230
1231  /**
1232   * Compare cell's column family against given comparator
1233   * @param cell       the cell to use for comparison
1234   * @param comparator the {@link CellComparator} to use for comparison
1235   * @return result comparing cell's column family
1236   */
1237  public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
1238    if (cell instanceof ByteBufferExtendedCell) {
1239      return comparator.compareTo(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
1240        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength());
1241    }
1242    return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
1243      cell.getFamilyLength());
1244  }
1245
1246  /**
1247   * Compare cell's qualifier against given comparator
1248   * @param cell       the cell to use for comparison
1249   * @param comparator the {@link CellComparator} to use for comparison
1250   * @return result comparing cell's qualifier
1251   */
1252  public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
1253    if (cell instanceof ByteBufferExtendedCell) {
1254      return comparator.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
1255        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
1256    }
1257    return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
1258      cell.getQualifierLength());
1259  }
1260
1261  public static Cell.Type toType(byte type) {
1262    KeyValue.Type codeToType = KeyValue.Type.codeToType(type);
1263    switch (codeToType) {
1264      case Put:
1265        return Cell.Type.Put;
1266      case Delete:
1267        return Cell.Type.Delete;
1268      case DeleteColumn:
1269        return Cell.Type.DeleteColumn;
1270      case DeleteFamily:
1271        return Cell.Type.DeleteFamily;
1272      case DeleteFamilyVersion:
1273        return Cell.Type.DeleteFamilyVersion;
1274      default:
1275        throw new UnsupportedOperationException("Invalid type of cell " + type);
1276    }
1277  }
1278
1279  public static KeyValue.Type toTypeByte(Cell.Type type) {
1280    switch (type) {
1281      case Put:
1282        return KeyValue.Type.Put;
1283      case Delete:
1284        return KeyValue.Type.Delete;
1285      case DeleteColumn:
1286        return KeyValue.Type.DeleteColumn;
1287      case DeleteFamilyVersion:
1288        return KeyValue.Type.DeleteFamilyVersion;
1289      case DeleteFamily:
1290        return KeyValue.Type.DeleteFamily;
1291      default:
1292        throw new UnsupportedOperationException("Unsupported data type:" + type);
1293    }
1294  }
1295
1296  /**
1297   * Compare cell's value against given comparator
1298   * @param cell       the cell to use for comparison
1299   * @param comparator the {@link CellComparator} to use for comparison
1300   * @return result comparing cell's value
1301   */
1302  public static int compareValue(Cell cell, ByteArrayComparable comparator) {
1303    if (cell instanceof ByteBufferExtendedCell) {
1304      return comparator.compareTo(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
1305        ((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
1306    }
1307    return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
1308  }
1309
1310  /**
1311   * These cells are used in reseeks/seeks to improve the read performance. They are not real cells
1312   * that are returned back to the clients
1313   */
1314  private static abstract class EmptyCell implements ExtendedCell {
1315
1316    @Override
1317    public void setSequenceId(long seqId) {
1318      // Fake cells don't need seqId, so leaving it as a noop.
1319    }
1320
1321    @Override
1322    public void setTimestamp(long ts) {
1323      // Fake cells can't be changed timestamp, so leaving it as a noop.
1324    }
1325
1326    @Override
1327    public void setTimestamp(byte[] ts) {
1328      // Fake cells can't be changed timestamp, so leaving it as a noop.
1329    }
1330
1331    @Override
1332    public byte[] getRowArray() {
1333      return EMPTY_BYTE_ARRAY;
1334    }
1335
1336    @Override
1337    public int getRowOffset() {
1338      return 0;
1339    }
1340
1341    @Override
1342    public short getRowLength() {
1343      return 0;
1344    }
1345
1346    @Override
1347    public byte[] getFamilyArray() {
1348      return EMPTY_BYTE_ARRAY;
1349    }
1350
1351    @Override
1352    public int getFamilyOffset() {
1353      return 0;
1354    }
1355
1356    @Override
1357    public byte getFamilyLength() {
1358      return 0;
1359    }
1360
1361    @Override
1362    public byte[] getQualifierArray() {
1363      return EMPTY_BYTE_ARRAY;
1364    }
1365
1366    @Override
1367    public int getQualifierOffset() {
1368      return 0;
1369    }
1370
1371    @Override
1372    public int getQualifierLength() {
1373      return 0;
1374    }
1375
1376    @Override
1377    public long getSequenceId() {
1378      return 0;
1379    }
1380
1381    @Override
1382    public byte[] getValueArray() {
1383      return EMPTY_BYTE_ARRAY;
1384    }
1385
1386    @Override
1387    public int getValueOffset() {
1388      return 0;
1389    }
1390
1391    @Override
1392    public int getValueLength() {
1393      return 0;
1394    }
1395
1396    @Override
1397    public byte[] getTagsArray() {
1398      return EMPTY_BYTE_ARRAY;
1399    }
1400
1401    @Override
1402    public int getTagsOffset() {
1403      return 0;
1404    }
1405
1406    @Override
1407    public int getTagsLength() {
1408      return 0;
1409    }
1410  }
1411
1412  /**
1413   * These cells are used in reseeks/seeks to improve the read performance. They are not real cells
1414   * that are returned back to the clients
1415   */
1416  private static abstract class EmptyByteBufferExtendedCell extends ByteBufferExtendedCell {
1417
1418    @Override
1419    public void setSequenceId(long seqId) {
1420      // Fake cells don't need seqId, so leaving it as a noop.
1421    }
1422
1423    @Override
1424    public void setTimestamp(long ts) {
1425      // Fake cells can't be changed timestamp, so leaving it as a noop.
1426    }
1427
1428    @Override
1429    public void setTimestamp(byte[] ts) {
1430      // Fake cells can't be changed timestamp, so leaving it as a noop.
1431    }
1432
1433    @Override
1434    public byte[] getRowArray() {
1435      return CellUtil.cloneRow(this);
1436    }
1437
1438    @Override
1439    public int getRowOffset() {
1440      return 0;
1441    }
1442
1443    @Override
1444    public short getRowLength() {
1445      return 0;
1446    }
1447
1448    @Override
1449    public byte[] getFamilyArray() {
1450      return CellUtil.cloneFamily(this);
1451    }
1452
1453    @Override
1454    public int getFamilyOffset() {
1455      return 0;
1456    }
1457
1458    @Override
1459    public byte getFamilyLength() {
1460      return 0;
1461    }
1462
1463    @Override
1464    public byte[] getQualifierArray() {
1465      return CellUtil.cloneQualifier(this);
1466    }
1467
1468    @Override
1469    public int getQualifierOffset() {
1470      return 0;
1471    }
1472
1473    @Override
1474    public int getQualifierLength() {
1475      return 0;
1476    }
1477
1478    @Override
1479    public long getSequenceId() {
1480      return 0;
1481    }
1482
1483    @Override
1484    public byte[] getValueArray() {
1485      return CellUtil.cloneValue(this);
1486    }
1487
1488    @Override
1489    public int getValueOffset() {
1490      return 0;
1491    }
1492
1493    @Override
1494    public int getValueLength() {
1495      return 0;
1496    }
1497
1498    @Override
1499    public byte[] getTagsArray() {
1500      return CellUtil.cloneTags(this);
1501    }
1502
1503    @Override
1504    public int getTagsOffset() {
1505      return 0;
1506    }
1507
1508    @Override
1509    public int getTagsLength() {
1510      return 0;
1511    }
1512
1513    @Override
1514    public ByteBuffer getRowByteBuffer() {
1515      return HConstants.EMPTY_BYTE_BUFFER;
1516    }
1517
1518    @Override
1519    public int getRowPosition() {
1520      return 0;
1521    }
1522
1523    @Override
1524    public ByteBuffer getFamilyByteBuffer() {
1525      return HConstants.EMPTY_BYTE_BUFFER;
1526    }
1527
1528    @Override
1529    public int getFamilyPosition() {
1530      return 0;
1531    }
1532
1533    @Override
1534    public ByteBuffer getQualifierByteBuffer() {
1535      return HConstants.EMPTY_BYTE_BUFFER;
1536    }
1537
1538    @Override
1539    public int getQualifierPosition() {
1540      return 0;
1541    }
1542
1543    @Override
1544    public ByteBuffer getTagsByteBuffer() {
1545      return HConstants.EMPTY_BYTE_BUFFER;
1546    }
1547
1548    @Override
1549    public int getTagsPosition() {
1550      return 0;
1551    }
1552
1553    @Override
1554    public ByteBuffer getValueByteBuffer() {
1555      return HConstants.EMPTY_BYTE_BUFFER;
1556    }
1557
1558    @Override
1559    public int getValuePosition() {
1560      return 0;
1561    }
1562  }
1563
1564  private static class FirstOnRowCell extends EmptyCell {
1565    // @formatter:off
1566    private static final int FIXED_HEAPSIZE =
1567        ClassSize.OBJECT // object
1568      + ClassSize.REFERENCE // row array
1569      + Bytes.SIZEOF_INT // row offset
1570      + Bytes.SIZEOF_SHORT;  // row length
1571    // @formatter:on
1572    private final byte[] rowArray;
1573    private final int roffset;
1574    private final short rlength;
1575
1576    public FirstOnRowCell(final byte[] row, int roffset, short rlength) {
1577      this.rowArray = row;
1578      this.roffset = roffset;
1579      this.rlength = rlength;
1580    }
1581
1582    @Override
1583    public long heapSize() {
1584      return ClassSize.align(FIXED_HEAPSIZE)
1585        // array overhead
1586        + (rlength == 0 ? ClassSize.sizeOfByteArray(rlength) : rlength);
1587    }
1588
1589    @Override
1590    public byte[] getRowArray() {
1591      return this.rowArray;
1592    }
1593
1594    @Override
1595    public int getRowOffset() {
1596      return this.roffset;
1597    }
1598
1599    @Override
1600    public short getRowLength() {
1601      return this.rlength;
1602    }
1603
1604    @Override
1605    public long getTimestamp() {
1606      return HConstants.LATEST_TIMESTAMP;
1607    }
1608
1609    @Override
1610    public byte getTypeByte() {
1611      return KeyValue.Type.Maximum.getCode();
1612    }
1613
1614    @Override
1615    public Type getType() {
1616      throw new UnsupportedOperationException();
1617    }
1618  }
1619
1620  private static class FirstOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
1621    // @formatter:off
1622    private static final int FIXED_OVERHEAD =
1623        ClassSize.OBJECT // object
1624        + ClassSize.REFERENCE // row buffer
1625        + Bytes.SIZEOF_INT // row offset
1626        + Bytes.SIZEOF_SHORT; // row length
1627    // @formatter:on
1628    private final ByteBuffer rowBuff;
1629    private final int roffset;
1630    private final short rlength;
1631
1632    public FirstOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
1633      this.rowBuff = row;
1634      this.roffset = roffset;
1635      this.rlength = rlength;
1636    }
1637
1638    @Override
1639    public long heapSize() {
1640      if (this.rowBuff.hasArray()) {
1641        return ClassSize.align(FIXED_OVERHEAD + rlength);
1642      }
1643      return ClassSize.align(FIXED_OVERHEAD);
1644    }
1645
1646    @Override
1647    public ByteBuffer getRowByteBuffer() {
1648      return this.rowBuff;
1649    }
1650
1651    @Override
1652    public int getRowPosition() {
1653      return this.roffset;
1654    }
1655
1656    @Override
1657    public short getRowLength() {
1658      return this.rlength;
1659    }
1660
1661    @Override
1662    public long getTimestamp() {
1663      return HConstants.LATEST_TIMESTAMP;
1664    }
1665
1666    @Override
1667    public byte getTypeByte() {
1668      return KeyValue.Type.Maximum.getCode();
1669    }
1670
1671    @Override
1672    public Type getType() {
1673      throw new UnsupportedOperationException();
1674    }
1675  }
1676
1677  private static class LastOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
1678    // @formatter:off
1679    private static final int FIXED_OVERHEAD = ClassSize.OBJECT // object
1680      + ClassSize.REFERENCE // rowBuff
1681      + Bytes.SIZEOF_INT // roffset
1682      + Bytes.SIZEOF_SHORT; // rlength
1683    // @formatter:on
1684    private final ByteBuffer rowBuff;
1685    private final int roffset;
1686    private final short rlength;
1687
1688    public LastOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
1689      this.rowBuff = row;
1690      this.roffset = roffset;
1691      this.rlength = rlength;
1692    }
1693
1694    @Override
1695    public long heapSize() {
1696      if (this.rowBuff.hasArray()) {
1697        return ClassSize.align(FIXED_OVERHEAD + rlength);
1698      }
1699      return ClassSize.align(FIXED_OVERHEAD);
1700    }
1701
1702    @Override
1703    public ByteBuffer getRowByteBuffer() {
1704      return this.rowBuff;
1705    }
1706
1707    @Override
1708    public int getRowPosition() {
1709      return this.roffset;
1710    }
1711
1712    @Override
1713    public short getRowLength() {
1714      return this.rlength;
1715    }
1716
1717    @Override
1718    public long getTimestamp() {
1719      return HConstants.OLDEST_TIMESTAMP;
1720    }
1721
1722    @Override
1723    public byte getTypeByte() {
1724      return KeyValue.Type.Minimum.getCode();
1725    }
1726
1727    @Override
1728    public Type getType() {
1729      throw new UnsupportedOperationException();
1730    }
1731  }
1732
1733  private static class FirstOnRowColByteBufferExtendedCell
1734    extends FirstOnRowByteBufferExtendedCell {
1735    // @formatter:off
1736    private static final int FIXED_OVERHEAD = FirstOnRowByteBufferExtendedCell.FIXED_OVERHEAD
1737      + ClassSize.REFERENCE * 2 // family buffer and column buffer
1738      + Bytes.SIZEOF_INT * 3 // famOffset, colOffset, colLength
1739      + Bytes.SIZEOF_BYTE; // famLength
1740    // @formatter:on
1741    private final ByteBuffer famBuff;
1742    private final int famOffset;
1743    private final byte famLength;
1744    private final ByteBuffer colBuff;
1745    private final int colOffset;
1746    private final int colLength;
1747
1748    public FirstOnRowColByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength,
1749      final ByteBuffer famBuff, final int famOffset, final byte famLength, final ByteBuffer col,
1750      final int colOffset, final int colLength) {
1751      super(row, roffset, rlength);
1752      this.famBuff = famBuff;
1753      this.famOffset = famOffset;
1754      this.famLength = famLength;
1755      this.colBuff = col;
1756      this.colOffset = colOffset;
1757      this.colLength = colLength;
1758    }
1759
1760    @Override
1761    public long heapSize() {
1762      if (famBuff.hasArray() && colBuff.hasArray()) {
1763        return ClassSize.align(FIXED_OVERHEAD + famLength + colLength);
1764      } else if (famBuff.hasArray()) {
1765        return ClassSize.align(FIXED_OVERHEAD + famLength);
1766      } else if (colBuff.hasArray()) {
1767        return ClassSize.align(FIXED_OVERHEAD + colLength);
1768      } else {
1769        return ClassSize.align(FIXED_OVERHEAD);
1770      }
1771    }
1772
1773    @Override
1774    public ByteBuffer getFamilyByteBuffer() {
1775      return this.famBuff;
1776    }
1777
1778    @Override
1779    public int getFamilyPosition() {
1780      return this.famOffset;
1781    }
1782
1783    @Override
1784    public byte getFamilyLength() {
1785      return famLength;
1786    }
1787
1788    @Override
1789    public ByteBuffer getQualifierByteBuffer() {
1790      return this.colBuff;
1791    }
1792
1793    @Override
1794    public int getQualifierPosition() {
1795      return this.colOffset;
1796    }
1797
1798    @Override
1799    public int getQualifierLength() {
1800      return this.colLength;
1801    }
1802  }
1803
1804  private static class FirstOnRowColCell extends FirstOnRowCell {
1805    // @formatter:off
1806    private static final long FIXED_HEAPSIZE = (long) FirstOnRowCell.FIXED_HEAPSIZE
1807      + Bytes.SIZEOF_BYTE // flength
1808      + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
1809      + ClassSize.REFERENCE * 2; // fArray, qArray
1810    // @formatter:on
1811    private final byte[] fArray;
1812    private final int foffset;
1813    private final byte flength;
1814    private final byte[] qArray;
1815    private final int qoffset;
1816    private final int qlength;
1817
1818    public FirstOnRowColCell(byte[] rArray, int roffset, short rlength, byte[] fArray, int foffset,
1819      byte flength, byte[] qArray, int qoffset, int qlength) {
1820      super(rArray, roffset, rlength);
1821      this.fArray = fArray;
1822      this.foffset = foffset;
1823      this.flength = flength;
1824      this.qArray = qArray;
1825      this.qoffset = qoffset;
1826      this.qlength = qlength;
1827    }
1828
1829    @Override
1830    public long heapSize() {
1831      return ClassSize.align(FIXED_HEAPSIZE)
1832        // array overhead
1833        + (flength == 0 ? ClassSize.sizeOfByteArray(flength) : flength)
1834        + (qlength == 0 ? ClassSize.sizeOfByteArray(qlength) : qlength);
1835    }
1836
1837    @Override
1838    public byte[] getFamilyArray() {
1839      return this.fArray;
1840    }
1841
1842    @Override
1843    public int getFamilyOffset() {
1844      return this.foffset;
1845    }
1846
1847    @Override
1848    public byte getFamilyLength() {
1849      return this.flength;
1850    }
1851
1852    @Override
1853    public byte[] getQualifierArray() {
1854      return this.qArray;
1855    }
1856
1857    @Override
1858    public int getQualifierOffset() {
1859      return this.qoffset;
1860    }
1861
1862    @Override
1863    public int getQualifierLength() {
1864      return this.qlength;
1865    }
1866  }
1867
1868  private static class FirstOnRowColTSCell extends FirstOnRowColCell {
1869    // @formatter:off
1870    private static final long FIXED_HEAPSIZE = FirstOnRowColCell.FIXED_HEAPSIZE
1871      + Bytes.SIZEOF_LONG; // ts
1872    private long ts;
1873    // @formatter:on
1874
1875    public FirstOnRowColTSCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
1876      int foffset, byte flength, byte[] qArray, int qoffset, int qlength, long ts) {
1877      super(rArray, roffset, rlength, fArray, foffset, flength, qArray, qoffset, qlength);
1878      this.ts = ts;
1879    }
1880
1881    @Override
1882    public long getTimestamp() {
1883      return this.ts;
1884    }
1885
1886    @Override
1887    public long heapSize() {
1888      return ClassSize.align(FIXED_HEAPSIZE);
1889    }
1890  }
1891
1892  private static class FirstOnRowColTSByteBufferExtendedCell
1893    extends FirstOnRowColByteBufferExtendedCell {
1894    // @formatter:off
1895    private static final int FIXED_OVERHEAD = FirstOnRowColByteBufferExtendedCell.FIXED_OVERHEAD
1896      + Bytes.SIZEOF_LONG; // ts
1897    private long ts;
1898    // @formatter:on
1899
1900    public FirstOnRowColTSByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
1901      ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset, int qlength,
1902      long ts) {
1903      super(rBuffer, roffset, rlength, fBuffer, foffset, flength, qBuffer, qoffset, qlength);
1904      this.ts = ts;
1905    }
1906
1907    @Override
1908    public long getTimestamp() {
1909      return this.ts;
1910    }
1911
1912    @Override
1913    public long heapSize() {
1914      return ClassSize.align(FIXED_OVERHEAD + super.heapSize());
1915    }
1916  }
1917
1918  private static class LastOnRowCell extends EmptyCell {
1919    // @formatter:off
1920    private static final int FIXED_OVERHEAD = ClassSize.OBJECT // object
1921      + ClassSize.REFERENCE // row array
1922      + Bytes.SIZEOF_INT // row offset
1923      + Bytes.SIZEOF_SHORT; // row length
1924    // @formatter:on
1925    private final byte[] rowArray;
1926    private final int roffset;
1927    private final short rlength;
1928
1929    public LastOnRowCell(byte[] row, int roffset, short rlength) {
1930      this.rowArray = row;
1931      this.roffset = roffset;
1932      this.rlength = rlength;
1933    }
1934
1935    @Override
1936    public long heapSize() {
1937      return ClassSize.align(FIXED_OVERHEAD)
1938        // array overhead
1939        + (rlength == 0 ? ClassSize.sizeOfByteArray(rlength) : rlength);
1940    }
1941
1942    @Override
1943    public byte[] getRowArray() {
1944      return this.rowArray;
1945    }
1946
1947    @Override
1948    public int getRowOffset() {
1949      return this.roffset;
1950    }
1951
1952    @Override
1953    public short getRowLength() {
1954      return this.rlength;
1955    }
1956
1957    @Override
1958    public long getTimestamp() {
1959      return HConstants.OLDEST_TIMESTAMP;
1960    }
1961
1962    @Override
1963    public byte getTypeByte() {
1964      return KeyValue.Type.Minimum.getCode();
1965    }
1966
1967    @Override
1968    public Type getType() {
1969      throw new UnsupportedOperationException();
1970    }
1971  }
1972
1973  private static class LastOnRowColCell extends LastOnRowCell {
1974    // @formatter:off
1975    private static final long FIXED_OVERHEAD = (long) LastOnRowCell.FIXED_OVERHEAD
1976      + ClassSize.REFERENCE * 2 // fArray and qArray
1977      + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
1978      + Bytes.SIZEOF_BYTE; // flength
1979    // @formatter:on
1980    private final byte[] fArray;
1981    private final int foffset;
1982    private final byte flength;
1983    private final byte[] qArray;
1984    private final int qoffset;
1985    private final int qlength;
1986
1987    public LastOnRowColCell(byte[] rArray, int roffset, short rlength, byte[] fArray, int foffset,
1988      byte flength, byte[] qArray, int qoffset, int qlength) {
1989      super(rArray, roffset, rlength);
1990      this.fArray = fArray;
1991      this.foffset = foffset;
1992      this.flength = flength;
1993      this.qArray = qArray;
1994      this.qoffset = qoffset;
1995      this.qlength = qlength;
1996    }
1997
1998    @Override
1999    public long heapSize() {
2000      return ClassSize.align(FIXED_OVERHEAD)
2001        // array overhead
2002        + (flength == 0 ? ClassSize.sizeOfByteArray(flength) : flength)
2003        + (qlength == 0 ? ClassSize.sizeOfByteArray(qlength) : qlength);
2004    }
2005
2006    @Override
2007    public byte[] getFamilyArray() {
2008      return this.fArray;
2009    }
2010
2011    @Override
2012    public int getFamilyOffset() {
2013      return this.foffset;
2014    }
2015
2016    @Override
2017    public byte getFamilyLength() {
2018      return this.flength;
2019    }
2020
2021    @Override
2022    public byte[] getQualifierArray() {
2023      return this.qArray;
2024    }
2025
2026    @Override
2027    public int getQualifierOffset() {
2028      return this.qoffset;
2029    }
2030
2031    @Override
2032    public int getQualifierLength() {
2033      return this.qlength;
2034    }
2035  }
2036
2037  private static class LastOnRowColByteBufferExtendedCell extends LastOnRowByteBufferExtendedCell {
2038    // @formatter:off
2039    private static final int FIXED_OVERHEAD = LastOnRowByteBufferExtendedCell.FIXED_OVERHEAD
2040      + ClassSize.REFERENCE * 2 // fBuffer and qBuffer
2041      + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
2042      + Bytes.SIZEOF_BYTE; // flength
2043    // @formatter:on
2044    private final ByteBuffer fBuffer;
2045    private final int foffset;
2046    private final byte flength;
2047    private final ByteBuffer qBuffer;
2048    private final int qoffset;
2049    private final int qlength;
2050
2051    public LastOnRowColByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
2052      ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset, int qlength) {
2053      super(rBuffer, roffset, rlength);
2054      this.fBuffer = fBuffer;
2055      this.foffset = foffset;
2056      this.flength = flength;
2057      this.qBuffer = qBuffer;
2058      this.qoffset = qoffset;
2059      this.qlength = qlength;
2060    }
2061
2062    @Override
2063    public long heapSize() {
2064      if (fBuffer.hasArray() && qBuffer.hasArray()) {
2065        return ClassSize.align(FIXED_OVERHEAD + flength + qlength);
2066      } else if (fBuffer.hasArray()) {
2067        return ClassSize.align(FIXED_OVERHEAD + flength);
2068      } else if (qBuffer.hasArray()) {
2069        return ClassSize.align(FIXED_OVERHEAD + qlength);
2070      } else {
2071        return ClassSize.align(FIXED_OVERHEAD);
2072      }
2073    }
2074
2075    @Override
2076    public ByteBuffer getFamilyByteBuffer() {
2077      return this.fBuffer;
2078    }
2079
2080    @Override
2081    public int getFamilyPosition() {
2082      return this.foffset;
2083    }
2084
2085    @Override
2086    public byte getFamilyLength() {
2087      return this.flength;
2088    }
2089
2090    @Override
2091    public ByteBuffer getQualifierByteBuffer() {
2092      return this.qBuffer;
2093    }
2094
2095    @Override
2096    public int getQualifierPosition() {
2097      return this.qoffset;
2098    }
2099
2100    @Override
2101    public int getQualifierLength() {
2102      return this.qlength;
2103    }
2104  }
2105
2106  private static class FirstOnRowDeleteFamilyCell extends EmptyCell {
2107    // @formatter:off
2108    private static final int FIXED_OVERHEAD = ClassSize.OBJECT // object
2109      + ClassSize.REFERENCE * 2 // fBuffer and qBuffer
2110      + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
2111      + Bytes.SIZEOF_BYTE; // flength
2112    // @formatter:on
2113    private final byte[] row;
2114    private final byte[] fam;
2115
2116    public FirstOnRowDeleteFamilyCell(byte[] row, byte[] fam) {
2117      this.row = row;
2118      this.fam = fam;
2119    }
2120
2121    @Override
2122    public long heapSize() {
2123      return ClassSize.align(FIXED_OVERHEAD)
2124        // array overhead
2125        + (getRowLength() == 0 ? ClassSize.sizeOfByteArray(getRowLength()) : getRowLength())
2126        + (getFamilyLength() == 0
2127          ? ClassSize.sizeOfByteArray(getFamilyLength())
2128          : getFamilyLength());
2129    }
2130
2131    @Override
2132    public byte[] getRowArray() {
2133      return this.row;
2134    }
2135
2136    @Override
2137    public short getRowLength() {
2138      return (short) this.row.length;
2139    }
2140
2141    @Override
2142    public byte[] getFamilyArray() {
2143      return this.fam;
2144    }
2145
2146    @Override
2147    public byte getFamilyLength() {
2148      return (byte) this.fam.length;
2149    }
2150
2151    @Override
2152    public long getTimestamp() {
2153      return HConstants.LATEST_TIMESTAMP;
2154    }
2155
2156    @Override
2157    public byte getTypeByte() {
2158      return KeyValue.Type.DeleteFamily.getCode();
2159    }
2160
2161    @Override
2162    public Type getType() {
2163      return Type.DeleteFamily;
2164    }
2165  }
2166
2167  /**
2168   * Writes the Cell's key part as it would have serialized in a KeyValue. The format is &lt;2 bytes
2169   * rk len&gt;&lt;rk&gt;&lt;1 byte cf len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes
2170   * timestamp&gt;&lt;1 byte type&gt;
2171   */
2172  public static void writeFlatKey(Cell cell, DataOutput out) throws IOException {
2173    short rowLen = cell.getRowLength();
2174    byte fLen = cell.getFamilyLength();
2175    int qLen = cell.getQualifierLength();
2176    // Using just one if/else loop instead of every time checking before writing every
2177    // component of cell
2178    if (cell instanceof ByteBufferExtendedCell) {
2179      out.writeShort(rowLen);
2180      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2181        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
2182      out.writeByte(fLen);
2183      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2184        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
2185      ByteBufferUtils.copyBufferToStream(out,
2186        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2187        ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
2188    } else {
2189      out.writeShort(rowLen);
2190      out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
2191      out.writeByte(fLen);
2192      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
2193      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen);
2194    }
2195    out.writeLong(cell.getTimestamp());
2196    out.writeByte(cell.getTypeByte());
2197  }
2198
2199  /**
2200   * Deep clones the given cell if the cell supports deep cloning
2201   * @param cell the cell to be cloned
2202   * @return the cloned cell
2203   */
2204  public static Cell deepClone(Cell cell) throws CloneNotSupportedException {
2205    if (cell instanceof ExtendedCell) {
2206      return ((ExtendedCell) cell).deepClone();
2207    }
2208    throw new CloneNotSupportedException();
2209  }
2210
2211  /**
2212   * Writes the cell to the given OutputStream
2213   * @param cell     the cell to be written
2214   * @param out      the outputstream
2215   * @param withTags if tags are to be written or not
2216   * @return the total bytes written
2217   */
2218  public static int writeCell(Cell cell, OutputStream out, boolean withTags) throws IOException {
2219    if (cell instanceof ExtendedCell) {
2220      return ((ExtendedCell) cell).write(out, withTags);
2221    } else {
2222      ByteBufferUtils.putInt(out, estimatedSerializedSizeOfKey(cell));
2223      ByteBufferUtils.putInt(out, cell.getValueLength());
2224      writeFlatKey(cell, out);
2225      writeValue(out, cell, cell.getValueLength());
2226      int tagsLength = cell.getTagsLength();
2227      if (withTags) {
2228        byte[] len = new byte[Bytes.SIZEOF_SHORT];
2229        Bytes.putAsShort(len, 0, tagsLength);
2230        out.write(len);
2231        if (tagsLength > 0) {
2232          writeTags(out, cell, tagsLength);
2233        }
2234      }
2235      int lenWritten =
2236        (2 * Bytes.SIZEOF_INT) + estimatedSerializedSizeOfKey(cell) + cell.getValueLength();
2237      if (withTags) {
2238        lenWritten += Bytes.SIZEOF_SHORT + tagsLength;
2239      }
2240      return lenWritten;
2241    }
2242  }
2243
2244  /**
2245   * Writes a cell to the buffer at the given offset
2246   * @param cell   the cell to be written
2247   * @param buf    the buffer to which the cell has to be wrriten
2248   * @param offset the offset at which the cell should be written
2249   */
2250  public static void writeCellToBuffer(Cell cell, ByteBuffer buf, int offset) {
2251    if (cell instanceof ExtendedCell) {
2252      ((ExtendedCell) cell).write(buf, offset);
2253    } else {
2254      // Using the KVUtil
2255      byte[] bytes = KeyValueUtil.copyToNewByteArray(cell);
2256      ByteBufferUtils.copyFromArrayToBuffer(buf, offset, bytes, 0, bytes.length);
2257    }
2258  }
2259
2260  public static int writeFlatKey(Cell cell, OutputStream out) throws IOException {
2261    short rowLen = cell.getRowLength();
2262    byte fLen = cell.getFamilyLength();
2263    int qLen = cell.getQualifierLength();
2264    // Using just one if/else loop instead of every time checking before writing every
2265    // component of cell
2266    if (cell instanceof ByteBufferExtendedCell) {
2267      StreamUtils.writeShort(out, rowLen);
2268      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2269        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
2270      out.write(fLen);
2271      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2272        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
2273      ByteBufferUtils.copyBufferToStream(out,
2274        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2275        ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
2276    } else {
2277      StreamUtils.writeShort(out, rowLen);
2278      out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
2279      out.write(fLen);
2280      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
2281      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen);
2282    }
2283    StreamUtils.writeLong(out, cell.getTimestamp());
2284    out.write(cell.getTypeByte());
2285    return Bytes.SIZEOF_SHORT + rowLen + Bytes.SIZEOF_BYTE + fLen + qLen + Bytes.SIZEOF_LONG
2286      + Bytes.SIZEOF_BYTE;
2287  }
2288
2289  /**
2290   * Sets the given seqId to the cell. Marked as audience Private as of 1.2.0. Setting a Cell
2291   * sequenceid is an internal implementation detail not for general public use.
2292   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2293   */
2294  public static void setSequenceId(Cell cell, long seqId) throws IOException {
2295    if (cell instanceof ExtendedCell) {
2296      ((ExtendedCell) cell).setSequenceId(seqId);
2297    } else {
2298      throw new IOException(
2299        new UnsupportedOperationException("Cell is not of type " + ExtendedCell.class.getName()));
2300    }
2301  }
2302
2303  /**
2304   * Sets the given timestamp to the cell.
2305   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2306   */
2307  public static void setTimestamp(Cell cell, long ts) throws IOException {
2308    if (cell instanceof ExtendedCell) {
2309      ((ExtendedCell) cell).setTimestamp(ts);
2310    } else {
2311      throw new IOException(
2312        new UnsupportedOperationException("Cell is not of type " + ExtendedCell.class.getName()));
2313    }
2314  }
2315
2316  /**
2317   * Sets the given timestamp to the cell.
2318   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2319   */
2320  public static void setTimestamp(Cell cell, byte[] ts) throws IOException {
2321    if (cell instanceof ExtendedCell) {
2322      ((ExtendedCell) cell).setTimestamp(ts);
2323    } else {
2324      throw new IOException(
2325        new UnsupportedOperationException("Cell is not of type " + ExtendedCell.class.getName()));
2326    }
2327  }
2328
2329  /**
2330   * Sets the given timestamp to the cell iff current timestamp is
2331   * {@link HConstants#LATEST_TIMESTAMP}.
2332   * @return True if cell timestamp is modified.
2333   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2334   */
2335  public static boolean updateLatestStamp(Cell cell, long ts) throws IOException {
2336    if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
2337      setTimestamp(cell, ts);
2338      return true;
2339    }
2340    return false;
2341  }
2342
2343  /**
2344   * Sets the given timestamp to the cell iff current timestamp is
2345   * {@link HConstants#LATEST_TIMESTAMP}.
2346   * @return True if cell timestamp is modified.
2347   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2348   */
2349  public static boolean updateLatestStamp(Cell cell, byte[] ts) throws IOException {
2350    if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
2351      setTimestamp(cell, ts);
2352      return true;
2353    }
2354    return false;
2355  }
2356
2357  /**
2358   * Writes the row from the given cell to the output stream
2359   * @param out     The outputstream to which the data has to be written
2360   * @param cell    The cell whose contents has to be written
2361   * @param rlength the row length
2362   */
2363  public static void writeRow(OutputStream out, Cell cell, short rlength) throws IOException {
2364    if (cell instanceof ByteBufferExtendedCell) {
2365      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2366        ((ByteBufferExtendedCell) cell).getRowPosition(), rlength);
2367    } else {
2368      out.write(cell.getRowArray(), cell.getRowOffset(), rlength);
2369    }
2370  }
2371
2372  /**
2373   * Writes the family from the given cell to the output stream
2374   * @param out     The outputstream to which the data has to be written
2375   * @param cell    The cell whose contents has to be written
2376   * @param flength the family length
2377   */
2378  public static void writeFamily(OutputStream out, Cell cell, byte flength) throws IOException {
2379    if (cell instanceof ByteBufferExtendedCell) {
2380      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2381        ((ByteBufferExtendedCell) cell).getFamilyPosition(), flength);
2382    } else {
2383      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flength);
2384    }
2385  }
2386
2387  /**
2388   * Writes the qualifier from the given cell to the output stream
2389   * @param out     The outputstream to which the data has to be written
2390   * @param cell    The cell whose contents has to be written
2391   * @param qlength the qualifier length
2392   */
2393  public static void writeQualifier(OutputStream out, Cell cell, int qlength) throws IOException {
2394    if (cell instanceof ByteBufferExtendedCell) {
2395      ByteBufferUtils.copyBufferToStream(out,
2396        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2397        ((ByteBufferExtendedCell) cell).getQualifierPosition(), qlength);
2398    } else {
2399      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlength);
2400    }
2401  }
2402
2403  /**
2404   * Writes the qualifier from the given cell to the output stream excluding the common prefix
2405   * @param out     The dataoutputstream to which the data has to be written
2406   * @param cell    The cell whose contents has to be written
2407   * @param qlength the qualifier length
2408   */
2409  public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell, int qlength,
2410    int commonPrefix) throws IOException {
2411    if (cell instanceof ByteBufferExtendedCell) {
2412      ByteBufferUtils.copyBufferToStream((DataOutput) out,
2413        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2414        ((ByteBufferExtendedCell) cell).getQualifierPosition() + commonPrefix,
2415        qlength - commonPrefix);
2416    } else {
2417      out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
2418        qlength - commonPrefix);
2419    }
2420  }
2421
2422  /**
2423   * Writes the value from the given cell to the output stream
2424   * @param out     The outputstream to which the data has to be written
2425   * @param cell    The cell whose contents has to be written
2426   * @param vlength the value length
2427   */
2428  public static void writeValue(OutputStream out, Cell cell, int vlength) throws IOException {
2429    if (cell instanceof ByteBufferExtendedCell) {
2430      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2431        ((ByteBufferExtendedCell) cell).getValuePosition(), vlength);
2432    } else {
2433      out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
2434    }
2435  }
2436
2437  /**
2438   * Writes the tag from the given cell to the output stream
2439   * @param out        The outputstream to which the data has to be written
2440   * @param cell       The cell whose contents has to be written
2441   * @param tagsLength the tag length
2442   */
2443  public static void writeTags(OutputStream out, Cell cell, int tagsLength) throws IOException {
2444    if (cell instanceof ByteBufferExtendedCell) {
2445      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
2446        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
2447    } else {
2448      out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
2449    }
2450  }
2451
2452  /**
2453   * special case for Cell.equals
2454   */
2455  public static boolean equalsIgnoreMvccVersion(Cell a, Cell b) {
2456    // row
2457    boolean res = CellUtil.matchingRows(a, b);
2458    if (!res) return res;
2459
2460    // family
2461    res = CellUtil.matchingColumn(a, b);
2462    if (!res) return res;
2463
2464    // timestamp: later sorts first
2465    if (!CellUtil.matchingTimestamp(a, b)) return false;
2466
2467    // type
2468    int c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
2469    if (c != 0) return false;
2470    else return true;
2471  }
2472
2473  /**
2474   * Converts the rowkey bytes of the given cell into an int value
2475   * @return rowkey as int
2476   */
2477  public static int getRowAsInt(Cell cell) {
2478    if (cell instanceof ByteBufferExtendedCell) {
2479      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2480        ((ByteBufferExtendedCell) cell).getRowPosition());
2481    }
2482    return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
2483  }
2484
2485  /**
2486   * Converts the value bytes of the given cell into a long value
2487   * @return value as long
2488   */
2489  public static long getValueAsLong(Cell cell) {
2490    if (cell instanceof ByteBufferExtendedCell) {
2491      return ByteBufferUtils.toLong(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2492        ((ByteBufferExtendedCell) cell).getValuePosition());
2493    }
2494    return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
2495  }
2496
2497  /**
2498   * Converts the value bytes of the given cell into a int value
2499   * @return value as int
2500   */
2501  public static int getValueAsInt(Cell cell) {
2502    if (cell instanceof ByteBufferExtendedCell) {
2503      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2504        ((ByteBufferExtendedCell) cell).getValuePosition());
2505    }
2506    return Bytes.toInt(cell.getValueArray(), cell.getValueOffset());
2507  }
2508
2509  /**
2510   * Converts the value bytes of the given cell into a double value
2511   * @return value as double
2512   */
2513  public static double getValueAsDouble(Cell cell) {
2514    if (cell instanceof ByteBufferExtendedCell) {
2515      return ByteBufferUtils.toDouble(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2516        ((ByteBufferExtendedCell) cell).getValuePosition());
2517    }
2518    return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
2519  }
2520
2521  /**
2522   * Converts the value bytes of the given cell into a BigDecimal
2523   * @return value as BigDecimal
2524   */
2525  public static BigDecimal getValueAsBigDecimal(Cell cell) {
2526    if (cell instanceof ByteBufferExtendedCell) {
2527      return ByteBufferUtils.toBigDecimal(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2528        ((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
2529    }
2530    return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
2531  }
2532
2533  /**
2534   * Compresses the tags to the given outputstream using the TagcompressionContext
2535   * @param out                   the outputstream to which the compression should happen
2536   * @param cell                  the cell which has tags
2537   * @param tagCompressionContext the TagCompressionContext
2538   * @throws IOException can throw IOException if the compression encounters issue
2539   */
2540  public static void compressTags(OutputStream out, Cell cell,
2541    TagCompressionContext tagCompressionContext) throws IOException {
2542    if (cell instanceof ByteBufferExtendedCell) {
2543      tagCompressionContext.compressTags(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
2544        ((ByteBufferExtendedCell) cell).getTagsPosition(), cell.getTagsLength());
2545    } else {
2546      tagCompressionContext.compressTags(out, cell.getTagsArray(), cell.getTagsOffset(),
2547        cell.getTagsLength());
2548    }
2549  }
2550
2551  public static void compressRow(OutputStream out, Cell cell, Dictionary dict) throws IOException {
2552    if (cell instanceof ByteBufferExtendedCell) {
2553      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2554        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(), dict);
2555    } else {
2556      Dictionary.write(out, cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), dict);
2557    }
2558  }
2559
2560  public static void compressFamily(OutputStream out, Cell cell, Dictionary dict)
2561    throws IOException {
2562    if (cell instanceof ByteBufferExtendedCell) {
2563      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2564        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(), dict);
2565    } else {
2566      Dictionary.write(out, cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2567        dict);
2568    }
2569  }
2570
2571  public static void compressQualifier(OutputStream out, Cell cell, Dictionary dict)
2572    throws IOException {
2573    if (cell instanceof ByteBufferExtendedCell) {
2574      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2575        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), dict);
2576    } else {
2577      Dictionary.write(out, cell.getQualifierArray(), cell.getQualifierOffset(),
2578        cell.getQualifierLength(), dict);
2579    }
2580  }
2581
2582  /**
2583   * Used when a cell needs to be compared with a key byte[] such as cases of finding the index from
2584   * the index block, bloom keys from the bloom blocks This byte[] is expected to be serialized in
2585   * the KeyValue serialization format If the KeyValue (Cell's) serialization format changes this
2586   * method cannot be used.
2587   * @param comparator the {@link CellComparator} to use for comparison
2588   * @param left       the cell to be compared
2589   * @param key        the serialized key part of a KeyValue
2590   * @param offset     the offset in the key byte[]
2591   * @param length     the length of the key byte[]
2592   * @return an int greater than 0 if left is greater than right lesser than 0 if left is lesser
2593   *         than right equal to 0 if left is equal to right
2594   */
2595  public static final int compare(CellComparator comparator, Cell left, byte[] key, int offset,
2596    int length) {
2597    // row
2598    short rrowlength = Bytes.toShort(key, offset);
2599    int c = comparator.compareRows(left, key, offset + Bytes.SIZEOF_SHORT, rrowlength);
2600    if (c != 0) return c;
2601
2602    // Compare the rest of the two KVs without making any assumptions about
2603    // the common prefix. This function will not compare rows anyway, so we
2604    // don't need to tell it that the common prefix includes the row.
2605    return compareWithoutRow(comparator, left, key, offset, length, rrowlength);
2606  }
2607
2608  /**
2609   * Compare columnFamily, qualifier, timestamp, and key type (everything except the row). This
2610   * method is used both in the normal comparator and the "same-prefix" comparator. Note that we are
2611   * assuming that row portions of both KVs have already been parsed and found identical, and we
2612   * don't validate that assumption here.
2613   * @param comparator the {@link CellComparator} to use for comparison
2614   * @param left       the cell to be compared
2615   * @param right      the serialized key part of a key-value
2616   * @param roffset    the offset in the key byte[]
2617   * @param rlength    the length of the key byte[]
2618   * @param rowlength  the row length
2619   * @return greater than 0 if left cell is bigger, less than 0 if right cell is bigger, 0 if both
2620   *         cells are equal
2621   */
2622  static final int compareWithoutRow(CellComparator comparator, Cell left, byte[] right,
2623    int roffset, int rlength, short rowlength) {
2624    /***
2625     * KeyValue Format and commonLength:
2626     * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
2627     * ------------------|-------commonLength--------|--------------
2628     */
2629    int commonLength = KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE + rowlength;
2630
2631    // commonLength + TIMESTAMP_TYPE_SIZE
2632    int commonLengthWithTSAndType = KeyValue.TIMESTAMP_TYPE_SIZE + commonLength;
2633    // ColumnFamily + Qualifier length.
2634    int lcolumnlength = left.getFamilyLength() + left.getQualifierLength();
2635    int rcolumnlength = rlength - commonLengthWithTSAndType;
2636
2637    byte ltype = left.getTypeByte();
2638    byte rtype = right[roffset + (rlength - 1)];
2639
2640    // If the column is not specified, the "minimum" key type appears the
2641    // latest in the sorted order, regardless of the timestamp. This is used
2642    // for specifying the last key/value in a given row, because there is no
2643    // "lexicographically last column" (it would be infinitely long). The
2644    // "maximum" key type does not need this behavior.
2645    if (lcolumnlength == 0 && ltype == KeyValue.Type.Minimum.getCode()) {
2646      // left is "bigger", i.e. it appears later in the sorted order
2647      return 1;
2648    }
2649    if (rcolumnlength == 0 && rtype == KeyValue.Type.Minimum.getCode()) {
2650      return -1;
2651    }
2652
2653    int rfamilyoffset = commonLength + roffset;
2654
2655    // Column family length.
2656    int lfamilylength = left.getFamilyLength();
2657    int rfamilylength = right[rfamilyoffset - 1];
2658    // If left family size is not equal to right family size, we need not
2659    // compare the qualifiers.
2660    boolean sameFamilySize = (lfamilylength == rfamilylength);
2661    if (!sameFamilySize) {
2662      // comparing column family is enough.
2663      return CellUtil.compareFamilies(left, right, rfamilyoffset, rfamilylength);
2664    }
2665    // Compare family & qualifier together.
2666    // Families are same. Compare on qualifiers.
2667    int comparison = CellUtil.compareColumns(left, right, rfamilyoffset, rfamilylength,
2668      rfamilyoffset + rfamilylength, (rcolumnlength - rfamilylength));
2669    if (comparison != 0) {
2670      return comparison;
2671    }
2672
2673    // //
2674    // Next compare timestamps.
2675    long rtimestamp = Bytes.toLong(right, roffset + (rlength - KeyValue.TIMESTAMP_TYPE_SIZE));
2676    int compare = comparator.compareTimestamps(left.getTimestamp(), rtimestamp);
2677    if (compare != 0) {
2678      return compare;
2679    }
2680
2681    // Compare types. Let the delete types sort ahead of puts; i.e. types
2682    // of higher numbers sort before those of lesser numbers. Maximum (255)
2683    // appears ahead of everything, and minimum (0) appears after
2684    // everything.
2685    return (0xff & rtype) - (0xff & ltype);
2686  }
2687
2688  /**
2689   * Return a new cell is located following input cell. If both of type and timestamp are minimum,
2690   * the input cell will be returned directly.
2691   */
2692  public static Cell createNextOnRowCol(Cell cell) {
2693    long ts = cell.getTimestamp();
2694    byte type = cell.getTypeByte();
2695    if (type != KeyValue.Type.Minimum.getCode()) {
2696      type = KeyValue.Type.values()[KeyValue.Type.codeToType(type).ordinal() - 1].getCode();
2697    } else if (ts != HConstants.OLDEST_TIMESTAMP) {
2698      ts = ts - 1;
2699      type = KeyValue.Type.Maximum.getCode();
2700    } else {
2701      return cell;
2702    }
2703    return createNextOnRowCol(cell, ts, type);
2704  }
2705
2706  static Cell createNextOnRowCol(Cell cell, long ts, byte type) {
2707    if (cell instanceof ByteBufferExtendedCell) {
2708      return new LastOnRowColByteBufferExtendedCell(
2709        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2710        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2711        ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2712        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2713        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2714        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength()) {
2715        @Override
2716        public long getTimestamp() {
2717          return ts;
2718        }
2719
2720        @Override
2721        public byte getTypeByte() {
2722          return type;
2723        }
2724      };
2725    }
2726    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2727      cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2728      cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()) {
2729      @Override
2730      public long getTimestamp() {
2731        return ts;
2732      }
2733
2734      @Override
2735      public byte getTypeByte() {
2736        return type;
2737      }
2738    };
2739  }
2740
2741  /**
2742   * Estimate based on keyvalue's serialization format in the RPC layer. Note that there is an extra
2743   * SIZEOF_INT added to the size here that indicates the actual length of the cell for cases where
2744   * cell's are serialized in a contiguous format (For eg in RPCs).
2745   * @return Estimate of the <code>cell</code> size in bytes plus an extra SIZEOF_INT indicating the
2746   *         actual cell length.
2747   */
2748  public static int estimatedSerializedSizeOf(final Cell cell) {
2749    return cell.getSerializedSize() + Bytes.SIZEOF_INT;
2750  }
2751
2752  /**
2753   * Calculates the serialized key size. We always serialize in the KeyValue's serialization format.
2754   * @param cell the cell for which the key size has to be calculated.
2755   * @return the key size
2756   */
2757  public static int estimatedSerializedSizeOfKey(final Cell cell) {
2758    if (cell instanceof KeyValue) return ((KeyValue) cell).getKeyLength();
2759    return cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength()
2760      + KeyValue.KEY_INFRASTRUCTURE_SIZE;
2761  }
2762
2763  /**
2764   * This method exists just to encapsulate how we serialize keys. To be replaced by a factory that
2765   * we query to figure what the Cell implementation is and then, what serialization engine to use
2766   * and further, how to serialize the key for inclusion in hfile index. TODO.
2767   * @return The key portion of the Cell serialized in the old-school KeyValue way or null if passed
2768   *         a null <code>cell</code>
2769   */
2770  public static byte[] getCellKeySerializedAsKeyValueKey(final Cell cell) {
2771    if (cell == null) return null;
2772    byte[] b = new byte[KeyValueUtil.keyLength(cell)];
2773    KeyValueUtil.appendKeyTo(cell, b, 0);
2774    return b;
2775  }
2776
2777  /**
2778   * Create a Cell that is smaller than all other possible Cells for the given Cell's row.
2779   * @return First possible Cell on passed Cell's row.
2780   */
2781  public static Cell createFirstOnRow(final Cell cell) {
2782    if (cell instanceof ByteBufferExtendedCell) {
2783      return new FirstOnRowByteBufferExtendedCell(
2784        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2785        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
2786    }
2787    return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2788  }
2789
2790  public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength) {
2791    return new FirstOnRowCell(row, roffset, rlength);
2792  }
2793
2794  public static Cell createFirstOnRow(final byte[] row, final byte[] family, final byte[] col) {
2795    return createFirstOnRow(row, 0, (short) row.length, family, 0, (byte) family.length, col, 0,
2796      col.length);
2797  }
2798
2799  public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength,
2800    final byte[] family, int foffset, byte flength, final byte[] col, int coffset, int clength) {
2801    return new FirstOnRowColCell(row, roffset, rlength, family, foffset, flength, col, coffset,
2802      clength);
2803  }
2804
2805  public static Cell createFirstOnRow(final byte[] row) {
2806    return createFirstOnRow(row, 0, (short) row.length);
2807  }
2808
2809  public static Cell createFirstOnRowFamily(Cell cell, byte[] fArray, int foff, int flen) {
2810    if (cell instanceof ByteBufferExtendedCell) {
2811      return new FirstOnRowColByteBufferExtendedCell(
2812        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2813        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2814        ByteBuffer.wrap(fArray), foff, (byte) flen, HConstants.EMPTY_BYTE_BUFFER, 0, 0);
2815    }
2816    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2817      fArray, foff, (byte) flen, HConstants.EMPTY_BYTE_ARRAY, 0, 0);
2818  }
2819
2820  public static Cell createFirstOnRowCol(final Cell cell) {
2821    if (cell instanceof ByteBufferExtendedCell) {
2822      return new FirstOnRowColByteBufferExtendedCell(
2823        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2824        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2825        HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
2826        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2827        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
2828    }
2829    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2830      HConstants.EMPTY_BYTE_ARRAY, 0, (byte) 0, cell.getQualifierArray(), cell.getQualifierOffset(),
2831      cell.getQualifierLength());
2832  }
2833
2834  public static Cell createFirstOnNextRow(final Cell cell) {
2835    byte[] nextRow = new byte[cell.getRowLength() + 1];
2836    CellUtil.copyRowTo(cell, nextRow, 0);
2837    nextRow[nextRow.length - 1] = 0;// maybe not necessary
2838    return new FirstOnRowCell(nextRow, 0, (short) nextRow.length);
2839  }
2840
2841  /**
2842   * Create a Cell that is smaller than all other possible Cells for the given Cell's rk:cf and
2843   * passed qualifier.
2844   * @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
2845   */
2846  public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int qoffest, int qlength) {
2847    if (cell instanceof ByteBufferExtendedCell) {
2848      return new FirstOnRowColByteBufferExtendedCell(
2849        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2850        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2851        ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2852        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2853        ByteBuffer.wrap(qArray), qoffest, qlength);
2854    }
2855    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2856      cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), qArray, qoffest,
2857      qlength);
2858  }
2859
2860  /**
2861   * Creates the first cell with the row/family/qualifier of this cell and the given timestamp. Uses
2862   * the "maximum" type that guarantees that the new cell is the lowest possible for this
2863   * combination of row, family, qualifier, and timestamp. This cell's own timestamp is ignored.
2864   * @param cell - cell
2865   */
2866  public static Cell createFirstOnRowColTS(Cell cell, long ts) {
2867    if (cell instanceof ByteBufferExtendedCell) {
2868      return new FirstOnRowColTSByteBufferExtendedCell(
2869        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2870        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2871        ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2872        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2873        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2874        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), ts);
2875    }
2876    return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2877      cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2878      cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), ts);
2879  }
2880
2881  /**
2882   * Create a Cell that is larger than all other possible Cells for the given Cell's row.
2883   * @return Last possible Cell on passed Cell's row.
2884   */
2885  public static Cell createLastOnRow(final Cell cell) {
2886    if (cell instanceof ByteBufferExtendedCell) {
2887      return new LastOnRowByteBufferExtendedCell(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2888        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
2889    }
2890    return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2891  }
2892
2893  public static Cell createLastOnRow(final byte[] row) {
2894    return new LastOnRowCell(row, 0, (short) row.length);
2895  }
2896
2897  /**
2898   * Create a Cell that is larger than all other possible Cells for the given Cell's rk:cf:q. Used
2899   * in creating "fake keys" for the multi-column Bloom filter optimization to skip the row/column
2900   * we already know is not in the file.
2901   * @return Last possible Cell on passed Cell's rk:cf:q.
2902   */
2903  public static Cell createLastOnRowCol(final Cell cell) {
2904    if (cell instanceof ByteBufferExtendedCell) {
2905      return new LastOnRowColByteBufferExtendedCell(
2906        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2907        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2908        ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2909        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2910        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2911        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
2912    }
2913    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2914      cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2915      cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
2916  }
2917
2918  /**
2919   * Create a Delete Family Cell for the specified row and family that would be smaller than all
2920   * other possible Delete Family KeyValues that have the same row and family. Used for seeking.
2921   * @param row - row key (arbitrary byte array)
2922   * @param fam - family name
2923   * @return First Delete Family possible key on passed <code>row</code>.
2924   */
2925  public static Cell createFirstDeleteFamilyCellOnRow(final byte[] row, final byte[] fam) {
2926    return new FirstOnRowDeleteFamilyCell(row, fam);
2927  }
2928}