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 */
018
019package org.apache.hadoop.hbase;
020
021import static org.apache.hadoop.hbase.KeyValue.COLUMN_FAMILY_DELIMITER;
022import static org.apache.hadoop.hbase.KeyValue.COLUMN_FAMILY_DELIM_ARRAY;
023import static org.apache.hadoop.hbase.KeyValue.getDelimiter;
024import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
025
026import java.io.DataOutput;
027import java.io.DataOutputStream;
028import java.io.IOException;
029import java.nio.ByteBuffer;
030import java.util.ArrayList;
031import java.util.Arrays;
032import java.util.Iterator;
033import java.util.List;
034import java.util.Map.Entry;
035import java.util.NavigableMap;
036import java.util.Optional;
037import java.util.function.Function;
038import org.apache.hadoop.hbase.KeyValue.Type;
039import org.apache.hadoop.hbase.io.HeapSize;
040import org.apache.hadoop.hbase.util.ByteBufferUtils;
041import org.apache.hadoop.hbase.util.ByteRange;
042import org.apache.hadoop.hbase.util.Bytes;
043import org.apache.yetus.audience.InterfaceAudience;
044import org.apache.yetus.audience.InterfaceAudience.Private;
045
046/**
047 * Utility methods helpful for slinging {@link Cell} instances. Some methods below are for internal
048 * use only and are marked InterfaceAudience.Private at the method level. Note that all such methods
049 * have been marked deprecated in HBase-2.0 which will be subsequently removed in HBase-3.0
050 */
051@InterfaceAudience.Public
052public final class CellUtil {
053
054  /**
055   * Private constructor to keep this class from being instantiated.
056   */
057  private CellUtil() {
058  }
059
060  /******************* ByteRange *******************************/
061
062  /**
063   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
064   */
065  @Deprecated
066  public static ByteRange fillRowRange(Cell cell, ByteRange range) {
067    return range.set(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
068  }
069
070  /**
071   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
072   */
073  @Deprecated
074  public static ByteRange fillFamilyRange(Cell cell, ByteRange range) {
075    return range.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
076  }
077
078  /**
079   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
080   */
081  @Deprecated
082  public static ByteRange fillQualifierRange(Cell cell, ByteRange range) {
083    return range.set(cell.getQualifierArray(), cell.getQualifierOffset(),
084      cell.getQualifierLength());
085  }
086
087  /**
088   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
089   */
090  @Deprecated
091  public static ByteRange fillValueRange(Cell cell, ByteRange range) {
092    return range.set(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
093  }
094
095  /**
096   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
097   */
098  @Deprecated
099  public static ByteRange fillTagRange(Cell cell, ByteRange range) {
100    return range.set(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
101  }
102
103  /***************** get individual arrays for tests ************/
104
105  public static byte[] cloneRow(Cell cell) {
106    byte[] output = new byte[cell.getRowLength()];
107    copyRowTo(cell, output, 0);
108    return output;
109  }
110
111  public static byte[] cloneFamily(Cell cell) {
112    byte[] output = new byte[cell.getFamilyLength()];
113    copyFamilyTo(cell, output, 0);
114    return output;
115  }
116
117  public static byte[] cloneQualifier(Cell cell) {
118    byte[] output = new byte[cell.getQualifierLength()];
119    copyQualifierTo(cell, output, 0);
120    return output;
121  }
122
123  public static byte[] cloneValue(Cell cell) {
124    byte[] output = new byte[cell.getValueLength()];
125    copyValueTo(cell, output, 0);
126    return output;
127  }
128
129  /**
130   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
131   *             Use {@link RawCell#cloneTags()}
132   */
133  @Deprecated
134  public static byte[] cloneTags(Cell cell) {
135    byte[] output = new byte[cell.getTagsLength()];
136    PrivateCellUtil.copyTagsTo(cell, output, 0);
137    return output;
138  }
139
140  /**
141   * Returns tag value in a new byte array. If server-side, use {@link Tag#getValueArray()} with
142   * appropriate {@link Tag#getValueOffset()} and {@link Tag#getValueLength()} instead to save on
143   * allocations.
144   * @param cell
145   * @return tag value in a new byte array.
146   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
147   */
148  @Deprecated
149  public static byte[] getTagArray(Cell cell) {
150    byte[] output = new byte[cell.getTagsLength()];
151    PrivateCellUtil.copyTagsTo(cell, output, 0);
152    return output;
153  }
154
155  /**
156   * Makes a column in family:qualifier form from separate byte arrays.
157   * <p>
158   * Not recommended for usage as this is old-style API.
159   * @param family
160   * @param qualifier
161   * @return family:qualifier
162   */
163  public static byte[] makeColumn(byte[] family, byte[] qualifier) {
164    return Bytes.add(family, COLUMN_FAMILY_DELIM_ARRAY, qualifier);
165  }
166
167  /**
168   * Splits a column in {@code family:qualifier} form into separate byte arrays. An empty qualifier
169   * (ie, {@code fam:}) is parsed as <code>{ fam, EMPTY_BYTE_ARRAY }</code> while no delimiter (ie,
170   * {@code fam}) is parsed as an array of one element, <code>{ fam }</code>.
171   * <p>
172   * Don't forget, HBase DOES support empty qualifiers. (see HBASE-9549)
173   * </p>
174   * <p>
175   * Not recommend to be used as this is old-style API.
176   * </p>
177   * @param c The column.
178   * @return The parsed column.
179   */
180  public static byte[][] parseColumn(byte[] c) {
181    final int index = getDelimiter(c, 0, c.length, COLUMN_FAMILY_DELIMITER);
182    if (index == -1) {
183      // If no delimiter, return array of size 1
184      return new byte[][] { c };
185    } else if (index == c.length - 1) {
186      // family with empty qualifier, return array size 2
187      byte[] family = new byte[c.length - 1];
188      System.arraycopy(c, 0, family, 0, family.length);
189      return new byte[][] { family, HConstants.EMPTY_BYTE_ARRAY };
190    }
191    // Family and column, return array size 2
192    final byte[][] result = new byte[2][];
193    result[0] = new byte[index];
194    System.arraycopy(c, 0, result[0], 0, index);
195    final int len = c.length - (index + 1);
196    result[1] = new byte[len];
197    System.arraycopy(c, index + 1 /* Skip delimiter */, result[1], 0, len);
198    return result;
199  }
200
201  /******************** copyTo **********************************/
202
203  /**
204   * Copies the row to the given byte[]
205   * @param cell the cell whose row has to be copied
206   * @param destination the destination byte[] to which the row has to be copied
207   * @param destinationOffset the offset in the destination byte[]
208   * @return the offset of the byte[] after the copy has happened
209   */
210  public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
211    short rowLen = cell.getRowLength();
212    if (cell instanceof ByteBufferExtendedCell) {
213      ByteBufferUtils.copyFromBufferToArray(destination,
214          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
215          ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
216    } else {
217      System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
218        rowLen);
219    }
220    return destinationOffset + rowLen;
221  }
222
223  /**
224   * Copies the row to the given bytebuffer
225   * @param cell cell the cell whose row has to be copied
226   * @param destination the destination bytebuffer to which the row has to be copied
227   * @param destinationOffset the offset in the destination byte[]
228   * @return the offset of the bytebuffer after the copy has happened
229   */
230  public static int copyRowTo(Cell cell, ByteBuffer destination, int destinationOffset) {
231    short rowLen = cell.getRowLength();
232    if (cell instanceof ByteBufferExtendedCell) {
233      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
234        destination, ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
235    } else {
236      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getRowArray(),
237        cell.getRowOffset(), rowLen);
238    }
239    return destinationOffset + rowLen;
240  }
241
242  /**
243   * Copies the row to a new byte[]
244   * @param cell the cell from which row has to copied
245   * @return the byte[] containing the row
246   */
247  public static byte[] copyRow(Cell cell) {
248    if (cell instanceof ByteBufferExtendedCell) {
249      return ByteBufferUtils.copyOfRange(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
250        ((ByteBufferExtendedCell) cell).getRowPosition(),
251        ((ByteBufferExtendedCell) cell).getRowPosition() + cell.getRowLength());
252    } else {
253      return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
254        cell.getRowOffset() + cell.getRowLength());
255    }
256  }
257
258  /**
259   * Copies the family to the given byte[]
260   * @param cell the cell whose family has to be copied
261   * @param destination the destination byte[] to which the family has to be copied
262   * @param destinationOffset the offset in the destination byte[]
263   * @return the offset of the byte[] after the copy has happened
264   */
265  public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
266    byte fLen = cell.getFamilyLength();
267    if (cell instanceof ByteBufferExtendedCell) {
268      ByteBufferUtils.copyFromBufferToArray(destination,
269          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
270          ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
271    } else {
272      System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
273        destinationOffset, fLen);
274    }
275    return destinationOffset + fLen;
276  }
277
278  /**
279   * Copies the family to the given bytebuffer
280   * @param cell the cell whose family has to be copied
281   * @param destination the destination bytebuffer to which the family has to be copied
282   * @param destinationOffset the offset in the destination bytebuffer
283   * @return the offset of the bytebuffer after the copy has happened
284   */
285  public static int copyFamilyTo(Cell cell, ByteBuffer destination, int destinationOffset) {
286    byte fLen = cell.getFamilyLength();
287    if (cell instanceof ByteBufferExtendedCell) {
288      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
289        destination, ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
290    } else {
291      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getFamilyArray(),
292        cell.getFamilyOffset(), fLen);
293    }
294    return destinationOffset + fLen;
295  }
296
297  /**
298   * Copies the qualifier to the given byte[]
299   * @param cell the cell whose qualifier has to be copied
300   * @param destination the destination byte[] to which the qualifier has to be copied
301   * @param destinationOffset the offset in the destination byte[]
302   * @return the offset of the byte[] after the copy has happened
303   */
304  public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
305    int qlen = cell.getQualifierLength();
306    if (cell instanceof ByteBufferExtendedCell) {
307      ByteBufferUtils.copyFromBufferToArray(destination,
308        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
309        ((ByteBufferExtendedCell) cell).getQualifierPosition(), destinationOffset, qlen);
310    } else {
311      System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
312        destinationOffset, qlen);
313    }
314    return destinationOffset + qlen;
315  }
316
317  /**
318   * Copies the qualifier to the given bytebuffer
319   * @param cell the cell whose qualifier has to be copied
320   * @param destination the destination bytebuffer to which the qualifier has to be copied
321   * @param destinationOffset the offset in the destination bytebuffer
322   * @return the offset of the bytebuffer after the copy has happened
323   */
324  public static int copyQualifierTo(Cell cell, ByteBuffer destination, int destinationOffset) {
325    int qlen = cell.getQualifierLength();
326    if (cell instanceof ByteBufferExtendedCell) {
327      ByteBufferUtils.copyFromBufferToBuffer(
328          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
329          destination, ((ByteBufferExtendedCell) cell).getQualifierPosition(),
330          destinationOffset, qlen);
331    } else {
332      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset,
333        cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
334    }
335    return destinationOffset + qlen;
336  }
337
338  /**
339   * Copies the value to the given byte[]
340   * @param cell the cell whose value has to be copied
341   * @param destination the destination byte[] to which the value has to be copied
342   * @param destinationOffset the offset in the destination byte[]
343   * @return the offset of the byte[] after the copy has happened
344   */
345  public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
346    int vlen = cell.getValueLength();
347    if (cell instanceof ByteBufferExtendedCell) {
348      ByteBufferUtils.copyFromBufferToArray(destination,
349          ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
350          ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
351    } else {
352      System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
353        vlen);
354    }
355    return destinationOffset + vlen;
356  }
357
358  /**
359   * Copies the value to the given bytebuffer
360   * @param cell the cell whose value has to be copied
361   * @param destination the destination bytebuffer to which the value has to be copied
362   * @param destinationOffset the offset in the destination bytebuffer
363   * @return the offset of the bytebuffer after the copy has happened
364   */
365  public static int copyValueTo(Cell cell, ByteBuffer destination, int destinationOffset) {
366    int vlen = cell.getValueLength();
367    if (cell instanceof ByteBufferExtendedCell) {
368      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
369        destination, ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
370    } else {
371      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getValueArray(),
372        cell.getValueOffset(), vlen);
373    }
374    return destinationOffset + vlen;
375  }
376
377  /**
378   * Copies the tags info into the tag portion of the cell
379   * @param cell
380   * @param destination
381   * @param destinationOffset
382   * @return position after tags
383   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
384   */
385  @Deprecated
386  public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
387    int tlen = cell.getTagsLength();
388    if (cell instanceof ByteBufferExtendedCell) {
389      ByteBufferUtils
390        .copyFromBufferToArray(destination, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
391          ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
392    } else {
393      System
394        .arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset, tlen);
395    }
396    return destinationOffset + tlen;
397  }
398
399  /**
400   * Copies the tags info into the tag portion of the cell
401   * @param cell
402   * @param destination
403   * @param destinationOffset
404   * @return position after tags
405   * @deprecated As of HBase-2.0. Will be removed in 3.0.
406   */
407  @Deprecated
408  public static int copyTagTo(Cell cell, ByteBuffer destination, int destinationOffset) {
409    int tlen = cell.getTagsLength();
410    if (cell instanceof ByteBufferExtendedCell) {
411      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
412        destination, ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
413    } else {
414      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
415        cell.getTagsOffset(), tlen);
416    }
417    return destinationOffset + tlen;
418  }
419
420  /********************* misc *************************************/
421
422  @Private
423  /**
424   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
425   */
426  @Deprecated
427  public static byte getRowByte(Cell cell, int index) {
428    if (cell instanceof ByteBufferExtendedCell) {
429      return ((ByteBufferExtendedCell) cell).getRowByteBuffer()
430          .get(((ByteBufferExtendedCell) cell).getRowPosition() + index);
431    }
432    return cell.getRowArray()[cell.getRowOffset() + index];
433  }
434
435  /**
436   * @deprecated As of HBase-2.0. Will be removed in 3.0.
437   */
438  @Deprecated
439  public static ByteBuffer getValueBufferShallowCopy(Cell cell) {
440    ByteBuffer buffer =
441        ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
442    return buffer;
443  }
444
445  /**
446   * @param cell
447   * @return cell's qualifier wrapped into a ByteBuffer.
448   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
449   */
450  @Deprecated
451  public static ByteBuffer getQualifierBufferShallowCopy(Cell cell) {
452    // No usage of this in code.
453    ByteBuffer buffer = ByteBuffer.wrap(cell.getQualifierArray(), cell.getQualifierOffset(),
454      cell.getQualifierLength());
455    return buffer;
456  }
457
458  /**
459   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use {@link CellBuilder}
460   *             instead
461   */
462  @Deprecated
463  public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
464      final long timestamp, final byte type, final byte[] value) {
465    return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
466            .setRow(row)
467            .setFamily(family)
468            .setQualifier(qualifier)
469            .setTimestamp(timestamp)
470            .setType(type)
471            .setValue(value)
472            .build();
473  }
474
475  /**
476   * Creates a cell with deep copy of all passed bytes.
477   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use {@link CellBuilder}
478   *             instead
479   */
480  @Deprecated
481  public static Cell createCell(final byte[] rowArray, final int rowOffset, final int rowLength,
482      final byte[] familyArray, final int familyOffset, final int familyLength,
483      final byte[] qualifierArray, final int qualifierOffset, final int qualifierLength) {
484    // See createCell(final byte [] row, final byte [] value) for why we default Maximum type.
485    return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
486            .setRow(rowArray, rowOffset, rowLength)
487            .setFamily(familyArray, familyOffset, familyLength)
488            .setQualifier(qualifierArray, qualifierOffset, qualifierLength)
489            .setTimestamp(HConstants.LATEST_TIMESTAMP)
490            .setType(KeyValue.Type.Maximum.getCode())
491            .setValue(HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
492            .build();
493  }
494
495  /**
496   * Marked as audience Private as of 1.2.0.
497   * Creating a Cell with a memstoreTS/mvcc is an internal
498   * implementation detail not for public use.
499   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
500   *             {@link ExtendedCellBuilder} instead
501   */
502  @InterfaceAudience.Private
503  @Deprecated
504  public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
505      final long timestamp, final byte type, final byte[] value, final long memstoreTS) {
506    return createCell(row, family, qualifier, timestamp, type, value, null, memstoreTS);
507  }
508
509  /**
510   * Marked as audience Private as of 1.2.0.
511   * Creating a Cell with tags and a memstoreTS/mvcc is an
512   * internal implementation detail not for public use.
513   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
514   *             {@link ExtendedCellBuilder} instead
515   */
516  @InterfaceAudience.Private
517  @Deprecated
518  public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
519      final long timestamp, final byte type, final byte[] value, byte[] tags,
520      final long memstoreTS) {
521    return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
522            .setRow(row)
523            .setFamily(family)
524            .setQualifier(qualifier)
525            .setTimestamp(timestamp)
526            .setType(type)
527            .setValue(value)
528            .setTags(tags)
529            .setSequenceId(memstoreTS)
530            .build();
531  }
532
533  /**
534   * Marked as audience Private as of 1.2.0.
535   * Creating a Cell with tags is an internal implementation detail not for public use.
536   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
537   *             {@link ExtendedCellBuilder} instead
538   */
539  @InterfaceAudience.Private
540  @Deprecated
541  public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
542      final long timestamp, Type type, final byte[] value, byte[] tags) {
543    return createCell(row, family, qualifier, timestamp, type.getCode(), value, tags, 0);
544  }
545
546  /**
547   * Create a Cell with specific row. Other fields defaulted.
548   * @param row
549   * @return Cell with passed row but all other fields are arbitrary
550   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use {@link CellBuilder}
551   *             instead
552   */
553  @Deprecated
554  public static Cell createCell(final byte[] row) {
555    return createCell(row, HConstants.EMPTY_BYTE_ARRAY);
556  }
557
558  /**
559   * Create a Cell with specific row and value. Other fields are defaulted.
560   * @param row
561   * @param value
562   * @return Cell with passed row and value but all other fields are arbitrary
563   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use {@link CellBuilder}
564   *             instead
565   */
566  @Deprecated
567  public static Cell createCell(final byte[] row, final byte[] value) {
568    // An empty family + empty qualifier + Type.Minimum is used as flag to indicate last on row.
569    // See the CellComparator and KeyValue comparator. Search for compareWithoutRow.
570    // Lets not make a last-on-row key as default but at same time, if you are making a key
571    // without specifying type, etc., flag it as weird by setting type to be Maximum.
572    return createCell(row, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
573      HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum.getCode(), value);
574  }
575
576  /**
577   * Create a Cell with specific row. Other fields defaulted.
578   * @param row
579   * @param family
580   * @param qualifier
581   * @return Cell with passed row but all other fields are arbitrary
582   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use {@link CellBuilder}
583   *             instead
584   */
585  @Deprecated
586  public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier) {
587    // See above in createCell(final byte [] row, final byte [] value) why we set type to Maximum.
588    return createCell(row, family, qualifier, HConstants.LATEST_TIMESTAMP,
589      KeyValue.Type.Maximum.getCode(), HConstants.EMPTY_BYTE_ARRAY);
590  }
591
592  /**
593   * Note : Now only CPs can create cell with tags using the CP environment
594   * @return A new cell which is having the extra tags also added to it.
595   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
596   *
597   */
598  @Deprecated
599  public static Cell createCell(Cell cell, List<Tag> tags) {
600    return PrivateCellUtil.createCell(cell, tags);
601  }
602
603  /**
604   * Now only CPs can create cell with tags using the CP environment
605   * @return A new cell which is having the extra tags also added to it.
606   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
607   */
608  @Deprecated
609  public static Cell createCell(Cell cell, byte[] tags) {
610    return PrivateCellUtil.createCell(cell, tags);
611  }
612
613  /**
614   * Now only CPs can create cell with tags using the CP environment
615   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
616   */
617  @Deprecated
618  public static Cell createCell(Cell cell, byte[] value, byte[] tags) {
619    return PrivateCellUtil.createCell(cell, value, tags);
620  }
621
622  /**
623   * @param cellScannerables
624   * @return CellScanner interface over <code>cellIterables</code>
625   */
626  public static CellScanner
627      createCellScanner(final List<? extends CellScannable> cellScannerables) {
628    return new CellScanner() {
629      private final Iterator<? extends CellScannable> iterator = cellScannerables.iterator();
630      private CellScanner cellScanner = null;
631
632      @Override
633      public Cell current() {
634        return this.cellScanner != null ? this.cellScanner.current() : null;
635      }
636
637      @Override
638      public boolean advance() throws IOException {
639        while (true) {
640          if (this.cellScanner == null) {
641            if (!this.iterator.hasNext()) return false;
642            this.cellScanner = this.iterator.next().cellScanner();
643          }
644          if (this.cellScanner.advance()) return true;
645          this.cellScanner = null;
646        }
647      }
648    };
649  }
650
651  /**
652   * @param cellIterable
653   * @return CellScanner interface over <code>cellIterable</code>
654   */
655  public static CellScanner createCellScanner(final Iterable<Cell> cellIterable) {
656    if (cellIterable == null) return null;
657    return createCellScanner(cellIterable.iterator());
658  }
659
660  /**
661   * @param cells
662   * @return CellScanner interface over <code>cellIterable</code> or null if <code>cells</code> is
663   *         null
664   */
665  public static CellScanner createCellScanner(final Iterator<Cell> cells) {
666    if (cells == null) return null;
667    return new CellScanner() {
668      private final Iterator<Cell> iterator = cells;
669      private Cell current = null;
670
671      @Override
672      public Cell current() {
673        return this.current;
674      }
675
676      @Override
677      public boolean advance() {
678        boolean hasNext = this.iterator.hasNext();
679        this.current = hasNext ? this.iterator.next() : null;
680        return hasNext;
681      }
682    };
683  }
684
685  /**
686   * @param cellArray
687   * @return CellScanner interface over <code>cellArray</code>
688   */
689  public static CellScanner createCellScanner(final Cell[] cellArray) {
690    return new CellScanner() {
691      private final Cell[] cells = cellArray;
692      private int index = -1;
693
694      @Override
695      public Cell current() {
696        if (cells == null) return null;
697        return (index < 0) ? null : this.cells[index];
698      }
699
700      @Override
701      public boolean advance() {
702        if (cells == null) return false;
703        return ++index < this.cells.length;
704      }
705    };
706  }
707
708  /**
709   * Flatten the map of cells out under the CellScanner
710   * @param map Map of Cell Lists; for example, the map of families to Cells that is used inside
711   *          Put, etc., keeping Cells organized by family.
712   * @return CellScanner interface over <code>cellIterable</code>
713   */
714  public static CellScanner createCellScanner(final NavigableMap<byte[], List<Cell>> map) {
715    return new CellScanner() {
716      private final Iterator<Entry<byte[], List<Cell>>> entries = map.entrySet().iterator();
717      private Iterator<Cell> currentIterator = null;
718      private Cell currentCell;
719
720      @Override
721      public Cell current() {
722        return this.currentCell;
723      }
724
725      @Override
726      public boolean advance() {
727        while (true) {
728          if (this.currentIterator == null) {
729            if (!this.entries.hasNext()) return false;
730            this.currentIterator = this.entries.next().getValue().iterator();
731          }
732          if (this.currentIterator.hasNext()) {
733            this.currentCell = this.currentIterator.next();
734            return true;
735          }
736          this.currentCell = null;
737          this.currentIterator = null;
738        }
739      }
740    };
741  }
742
743  /**
744   * @param left
745   * @param right
746   * @return True if the rows in <code>left</code> and <code>right</code> Cells match
747   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
748   *             {@link #matchingRows(Cell, Cell)}
749   */
750  @Deprecated
751  public static boolean matchingRow(final Cell left, final Cell right) {
752    return matchingRows(left, right);
753  }
754
755  /**
756   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
757   *   {@link #matchingRows(Cell, byte[])}
758   */
759  @Deprecated
760  public static boolean matchingRow(final Cell left, final byte[] buf) {
761    return matchingRows(left, buf);
762  }
763
764  public static boolean matchingRows(final Cell left, final byte[] buf) {
765    if (buf == null) {
766      return left.getRowLength() == 0;
767    }
768    return PrivateCellUtil.matchingRows(left, buf, 0, buf.length);
769  }
770
771  /**
772   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
773   *             {@link #matchingRows(Cell, Cell)}
774   * @return true if the row is matching
775   */
776  @Deprecated
777  public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
778      final int length) {
779    if (left instanceof ByteBufferExtendedCell) {
780      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
781        ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), buf, offset, length);
782    }
783    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
784      length);
785  }
786
787  public static boolean matchingFamily(final Cell left, final Cell right) {
788    byte lfamlength = left.getFamilyLength();
789    byte rfamlength = right.getFamilyLength();
790    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
791      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
792        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
793        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
794        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength);
795    }
796    if (left instanceof ByteBufferExtendedCell) {
797      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
798        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength, right.getFamilyArray(),
799        right.getFamilyOffset(), rfamlength);
800    }
801    if (right instanceof ByteBufferExtendedCell) {
802      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
803        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength, left.getFamilyArray(),
804        left.getFamilyOffset(), lfamlength);
805    }
806    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
807      right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
808  }
809
810  public static boolean matchingFamily(final Cell left, final byte[] buf) {
811    if (buf == null) {
812      return left.getFamilyLength() == 0;
813    }
814    return PrivateCellUtil.matchingFamily(left, buf, 0, buf.length);
815  }
816
817  /**
818   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
819   */
820  @Deprecated
821  public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
822    final int length) {
823    if (left instanceof ByteBufferExtendedCell) {
824      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
825        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), buf, offset,
826        length);
827    }
828    return Bytes
829      .equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf, offset,
830        length);
831  }
832
833  public static boolean matchingQualifier(final Cell left, final Cell right) {
834    int lqlength = left.getQualifierLength();
835    int rqlength = right.getQualifierLength();
836    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
837      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
838        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
839        ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
840        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength);
841    }
842    if (left instanceof ByteBufferExtendedCell) {
843      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
844        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength, right.getQualifierArray(),
845        right.getQualifierOffset(), rqlength);
846    }
847    if (right instanceof ByteBufferExtendedCell) {
848      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
849        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength, left.getQualifierArray(),
850        left.getQualifierOffset(), lqlength);
851    }
852    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
853      right.getQualifierArray(), right.getQualifierOffset(), rqlength);
854  }
855
856  /**
857   * Finds if the qualifier part of the cell and the KV serialized byte[] are equal
858   * @param left
859   * @param buf the serialized keyvalue format byte[]
860   * @return true if the qualifier matches, false otherwise
861   */
862  public static boolean matchingQualifier(final Cell left, final byte[] buf) {
863    if (buf == null) {
864      return left.getQualifierLength() == 0;
865    }
866    return PrivateCellUtil.matchingQualifier(left, buf, 0, buf.length);
867  }
868
869  /**
870   * Finds if the qualifier part of the cell and the KV serialized byte[] are equal
871   * @param left
872   * @param buf the serialized keyvalue format byte[]
873   * @param offset the offset of the qualifier in the byte[]
874   * @param length the length of the qualifier in the byte[]
875   * @return true if the qualifier matches, false otherwise
876   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
877   */
878  @Deprecated
879  public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
880    final int length) {
881    if (buf == null) {
882      return left.getQualifierLength() == 0;
883    }
884    if (left instanceof ByteBufferExtendedCell) {
885      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
886        ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(), buf,
887        offset, length);
888    }
889    return Bytes
890      .equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(), buf,
891        offset, length);
892  }
893
894  public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
895    return matchingFamily(left, fam) && matchingQualifier(left, qual);
896  }
897
898  /**
899   * @return True if matching column family and the qualifier starts with <code>qual</code>
900   */
901  public static boolean matchingColumnFamilyAndQualifierPrefix(final Cell left, final byte[] fam,
902      final byte[] qual) {
903    return matchingFamily(left, fam) && PrivateCellUtil.qualifierStartsWith(left, qual);
904  }
905
906  /**
907   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
908   */
909  @Deprecated
910  public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
911      final int flength, final byte[] qual, final int qoffset, final int qlength) {
912    if (!PrivateCellUtil.matchingFamily(left, fam, foffset, flength)) return false;
913    return PrivateCellUtil.matchingQualifier(left, qual, qoffset, qlength);
914  }
915
916  public static boolean matchingColumn(final Cell left, final Cell right) {
917    if (!matchingFamily(left, right)) return false;
918    return matchingQualifier(left, right);
919  }
920
921  public static boolean matchingValue(final Cell left, final Cell right) {
922    return PrivateCellUtil.matchingValue(left, right, left.getValueLength(),
923      right.getValueLength());
924  }
925
926  public static boolean matchingValue(final Cell left, final byte[] buf) {
927    if (left instanceof ByteBufferExtendedCell) {
928      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getValueByteBuffer(),
929        ((ByteBufferExtendedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
930        buf.length) == 0;
931    }
932    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
933      buf.length);
934  }
935
936  /**
937   * @return True if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
938   *         or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
939   */
940  @SuppressWarnings("deprecation")
941  public static boolean isDelete(final Cell cell) {
942    return PrivateCellUtil.isDelete(cell.getTypeByte());
943  }
944
945  /**
946   * @return True if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
947   *         or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
948   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
949   */
950  @Deprecated
951  public static boolean isDelete(final byte type) {
952    return Type.Delete.getCode() <= type && type <= Type.DeleteFamily.getCode();
953  }
954
955  /**
956   * @return True if this cell is a {@link KeyValue.Type#Delete} type.
957   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
958   */
959  @Deprecated
960  public static boolean isDeleteType(Cell cell) {
961    return cell.getTypeByte() == Type.Delete.getCode();
962  }
963
964  /**
965   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
966   */
967  @Deprecated
968  public static boolean isDeleteFamily(final Cell cell) {
969    return cell.getTypeByte() == Type.DeleteFamily.getCode();
970  }
971
972  /**
973   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
974   */
975  @Deprecated
976  public static boolean isDeleteFamilyVersion(final Cell cell) {
977    return cell.getTypeByte() == Type.DeleteFamilyVersion.getCode();
978  }
979
980  /**
981   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
982   */
983  @Deprecated
984  public static boolean isDeleteColumns(final Cell cell) {
985    return cell.getTypeByte() == Type.DeleteColumn.getCode();
986  }
987
988  /**
989   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
990   */
991  @Deprecated
992  public static boolean isDeleteColumnVersion(final Cell cell) {
993    return cell.getTypeByte() == Type.Delete.getCode();
994  }
995
996  /**
997   * @return True if this cell is a delete family or column type.
998   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
999   */
1000  @Deprecated
1001  public static boolean isDeleteColumnOrFamily(Cell cell) {
1002    int t = cell.getTypeByte();
1003    return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
1004  }
1005
1006  /**
1007   * @return True if this cell is a Put.
1008   */
1009  @SuppressWarnings("deprecation")
1010  public static boolean isPut(Cell cell) {
1011    return cell.getTypeByte() == Type.Put.getCode();
1012  }
1013
1014  /**
1015   * Estimate based on keyvalue's serialization format in the RPC layer. Note that there is an extra
1016   * SIZEOF_INT added to the size here that indicates the actual length of the cell for cases where
1017   * cell's are serialized in a contiguous format (For eg in RPCs).
1018   * @param cell
1019   * @return Estimate of the <code>cell</code> size in bytes plus an extra SIZEOF_INT indicating the
1020   *         actual cell length.
1021   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
1022   */
1023  @Deprecated
1024  public static int estimatedSerializedSizeOf(final Cell cell) {
1025    if (cell instanceof ExtendedCell) {
1026      return ((ExtendedCell) cell).getSerializedSize(true) + Bytes.SIZEOF_INT;
1027    }
1028
1029    return getSumOfCellElementLengths(cell) +
1030    // Use the KeyValue's infrastructure size presuming that another implementation would have
1031    // same basic cost.
1032        KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE +
1033        // Serialization is probably preceded by a length (it is in the KeyValueCodec at least).
1034        Bytes.SIZEOF_INT;
1035  }
1036
1037  /**
1038   * @param cell
1039   * @return Sum of the lengths of all the elements in a Cell; does not count in any infrastructure
1040   */
1041  private static int getSumOfCellElementLengths(final Cell cell) {
1042    return getSumOfCellKeyElementLengths(cell) + cell.getValueLength() + cell.getTagsLength();
1043  }
1044
1045  /**
1046   * @param cell
1047   * @return Sum of all elements that make up a key; does not include infrastructure, tags or
1048   *         values.
1049   */
1050  private static int getSumOfCellKeyElementLengths(final Cell cell) {
1051    return cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength()
1052        + KeyValue.TIMESTAMP_TYPE_SIZE;
1053  }
1054
1055  /**
1056   * Calculates the serialized key size. We always serialize in the KeyValue's serialization format.
1057   * @param cell the cell for which the key size has to be calculated.
1058   * @return the key size
1059   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
1060   */
1061  @Deprecated
1062  public static int estimatedSerializedSizeOfKey(final Cell cell) {
1063    if (cell instanceof KeyValue) return ((KeyValue) cell).getKeyLength();
1064    return cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength()
1065        + KeyValue.KEY_INFRASTRUCTURE_SIZE;
1066  }
1067
1068  /**
1069   * This is an estimate of the heap space occupied by a cell. When the cell is of type
1070   * {@link HeapSize} we call {@link HeapSize#heapSize()} so cell can give a correct value. In other
1071   * cases we just consider the bytes occupied by the cell components ie. row, CF, qualifier,
1072   * timestamp, type, value and tags.
1073   * @param cell
1074   * @return estimate of the heap space
1075   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
1076   *             Use {@link RawCell#getTags()}
1077   */
1078  @Deprecated
1079  public static long estimatedHeapSizeOf(final Cell cell) {
1080    return cell.heapSize();
1081  }
1082
1083  /********************* tags *************************************/
1084  /**
1085   * Util method to iterate through the tags
1086   * @param tags
1087   * @param offset
1088   * @param length
1089   * @return iterator for the tags
1090   * @deprecated As of 2.0.0 and will be removed in 3.0.0 Instead use
1091   *             {@link PrivateCellUtil#tagsIterator(Cell)}
1092   */
1093  @Deprecated
1094  public static Iterator<Tag> tagsIterator(final byte[] tags, final int offset, final int length) {
1095    return new Iterator<Tag>() {
1096      private int pos = offset;
1097      private int endOffset = offset + length - 1;
1098
1099      @Override
1100      public boolean hasNext() {
1101        return this.pos < endOffset;
1102      }
1103
1104      @Override
1105      public Tag next() {
1106        if (hasNext()) {
1107          int curTagLen = Bytes.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
1108          Tag tag = new ArrayBackedTag(tags, pos, curTagLen + TAG_LENGTH_SIZE);
1109          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
1110          return tag;
1111        }
1112        return null;
1113      }
1114
1115      @Override
1116      public void remove() {
1117        throw new UnsupportedOperationException();
1118      }
1119    };
1120  }
1121
1122  /**
1123   * @param cell The Cell
1124   * @return Tags in the given Cell as a List
1125   * @deprecated As of 2.0.0 and will be removed in 3.0.0
1126   */
1127  @Deprecated
1128  public static List<Tag> getTags(Cell cell) {
1129    List<Tag> tags = new ArrayList<>();
1130    Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cell);
1131    while (tagsItr.hasNext()) {
1132      tags.add(tagsItr.next());
1133    }
1134    return tags;
1135  }
1136
1137  /**
1138   * Retrieve Cell's first tag, matching the passed in type
1139   * @param cell The Cell
1140   * @param type Type of the Tag to retrieve
1141   * @return null if there is no tag of the passed in tag type
1142   * @deprecated As of 2.0.0 and will be removed in HBase-3.0.0
1143   *             Use {@link RawCell#getTag(byte)}
1144   */
1145  @Deprecated
1146  public static Tag getTag(Cell cell, byte type) {
1147    Optional<Tag> tag = PrivateCellUtil.getTag(cell, type);
1148    if (tag.isPresent()) {
1149      return tag.get();
1150    } else {
1151      return null;
1152    }
1153  }
1154
1155  /**
1156   * Returns true if the first range start1...end1 overlaps with the second range start2...end2,
1157   * assuming the byte arrays represent row keys
1158   * @deprecated As of 2.0.0 and will be removed in 3.0.0
1159   */
1160  @Deprecated
1161  public static boolean overlappingKeys(final byte[] start1, final byte[] end1, final byte[] start2,
1162      final byte[] end2) {
1163    return (end2.length == 0 || start1.length == 0 || Bytes.compareTo(start1, end2) < 0)
1164        && (end1.length == 0 || start2.length == 0 || Bytes.compareTo(start2, end1) < 0);
1165  }
1166
1167  /**
1168   * Sets the given seqId to the cell. Marked as audience Private as of 1.2.0. Setting a Cell
1169   * sequenceid is an internal implementation detail not for general public use.
1170   * @param cell
1171   * @param seqId
1172   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1173   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1174   */
1175  @Deprecated
1176  public static void setSequenceId(Cell cell, long seqId) throws IOException {
1177    PrivateCellUtil.setSequenceId(cell, seqId);
1178  }
1179
1180  /**
1181   * Sets the given timestamp to the cell.
1182   * @param cell
1183   * @param ts
1184   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1185   * @deprecated As of HBase-2.0. Will be a LimitedPrivate API in HBase-3.0.
1186   */
1187  @Deprecated
1188  public static void setTimestamp(Cell cell, long ts) throws IOException {
1189    PrivateCellUtil.setTimestamp(cell, ts);
1190  }
1191
1192  /**
1193   * Sets the given timestamp to the cell.
1194   * @param cell
1195   * @param ts buffer containing the timestamp value
1196   * @param tsOffset offset to the new timestamp
1197   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1198   * @deprecated As of HBase-2.0. Will be a LimitedPrivate API in HBase-3.0.
1199   */
1200  @Deprecated
1201  public static void setTimestamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
1202    PrivateCellUtil.setTimestamp(cell, Bytes.toLong(ts, tsOffset));
1203  }
1204
1205  /**
1206   * Sets the given timestamp to the cell iff current timestamp is
1207   * {@link HConstants#LATEST_TIMESTAMP}.
1208   * @param cell
1209   * @param ts
1210   * @return True if cell timestamp is modified.
1211   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1212   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1213   */
1214  @Deprecated
1215  public static boolean updateLatestStamp(Cell cell, long ts) throws IOException {
1216    return PrivateCellUtil.updateLatestStamp(cell, ts);
1217  }
1218
1219  /**
1220   * Sets the given timestamp to the cell iff current timestamp is
1221   * {@link HConstants#LATEST_TIMESTAMP}.
1222   * @param cell
1223   * @param ts buffer containing the timestamp value
1224   * @param tsOffset offset to the new timestamp
1225   * @return True if cell timestamp is modified.
1226   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1227   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1228   */
1229  @Deprecated
1230  public static boolean updateLatestStamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
1231    return PrivateCellUtil.updateLatestStamp(cell, Bytes.toLong(ts, tsOffset));
1232  }
1233
1234  /**
1235   * Writes the Cell's key part as it would have serialized in a KeyValue. The format is &lt;2 bytes
1236   * rk len&gt;&lt;rk&gt;&lt;1 byte cf len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes
1237   * timestamp&gt;&lt;1 byte type&gt;
1238   * @param cell
1239   * @param out
1240   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1241   * @throws IOException
1242   */
1243  @Deprecated
1244  public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException {
1245    short rowLen = cell.getRowLength();
1246    byte fLen = cell.getFamilyLength();
1247    int qLen = cell.getQualifierLength();
1248    // Using just one if/else loop instead of every time checking before writing every
1249    // component of cell
1250    if (cell instanceof ByteBufferExtendedCell) {
1251      out.writeShort(rowLen);
1252      ByteBufferUtils
1253        .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1254          ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
1255      out.writeByte(fLen);
1256      ByteBufferUtils
1257        .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
1258          ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
1259      ByteBufferUtils.copyBufferToStream((DataOutput) out,
1260        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
1261        ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
1262    } else {
1263      out.writeShort(rowLen);
1264      out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
1265      out.writeByte(fLen);
1266      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
1267      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen);
1268    }
1269    out.writeLong(cell.getTimestamp());
1270    out.writeByte(cell.getTypeByte());
1271  }
1272
1273  /**
1274   * Writes the row from the given cell to the output stream excluding the common prefix
1275   * @param out The dataoutputstream to which the data has to be written
1276   * @param cell The cell whose contents has to be written
1277   * @param rlength the row length
1278   * @throws IOException
1279   * @deprecated As of 2.0. Will be removed in hbase-3.0
1280   */
1281  @Deprecated
1282  public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
1283    int commonPrefix) throws IOException {
1284    if (cell instanceof ByteBufferExtendedCell) {
1285      ByteBufferUtils
1286        .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1287          ((ByteBufferExtendedCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
1288    } else {
1289      out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
1290    }
1291  }
1292
1293  /**
1294   * @param cell
1295   * @return The Key portion of the passed <code>cell</code> as a String.
1296   */
1297  public static String getCellKeyAsString(Cell cell) {
1298    return getCellKeyAsString(cell,
1299      c -> Bytes.toStringBinary(c.getRowArray(), c.getRowOffset(), c.getRowLength()));
1300  }
1301
1302  /**
1303   * @param cell the cell to convert
1304   * @param rowConverter used to convert the row of the cell to a string
1305   * @return The Key portion of the passed <code>cell</code> as a String.
1306   */
1307  public static String getCellKeyAsString(Cell cell, Function<Cell, String> rowConverter) {
1308    StringBuilder sb = new StringBuilder(rowConverter.apply(cell));
1309    sb.append('/');
1310    sb.append(cell.getFamilyLength() == 0 ? "" :
1311      Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()));
1312    // KeyValue only added ':' if family is non-null. Do same.
1313    if (cell.getFamilyLength() > 0) sb.append(':');
1314    sb.append(cell.getQualifierLength() == 0 ? "" :
1315      Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
1316        cell.getQualifierLength()));
1317    sb.append('/');
1318    sb.append(KeyValue.humanReadableTimestamp(cell.getTimestamp()));
1319    sb.append('/');
1320    sb.append(Type.codeToType(cell.getTypeByte()));
1321    if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
1322      sb.append("/vlen=");
1323      sb.append(cell.getValueLength());
1324    }
1325    sb.append("/seqid=");
1326    sb.append(cell.getSequenceId());
1327    return sb.toString();
1328  }
1329
1330  /**
1331   * This method exists just to encapsulate how we serialize keys. To be replaced by a factory that
1332   * we query to figure what the Cell implementation is and then, what serialization engine to use
1333   * and further, how to serialize the key for inclusion in hfile index. TODO.
1334   * @param cell
1335   * @return The key portion of the Cell serialized in the old-school KeyValue way or null if passed
1336   *         a null <code>cell</code>
1337   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1338   */
1339  @Deprecated
1340  public static byte[] getCellKeySerializedAsKeyValueKey(final Cell cell) {
1341    if (cell == null) return null;
1342    byte[] b = new byte[KeyValueUtil.keyLength(cell)];
1343    KeyValueUtil.appendKeyTo(cell, b, 0);
1344    return b;
1345  }
1346
1347  /**
1348   * Write rowkey excluding the common part.
1349   * @param cell
1350   * @param rLen
1351   * @param commonPrefix
1352   * @param out
1353   * @throws IOException
1354   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1355   */
1356  @Deprecated
1357  public static void writeRowKeyExcludingCommon(Cell cell, short rLen, int commonPrefix,
1358      DataOutputStream out) throws IOException {
1359    if (commonPrefix == 0) {
1360      out.writeShort(rLen);
1361    } else if (commonPrefix == 1) {
1362      out.writeByte((byte) rLen);
1363      commonPrefix--;
1364    } else {
1365      commonPrefix -= KeyValue.ROW_LENGTH_SIZE;
1366    }
1367    if (rLen > commonPrefix) {
1368      PrivateCellUtil.writeRowSkippingBytes(out, cell, rLen, commonPrefix);
1369    }
1370  }
1371
1372  /**
1373   * Find length of common prefix in keys of the cells, considering key as byte[] if serialized in
1374   * {@link KeyValue}. The key format is &lt;2 bytes rk len&gt;&lt;rk&gt;&lt;1 byte cf
1375   * len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes timestamp&gt;&lt;1 byte type&gt;
1376   * @param c1 the cell
1377   * @param c2 the cell
1378   * @param bypassFamilyCheck when true assume the family bytes same in both cells. Pass it as true
1379   *          when dealing with Cells in same CF so as to avoid some checks
1380   * @param withTsType when true check timestamp and type bytes also.
1381   * @return length of common prefix
1382   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1383   */
1384  @Deprecated
1385  public static int findCommonPrefixInFlatKey(Cell c1, Cell c2, boolean bypassFamilyCheck,
1386    boolean withTsType) {
1387    // Compare the 2 bytes in RK length part
1388    short rLen1 = c1.getRowLength();
1389    short rLen2 = c2.getRowLength();
1390    int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
1391    if (rLen1 != rLen2) {
1392      // early out when the RK length itself is not matching
1393      return ByteBufferUtils
1394        .findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE, Bytes.toBytes(rLen2),
1395          0, KeyValue.ROW_LENGTH_SIZE);
1396    }
1397    // Compare the RKs
1398    int rkCommonPrefix = 0;
1399    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1400      rkCommonPrefix = ByteBufferUtils
1401        .findCommonPrefix(((ByteBufferExtendedCell) c1).getRowByteBuffer(),
1402          ((ByteBufferExtendedCell) c1).getRowPosition(), rLen1,
1403          ((ByteBufferExtendedCell) c2).getRowByteBuffer(),
1404          ((ByteBufferExtendedCell) c2).getRowPosition(), rLen2);
1405    } else {
1406      // There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes
1407      // either
1408      // in flush or compactions. In flushes both cells are KV and in case of compaction it will be
1409      // either
1410      // KV or BBCell
1411      rkCommonPrefix = ByteBufferUtils
1412        .findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1, c2.getRowArray(),
1413          c2.getRowOffset(), rLen2);
1414    }
1415    commonPrefix += rkCommonPrefix;
1416    if (rkCommonPrefix != rLen1) {
1417      // Early out when RK is not fully matching.
1418      return commonPrefix;
1419    }
1420    // Compare 1 byte CF length part
1421    byte fLen1 = c1.getFamilyLength();
1422    if (bypassFamilyCheck) {
1423      // This flag will be true when caller is sure that the family will be same for both the cells
1424      // Just make commonPrefix to increment by the family part
1425      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE + fLen1;
1426    } else {
1427      byte fLen2 = c2.getFamilyLength();
1428      if (fLen1 != fLen2) {
1429        // early out when the CF length itself is not matching
1430        return commonPrefix;
1431      }
1432      // CF lengths are same so there is one more byte common in key part
1433      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
1434      // Compare the CF names
1435      int fCommonPrefix;
1436      if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1437        fCommonPrefix = ByteBufferUtils
1438          .findCommonPrefix(((ByteBufferExtendedCell) c1).getFamilyByteBuffer(),
1439            ((ByteBufferExtendedCell) c1).getFamilyPosition(), fLen1,
1440            ((ByteBufferExtendedCell) c2).getFamilyByteBuffer(),
1441            ((ByteBufferExtendedCell) c2).getFamilyPosition(), fLen2);
1442      } else {
1443        fCommonPrefix = ByteBufferUtils
1444          .findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(), fLen1, c2.getFamilyArray(),
1445            c2.getFamilyOffset(), fLen2);
1446      }
1447      commonPrefix += fCommonPrefix;
1448      if (fCommonPrefix != fLen1) {
1449        return commonPrefix;
1450      }
1451    }
1452    // Compare the Qualifiers
1453    int qLen1 = c1.getQualifierLength();
1454    int qLen2 = c2.getQualifierLength();
1455    int qCommon;
1456    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1457      qCommon = ByteBufferUtils
1458        .findCommonPrefix(((ByteBufferExtendedCell) c1).getQualifierByteBuffer(),
1459          ((ByteBufferExtendedCell) c1).getQualifierPosition(), qLen1,
1460          ((ByteBufferExtendedCell) c2).getQualifierByteBuffer(),
1461          ((ByteBufferExtendedCell) c2).getQualifierPosition(), qLen2);
1462    } else {
1463      qCommon = ByteBufferUtils
1464        .findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(), qLen1,
1465          c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
1466    }
1467    commonPrefix += qCommon;
1468    if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
1469      return commonPrefix;
1470    }
1471    // Compare the timestamp parts
1472    int tsCommonPrefix = ByteBufferUtils
1473      .findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE,
1474        Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
1475    commonPrefix += tsCommonPrefix;
1476    if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
1477      return commonPrefix;
1478    }
1479    // Compare the type
1480    if (c1.getTypeByte() == c2.getTypeByte()) {
1481      commonPrefix += KeyValue.TYPE_SIZE;
1482    }
1483    return commonPrefix;
1484  }
1485
1486  /** Returns a string representation of the cell */
1487  public static String toString(Cell cell, boolean verbose) {
1488    if (cell == null) {
1489      return "";
1490    }
1491    StringBuilder builder = new StringBuilder();
1492    String keyStr = getCellKeyAsString(cell);
1493
1494    String tag = null;
1495    String value = null;
1496    if (verbose) {
1497      // TODO: pretty print tags as well
1498      if (cell.getTagsLength() > 0) {
1499        tag = Bytes.toStringBinary(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
1500      }
1501      if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
1502        value = Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
1503          cell.getValueLength());
1504      }
1505    }
1506
1507    builder.append(keyStr);
1508    if (tag != null && !tag.isEmpty()) {
1509      builder.append("/").append(tag);
1510    }
1511    if (value != null) {
1512      builder.append("/").append(value);
1513    }
1514
1515    return builder.toString();
1516  }
1517
1518  /***************** special cases ****************************/
1519
1520  /**
1521   * special case for Cell.equals
1522   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1523   */
1524  @Deprecated
1525  public static boolean equalsIgnoreMvccVersion(Cell a, Cell b) {
1526    // row
1527    boolean res = matchingRows(a, b);
1528    if (!res) return res;
1529
1530    // family
1531    res = matchingColumn(a, b);
1532    if (!res) return res;
1533
1534    // timestamp: later sorts first
1535    if (!matchingTimestamp(a, b)) return false;
1536
1537    // type
1538    int c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
1539    if (c != 0) return false;
1540    else return true;
1541  }
1542
1543  /**************** equals ****************************/
1544
1545  public static boolean equals(Cell a, Cell b) {
1546    return matchingRows(a, b) && matchingFamily(a, b) && matchingQualifier(a, b)
1547        && matchingTimestamp(a, b) && PrivateCellUtil.matchingType(a, b);
1548  }
1549
1550  public static boolean matchingTimestamp(Cell a, Cell b) {
1551    return CellComparator.getInstance().compareTimestamps(a.getTimestamp(), b.getTimestamp()) == 0;
1552  }
1553
1554  /**
1555   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1556   */
1557  @Deprecated
1558  public static boolean matchingType(Cell a, Cell b) {
1559    return a.getTypeByte() == b.getTypeByte();
1560  }
1561
1562  /**
1563   * Compares the row of two keyvalues for equality
1564   * @param left
1565   * @param right
1566   * @return True if rows match.
1567   */
1568  public static boolean matchingRows(final Cell left, final Cell right) {
1569    short lrowlength = left.getRowLength();
1570    short rrowlength = right.getRowLength();
1571    if (lrowlength != rrowlength) return false;
1572    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
1573      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
1574          ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength,
1575          ((ByteBufferExtendedCell) right).getRowByteBuffer(),
1576          ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength);
1577    }
1578    if (left instanceof ByteBufferExtendedCell) {
1579      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
1580          ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
1581          right.getRowOffset(), rrowlength);
1582    }
1583    if (right instanceof ByteBufferExtendedCell) {
1584      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getRowByteBuffer(),
1585          ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
1586          left.getRowOffset(), lrowlength);
1587    }
1588    return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
1589        right.getRowOffset(), rrowlength);
1590  }
1591
1592  /**
1593   * Compares the row and column of two keyvalues for equality
1594   * @param left
1595   * @param right
1596   * @return True if same row and column.
1597   */
1598  public static boolean matchingRowColumn(final Cell left, final Cell right) {
1599    if ((left.getRowLength() + left.getFamilyLength()
1600        + left.getQualifierLength()) != (right.getRowLength() + right.getFamilyLength()
1601            + right.getQualifierLength())) {
1602      return false;
1603    }
1604
1605    if (!matchingRows(left, right)) {
1606      return false;
1607    }
1608    return matchingColumn(left, right);
1609  }
1610
1611  public static boolean matchingRowColumnBytes(final Cell left, final Cell right) {
1612    int lrowlength = left.getRowLength();
1613    int rrowlength = right.getRowLength();
1614    int lfamlength = left.getFamilyLength();
1615    int rfamlength = right.getFamilyLength();
1616    int lqlength = left.getQualifierLength();
1617    int rqlength = right.getQualifierLength();
1618    // match length
1619    if ((lrowlength + lfamlength + lqlength) !=
1620        (rrowlength + rfamlength + rqlength)) {
1621      return false;
1622    }
1623
1624    // match row
1625    if (!Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
1626        right.getRowOffset(), rrowlength)) {
1627      return false;
1628    }
1629    //match family
1630    if (!Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
1631        right.getFamilyArray(), right.getFamilyOffset(), rfamlength)) {
1632      return false;
1633    }
1634    //match qualifier
1635    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
1636        lqlength, right.getQualifierArray(), right.getQualifierOffset(),
1637        rqlength);
1638  }
1639
1640  /**
1641   * Compares the cell's qualifier with the given byte[]
1642   * @param left the cell for which the qualifier has to be compared
1643   * @param right the byte[] having the qualifier
1644   * @param rOffset the offset of the qualifier
1645   * @param rLength the length of the qualifier
1646   * @return greater than 0 if left cell's qualifier is bigger than byte[], lesser than 0 if left
1647   *         cell's qualifier is lesser than byte[] and 0 otherwise
1648   */
1649  public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
1650    if (left instanceof ByteBufferExtendedCell) {
1651      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
1652          ((ByteBufferExtendedCell) left).getQualifierPosition(),
1653          left.getQualifierLength(), right, rOffset, rLength);
1654    }
1655    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
1656      left.getQualifierLength(), right, rOffset, rLength);
1657  }
1658
1659  /**
1660   * Used when a cell needs to be compared with a key byte[] such as cases of finding the index from
1661   * the index block, bloom keys from the bloom blocks This byte[] is expected to be serialized in
1662   * the KeyValue serialization format If the KeyValue (Cell's) serialization format changes this
1663   * method cannot be used.
1664   * @param comparator the cell comparator
1665   * @param left the cell to be compared
1666   * @param key the serialized key part of a KeyValue
1667   * @param offset the offset in the key byte[]
1668   * @param length the length of the key byte[]
1669   * @return an int greater than 0 if left is greater than right lesser than 0 if left is lesser
1670   *         than right equal to 0 if left is equal to right
1671   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1672   */
1673  @InterfaceAudience.Private
1674  @Deprecated
1675  public static final int compare(CellComparator comparator, Cell left, byte[] key, int offset,
1676      int length) {
1677    // row
1678    short rrowlength = Bytes.toShort(key, offset);
1679    int c = comparator.compareRows(left, key, offset + Bytes.SIZEOF_SHORT, rrowlength);
1680    if (c != 0) return c;
1681
1682    // Compare the rest of the two KVs without making any assumptions about
1683    // the common prefix. This function will not compare rows anyway, so we
1684    // don't need to tell it that the common prefix includes the row.
1685    return PrivateCellUtil.compareWithoutRow(comparator, left, key, offset, length, rrowlength);
1686  }
1687
1688  /**
1689   * Compares the cell's family with the given byte[]
1690   * @param left the cell for which the family has to be compared
1691   * @param right the byte[] having the family
1692   * @param roffset the offset of the family
1693   * @param rlength the length of the family
1694   * @return greater than 0 if left cell's family is bigger than byte[], lesser than 0 if left
1695   *         cell's family is lesser than byte[] and 0 otherwise
1696   */
1697  public final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
1698    if (left instanceof ByteBufferExtendedCell) {
1699      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
1700        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
1701        rlength);
1702    }
1703    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
1704      right, roffset, rlength);
1705  }
1706
1707  /**
1708   * Compares the cell's column (family and qualifier) with the given byte[]
1709   * @param left the cell for which the column has to be compared
1710   * @param right the byte[] having the column
1711   * @param rfoffset the offset of the family
1712   * @param rflength the length of the family
1713   * @param rqoffset the offset of the qualifier
1714   * @param rqlength the length of the qualifier
1715   * @return greater than 0 if left cell's column is bigger than byte[], lesser than 0 if left
1716   *         cell's column is lesser than byte[] and 0 otherwise
1717   */
1718  public final static int compareColumns(Cell left, byte[] right, int rfoffset, int rflength,
1719      int rqoffset, int rqlength) {
1720    int diff = compareFamilies(left, right, rfoffset, rflength);
1721    if (diff != 0) return diff;
1722    return compareQualifiers(left, right, rqoffset, rqlength);
1723  }
1724}