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