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