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