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