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.getQualifierLength() < startsWith.length) {
772      return false;
773    }
774    if (left instanceof ByteBufferExtendedCell) {
775      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
776          ((ByteBufferExtendedCell) left).getQualifierPosition(), startsWith.length,
777          startsWith, 0, startsWith.length);
778    }
779    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
780        startsWith.length, startsWith, 0, startsWith.length);
781  }
782
783  public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
784      final int flength, final byte[] qual, final int qoffset, final int qlength) {
785    if (!matchingFamily(left, fam, foffset, flength)) {
786      return false;
787    }
788    return matchingQualifier(left, qual, qoffset, qlength);
789  }
790
791  public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
792      int rvlength) {
793    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
794      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
795          ((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
796          ((ByteBufferExtendedCell) right).getValueByteBuffer(),
797          ((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
798    }
799    if (left instanceof ByteBufferExtendedCell) {
800      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
801          ((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
802          right.getValueOffset(), rvlength);
803    }
804    if (right instanceof ByteBufferExtendedCell) {
805      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
806          ((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
807          left.getValueOffset(), lvlength);
808    }
809    return Bytes
810        .equals(left.getValueArray(), left.getValueOffset(), lvlength, right.getValueArray(),
811            right.getValueOffset(), rvlength);
812  }
813
814  public static boolean matchingType(Cell a, Cell b) {
815    return a.getTypeByte() == b.getTypeByte();
816  }
817
818  /**
819   * @return True if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
820   *         or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
821   */
822  public static boolean isDelete(final byte type) {
823    return KeyValue.Type.Delete.getCode() <= type && type <= KeyValue.Type.DeleteFamily.getCode();
824  }
825
826  /**
827   * @return True if this cell is a {@link KeyValue.Type#Delete} type.
828   */
829  public static boolean isDeleteType(Cell cell) {
830    return cell.getTypeByte() == KeyValue.Type.Delete.getCode();
831  }
832
833  public static boolean isDeleteFamily(final Cell cell) {
834    return cell.getTypeByte() == KeyValue.Type.DeleteFamily.getCode();
835  }
836
837  public static boolean isDeleteFamilyVersion(final Cell cell) {
838    return cell.getTypeByte() == KeyValue.Type.DeleteFamilyVersion.getCode();
839  }
840
841  public static boolean isDeleteColumns(final Cell cell) {
842    return cell.getTypeByte() == KeyValue.Type.DeleteColumn.getCode();
843  }
844
845  public static boolean isDeleteColumnVersion(final Cell cell) {
846    return cell.getTypeByte() == KeyValue.Type.Delete.getCode();
847  }
848
849  /**
850   * @return True if this cell is a delete family or column type.
851   */
852  public static boolean isDeleteColumnOrFamily(Cell cell) {
853    int t = cell.getTypeByte();
854    return t == KeyValue.Type.DeleteColumn.getCode() || t == KeyValue.Type.DeleteFamily.getCode();
855  }
856
857  public static byte[] cloneTags(Cell cell) {
858    byte[] output = new byte[cell.getTagsLength()];
859    copyTagsTo(cell, output, 0);
860    return output;
861  }
862
863  /**
864   * Copies the tags info into the tag portion of the cell
865   * @param cell
866   * @param destination
867   * @param destinationOffset
868   * @return position after tags
869   */
870  public static int copyTagsTo(Cell cell, byte[] destination, int destinationOffset) {
871    int tlen = cell.getTagsLength();
872    if (cell instanceof ByteBufferExtendedCell) {
873      ByteBufferUtils
874        .copyFromBufferToArray(destination, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
875          ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
876    } else {
877      System
878        .arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset, tlen);
879    }
880    return destinationOffset + tlen;
881  }
882
883  /**
884   * Copies the tags info into the tag portion of the cell
885   * @param cell
886   * @param destination
887   * @param destinationOffset
888   * @return the position after tags
889   */
890  public static int copyTagsTo(Cell cell, ByteBuffer destination, int destinationOffset) {
891    int tlen = cell.getTagsLength();
892    if (cell instanceof ByteBufferExtendedCell) {
893      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
894        destination, ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
895    } else {
896      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
897        cell.getTagsOffset(), tlen);
898    }
899    return destinationOffset + tlen;
900  }
901
902  /**
903   * @param cell The Cell
904   * @return Tags in the given Cell as a List
905   */
906  public static List<Tag> getTags(Cell cell) {
907    List<Tag> tags = new ArrayList<>();
908    Iterator<Tag> tagsItr = tagsIterator(cell);
909    while (tagsItr.hasNext()) {
910      tags.add(tagsItr.next());
911    }
912    return tags;
913  }
914
915  /**
916   * Retrieve Cell's first tag, matching the passed in type
917   * @param cell The Cell
918   * @param type Type of the Tag to retrieve
919   * @return null if there is no tag of the passed in tag type
920   */
921  public static Optional<Tag> getTag(Cell cell, byte type) {
922    boolean bufferBacked = cell instanceof ByteBufferExtendedCell;
923    int length = cell.getTagsLength();
924    int offset =
925      bufferBacked ? ((ByteBufferExtendedCell) cell).getTagsPosition() : cell.getTagsOffset();
926    int pos = offset;
927    while (pos < offset + length) {
928      int tagLen;
929      if (bufferBacked) {
930        ByteBuffer tagsBuffer = ((ByteBufferExtendedCell) cell).getTagsByteBuffer();
931        tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
932        if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
933          return Optional.of(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
934        }
935      } else {
936        tagLen = Bytes.readAsInt(cell.getTagsArray(), pos, TAG_LENGTH_SIZE);
937        if (cell.getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
938          return Optional
939            .of(new ArrayBackedTag(cell.getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
940        }
941      }
942      pos += TAG_LENGTH_SIZE + tagLen;
943    }
944    return Optional.empty();
945  }
946
947  /**
948   * Util method to iterate through the tags in the given cell.
949   * @param cell The Cell over which tags iterator is needed.
950   * @return iterator for the tags
951   */
952  public static Iterator<Tag> tagsIterator(final Cell cell) {
953    final int tagsLength = cell.getTagsLength();
954    // Save an object allocation where we can
955    if (tagsLength == 0) {
956      return TagUtil.EMPTY_TAGS_ITR;
957    }
958    if (cell instanceof ByteBufferExtendedCell) {
959      return tagsIterator(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
960        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
961    }
962    return CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
963  }
964
965  public static Iterator<Tag> tagsIterator(final ByteBuffer tags, final int offset,
966      final int length) {
967    return new Iterator<Tag>() {
968      private int pos = offset;
969      private int endOffset = offset + length - 1;
970
971      @Override
972      public boolean hasNext() {
973        return this.pos < endOffset;
974      }
975
976      @Override
977      public Tag next() {
978        if (hasNext()) {
979          int curTagLen = ByteBufferUtils.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
980          Tag tag = new ByteBufferTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
981          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
982          return tag;
983        }
984        return null;
985      }
986
987      @Override
988      public void remove() {
989        throw new UnsupportedOperationException();
990      }
991    };
992  }
993
994  /**
995   * Returns true if the first range start1...end1 overlaps with the second range start2...end2,
996   * assuming the byte arrays represent row keys
997   */
998  public static boolean overlappingKeys(final byte[] start1, final byte[] end1, final byte[] start2,
999      final byte[] end2) {
1000    return (end2.length == 0 || start1.length == 0 || Bytes.compareTo(start1, end2) < 0)
1001        && (end1.length == 0 || start2.length == 0 || Bytes.compareTo(start2, end1) < 0);
1002  }
1003
1004  /**
1005   * Write rowkey excluding the common part.
1006   * @param cell
1007   * @param rLen
1008   * @param commonPrefix
1009   * @param out
1010   * @throws IOException
1011   */
1012  public static void writeRowKeyExcludingCommon(Cell cell, short rLen, int commonPrefix,
1013      DataOutputStream out) throws IOException {
1014    if (commonPrefix == 0) {
1015      out.writeShort(rLen);
1016    } else if (commonPrefix == 1) {
1017      out.writeByte((byte) rLen);
1018      commonPrefix--;
1019    } else {
1020      commonPrefix -= KeyValue.ROW_LENGTH_SIZE;
1021    }
1022    if (rLen > commonPrefix) {
1023      writeRowSkippingBytes(out, cell, rLen, commonPrefix);
1024    }
1025  }
1026
1027  /**
1028   * Writes the row from the given cell to the output stream excluding the common prefix
1029   * @param out The dataoutputstream to which the data has to be written
1030   * @param cell The cell whose contents has to be written
1031   * @param rlength the row length
1032   * @throws IOException
1033   */
1034  public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
1035      int commonPrefix) throws IOException {
1036    if (cell instanceof ByteBufferExtendedCell) {
1037      ByteBufferUtils
1038          .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1039              ((ByteBufferExtendedCell) cell).getRowPosition() + commonPrefix,
1040              rlength - commonPrefix);
1041    } else {
1042      out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
1043    }
1044  }
1045
1046  /**
1047   * Find length of common prefix in keys of the cells, considering key as byte[] if serialized in
1048   * {@link KeyValue}. The key format is &lt;2 bytes rk len&gt;&lt;rk&gt;&lt;1 byte cf
1049   * len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes timestamp&gt;&lt;1 byte type&gt;
1050   * @param c1 the cell
1051   * @param c2 the cell
1052   * @param bypassFamilyCheck when true assume the family bytes same in both cells. Pass it as true
1053   *          when dealing with Cells in same CF so as to avoid some checks
1054   * @param withTsType when true check timestamp and type bytes also.
1055   * @return length of common prefix
1056   */
1057  public static int findCommonPrefixInFlatKey(Cell c1, Cell c2, boolean bypassFamilyCheck,
1058    boolean withTsType) {
1059    // Compare the 2 bytes in RK length part
1060    short rLen1 = c1.getRowLength();
1061    short rLen2 = c2.getRowLength();
1062    int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
1063    if (rLen1 != rLen2) {
1064      // early out when the RK length itself is not matching
1065      return ByteBufferUtils
1066        .findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE, Bytes.toBytes(rLen2),
1067          0, KeyValue.ROW_LENGTH_SIZE);
1068    }
1069    // Compare the RKs
1070    int rkCommonPrefix = 0;
1071    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1072      rkCommonPrefix = ByteBufferUtils
1073        .findCommonPrefix(((ByteBufferExtendedCell) c1).getRowByteBuffer(),
1074          ((ByteBufferExtendedCell) c1).getRowPosition(), rLen1,
1075          ((ByteBufferExtendedCell) c2).getRowByteBuffer(),
1076          ((ByteBufferExtendedCell) c2).getRowPosition(), rLen2);
1077    } else {
1078      // There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes
1079      // either
1080      // in flush or compactions. In flushes both cells are KV and in case of compaction it will be
1081      // either
1082      // KV or BBCell
1083      rkCommonPrefix = ByteBufferUtils
1084        .findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1, c2.getRowArray(),
1085          c2.getRowOffset(), rLen2);
1086    }
1087    commonPrefix += rkCommonPrefix;
1088    if (rkCommonPrefix != rLen1) {
1089      // Early out when RK is not fully matching.
1090      return commonPrefix;
1091    }
1092    // Compare 1 byte CF length part
1093    byte fLen1 = c1.getFamilyLength();
1094    if (bypassFamilyCheck) {
1095      // This flag will be true when caller is sure that the family will be same for both the cells
1096      // Just make commonPrefix to increment by the family part
1097      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE + fLen1;
1098    } else {
1099      byte fLen2 = c2.getFamilyLength();
1100      if (fLen1 != fLen2) {
1101        // early out when the CF length itself is not matching
1102        return commonPrefix;
1103      }
1104      // CF lengths are same so there is one more byte common in key part
1105      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
1106      // Compare the CF names
1107      int fCommonPrefix;
1108      if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1109        fCommonPrefix = ByteBufferUtils
1110          .findCommonPrefix(((ByteBufferExtendedCell) c1).getFamilyByteBuffer(),
1111            ((ByteBufferExtendedCell) c1).getFamilyPosition(), fLen1,
1112            ((ByteBufferExtendedCell) c2).getFamilyByteBuffer(),
1113            ((ByteBufferExtendedCell) c2).getFamilyPosition(), fLen2);
1114      } else {
1115        fCommonPrefix = ByteBufferUtils
1116          .findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(), fLen1, c2.getFamilyArray(),
1117            c2.getFamilyOffset(), fLen2);
1118      }
1119      commonPrefix += fCommonPrefix;
1120      if (fCommonPrefix != fLen1) {
1121        return commonPrefix;
1122      }
1123    }
1124    // Compare the Qualifiers
1125    int qLen1 = c1.getQualifierLength();
1126    int qLen2 = c2.getQualifierLength();
1127    int qCommon;
1128    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1129      qCommon = ByteBufferUtils
1130        .findCommonPrefix(((ByteBufferExtendedCell) c1).getQualifierByteBuffer(),
1131          ((ByteBufferExtendedCell) c1).getQualifierPosition(), qLen1,
1132          ((ByteBufferExtendedCell) c2).getQualifierByteBuffer(),
1133          ((ByteBufferExtendedCell) c2).getQualifierPosition(), qLen2);
1134    } else {
1135      qCommon = ByteBufferUtils
1136        .findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(), qLen1,
1137          c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
1138    }
1139    commonPrefix += qCommon;
1140    if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
1141      return commonPrefix;
1142    }
1143    // Compare the timestamp parts
1144    int tsCommonPrefix = ByteBufferUtils
1145      .findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE,
1146        Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
1147    commonPrefix += tsCommonPrefix;
1148    if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
1149      return commonPrefix;
1150    }
1151    // Compare the type
1152    if (c1.getTypeByte() == c2.getTypeByte()) {
1153      commonPrefix += KeyValue.TYPE_SIZE;
1154    }
1155    return commonPrefix;
1156  }
1157
1158  /**
1159   * Used to compare two cells based on the column hint provided. This is specifically used when we
1160   * need to optimize the seeks based on the next indexed key. This is an advanced usage API
1161   * specifically needed for some optimizations.
1162   * @param nextIndexedCell the next indexed cell
1163   * @param currentCell the cell to be compared
1164   * @param foff the family offset of the currentCell
1165   * @param flen the family length of the currentCell
1166   * @param colHint the column hint provided - could be null
1167   * @param coff the offset of the column hint if provided, if not offset of the currentCell's
1168   *          qualifier
1169   * @param clen the length of the column hint if provided, if not length of the currentCell's
1170   *          qualifier
1171   * @param ts the timestamp to be seeked
1172   * @param type the type to be seeked
1173   * @return an int based on the given column hint TODO : To be moved out of here because this is a
1174   *         special API used in scan optimization.
1175   */
1176  // compare a key against row/fam/qual/ts/type
1177  public static final int compareKeyBasedOnColHint(CellComparator comparator, Cell nextIndexedCell,
1178      Cell currentCell, int foff, int flen, byte[] colHint, int coff, int clen, long ts,
1179      byte type) {
1180    int compare = comparator.compareRows(nextIndexedCell, currentCell);
1181    if (compare != 0) {
1182      return compare;
1183    }
1184    // If the column is not specified, the "minimum" key type appears the
1185    // latest in the sorted order, regardless of the timestamp. This is used
1186    // for specifying the last key/value in a given row, because there is no
1187    // "lexicographically last column" (it would be infinitely long). The
1188    // "maximum" key type does not need this behavior.
1189    if (nextIndexedCell.getFamilyLength() + nextIndexedCell.getQualifierLength() == 0
1190        && nextIndexedCell.getTypeByte() == KeyValue.Type.Minimum.getCode()) {
1191      // left is "bigger", i.e. it appears later in the sorted order
1192      return 1;
1193    }
1194    if (flen + clen == 0 && type == KeyValue.Type.Minimum.getCode()) {
1195      return -1;
1196    }
1197
1198    compare = comparator.compareFamilies(nextIndexedCell, currentCell);
1199    if (compare != 0) {
1200      return compare;
1201    }
1202    if (colHint == null) {
1203      compare = comparator.compareQualifiers(nextIndexedCell, currentCell);
1204    } else {
1205      compare = CellUtil.compareQualifiers(nextIndexedCell, colHint, coff, clen);
1206    }
1207    if (compare != 0) {
1208      return compare;
1209    }
1210    // Next compare timestamps.
1211    compare = comparator.compareTimestamps(nextIndexedCell.getTimestamp(), ts);
1212    if (compare != 0) {
1213      return compare;
1214    }
1215
1216    // Compare types. Let the delete types sort ahead of puts; i.e. types
1217    // of higher numbers sort before those of lesser numbers. Maximum (255)
1218    // appears ahead of everything, and minimum (0) appears after
1219    // everything.
1220    return (0xff & type) - (0xff & nextIndexedCell.getTypeByte());
1221  }
1222
1223  /**
1224   * Compares only the key portion of a cell. It does not include the sequence id/mvcc of the cell
1225   * @param left
1226   * @param right
1227   * @return an int greater than 0 if left &gt; than right lesser than 0 if left &lt; than right
1228   *         equal to 0 if left is equal to right
1229   */
1230  public static final int compareKeyIgnoresMvcc(CellComparator comparator, Cell left, Cell right) {
1231    return ((CellComparatorImpl) comparator).compare(left, right, true);
1232  }
1233
1234  /**
1235   * Compare cell's row against given comparator
1236   * @param cell the cell to use for comparison
1237   * @param comparator the {@link CellComparator} to use for comparison
1238   * @return result comparing cell's row
1239   */
1240  public static int compareRow(Cell cell, ByteArrayComparable comparator) {
1241    if (cell instanceof ByteBufferExtendedCell) {
1242      return comparator.compareTo(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1243        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
1244    }
1245    return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
1246  }
1247
1248  /**
1249   * Compare cell's column family against given comparator
1250   * @param cell the cell to use for comparison
1251   * @param comparator the {@link CellComparator} to use for comparison
1252   * @return result comparing cell's column family
1253   */
1254  public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
1255    if (cell instanceof ByteBufferExtendedCell) {
1256      return comparator.compareTo(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
1257        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength());
1258    }
1259    return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
1260      cell.getFamilyLength());
1261  }
1262
1263  /**
1264   * Compare cell's qualifier against given comparator
1265   * @param cell the cell to use for comparison
1266   * @param comparator the {@link CellComparator} to use for comparison
1267   * @return result comparing cell's qualifier
1268   */
1269  public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
1270    if (cell instanceof ByteBufferExtendedCell) {
1271      return comparator.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
1272        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
1273    }
1274    return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
1275      cell.getQualifierLength());
1276  }
1277
1278  public static Cell.Type toType(byte type) {
1279    KeyValue.Type codeToType = KeyValue.Type.codeToType(type);
1280    switch (codeToType) {
1281      case Put: return Cell.Type.Put;
1282      case Delete: return Cell.Type.Delete;
1283      case DeleteColumn: return Cell.Type.DeleteColumn;
1284      case DeleteFamily: return Cell.Type.DeleteFamily;
1285      case DeleteFamilyVersion: return Cell.Type.DeleteFamilyVersion;
1286      default: throw new UnsupportedOperationException("Invalid type of cell "+type);
1287    }
1288  }
1289
1290  public static KeyValue.Type toTypeByte(Cell.Type type) {
1291    switch (type) {
1292      case Put: return KeyValue.Type.Put;
1293      case Delete: return KeyValue.Type.Delete;
1294      case DeleteColumn: return KeyValue.Type.DeleteColumn;
1295      case DeleteFamilyVersion: return KeyValue.Type.DeleteFamilyVersion;
1296      case DeleteFamily: return KeyValue.Type.DeleteFamily;
1297      default: throw new UnsupportedOperationException("Unsupported data type:" + type);
1298    }
1299  }
1300
1301  /**
1302   * Compare cell's value against given comparator
1303   * @param cell the cell to use for comparison
1304   * @param comparator the {@link CellComparator} to use for comparison
1305   * @return result comparing cell's value
1306   */
1307  public static int compareValue(Cell cell, ByteArrayComparable comparator) {
1308    if (cell instanceof ByteBufferExtendedCell) {
1309      return comparator.compareTo(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
1310        ((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
1311    }
1312    return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
1313  }
1314
1315  /**
1316   * These cells are used in reseeks/seeks to improve the read performance. They are not real cells
1317   * that are returned back to the clients
1318   */
1319  private static abstract class EmptyCell implements ExtendedCell {
1320
1321    @Override
1322    public void setSequenceId(long seqId) {
1323      // Fake cells don't need seqId, so leaving it as a noop.
1324    }
1325
1326    @Override
1327    public void setTimestamp(long ts) {
1328      // Fake cells can't be changed timestamp, so leaving it as a noop.
1329    }
1330
1331    @Override
1332    public void setTimestamp(byte[] ts) {
1333      // Fake cells can't be changed timestamp, so leaving it as a noop.
1334    }
1335
1336    @Override
1337    public byte[] getRowArray() {
1338      return EMPTY_BYTE_ARRAY;
1339    }
1340
1341    @Override
1342    public int getRowOffset() {
1343      return 0;
1344    }
1345
1346    @Override
1347    public short getRowLength() {
1348      return 0;
1349    }
1350
1351    @Override
1352    public byte[] getFamilyArray() {
1353      return EMPTY_BYTE_ARRAY;
1354    }
1355
1356    @Override
1357    public int getFamilyOffset() {
1358      return 0;
1359    }
1360
1361    @Override
1362    public byte getFamilyLength() {
1363      return 0;
1364    }
1365
1366    @Override
1367    public byte[] getQualifierArray() {
1368      return EMPTY_BYTE_ARRAY;
1369    }
1370
1371    @Override
1372    public int getQualifierOffset() {
1373      return 0;
1374    }
1375
1376    @Override
1377    public int getQualifierLength() {
1378      return 0;
1379    }
1380
1381    @Override
1382    public long getSequenceId() {
1383      return 0;
1384    }
1385
1386    @Override
1387    public byte[] getValueArray() {
1388      return EMPTY_BYTE_ARRAY;
1389    }
1390
1391    @Override
1392    public int getValueOffset() {
1393      return 0;
1394    }
1395
1396    @Override
1397    public int getValueLength() {
1398      return 0;
1399    }
1400
1401    @Override
1402    public byte[] getTagsArray() {
1403      return EMPTY_BYTE_ARRAY;
1404    }
1405
1406    @Override
1407    public int getTagsOffset() {
1408      return 0;
1409    }
1410
1411    @Override
1412    public int getTagsLength() {
1413      return 0;
1414    }
1415  }
1416
1417  /**
1418   * These cells are used in reseeks/seeks to improve the read performance. They are not real cells
1419   * that are returned back to the clients
1420   */
1421  private static abstract class EmptyByteBufferExtendedCell extends ByteBufferExtendedCell {
1422
1423    @Override
1424    public void setSequenceId(long seqId) {
1425      // Fake cells don't need seqId, so leaving it as a noop.
1426    }
1427
1428    @Override
1429    public void setTimestamp(long ts) {
1430      // Fake cells can't be changed timestamp, so leaving it as a noop.
1431    }
1432
1433    @Override
1434    public void setTimestamp(byte[] ts) {
1435      // Fake cells can't be changed timestamp, so leaving it as a noop.
1436    }
1437
1438    @Override
1439    public byte[] getRowArray() {
1440      return CellUtil.cloneRow(this);
1441    }
1442
1443    @Override
1444    public int getRowOffset() {
1445      return 0;
1446    }
1447
1448    @Override
1449    public short getRowLength() {
1450      return 0;
1451    }
1452
1453    @Override
1454    public byte[] getFamilyArray() {
1455      return CellUtil.cloneFamily(this);
1456    }
1457
1458    @Override
1459    public int getFamilyOffset() {
1460      return 0;
1461    }
1462
1463    @Override
1464    public byte getFamilyLength() {
1465      return 0;
1466    }
1467
1468    @Override
1469    public byte[] getQualifierArray() {
1470      return CellUtil.cloneQualifier(this);
1471    }
1472
1473    @Override
1474    public int getQualifierOffset() {
1475      return 0;
1476    }
1477
1478    @Override
1479    public int getQualifierLength() {
1480      return 0;
1481    }
1482
1483    @Override
1484    public long getSequenceId() {
1485      return 0;
1486    }
1487
1488    @Override
1489    public byte[] getValueArray() {
1490      return CellUtil.cloneValue(this);
1491    }
1492
1493    @Override
1494    public int getValueOffset() {
1495      return 0;
1496    }
1497
1498    @Override
1499    public int getValueLength() {
1500      return 0;
1501    }
1502
1503    @Override
1504    public byte[] getTagsArray() {
1505      return CellUtil.cloneTags(this);
1506    }
1507
1508    @Override
1509    public int getTagsOffset() {
1510      return 0;
1511    }
1512
1513    @Override
1514    public int getTagsLength() {
1515      return 0;
1516    }
1517
1518    @Override
1519    public ByteBuffer getRowByteBuffer() {
1520      return HConstants.EMPTY_BYTE_BUFFER;
1521    }
1522
1523    @Override
1524    public int getRowPosition() {
1525      return 0;
1526    }
1527
1528    @Override
1529    public ByteBuffer getFamilyByteBuffer() {
1530      return HConstants.EMPTY_BYTE_BUFFER;
1531    }
1532
1533    @Override
1534    public int getFamilyPosition() {
1535      return 0;
1536    }
1537
1538    @Override
1539    public ByteBuffer getQualifierByteBuffer() {
1540      return HConstants.EMPTY_BYTE_BUFFER;
1541    }
1542
1543    @Override
1544    public int getQualifierPosition() {
1545      return 0;
1546    }
1547
1548    @Override
1549    public ByteBuffer getTagsByteBuffer() {
1550      return HConstants.EMPTY_BYTE_BUFFER;
1551    }
1552
1553    @Override
1554    public int getTagsPosition() {
1555      return 0;
1556    }
1557
1558    @Override
1559    public ByteBuffer getValueByteBuffer() {
1560      return HConstants.EMPTY_BYTE_BUFFER;
1561    }
1562
1563    @Override
1564    public int getValuePosition() {
1565      return 0;
1566    }
1567  }
1568
1569  private static class FirstOnRowCell extends EmptyCell {
1570    private static final int FIXED_HEAPSIZE =
1571        ClassSize.OBJECT // object
1572      + ClassSize.REFERENCE // row array
1573      + Bytes.SIZEOF_INT // row offset
1574      + Bytes.SIZEOF_SHORT;  // row length
1575    private final byte[] rowArray;
1576    private final int roffset;
1577    private final short rlength;
1578
1579    public FirstOnRowCell(final byte[] row, int roffset, short rlength) {
1580      this.rowArray = row;
1581      this.roffset = roffset;
1582      this.rlength = rlength;
1583    }
1584
1585    @Override
1586    public long heapSize() {
1587      return ClassSize.align(FIXED_HEAPSIZE)
1588          // array overhead
1589          + (rlength == 0 ? ClassSize.sizeOfByteArray(rlength) : rlength);
1590    }
1591
1592    @Override
1593    public byte[] getRowArray() {
1594      return this.rowArray;
1595    }
1596
1597    @Override
1598    public int getRowOffset() {
1599      return this.roffset;
1600    }
1601
1602    @Override
1603    public short getRowLength() {
1604      return this.rlength;
1605    }
1606
1607    @Override
1608    public long getTimestamp() {
1609      return HConstants.LATEST_TIMESTAMP;
1610    }
1611
1612    @Override
1613    public byte getTypeByte() {
1614      return KeyValue.Type.Maximum.getCode();
1615    }
1616
1617    @Override
1618    public Type getType() {
1619      throw new UnsupportedOperationException();
1620    }
1621  }
1622
1623  private static class FirstOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
1624    private static final int FIXED_OVERHEAD =
1625        ClassSize.OBJECT // object
1626        + ClassSize.REFERENCE // row buffer
1627        + Bytes.SIZEOF_INT // row offset
1628        + Bytes.SIZEOF_SHORT; // row length
1629    private final ByteBuffer rowBuff;
1630    private final int roffset;
1631    private final short rlength;
1632
1633    public FirstOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
1634      this.rowBuff = row;
1635      this.roffset = roffset;
1636      this.rlength = rlength;
1637    }
1638
1639    @Override
1640    public long heapSize() {
1641      if (this.rowBuff.hasArray()) {
1642        return ClassSize.align(FIXED_OVERHEAD + rlength);
1643      }
1644      return ClassSize.align(FIXED_OVERHEAD);
1645    }
1646
1647    @Override
1648    public ByteBuffer getRowByteBuffer() {
1649      return this.rowBuff;
1650    }
1651
1652    @Override
1653    public int getRowPosition() {
1654      return this.roffset;
1655    }
1656
1657    @Override
1658    public short getRowLength() {
1659      return this.rlength;
1660    }
1661
1662    @Override
1663    public long getTimestamp() {
1664      return HConstants.LATEST_TIMESTAMP;
1665    }
1666
1667    @Override
1668    public byte getTypeByte() {
1669      return KeyValue.Type.Maximum.getCode();
1670    }
1671
1672    @Override
1673    public Type getType() {
1674      throw new UnsupportedOperationException();
1675    }
1676  }
1677
1678  private static class LastOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
1679    private static final int FIXED_OVERHEAD =
1680        ClassSize.OBJECT // object
1681      + ClassSize.REFERENCE // rowBuff
1682      + Bytes.SIZEOF_INT // roffset
1683      + Bytes.SIZEOF_SHORT; // rlength
1684    private final ByteBuffer rowBuff;
1685    private final int roffset;
1686    private final short rlength;
1687
1688    public LastOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
1689      this.rowBuff = row;
1690      this.roffset = roffset;
1691      this.rlength = rlength;
1692    }
1693
1694    @Override
1695    public long heapSize() {
1696      if (this.rowBuff.hasArray()) {
1697        return ClassSize.align(FIXED_OVERHEAD + rlength);
1698      }
1699      return ClassSize.align(FIXED_OVERHEAD);
1700    }
1701
1702    @Override
1703    public ByteBuffer getRowByteBuffer() {
1704      return this.rowBuff;
1705    }
1706
1707    @Override
1708    public int getRowPosition() {
1709      return this.roffset;
1710    }
1711
1712    @Override
1713    public short getRowLength() {
1714      return this.rlength;
1715    }
1716
1717    @Override
1718    public long getTimestamp() {
1719      return HConstants.OLDEST_TIMESTAMP;
1720    }
1721
1722    @Override
1723    public byte getTypeByte() {
1724      return KeyValue.Type.Minimum.getCode();
1725    }
1726
1727    @Override
1728    public Type getType() {
1729      throw new UnsupportedOperationException();
1730    }
1731  }
1732
1733  private static class FirstOnRowColByteBufferExtendedCell
1734      extends FirstOnRowByteBufferExtendedCell {
1735    private static final int FIXED_OVERHEAD =
1736        FirstOnRowByteBufferExtendedCell.FIXED_OVERHEAD
1737        + ClassSize.REFERENCE * 2 // family buffer and column buffer
1738        + Bytes.SIZEOF_INT * 3 // famOffset, colOffset, colLength
1739        + Bytes.SIZEOF_BYTE; // famLength
1740    private final ByteBuffer famBuff;
1741    private final int famOffset;
1742    private final byte famLength;
1743    private final ByteBuffer colBuff;
1744    private final int colOffset;
1745    private final int colLength;
1746
1747    public FirstOnRowColByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength,
1748        final ByteBuffer famBuff, final int famOffset, final byte famLength, final ByteBuffer col,
1749        final int colOffset, final int colLength) {
1750      super(row, roffset, rlength);
1751      this.famBuff = famBuff;
1752      this.famOffset = famOffset;
1753      this.famLength = famLength;
1754      this.colBuff = col;
1755      this.colOffset = colOffset;
1756      this.colLength = colLength;
1757    }
1758
1759    @Override
1760    public long heapSize() {
1761      if (famBuff.hasArray() && colBuff.hasArray()) {
1762        return ClassSize.align(FIXED_OVERHEAD + famLength + colLength);
1763      } else if (famBuff.hasArray()) {
1764        return ClassSize.align(FIXED_OVERHEAD + famLength);
1765      } else if (colBuff.hasArray()) {
1766        return ClassSize.align(FIXED_OVERHEAD + colLength);
1767      } else {
1768        return ClassSize.align(FIXED_OVERHEAD);
1769      }
1770    }
1771
1772    @Override
1773    public ByteBuffer getFamilyByteBuffer() {
1774      return this.famBuff;
1775    }
1776
1777    @Override
1778    public int getFamilyPosition() {
1779      return this.famOffset;
1780    }
1781
1782    @Override
1783    public byte getFamilyLength() {
1784      return famLength;
1785    }
1786
1787    @Override
1788    public ByteBuffer getQualifierByteBuffer() {
1789      return this.colBuff;
1790    }
1791
1792    @Override
1793    public int getQualifierPosition() {
1794      return this.colOffset;
1795    }
1796
1797    @Override
1798    public int getQualifierLength() {
1799      return this.colLength;
1800    }
1801  }
1802
1803  private static class FirstOnRowColCell extends FirstOnRowCell {
1804    private static final long FIXED_HEAPSIZE =
1805        FirstOnRowCell.FIXED_HEAPSIZE
1806      + Bytes.SIZEOF_BYTE // flength
1807      + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
1808      + ClassSize.REFERENCE * 2; // fArray, qArray
1809    private final byte[] fArray;
1810    private final int foffset;
1811    private final byte flength;
1812    private final byte[] qArray;
1813    private final int qoffset;
1814    private final int qlength;
1815
1816    public FirstOnRowColCell(byte[] rArray, int roffset, short rlength, byte[] fArray, int foffset,
1817        byte flength, byte[] qArray, int qoffset, int qlength) {
1818      super(rArray, roffset, rlength);
1819      this.fArray = fArray;
1820      this.foffset = foffset;
1821      this.flength = flength;
1822      this.qArray = qArray;
1823      this.qoffset = qoffset;
1824      this.qlength = qlength;
1825    }
1826
1827    @Override
1828    public long heapSize() {
1829      return ClassSize.align(FIXED_HEAPSIZE)
1830          // array overhead
1831          + (flength == 0 ? ClassSize.sizeOfByteArray(flength) : flength)
1832          + (qlength == 0 ? ClassSize.sizeOfByteArray(qlength) : qlength);
1833    }
1834
1835    @Override
1836    public byte[] getFamilyArray() {
1837      return this.fArray;
1838    }
1839
1840    @Override
1841    public int getFamilyOffset() {
1842      return this.foffset;
1843    }
1844
1845    @Override
1846    public byte getFamilyLength() {
1847      return this.flength;
1848    }
1849
1850    @Override
1851    public byte[] getQualifierArray() {
1852      return this.qArray;
1853    }
1854
1855    @Override
1856    public int getQualifierOffset() {
1857      return this.qoffset;
1858    }
1859
1860    @Override
1861    public int getQualifierLength() {
1862      return this.qlength;
1863    }
1864  }
1865
1866  private static class FirstOnRowColTSCell extends FirstOnRowColCell {
1867    private static final long FIXED_HEAPSIZE =
1868        FirstOnRowColCell.FIXED_HEAPSIZE
1869            + Bytes.SIZEOF_LONG; // ts
1870    private long ts;
1871
1872    public FirstOnRowColTSCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
1873        int foffset, byte flength, byte[] qArray, int qoffset, int qlength, long ts) {
1874      super(rArray, roffset, rlength, fArray, foffset, flength, qArray, qoffset, qlength);
1875      this.ts = ts;
1876    }
1877
1878    @Override
1879    public long getTimestamp() {
1880      return this.ts;
1881    }
1882
1883    @Override
1884    public long heapSize() {
1885      return ClassSize.align(FIXED_HEAPSIZE);
1886    }
1887  }
1888
1889  private static class FirstOnRowColTSByteBufferExtendedCell
1890      extends FirstOnRowColByteBufferExtendedCell {
1891    private static final int FIXED_OVERHEAD =
1892        FirstOnRowColByteBufferExtendedCell.FIXED_OVERHEAD
1893            + Bytes.SIZEOF_LONG; // ts
1894    private long ts;
1895
1896    public FirstOnRowColTSByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
1897        ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset, int qlength,
1898        long ts) {
1899      super(rBuffer, roffset, rlength, fBuffer, foffset, flength, qBuffer, qoffset, qlength);
1900      this.ts = ts;
1901    }
1902
1903    @Override
1904    public long getTimestamp() {
1905      return this.ts;
1906    }
1907
1908    @Override
1909    public long heapSize() {
1910      return ClassSize.align(FIXED_OVERHEAD + super.heapSize());
1911    }
1912  }
1913
1914  private static class LastOnRowCell extends EmptyCell {
1915    private static final int FIXED_OVERHEAD =
1916        ClassSize.OBJECT // object
1917      + ClassSize.REFERENCE // row array
1918      + Bytes.SIZEOF_INT // row offset
1919      + Bytes.SIZEOF_SHORT; // row length
1920    private final byte[] rowArray;
1921    private final int roffset;
1922    private final short rlength;
1923
1924    public LastOnRowCell(byte[] row, int roffset, short rlength) {
1925      this.rowArray = row;
1926      this.roffset = roffset;
1927      this.rlength = rlength;
1928    }
1929
1930    @Override
1931    public long heapSize() {
1932      return ClassSize.align(FIXED_OVERHEAD)
1933          // array overhead
1934          + (rlength == 0 ? ClassSize.sizeOfByteArray(rlength) : rlength);
1935    }
1936
1937    @Override
1938    public byte[] getRowArray() {
1939      return this.rowArray;
1940    }
1941
1942    @Override
1943    public int getRowOffset() {
1944      return this.roffset;
1945    }
1946
1947    @Override
1948    public short getRowLength() {
1949      return this.rlength;
1950    }
1951
1952    @Override
1953    public long getTimestamp() {
1954      return HConstants.OLDEST_TIMESTAMP;
1955    }
1956
1957    @Override
1958    public byte getTypeByte() {
1959      return KeyValue.Type.Minimum.getCode();
1960    }
1961
1962    @Override
1963    public Type getType() {
1964      throw new UnsupportedOperationException();
1965    }
1966  }
1967
1968  private static class LastOnRowColCell extends LastOnRowCell {
1969    private static final long FIXED_OVERHEAD = LastOnRowCell.FIXED_OVERHEAD
1970        + ClassSize.REFERENCE * 2 // fArray and qArray
1971        + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
1972        + Bytes.SIZEOF_BYTE; // flength
1973    private final byte[] fArray;
1974    private final int foffset;
1975    private final byte flength;
1976    private final byte[] qArray;
1977    private final int qoffset;
1978    private final int qlength;
1979
1980    public LastOnRowColCell(byte[] rArray, int roffset, short rlength, byte[] fArray, int foffset,
1981        byte flength, byte[] qArray, int qoffset, int qlength) {
1982      super(rArray, roffset, rlength);
1983      this.fArray = fArray;
1984      this.foffset = foffset;
1985      this.flength = flength;
1986      this.qArray = qArray;
1987      this.qoffset = qoffset;
1988      this.qlength = qlength;
1989    }
1990
1991    @Override
1992    public long heapSize() {
1993      return ClassSize.align(FIXED_OVERHEAD)
1994          // array overhead
1995          + (flength == 0 ? ClassSize.sizeOfByteArray(flength) : flength)
1996          + (qlength == 0 ? ClassSize.sizeOfByteArray(qlength) : qlength);
1997    }
1998
1999    @Override
2000    public byte[] getFamilyArray() {
2001      return this.fArray;
2002    }
2003
2004    @Override
2005    public int getFamilyOffset() {
2006      return this.foffset;
2007    }
2008
2009    @Override
2010    public byte getFamilyLength() {
2011      return this.flength;
2012    }
2013
2014    @Override
2015    public byte[] getQualifierArray() {
2016      return this.qArray;
2017    }
2018
2019    @Override
2020    public int getQualifierOffset() {
2021      return this.qoffset;
2022    }
2023
2024    @Override
2025    public int getQualifierLength() {
2026      return this.qlength;
2027    }
2028  }
2029
2030  private static class LastOnRowColByteBufferExtendedCell extends LastOnRowByteBufferExtendedCell {
2031    private static final int FIXED_OVERHEAD =
2032        LastOnRowByteBufferExtendedCell.FIXED_OVERHEAD
2033            + ClassSize.REFERENCE * 2 // fBuffer and qBuffer
2034            + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
2035            + Bytes.SIZEOF_BYTE; // flength
2036    private final ByteBuffer fBuffer;
2037    private final int foffset;
2038    private final byte flength;
2039    private final ByteBuffer qBuffer;
2040    private final int qoffset;
2041    private final int qlength;
2042
2043    public LastOnRowColByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
2044        ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset,
2045        int qlength) {
2046      super(rBuffer, roffset, rlength);
2047      this.fBuffer = fBuffer;
2048      this.foffset = foffset;
2049      this.flength = flength;
2050      this.qBuffer = qBuffer;
2051      this.qoffset = qoffset;
2052      this.qlength = qlength;
2053    }
2054
2055    @Override
2056    public long heapSize() {
2057      if (fBuffer.hasArray() && qBuffer.hasArray()) {
2058        return ClassSize.align(FIXED_OVERHEAD + flength + qlength);
2059      } else if (fBuffer.hasArray()) {
2060        return ClassSize.align(FIXED_OVERHEAD + flength);
2061      } else if (qBuffer.hasArray()) {
2062        return ClassSize.align(FIXED_OVERHEAD + qlength);
2063      } else {
2064        return ClassSize.align(FIXED_OVERHEAD);
2065      }
2066    }
2067
2068    @Override
2069    public ByteBuffer getFamilyByteBuffer() {
2070      return this.fBuffer;
2071    }
2072
2073    @Override
2074    public int getFamilyPosition() {
2075      return this.foffset;
2076    }
2077
2078    @Override
2079    public byte getFamilyLength() {
2080      return this.flength;
2081    }
2082
2083    @Override
2084    public ByteBuffer getQualifierByteBuffer() {
2085      return this.qBuffer;
2086    }
2087
2088    @Override
2089    public int getQualifierPosition() {
2090      return this.qoffset;
2091    }
2092
2093    @Override
2094    public int getQualifierLength() {
2095      return this.qlength;
2096    }
2097  }
2098
2099  private static class FirstOnRowDeleteFamilyCell extends EmptyCell {
2100    private static final int FIXED_OVERHEAD =
2101        ClassSize.OBJECT // object
2102      + ClassSize.REFERENCE * 2 // fBuffer and qBuffer
2103      + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
2104      + Bytes.SIZEOF_BYTE; // flength
2105    private final byte[] row;
2106    private final byte[] fam;
2107
2108    public FirstOnRowDeleteFamilyCell(byte[] row, byte[] fam) {
2109      this.row = row;
2110      this.fam = fam;
2111    }
2112
2113    @Override
2114    public long heapSize() {
2115      return ClassSize.align(FIXED_OVERHEAD)
2116        // array overhead
2117        + (getRowLength() == 0 ? ClassSize.sizeOfByteArray(getRowLength()) : getRowLength())
2118        + (getFamilyLength() == 0 ?
2119          ClassSize.sizeOfByteArray(getFamilyLength()) : getFamilyLength());
2120    }
2121
2122    @Override
2123    public byte[] getRowArray() {
2124      return this.row;
2125    }
2126
2127    @Override
2128    public short getRowLength() {
2129      return (short) this.row.length;
2130    }
2131
2132    @Override
2133    public byte[] getFamilyArray() {
2134      return this.fam;
2135    }
2136
2137    @Override
2138    public byte getFamilyLength() {
2139      return (byte) this.fam.length;
2140    }
2141
2142    @Override
2143    public long getTimestamp() {
2144      return HConstants.LATEST_TIMESTAMP;
2145    }
2146
2147    @Override
2148    public byte getTypeByte() {
2149      return KeyValue.Type.DeleteFamily.getCode();
2150    }
2151
2152    @Override
2153    public Type getType() {
2154      return Type.DeleteFamily;
2155    }
2156  }
2157
2158  /**
2159   * Writes the Cell's key part as it would have serialized in a KeyValue. The format is &lt;2 bytes
2160   * rk len&gt;&lt;rk&gt;&lt;1 byte cf len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes
2161   * timestamp&gt;&lt;1 byte type&gt;
2162   * @param cell
2163   * @param out
2164   * @throws IOException
2165   */
2166  public static void writeFlatKey(Cell cell, DataOutput out) throws IOException {
2167    short rowLen = cell.getRowLength();
2168    byte fLen = cell.getFamilyLength();
2169    int qLen = cell.getQualifierLength();
2170    // Using just one if/else loop instead of every time checking before writing every
2171    // component of cell
2172    if (cell instanceof ByteBufferExtendedCell) {
2173      out.writeShort(rowLen);
2174      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2175        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
2176      out.writeByte(fLen);
2177      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2178        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
2179      ByteBufferUtils
2180        .copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2181          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
2182    } else {
2183      out.writeShort(rowLen);
2184      out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
2185      out.writeByte(fLen);
2186      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
2187      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen);
2188    }
2189    out.writeLong(cell.getTimestamp());
2190    out.writeByte(cell.getTypeByte());
2191  }
2192
2193  /**
2194   * Deep clones the given cell if the cell supports deep cloning
2195   * @param cell the cell to be cloned
2196   * @return the cloned cell
2197   * @throws CloneNotSupportedException
2198   */
2199  public static Cell deepClone(Cell cell) throws CloneNotSupportedException {
2200    if (cell instanceof ExtendedCell) {
2201      return ((ExtendedCell) cell).deepClone();
2202    }
2203    throw new CloneNotSupportedException();
2204  }
2205
2206  /**
2207   * Writes the cell to the given OutputStream
2208   * @param cell the cell to be written
2209   * @param out the outputstream
2210   * @param withTags if tags are to be written or not
2211   * @return the total bytes written
2212   * @throws IOException
2213   */
2214  public static int writeCell(Cell cell, OutputStream out, boolean withTags) throws IOException {
2215    if (cell instanceof ExtendedCell) {
2216      return ((ExtendedCell) cell).write(out, withTags);
2217    } else {
2218      ByteBufferUtils.putInt(out, estimatedSerializedSizeOfKey(cell));
2219      ByteBufferUtils.putInt(out, cell.getValueLength());
2220      writeFlatKey(cell, out);
2221      writeValue(out, cell, cell.getValueLength());
2222      int tagsLength = cell.getTagsLength();
2223      if (withTags) {
2224        byte[] len = new byte[Bytes.SIZEOF_SHORT];
2225        Bytes.putAsShort(len, 0, tagsLength);
2226        out.write(len);
2227        if (tagsLength > 0) {
2228          writeTags(out, cell, tagsLength);
2229        }
2230      }
2231      int lenWritten = (2 * Bytes.SIZEOF_INT) + estimatedSerializedSizeOfKey(cell)
2232          + cell.getValueLength();
2233      if (withTags) {
2234        lenWritten += Bytes.SIZEOF_SHORT + tagsLength;
2235      }
2236      return lenWritten;
2237    }
2238  }
2239
2240  /**
2241   * Writes a cell to the buffer at the given offset
2242   * @param cell the cell to be written
2243   * @param buf the buffer to which the cell has to be wrriten
2244   * @param offset the offset at which the cell should be written
2245   */
2246  public static void writeCellToBuffer(Cell cell, ByteBuffer buf, int offset) {
2247    if (cell instanceof ExtendedCell) {
2248      ((ExtendedCell) cell).write(buf, offset);
2249    } else {
2250      // Using the KVUtil
2251      byte[] bytes = KeyValueUtil.copyToNewByteArray(cell);
2252      ByteBufferUtils.copyFromArrayToBuffer(buf, offset, bytes, 0, bytes.length);
2253    }
2254  }
2255
2256  public static int writeFlatKey(Cell cell, OutputStream out) throws IOException {
2257    short rowLen = cell.getRowLength();
2258    byte fLen = cell.getFamilyLength();
2259    int qLen = cell.getQualifierLength();
2260    // Using just one if/else loop instead of every time checking before writing every
2261    // component of cell
2262    if (cell instanceof ByteBufferExtendedCell) {
2263      StreamUtils.writeShort(out, rowLen);
2264      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2265        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
2266      out.write(fLen);
2267      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2268        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
2269      ByteBufferUtils
2270        .copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2271          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
2272    } else {
2273      StreamUtils.writeShort(out, rowLen);
2274      out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
2275      out.write(fLen);
2276      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
2277      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen);
2278    }
2279    StreamUtils.writeLong(out, cell.getTimestamp());
2280    out.write(cell.getTypeByte());
2281    return Bytes.SIZEOF_SHORT + rowLen + Bytes.SIZEOF_BYTE + fLen + qLen + Bytes.SIZEOF_LONG
2282      + Bytes.SIZEOF_BYTE;
2283  }
2284
2285  /**
2286   * Sets the given seqId to the cell. Marked as audience Private as of 1.2.0. Setting a Cell
2287   * sequenceid is an internal implementation detail not for general public use.
2288   * @param cell
2289   * @param seqId
2290   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2291   */
2292  public static void setSequenceId(Cell cell, long seqId) throws IOException {
2293    if (cell instanceof ExtendedCell) {
2294      ((ExtendedCell) cell).setSequenceId(seqId);
2295    } else {
2296      throw new IOException(new UnsupportedOperationException(
2297          "Cell is not of type " + ExtendedCell.class.getName()));
2298    }
2299  }
2300
2301  /**
2302   * Sets the given timestamp to the cell.
2303   * @param cell
2304   * @param ts
2305   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2306   */
2307  public static void setTimestamp(Cell cell, long ts) throws IOException {
2308    if (cell instanceof ExtendedCell) {
2309      ((ExtendedCell) cell).setTimestamp(ts);
2310    } else {
2311      throw new IOException(new UnsupportedOperationException(
2312          "Cell is not of type " + ExtendedCell.class.getName()));
2313    }
2314  }
2315
2316  /**
2317   * Sets the given timestamp to the cell.
2318   * @param cell
2319   * @param ts buffer containing the timestamp value
2320   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2321   */
2322  public static void setTimestamp(Cell cell, byte[] ts) throws IOException {
2323    if (cell instanceof ExtendedCell) {
2324      ((ExtendedCell) cell).setTimestamp(ts);
2325    } else {
2326      throw new IOException(new UnsupportedOperationException(
2327          "Cell is not of type " + ExtendedCell.class.getName()));
2328    }
2329  }
2330
2331  /**
2332   * Sets the given timestamp to the cell iff current timestamp is
2333   * {@link HConstants#LATEST_TIMESTAMP}.
2334   * @param cell
2335   * @param ts
2336   * @return True if cell timestamp is modified.
2337   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2338   */
2339  public static boolean updateLatestStamp(Cell cell, long ts) throws IOException {
2340    if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
2341      setTimestamp(cell, ts);
2342      return true;
2343    }
2344    return false;
2345  }
2346
2347  /**
2348   * Sets the given timestamp to the cell iff current timestamp is
2349   * {@link HConstants#LATEST_TIMESTAMP}.
2350   * @param cell
2351   * @param ts buffer containing the timestamp value
2352   * @return True if cell timestamp is modified.
2353   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
2354   */
2355  public static boolean updateLatestStamp(Cell cell, byte[] ts) throws IOException {
2356    if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
2357      setTimestamp(cell, ts);
2358      return true;
2359    }
2360    return false;
2361  }
2362
2363  /**
2364   * Writes the row from the given cell to the output stream
2365   * @param out The outputstream to which the data has to be written
2366   * @param cell The cell whose contents has to be written
2367   * @param rlength the row length
2368   * @throws IOException
2369   */
2370  public static void writeRow(OutputStream out, Cell cell, short rlength) throws IOException {
2371    if (cell instanceof ByteBufferExtendedCell) {
2372      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2373        ((ByteBufferExtendedCell) cell).getRowPosition(), rlength);
2374    } else {
2375      out.write(cell.getRowArray(), cell.getRowOffset(), rlength);
2376    }
2377  }
2378
2379  /**
2380   * Writes the family from the given cell to the output stream
2381   * @param out The outputstream to which the data has to be written
2382   * @param cell The cell whose contents has to be written
2383   * @param flength the family length
2384   * @throws IOException
2385   */
2386  public static void writeFamily(OutputStream out, Cell cell, byte flength) throws IOException {
2387    if (cell instanceof ByteBufferExtendedCell) {
2388      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2389        ((ByteBufferExtendedCell) cell).getFamilyPosition(), flength);
2390    } else {
2391      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flength);
2392    }
2393  }
2394
2395  /**
2396   * Writes the qualifier from the given cell to the output stream
2397   * @param out The outputstream to which the data has to be written
2398   * @param cell The cell whose contents has to be written
2399   * @param qlength the qualifier length
2400   * @throws IOException
2401   */
2402  public static void writeQualifier(OutputStream out, Cell cell, int qlength) throws IOException {
2403    if (cell instanceof ByteBufferExtendedCell) {
2404      ByteBufferUtils
2405        .copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2406          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qlength);
2407    } else {
2408      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlength);
2409    }
2410  }
2411
2412  /**
2413   * Writes the qualifier from the given cell to the output stream excluding the common prefix
2414   * @param out The dataoutputstream to which the data has to be written
2415   * @param cell The cell whose contents has to be written
2416   * @param qlength the qualifier length
2417   * @throws IOException
2418   */
2419  public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell, int qlength,
2420      int commonPrefix) throws IOException {
2421    if (cell instanceof ByteBufferExtendedCell) {
2422      ByteBufferUtils.copyBufferToStream((DataOutput) out,
2423          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2424          ((ByteBufferExtendedCell) cell).getQualifierPosition() + commonPrefix,
2425          qlength - commonPrefix);
2426    } else {
2427      out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
2428          qlength - commonPrefix);
2429    }
2430  }
2431
2432  /**
2433   * Writes the value from the given cell to the output stream
2434   * @param out The outputstream to which the data has to be written
2435   * @param cell The cell whose contents has to be written
2436   * @param vlength the value length
2437   * @throws IOException
2438   */
2439  public static void writeValue(OutputStream out, Cell cell, int vlength) throws IOException {
2440    if (cell instanceof ByteBufferExtendedCell) {
2441      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2442        ((ByteBufferExtendedCell) cell).getValuePosition(), vlength);
2443    } else {
2444      out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
2445    }
2446  }
2447
2448  /**
2449   * Writes the tag from the given cell to the output stream
2450   * @param out The outputstream to which the data has to be written
2451   * @param cell The cell whose contents has to be written
2452   * @param tagsLength the tag length
2453   * @throws IOException
2454   */
2455  public static void writeTags(OutputStream out, Cell cell, int tagsLength) throws IOException {
2456    if (cell instanceof ByteBufferExtendedCell) {
2457      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
2458        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
2459    } else {
2460      out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
2461    }
2462  }
2463
2464  /**
2465   * special case for Cell.equals
2466   */
2467  public static boolean equalsIgnoreMvccVersion(Cell a, Cell b) {
2468    // row
2469    boolean res = CellUtil.matchingRows(a, b);
2470    if (!res) return res;
2471
2472    // family
2473    res = CellUtil.matchingColumn(a, b);
2474    if (!res) return res;
2475
2476    // timestamp: later sorts first
2477    if (!CellUtil.matchingTimestamp(a, b)) return false;
2478
2479    // type
2480    int c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
2481    if (c != 0) return false;
2482    else return true;
2483  }
2484
2485  /**
2486   * Converts the rowkey bytes of the given cell into an int value
2487   * @param cell
2488   * @return rowkey as int
2489   */
2490  public static int getRowAsInt(Cell cell) {
2491    if (cell instanceof ByteBufferExtendedCell) {
2492      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2493        ((ByteBufferExtendedCell) cell).getRowPosition());
2494    }
2495    return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
2496  }
2497
2498  /**
2499   * Converts the value bytes of the given cell into a long value
2500   * @param cell
2501   * @return value as long
2502   */
2503  public static long getValueAsLong(Cell cell) {
2504    if (cell instanceof ByteBufferExtendedCell) {
2505      return ByteBufferUtils.toLong(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2506        ((ByteBufferExtendedCell) cell).getValuePosition());
2507    }
2508    return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
2509  }
2510
2511  /**
2512   * Converts the value bytes of the given cell into a int value
2513   * @param cell
2514   * @return value as int
2515   */
2516  public static int getValueAsInt(Cell cell) {
2517    if (cell instanceof ByteBufferExtendedCell) {
2518      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2519        ((ByteBufferExtendedCell) cell).getValuePosition());
2520    }
2521    return Bytes.toInt(cell.getValueArray(), cell.getValueOffset());
2522  }
2523
2524  /**
2525   * Converts the value bytes of the given cell into a double value
2526   * @param cell
2527   * @return value as double
2528   */
2529  public static double getValueAsDouble(Cell cell) {
2530    if (cell instanceof ByteBufferExtendedCell) {
2531      return ByteBufferUtils.toDouble(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2532        ((ByteBufferExtendedCell) cell).getValuePosition());
2533    }
2534    return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
2535  }
2536
2537  /**
2538   * Converts the value bytes of the given cell into a BigDecimal
2539   * @param cell
2540   * @return value as BigDecimal
2541   */
2542  public static BigDecimal getValueAsBigDecimal(Cell cell) {
2543    if (cell instanceof ByteBufferExtendedCell) {
2544      return ByteBufferUtils.toBigDecimal(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
2545        ((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
2546    }
2547    return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
2548  }
2549
2550  /**
2551   * Compresses the tags to the given outputstream using the TagcompressionContext
2552   * @param out the outputstream to which the compression should happen
2553   * @param cell the cell which has tags
2554   * @param tagCompressionContext the TagCompressionContext
2555   * @throws IOException can throw IOException if the compression encounters issue
2556   */
2557  public static void compressTags(OutputStream out, Cell cell,
2558      TagCompressionContext tagCompressionContext) throws IOException {
2559    if (cell instanceof ByteBufferExtendedCell) {
2560      tagCompressionContext.compressTags(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
2561        ((ByteBufferExtendedCell) cell).getTagsPosition(), cell.getTagsLength());
2562    } else {
2563      tagCompressionContext.compressTags(out, cell.getTagsArray(), cell.getTagsOffset(),
2564        cell.getTagsLength());
2565    }
2566  }
2567
2568  public static void compressRow(OutputStream out, Cell cell, Dictionary dict) throws IOException {
2569    if (cell instanceof ByteBufferExtendedCell) {
2570      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2571        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(), dict);
2572    } else {
2573      Dictionary.write(out, cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), dict);
2574    }
2575  }
2576
2577  public static void compressFamily(OutputStream out, Cell cell, Dictionary dict)
2578      throws IOException {
2579    if (cell instanceof ByteBufferExtendedCell) {
2580      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2581        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(), dict);
2582    } else {
2583      Dictionary.write(out, cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2584        dict);
2585    }
2586  }
2587
2588  public static void compressQualifier(OutputStream out, Cell cell, Dictionary dict)
2589      throws IOException {
2590    if (cell instanceof ByteBufferExtendedCell) {
2591      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2592        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), dict);
2593    } else {
2594      Dictionary.write(out, cell.getQualifierArray(), cell.getQualifierOffset(),
2595        cell.getQualifierLength(), dict);
2596    }
2597  }
2598
2599  /**
2600   * Used when a cell needs to be compared with a key byte[] such as cases of finding the index from
2601   * the index block, bloom keys from the bloom blocks This byte[] is expected to be serialized in
2602   * the KeyValue serialization format If the KeyValue (Cell's) serialization format changes this
2603   * method cannot be used.
2604   * @param comparator the {@link CellComparator} to use for comparison
2605   * @param left the cell to be compared
2606   * @param key the serialized key part of a KeyValue
2607   * @param offset the offset in the key byte[]
2608   * @param length the length of the key byte[]
2609   * @return an int greater than 0 if left is greater than right lesser than 0 if left is lesser
2610   *         than right equal to 0 if left is equal to right
2611   */
2612  @VisibleForTesting
2613  public static final int compare(CellComparator comparator, Cell left, byte[] key, int offset,
2614      int length) {
2615    // row
2616    short rrowlength = Bytes.toShort(key, offset);
2617    int c = comparator.compareRows(left, key, offset + Bytes.SIZEOF_SHORT, rrowlength);
2618    if (c != 0) return c;
2619
2620    // Compare the rest of the two KVs without making any assumptions about
2621    // the common prefix. This function will not compare rows anyway, so we
2622    // don't need to tell it that the common prefix includes the row.
2623    return compareWithoutRow(comparator, left, key, offset, length, rrowlength);
2624  }
2625
2626  /**
2627   * Compare columnFamily, qualifier, timestamp, and key type (everything except the row). This
2628   * method is used both in the normal comparator and the "same-prefix" comparator. Note that we are
2629   * assuming that row portions of both KVs have already been parsed and found identical, and we
2630   * don't validate that assumption here.
2631   * @param comparator the {@link CellComparator} to use for comparison
2632   * @param left the cell to be compared
2633   * @param right the serialized key part of a key-value
2634   * @param roffset the offset in the key byte[]
2635   * @param rlength the length of the key byte[]
2636   * @param rowlength the row length
2637   * @return greater than 0 if left cell is bigger, less than 0 if right cell is bigger, 0 if both
2638   *         cells are equal
2639   */
2640  static final int compareWithoutRow(CellComparator comparator, Cell left, byte[] right,
2641      int roffset, int rlength, short rowlength) {
2642    /***
2643     * KeyValue Format and commonLength:
2644     * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
2645     * ------------------|-------commonLength--------|--------------
2646     */
2647    int commonLength = KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE + rowlength;
2648
2649    // commonLength + TIMESTAMP_TYPE_SIZE
2650    int commonLengthWithTSAndType = KeyValue.TIMESTAMP_TYPE_SIZE + commonLength;
2651    // ColumnFamily + Qualifier length.
2652    int lcolumnlength = left.getFamilyLength() + left.getQualifierLength();
2653    int rcolumnlength = rlength - commonLengthWithTSAndType;
2654
2655    byte ltype = left.getTypeByte();
2656    byte rtype = right[roffset + (rlength - 1)];
2657
2658    // If the column is not specified, the "minimum" key type appears the
2659    // latest in the sorted order, regardless of the timestamp. This is used
2660    // for specifying the last key/value in a given row, because there is no
2661    // "lexicographically last column" (it would be infinitely long). The
2662    // "maximum" key type does not need this behavior.
2663    if (lcolumnlength == 0 && ltype == KeyValue.Type.Minimum.getCode()) {
2664      // left is "bigger", i.e. it appears later in the sorted order
2665      return 1;
2666    }
2667    if (rcolumnlength == 0 && rtype == KeyValue.Type.Minimum.getCode()) {
2668      return -1;
2669    }
2670
2671    int rfamilyoffset = commonLength + roffset;
2672
2673    // Column family length.
2674    int lfamilylength = left.getFamilyLength();
2675    int rfamilylength = right[rfamilyoffset - 1];
2676    // If left family size is not equal to right family size, we need not
2677    // compare the qualifiers.
2678    boolean sameFamilySize = (lfamilylength == rfamilylength);
2679    if (!sameFamilySize) {
2680      // comparing column family is enough.
2681      return CellUtil.compareFamilies(left, right, rfamilyoffset, rfamilylength);
2682    }
2683    // Compare family & qualifier together.
2684    // Families are same. Compare on qualifiers.
2685    int comparison = CellUtil.compareColumns(left, right, rfamilyoffset, rfamilylength,
2686      rfamilyoffset + rfamilylength, (rcolumnlength - rfamilylength));
2687    if (comparison != 0) {
2688      return comparison;
2689    }
2690
2691    // //
2692    // Next compare timestamps.
2693    long rtimestamp = Bytes.toLong(right, roffset + (rlength - KeyValue.TIMESTAMP_TYPE_SIZE));
2694    int compare = comparator.compareTimestamps(left.getTimestamp(), rtimestamp);
2695    if (compare != 0) {
2696      return compare;
2697    }
2698
2699    // Compare types. Let the delete types sort ahead of puts; i.e. types
2700    // of higher numbers sort before those of lesser numbers. Maximum (255)
2701    // appears ahead of everything, and minimum (0) appears after
2702    // everything.
2703    return (0xff & rtype) - (0xff & ltype);
2704  }
2705
2706  /**
2707   * @return An new cell is located following input cell. If both of type and timestamp are minimum,
2708   *         the input cell will be returned directly.
2709   */
2710  public static Cell createNextOnRowCol(Cell cell) {
2711    long ts = cell.getTimestamp();
2712    byte type = cell.getTypeByte();
2713    if (type != KeyValue.Type.Minimum.getCode()) {
2714      type = KeyValue.Type.values()[KeyValue.Type.codeToType(type).ordinal() - 1].getCode();
2715    } else if (ts != HConstants.OLDEST_TIMESTAMP) {
2716      ts = ts - 1;
2717      type = KeyValue.Type.Maximum.getCode();
2718    } else {
2719      return cell;
2720    }
2721    return createNextOnRowCol(cell, ts, type);
2722  }
2723
2724  static Cell createNextOnRowCol(Cell cell, long ts, byte type) {
2725    if (cell instanceof ByteBufferExtendedCell) {
2726      return new LastOnRowColByteBufferExtendedCell(
2727          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2728          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2729          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2730          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2731          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2732          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength()) {
2733        @Override
2734        public long getTimestamp() {
2735          return ts;
2736        }
2737
2738        @Override
2739        public byte getTypeByte() {
2740          return type;
2741        }
2742      };
2743    }
2744    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2745        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2746        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()) {
2747      @Override
2748      public long getTimestamp() {
2749        return ts;
2750      }
2751
2752      @Override
2753      public byte getTypeByte() {
2754        return type;
2755      }
2756    };
2757  }
2758
2759  /**
2760   * Estimate based on keyvalue's serialization format in the RPC layer. Note that there is an extra
2761   * SIZEOF_INT added to the size here that indicates the actual length of the cell for cases where
2762   * cell's are serialized in a contiguous format (For eg in RPCs).
2763   * @param cell
2764   * @return Estimate of the <code>cell</code> size in bytes plus an extra SIZEOF_INT indicating the
2765   *         actual cell length.
2766   */
2767  public static int estimatedSerializedSizeOf(final Cell cell) {
2768    if (cell instanceof ExtendedCell) {
2769      return ((ExtendedCell) cell).getSerializedSize(true) + Bytes.SIZEOF_INT;
2770    }
2771
2772    return getSumOfCellElementLengths(cell) +
2773    // Use the KeyValue's infrastructure size presuming that another implementation would have
2774    // same basic cost.
2775        KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE +
2776        // Serialization is probably preceded by a length (it is in the KeyValueCodec at least).
2777        Bytes.SIZEOF_INT;
2778  }
2779
2780  /**
2781   * @param cell
2782   * @return Sum of the lengths of all the elements in a Cell; does not count in any infrastructure
2783   */
2784  private static int getSumOfCellElementLengths(final Cell cell) {
2785    return getSumOfCellKeyElementLengths(cell) + cell.getValueLength() + cell.getTagsLength();
2786  }
2787
2788  /**
2789   * @param cell
2790   * @return Sum of all elements that make up a key; does not include infrastructure, tags or
2791   *         values.
2792   */
2793  private static int getSumOfCellKeyElementLengths(final Cell cell) {
2794    return cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength()
2795        + KeyValue.TIMESTAMP_TYPE_SIZE;
2796  }
2797
2798  /**
2799   * Calculates the serialized key size. We always serialize in the KeyValue's serialization format.
2800   * @param cell the cell for which the key size has to be calculated.
2801   * @return the key size
2802   */
2803  public static int estimatedSerializedSizeOfKey(final Cell cell) {
2804    if (cell instanceof KeyValue) return ((KeyValue) cell).getKeyLength();
2805    return cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength()
2806        + KeyValue.KEY_INFRASTRUCTURE_SIZE;
2807  }
2808
2809  /**
2810   * This is an estimate of the heap space occupied by a cell. When the cell is of type
2811   * {@link HeapSize} we call {@link HeapSize#heapSize()} so cell can give a correct value. In other
2812   * cases we just consider the bytes occupied by the cell components ie. row, CF, qualifier,
2813   * timestamp, type, value and tags.
2814   * Note that this can be the JVM heap space (on-heap) or the OS heap (off-heap)
2815   * @param cell
2816   * @return estimate of the heap space
2817   */
2818  public static long estimatedSizeOfCell(final Cell cell) {
2819    if (cell instanceof HeapSize) {
2820      return ((HeapSize) cell).heapSize();
2821    }
2822    // TODO: Add sizing of references that hold the row, family, etc., arrays.
2823    return estimatedSerializedSizeOf(cell);
2824  }
2825
2826  /**
2827   * This method exists just to encapsulate how we serialize keys. To be replaced by a factory that
2828   * we query to figure what the Cell implementation is and then, what serialization engine to use
2829   * and further, how to serialize the key for inclusion in hfile index. TODO.
2830   * @param cell
2831   * @return The key portion of the Cell serialized in the old-school KeyValue way or null if passed
2832   *         a null <code>cell</code>
2833   */
2834  public static byte[] getCellKeySerializedAsKeyValueKey(final Cell cell) {
2835    if (cell == null) return null;
2836    byte[] b = new byte[KeyValueUtil.keyLength(cell)];
2837    KeyValueUtil.appendKeyTo(cell, b, 0);
2838    return b;
2839  }
2840
2841  /**
2842   * Create a Cell that is smaller than all other possible Cells for the given Cell's row.
2843   * @param cell
2844   * @return First possible Cell on passed Cell's row.
2845   */
2846  public static Cell createFirstOnRow(final Cell cell) {
2847    if (cell instanceof ByteBufferExtendedCell) {
2848      return new FirstOnRowByteBufferExtendedCell(
2849          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2850          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
2851    }
2852    return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2853  }
2854
2855  public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength) {
2856    return new FirstOnRowCell(row, roffset, rlength);
2857  }
2858
2859  public static Cell createFirstOnRow(final byte[] row, final byte[] family, final byte[] col) {
2860    return createFirstOnRow(row, 0, (short) row.length, family, 0, (byte) family.length, col, 0,
2861        col.length);
2862  }
2863
2864  public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength,
2865      final byte[] family, int foffset, byte flength, final byte[] col, int coffset, int clength) {
2866    return new FirstOnRowColCell(row, roffset, rlength, family, foffset, flength, col, coffset,
2867        clength);
2868  }
2869
2870  public static Cell createFirstOnRow(final byte[] row) {
2871    return createFirstOnRow(row, 0, (short) row.length);
2872  }
2873
2874  public static Cell createFirstOnRowFamily(Cell cell, byte[] fArray, int foff, int flen) {
2875    if (cell instanceof ByteBufferExtendedCell) {
2876      return new FirstOnRowColByteBufferExtendedCell(
2877        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2878        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2879        ByteBuffer.wrap(fArray), foff, (byte) flen, HConstants.EMPTY_BYTE_BUFFER, 0, 0);
2880    }
2881    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2882      fArray, foff, (byte) flen, HConstants.EMPTY_BYTE_ARRAY, 0, 0);
2883  }
2884
2885  public static Cell createFirstOnRowCol(final Cell cell) {
2886    if (cell instanceof ByteBufferExtendedCell) {
2887      return new FirstOnRowColByteBufferExtendedCell(
2888          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2889          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2890          HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
2891          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2892          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
2893    }
2894    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2895        HConstants.EMPTY_BYTE_ARRAY, 0, (byte) 0, cell.getQualifierArray(),
2896        cell.getQualifierOffset(), cell.getQualifierLength());
2897  }
2898
2899  public static Cell createFirstOnNextRow(final Cell cell) {
2900    byte[] nextRow = new byte[cell.getRowLength() + 1];
2901    CellUtil.copyRowTo(cell, nextRow, 0);
2902    nextRow[nextRow.length - 1] = 0;// maybe not necessary
2903    return new FirstOnRowCell(nextRow, 0, (short) nextRow.length);
2904  }
2905
2906  /**
2907   * Create a Cell that is smaller than all other possible Cells for the given Cell's rk:cf and
2908   * passed qualifier.
2909   * @param cell
2910   * @param qArray
2911   * @param qoffest
2912   * @param qlength
2913   * @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
2914   */
2915  public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int qoffest, int qlength) {
2916    if (cell instanceof ByteBufferExtendedCell) {
2917      return new FirstOnRowColByteBufferExtendedCell(
2918          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2919          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2920          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2921          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2922          ByteBuffer.wrap(qArray), qoffest, qlength);
2923    }
2924    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2925        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), qArray, qoffest,
2926        qlength);
2927  }
2928
2929  /**
2930   * Creates the first cell with the row/family/qualifier of this cell and the given timestamp. Uses
2931   * the "maximum" type that guarantees that the new cell is the lowest possible for this
2932   * combination of row, family, qualifier, and timestamp. This cell's own timestamp is ignored.
2933   * @param cell - cell
2934   * @param ts
2935   */
2936  public static Cell createFirstOnRowColTS(Cell cell, long ts) {
2937    if (cell instanceof ByteBufferExtendedCell) {
2938      return new FirstOnRowColTSByteBufferExtendedCell(
2939          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2940          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2941          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2942          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2943          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2944          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), ts);
2945    }
2946    return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2947        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2948        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), ts);
2949  }
2950
2951  /**
2952   * Create a Cell that is larger than all other possible Cells for the given Cell's row.
2953   * @param cell
2954   * @return Last possible Cell on passed Cell's row.
2955   */
2956  public static Cell createLastOnRow(final Cell cell) {
2957    if (cell instanceof ByteBufferExtendedCell) {
2958      return new LastOnRowByteBufferExtendedCell(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2959          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
2960    }
2961    return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2962  }
2963
2964  public static Cell createLastOnRow(final byte[] row) {
2965    return new LastOnRowCell(row, 0, (short) row.length);
2966  }
2967
2968  /**
2969   * Create a Cell that is larger than all other possible Cells for the given Cell's rk:cf:q. Used
2970   * in creating "fake keys" for the multi-column Bloom filter optimization to skip the row/column
2971   * we already know is not in the file.
2972   * @param cell
2973   * @return Last possible Cell on passed Cell's rk:cf:q.
2974   */
2975  public static Cell createLastOnRowCol(final Cell cell) {
2976    if (cell instanceof ByteBufferExtendedCell) {
2977      return new LastOnRowColByteBufferExtendedCell(
2978          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
2979          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
2980          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
2981          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
2982          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
2983          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
2984    }
2985    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
2986        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
2987        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
2988  }
2989
2990  /**
2991   * Create a Delete Family Cell for the specified row and family that would be smaller than all
2992   * other possible Delete Family KeyValues that have the same row and family. Used for seeking.
2993   * @param row - row key (arbitrary byte array)
2994   * @param fam - family name
2995   * @return First Delete Family possible key on passed <code>row</code>.
2996   */
2997  public static Cell createFirstDeleteFamilyCellOnRow(final byte[] row, final byte[] fam) {
2998    return new FirstOnRowDeleteFamilyCell(row, fam);
2999  }
3000}