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