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