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