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  /** Returns CellScanner interface over <code>cellIterables</code> */
588  public static CellScanner
589    createCellScanner(final List<? extends CellScannable> cellScannerables) {
590    return new CellScanner() {
591      private final Iterator<? extends CellScannable> iterator = cellScannerables.iterator();
592      private CellScanner cellScanner = null;
593
594      @Override
595      public Cell current() {
596        return this.cellScanner != null ? this.cellScanner.current() : null;
597      }
598
599      @Override
600      public boolean advance() throws IOException {
601        while (true) {
602          if (this.cellScanner == null) {
603            if (!this.iterator.hasNext()) return false;
604            this.cellScanner = this.iterator.next().cellScanner();
605          }
606          if (this.cellScanner.advance()) return true;
607          this.cellScanner = null;
608        }
609      }
610    };
611  }
612
613  /** Returns CellScanner interface over <code>cellIterable</code> */
614  public static CellScanner createCellScanner(final Iterable<Cell> cellIterable) {
615    if (cellIterable == null) return null;
616    return createCellScanner(cellIterable.iterator());
617  }
618
619  /**
620   * Returns CellScanner interface over <code>cellIterable</code> or null if <code>cells</code> is
621   * null
622   */
623  public static CellScanner createCellScanner(final Iterator<Cell> cells) {
624    if (cells == null) return null;
625    return new CellScanner() {
626      private final Iterator<Cell> iterator = cells;
627      private Cell current = null;
628
629      @Override
630      public Cell current() {
631        return this.current;
632      }
633
634      @Override
635      public boolean advance() {
636        boolean hasNext = this.iterator.hasNext();
637        this.current = hasNext ? this.iterator.next() : null;
638        return hasNext;
639      }
640    };
641  }
642
643  /** Returns CellScanner interface over <code>cellArray</code> */
644  public static CellScanner createCellScanner(final Cell[] cellArray) {
645    return new CellScanner() {
646      private final Cell[] cells = cellArray;
647      private int index = -1;
648
649      @Override
650      public Cell current() {
651        if (cells == null) return null;
652        return (index < 0) ? null : this.cells[index];
653      }
654
655      @Override
656      public boolean advance() {
657        if (cells == null) return false;
658        return ++index < this.cells.length;
659      }
660    };
661  }
662
663  /**
664   * Flatten the map of cells out under the CellScanner
665   * @param map Map of Cell Lists; for example, the map of families to Cells that is used inside
666   *            Put, etc., keeping Cells organized by family.
667   * @return CellScanner interface over <code>cellIterable</code>
668   */
669  public static CellScanner createCellScanner(final NavigableMap<byte[], List<Cell>> map) {
670    return new CellScanner() {
671      private final Iterator<Entry<byte[], List<Cell>>> entries = map.entrySet().iterator();
672      private Iterator<Cell> currentIterator = null;
673      private Cell currentCell;
674
675      @Override
676      public Cell current() {
677        return this.currentCell;
678      }
679
680      @Override
681      public boolean advance() {
682        while (true) {
683          if (this.currentIterator == null) {
684            if (!this.entries.hasNext()) return false;
685            this.currentIterator = this.entries.next().getValue().iterator();
686          }
687          if (this.currentIterator.hasNext()) {
688            this.currentCell = this.currentIterator.next();
689            return true;
690          }
691          this.currentCell = null;
692          this.currentIterator = null;
693        }
694      }
695    };
696  }
697
698  /**
699   * Return true if the rows in <code>left</code> and <code>right</code> Cells match
700   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
701   *             {@link #matchingRows(Cell, Cell)}
702   */
703  @Deprecated
704  public static boolean matchingRow(final Cell left, final Cell right) {
705    return matchingRows(left, right);
706  }
707
708  /**
709   * Return true if the row is matching
710   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
711   *             {@link #matchingRows(Cell, byte[])}
712   */
713  @Deprecated
714  public static boolean matchingRow(final Cell left, final byte[] buf) {
715    return matchingRows(left, buf);
716  }
717
718  public static boolean matchingRows(final Cell left, final byte[] buf) {
719    if (buf == null) {
720      return left.getRowLength() == 0;
721    }
722    return PrivateCellUtil.matchingRows(left, buf, 0, buf.length);
723  }
724
725  /**
726   * Return true if the row is matching
727   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
728   *             {@link #matchingRows(Cell, Cell)}
729   */
730  @Deprecated
731  public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
732    final int length) {
733    if (left instanceof ByteBufferExtendedCell) {
734      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
735        ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), buf, offset, length);
736    }
737    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
738      length);
739  }
740
741  public static boolean matchingFamily(final Cell left, final Cell right) {
742    byte lfamlength = left.getFamilyLength();
743    byte rfamlength = right.getFamilyLength();
744    return matchingFamily(left, lfamlength, right, rfamlength);
745  }
746
747  public static boolean matchingFamily(final Cell left, final byte lfamlength, final Cell right,
748    final byte rfamlength) {
749    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
750      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
751        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
752        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
753        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength);
754    }
755    if (left instanceof ByteBufferExtendedCell) {
756      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
757        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength, right.getFamilyArray(),
758        right.getFamilyOffset(), rfamlength);
759    }
760    if (right instanceof ByteBufferExtendedCell) {
761      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
762        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength, left.getFamilyArray(),
763        left.getFamilyOffset(), lfamlength);
764    }
765    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
766      right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
767  }
768
769  public static boolean matchingFamily(final Cell left, final byte[] buf) {
770    if (buf == null) {
771      return left.getFamilyLength() == 0;
772    }
773    return PrivateCellUtil.matchingFamily(left, buf, 0, buf.length);
774  }
775
776  /**
777   * Return true if the family is matching
778   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
779   */
780  @Deprecated
781  public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
782    final int length) {
783    if (left instanceof ByteBufferExtendedCell) {
784      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
785        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), buf, offset,
786        length);
787    }
788    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
789      offset, length);
790  }
791
792  public static boolean matchingQualifier(final Cell left, final Cell right) {
793    int lqlength = left.getQualifierLength();
794    int rqlength = right.getQualifierLength();
795    return matchingQualifier(left, lqlength, right, rqlength);
796  }
797
798  private static boolean matchingQualifier(final Cell left, final int lqlength, final Cell right,
799    final int rqlength) {
800    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
801      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
802        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
803        ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
804        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength);
805    }
806    if (left instanceof ByteBufferExtendedCell) {
807      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
808        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength, right.getQualifierArray(),
809        right.getQualifierOffset(), rqlength);
810    }
811    if (right instanceof ByteBufferExtendedCell) {
812      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
813        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength, left.getQualifierArray(),
814        left.getQualifierOffset(), lqlength);
815    }
816    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
817      right.getQualifierArray(), right.getQualifierOffset(), rqlength);
818  }
819
820  /**
821   * Finds if the qualifier part of the cell and the KV serialized byte[] are equal.
822   * @return true if the qualifier matches, false otherwise
823   */
824  public static boolean matchingQualifier(final Cell left, final byte[] buf) {
825    if (buf == null) {
826      return left.getQualifierLength() == 0;
827    }
828    return PrivateCellUtil.matchingQualifier(left, buf, 0, buf.length);
829  }
830
831  /**
832   * Finds if the qualifier part of the cell and the KV serialized byte[] are equal
833   * @param buf    the serialized keyvalue format byte[]
834   * @param offset the offset of the qualifier in the byte[]
835   * @param length the length of the qualifier in the byte[]
836   * @return true if the qualifier matches, false otherwise
837   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
838   */
839  @Deprecated
840  public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
841    final int length) {
842    if (buf == null) {
843      return left.getQualifierLength() == 0;
844    }
845    if (left instanceof ByteBufferExtendedCell) {
846      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
847        ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(), buf,
848        offset, length);
849    }
850    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
851      left.getQualifierLength(), buf, offset, length);
852  }
853
854  public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
855    return matchingFamily(left, fam) && matchingQualifier(left, qual);
856  }
857
858  /** Returns True if matching column family and the qualifier starts with <code>qual</code> */
859  public static boolean matchingColumnFamilyAndQualifierPrefix(final Cell left, final byte[] fam,
860    final byte[] qual) {
861    return matchingFamily(left, fam) && PrivateCellUtil.qualifierStartsWith(left, qual);
862  }
863
864  /**
865   * Return true if the column is matching
866   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
867   */
868  @Deprecated
869  public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
870    final int flength, final byte[] qual, final int qoffset, final int qlength) {
871    if (!PrivateCellUtil.matchingFamily(left, fam, foffset, flength)) return false;
872    return PrivateCellUtil.matchingQualifier(left, qual, qoffset, qlength);
873  }
874
875  public static boolean matchingColumn(final Cell left, final Cell right) {
876    if (!matchingFamily(left, right)) return false;
877    return matchingQualifier(left, right);
878  }
879
880  private static boolean matchingColumn(final Cell left, final byte lFamLen, final int lQualLength,
881    final Cell right, final byte rFamLen, final int rQualLength) {
882    if (!matchingFamily(left, lFamLen, right, rFamLen)) {
883      return false;
884    }
885    return matchingQualifier(left, lQualLength, right, rQualLength);
886  }
887
888  public static boolean matchingValue(final Cell left, final Cell right) {
889    return PrivateCellUtil.matchingValue(left, right, left.getValueLength(),
890      right.getValueLength());
891  }
892
893  public static boolean matchingValue(final Cell left, final byte[] buf) {
894    if (left instanceof ByteBufferExtendedCell) {
895      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getValueByteBuffer(),
896        ((ByteBufferExtendedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
897        buf.length) == 0;
898    }
899    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
900      buf.length);
901  }
902
903  public static boolean matchingTags(final Cell left, final Cell right) {
904    return PrivateCellUtil.matchingTags(left, right, left.getTagsLength(), right.getTagsLength());
905  }
906
907  /**
908   * Return true if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
909   * or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
910   */
911  @SuppressWarnings("deprecation")
912  public static boolean isDelete(final Cell cell) {
913    return PrivateCellUtil.isDelete(cell.getTypeByte());
914  }
915
916  /**
917   * Return true if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
918   * or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
919   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
920   */
921  @Deprecated
922  public static boolean isDelete(final byte type) {
923    return KeyValue.Type.Delete.getCode() <= type && type <= KeyValue.Type.DeleteFamily.getCode();
924  }
925
926  /**
927   * Return true if this cell is a {@link KeyValue.Type#Delete} type.
928   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
929   */
930  @Deprecated
931  public static boolean isDeleteType(Cell cell) {
932    return cell.getTypeByte() == KeyValue.Type.Delete.getCode();
933  }
934
935  /**
936   * Check whether the given cell is a delete family
937   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
938   */
939  @Deprecated
940  public static boolean isDeleteFamily(final Cell cell) {
941    return cell.getTypeByte() == KeyValue.Type.DeleteFamily.getCode();
942  }
943
944  /**
945   * Check whether the given cell is a delete family version
946   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
947   */
948  @Deprecated
949  public static boolean isDeleteFamilyVersion(final Cell cell) {
950    return cell.getTypeByte() == KeyValue.Type.DeleteFamilyVersion.getCode();
951  }
952
953  /**
954   * Check whether the given cell is a delete columns
955   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
956   */
957  @Deprecated
958  public static boolean isDeleteColumns(final Cell cell) {
959    return cell.getTypeByte() == KeyValue.Type.DeleteColumn.getCode();
960  }
961
962  /**
963   * Check whether the given cell is a delete column version
964   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
965   */
966  @Deprecated
967  public static boolean isDeleteColumnVersion(final Cell cell) {
968    return cell.getTypeByte() == KeyValue.Type.Delete.getCode();
969  }
970
971  /**
972   * Return true if this cell is a delete family or column type.
973   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
974   */
975  @Deprecated
976  public static boolean isDeleteColumnOrFamily(Cell cell) {
977    int t = cell.getTypeByte();
978    return t == KeyValue.Type.DeleteColumn.getCode() || t == KeyValue.Type.DeleteFamily.getCode();
979  }
980
981  /** Returns True if this cell is a Put. */
982  @SuppressWarnings("deprecation")
983  public static boolean isPut(Cell cell) {
984    return cell.getTypeByte() == KeyValue.Type.Put.getCode();
985  }
986
987  /**
988   * Estimate based on keyvalue's serialization format in the RPC layer. Note that there is an extra
989   * SIZEOF_INT added to the size here that indicates the actual length of the cell for cases where
990   * cell's are serialized in a contiguous format (For eg in RPCs).
991   * @return Estimate of the <code>cell</code> size in bytes plus an extra SIZEOF_INT indicating the
992   *         actual cell length.
993   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
994   */
995  @Deprecated
996  public static int estimatedSerializedSizeOf(final Cell cell) {
997    if (cell instanceof ExtendedCell) {
998      return ((ExtendedCell) cell).getSerializedSize(true) + Bytes.SIZEOF_INT;
999    }
1000
1001    return getSumOfCellElementLengths(cell) +
1002    // Use the KeyValue's infrastructure size presuming that another implementation would have
1003    // same basic cost.
1004      KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE +
1005      // Serialization is probably preceded by a length (it is in the KeyValueCodec at least).
1006      Bytes.SIZEOF_INT;
1007  }
1008
1009  /**
1010   * Return sum of the lengths of all the elements in a Cell; does not count in any infrastructure
1011   */
1012  private static int getSumOfCellElementLengths(final Cell cell) {
1013    return getSumOfCellKeyElementLengths(cell) + cell.getValueLength() + cell.getTagsLength();
1014  }
1015
1016  /**
1017   * Return sum of all elements that make up a key; does not include infrastructure, tags or values.
1018   */
1019  private static int getSumOfCellKeyElementLengths(final Cell cell) {
1020    return cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength()
1021      + KeyValue.TIMESTAMP_TYPE_SIZE;
1022  }
1023
1024  /**
1025   * Calculates the serialized key size. We always serialize in the KeyValue's serialization format.
1026   * @param cell the cell for which the key size has to be calculated.
1027   * @return the key size
1028   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
1029   */
1030  @Deprecated
1031  public static int estimatedSerializedSizeOfKey(final Cell cell) {
1032    if (cell instanceof KeyValue) return ((KeyValue) cell).getKeyLength();
1033    return cell.getRowLength() + cell.getFamilyLength() + cell.getQualifierLength()
1034      + KeyValue.KEY_INFRASTRUCTURE_SIZE;
1035  }
1036
1037  /**
1038   * This is an estimate of the heap space occupied by a cell. When the cell is of type
1039   * {@link HeapSize} we call {@link HeapSize#heapSize()} so cell can give a correct value. In other
1040   * cases we just consider the bytes occupied by the cell components ie. row, CF, qualifier,
1041   * timestamp, type, value and tags.
1042   * @return estimate of the heap space
1043   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
1044   *             {@link RawCell#getTags()}
1045   */
1046  @Deprecated
1047  public static long estimatedHeapSizeOf(final Cell cell) {
1048    return cell.heapSize();
1049  }
1050
1051  /********************* tags *************************************/
1052  /**
1053   * Util method to iterate through the tags
1054   * @return iterator for the tags
1055   * @deprecated As of 2.0.0 and will be removed in 3.0.0 Instead use
1056   *             {@link PrivateCellUtil#tagsIterator(Cell)}
1057   */
1058  @Deprecated
1059  public static Iterator<Tag> tagsIterator(final byte[] tags, final int offset, final int length) {
1060    return new Iterator<Tag>() {
1061      private int pos = offset;
1062      private int endOffset = offset + length - 1;
1063
1064      @Override
1065      public boolean hasNext() {
1066        return this.pos < endOffset;
1067      }
1068
1069      @Override
1070      public Tag next() {
1071        if (hasNext()) {
1072          int curTagLen = Bytes.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
1073          Tag tag = new ArrayBackedTag(tags, pos, curTagLen + TAG_LENGTH_SIZE);
1074          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
1075          return tag;
1076        }
1077        return null;
1078      }
1079
1080      @Override
1081      public void remove() {
1082        throw new UnsupportedOperationException();
1083      }
1084    };
1085  }
1086
1087  /**
1088   * Return tags in the given Cell as a List
1089   * @deprecated As of 2.0.0 and will be removed in 3.0.0
1090   */
1091  @Deprecated
1092  public static List<Tag> getTags(Cell cell) {
1093    List<Tag> tags = new ArrayList<>();
1094    Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(cell);
1095    while (tagsItr.hasNext()) {
1096      tags.add(tagsItr.next());
1097    }
1098    return tags;
1099  }
1100
1101  /**
1102   * Retrieve Cell's first tag, matching the passed in type
1103   * @param cell The Cell
1104   * @param type Type of the Tag to retrieve
1105   * @return null if there is no tag of the passed in tag type
1106   * @deprecated As of 2.0.0 and will be removed in HBase-3.0.0 Use {@link RawCell#getTag(byte)}
1107   */
1108  @Deprecated
1109  public static Tag getTag(Cell cell, byte type) {
1110    Optional<Tag> tag = PrivateCellUtil.getTag(cell, type);
1111    if (tag.isPresent()) {
1112      return tag.get();
1113    } else {
1114      return null;
1115    }
1116  }
1117
1118  /**
1119   * Returns true if the first range start1...end1 overlaps with the second range start2...end2,
1120   * assuming the byte arrays represent row keys
1121   * @deprecated As of 2.0.0 and will be removed in 3.0.0
1122   */
1123  @Deprecated
1124  public static boolean overlappingKeys(final byte[] start1, final byte[] end1, final byte[] start2,
1125    final byte[] end2) {
1126    return (end2.length == 0 || start1.length == 0 || Bytes.compareTo(start1, end2) < 0)
1127      && (end1.length == 0 || start2.length == 0 || Bytes.compareTo(start2, end1) < 0);
1128  }
1129
1130  /**
1131   * Sets the given seqId to the cell. Marked as audience Private as of 1.2.0. Setting a Cell
1132   * sequenceid is an internal implementation detail not for general public use.
1133   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1134   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1135   */
1136  @Deprecated
1137  public static void setSequenceId(Cell cell, long seqId) throws IOException {
1138    PrivateCellUtil.setSequenceId(cell, seqId);
1139  }
1140
1141  /**
1142   * Sets the given timestamp to the cell. Note that this method is a LimitedPrivate API and may
1143   * change between minor releases.
1144   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1145   * @deprecated As of HBase-2.0. Will be a LimitedPrivate API in HBase-3.0.
1146   */
1147  @Deprecated
1148  public static void setTimestamp(Cell cell, long ts) throws IOException {
1149    PrivateCellUtil.setTimestamp(cell, ts);
1150  }
1151
1152  /**
1153   * Sets the given timestamp to the cell. Note that this method is a LimitedPrivate API and may
1154   * change between minor releases.
1155   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1156   * @deprecated As of HBase-2.0. Will be a LimitedPrivate API in HBase-3.0.
1157   */
1158  @Deprecated
1159  public static void setTimestamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
1160    PrivateCellUtil.setTimestamp(cell, Bytes.toLong(ts, tsOffset));
1161  }
1162
1163  /**
1164   * Sets the given timestamp to the cell iff current timestamp is
1165   * {@link HConstants#LATEST_TIMESTAMP}.
1166   * @return True if cell timestamp is modified.
1167   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1168   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1169   */
1170  @Deprecated
1171  public static boolean updateLatestStamp(Cell cell, long ts) throws IOException {
1172    return PrivateCellUtil.updateLatestStamp(cell, ts);
1173  }
1174
1175  /**
1176   * Sets the given timestamp to the cell iff current timestamp is
1177   * {@link HConstants#LATEST_TIMESTAMP}.
1178   * @param ts       buffer containing the timestamp value
1179   * @param tsOffset offset to the new timestamp
1180   * @return True if cell timestamp is modified.
1181   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
1182   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1183   */
1184  @Deprecated
1185  public static boolean updateLatestStamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
1186    return PrivateCellUtil.updateLatestStamp(cell, Bytes.toLong(ts, tsOffset));
1187  }
1188
1189  /**
1190   * Writes the Cell's key part as it would have serialized in a KeyValue. The format is &lt;2 bytes
1191   * rk len&gt;&lt;rk&gt;&lt;1 byte cf len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes
1192   * timestamp&gt;&lt;1 byte type&gt;
1193   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1194   */
1195  @Deprecated
1196  public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException {
1197    short rowLen = cell.getRowLength();
1198    byte fLen = cell.getFamilyLength();
1199    int qLen = cell.getQualifierLength();
1200    // Using just one if/else loop instead of every time checking before writing every
1201    // component of cell
1202    if (cell instanceof ByteBufferExtendedCell) {
1203      out.writeShort(rowLen);
1204      ByteBufferUtils.copyBufferToStream((DataOutput) out,
1205        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1206        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
1207      out.writeByte(fLen);
1208      ByteBufferUtils.copyBufferToStream((DataOutput) out,
1209        ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
1210        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
1211      ByteBufferUtils.copyBufferToStream((DataOutput) out,
1212        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
1213        ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
1214    } else {
1215      out.writeShort(rowLen);
1216      out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
1217      out.writeByte(fLen);
1218      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
1219      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen);
1220    }
1221    out.writeLong(cell.getTimestamp());
1222    out.writeByte(cell.getTypeByte());
1223  }
1224
1225  /**
1226   * Writes the row from the given cell to the output stream excluding the common prefix
1227   * @param out     The dataoutputstream to which the data has to be written
1228   * @param cell    The cell whose contents has to be written
1229   * @param rlength the row length
1230   * @deprecated As of 2.0. Will be removed in hbase-3.0
1231   */
1232  @Deprecated
1233  public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
1234    int commonPrefix) throws IOException {
1235    if (cell instanceof ByteBufferExtendedCell) {
1236      ByteBufferUtils.copyBufferToStream((DataOutput) out,
1237        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
1238        ((ByteBufferExtendedCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
1239    } else {
1240      out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
1241    }
1242  }
1243
1244  /**
1245   * Return the key portion of the passed <code>cell</code> as a String.
1246   */
1247  public static String getCellKeyAsString(Cell cell) {
1248    return getCellKeyAsString(cell,
1249      c -> Bytes.toStringBinary(c.getRowArray(), c.getRowOffset(), c.getRowLength()));
1250  }
1251
1252  /**
1253   * Return the Key portion of the passed <code>cell</code> as a String.
1254   * @param cell         the cell to convert
1255   * @param rowConverter used to convert the row of the cell to a string
1256   * @return The Key portion of the passed <code>cell</code> as a String.
1257   */
1258  public static String getCellKeyAsString(Cell cell, Function<Cell, String> rowConverter) {
1259    StringBuilder sb = new StringBuilder(rowConverter.apply(cell));
1260    sb.append('/');
1261    sb.append(cell.getFamilyLength() == 0
1262      ? ""
1263      : Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
1264        cell.getFamilyLength()));
1265    // KeyValue only added ':' if family is non-null. Do same.
1266    if (cell.getFamilyLength() > 0) sb.append(':');
1267    sb.append(cell.getQualifierLength() == 0
1268      ? ""
1269      : Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
1270        cell.getQualifierLength()));
1271    sb.append('/');
1272    sb.append(KeyValue.humanReadableTimestamp(cell.getTimestamp()));
1273    sb.append('/');
1274    sb.append(KeyValue.Type.codeToType(cell.getTypeByte()));
1275    if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
1276      sb.append("/vlen=");
1277      sb.append(cell.getValueLength());
1278    }
1279    sb.append("/seqid=");
1280    sb.append(cell.getSequenceId());
1281    return sb.toString();
1282  }
1283
1284  /**
1285   * This method exists just to encapsulate how we serialize keys. To be replaced by a factory that
1286   * we query to figure what the Cell implementation is and then, what serialization engine to use
1287   * and further, how to serialize the key for inclusion in hfile index. TODO.
1288   * @return The key portion of the Cell serialized in the old-school KeyValue way or null if passed
1289   *         a null <code>cell</code>
1290   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1291   */
1292  @Deprecated
1293  public static byte[] getCellKeySerializedAsKeyValueKey(final Cell cell) {
1294    if (cell == null) return null;
1295    byte[] b = new byte[KeyValueUtil.keyLength(cell)];
1296    KeyValueUtil.appendKeyTo(cell, b, 0);
1297    return b;
1298  }
1299
1300  /**
1301   * Write rowkey excluding the common part.
1302   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1303   */
1304  @Deprecated
1305  public static void writeRowKeyExcludingCommon(Cell cell, short rLen, int commonPrefix,
1306    DataOutputStream out) throws IOException {
1307    if (commonPrefix == 0) {
1308      out.writeShort(rLen);
1309    } else if (commonPrefix == 1) {
1310      out.writeByte((byte) rLen);
1311      commonPrefix--;
1312    } else {
1313      commonPrefix -= KeyValue.ROW_LENGTH_SIZE;
1314    }
1315    if (rLen > commonPrefix) {
1316      PrivateCellUtil.writeRowSkippingBytes(out, cell, rLen, commonPrefix);
1317    }
1318  }
1319
1320  /**
1321   * Find length of common prefix in keys of the cells, considering key as byte[] if serialized in
1322   * {@link KeyValue}. The key format is &lt;2 bytes rk len&gt;&lt;rk&gt;&lt;1 byte cf
1323   * len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes timestamp&gt;&lt;1 byte type&gt;
1324   * @param c1                the cell
1325   * @param c2                the cell
1326   * @param bypassFamilyCheck when true assume the family bytes same in both cells. Pass it as true
1327   *                          when dealing with Cells in same CF so as to avoid some checks
1328   * @param withTsType        when true check timestamp and type bytes also.
1329   * @return length of common prefix
1330   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1331   */
1332  @Deprecated
1333  public static int findCommonPrefixInFlatKey(Cell c1, Cell c2, boolean bypassFamilyCheck,
1334    boolean withTsType) {
1335    // Compare the 2 bytes in RK length part
1336    short rLen1 = c1.getRowLength();
1337    short rLen2 = c2.getRowLength();
1338    int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
1339    if (rLen1 != rLen2) {
1340      // early out when the RK length itself is not matching
1341      return ByteBufferUtils.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE,
1342        Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
1343    }
1344    // Compare the RKs
1345    int rkCommonPrefix = 0;
1346    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1347      rkCommonPrefix =
1348        ByteBufferUtils.findCommonPrefix(((ByteBufferExtendedCell) c1).getRowByteBuffer(),
1349          ((ByteBufferExtendedCell) c1).getRowPosition(), rLen1,
1350          ((ByteBufferExtendedCell) c2).getRowByteBuffer(),
1351          ((ByteBufferExtendedCell) c2).getRowPosition(), rLen2);
1352    } else {
1353      // There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes
1354      // either
1355      // in flush or compactions. In flushes both cells are KV and in case of compaction it will be
1356      // either
1357      // KV or BBCell
1358      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1,
1359        c2.getRowArray(), c2.getRowOffset(), rLen2);
1360    }
1361    commonPrefix += rkCommonPrefix;
1362    if (rkCommonPrefix != rLen1) {
1363      // Early out when RK is not fully matching.
1364      return commonPrefix;
1365    }
1366    // Compare 1 byte CF length part
1367    byte fLen1 = c1.getFamilyLength();
1368    if (bypassFamilyCheck) {
1369      // This flag will be true when caller is sure that the family will be same for both the cells
1370      // Just make commonPrefix to increment by the family part
1371      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE + fLen1;
1372    } else {
1373      byte fLen2 = c2.getFamilyLength();
1374      if (fLen1 != fLen2) {
1375        // early out when the CF length itself is not matching
1376        return commonPrefix;
1377      }
1378      // CF lengths are same so there is one more byte common in key part
1379      commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
1380      // Compare the CF names
1381      int fCommonPrefix;
1382      if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1383        fCommonPrefix =
1384          ByteBufferUtils.findCommonPrefix(((ByteBufferExtendedCell) c1).getFamilyByteBuffer(),
1385            ((ByteBufferExtendedCell) c1).getFamilyPosition(), fLen1,
1386            ((ByteBufferExtendedCell) c2).getFamilyByteBuffer(),
1387            ((ByteBufferExtendedCell) c2).getFamilyPosition(), fLen2);
1388      } else {
1389        fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(),
1390          fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
1391      }
1392      commonPrefix += fCommonPrefix;
1393      if (fCommonPrefix != fLen1) {
1394        return commonPrefix;
1395      }
1396    }
1397    // Compare the Qualifiers
1398    int qLen1 = c1.getQualifierLength();
1399    int qLen2 = c2.getQualifierLength();
1400    int qCommon;
1401    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
1402      qCommon =
1403        ByteBufferUtils.findCommonPrefix(((ByteBufferExtendedCell) c1).getQualifierByteBuffer(),
1404          ((ByteBufferExtendedCell) c1).getQualifierPosition(), qLen1,
1405          ((ByteBufferExtendedCell) c2).getQualifierByteBuffer(),
1406          ((ByteBufferExtendedCell) c2).getQualifierPosition(), qLen2);
1407    } else {
1408      qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
1409        qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
1410    }
1411    commonPrefix += qCommon;
1412    if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
1413      return commonPrefix;
1414    }
1415    // Compare the timestamp parts
1416    int tsCommonPrefix = ByteBufferUtils.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0,
1417      KeyValue.TIMESTAMP_SIZE, Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
1418    commonPrefix += tsCommonPrefix;
1419    if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
1420      return commonPrefix;
1421    }
1422    // Compare the type
1423    if (c1.getTypeByte() == c2.getTypeByte()) {
1424      commonPrefix += KeyValue.TYPE_SIZE;
1425    }
1426    return commonPrefix;
1427  }
1428
1429  /** Returns a string representation of the cell */
1430  public static String toString(Cell cell, boolean verbose) {
1431    if (cell == null) {
1432      return "";
1433    }
1434    StringBuilder builder = new StringBuilder();
1435    String keyStr = getCellKeyAsString(cell);
1436
1437    String tag = null;
1438    String value = null;
1439    if (verbose) {
1440      // TODO: pretty print tags as well
1441      if (cell.getTagsLength() > 0) {
1442        tag = Bytes.toStringBinary(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
1443      }
1444      if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
1445        value =
1446          Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
1447      }
1448    }
1449
1450    builder.append(keyStr);
1451    if (tag != null && !tag.isEmpty()) {
1452      builder.append("/").append(tag);
1453    }
1454    if (value != null) {
1455      builder.append("/").append(value);
1456    }
1457
1458    return builder.toString();
1459  }
1460
1461  /***************** special cases ****************************/
1462
1463  /**
1464   * special case for Cell.equals
1465   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1466   */
1467  @Deprecated
1468  public static boolean equalsIgnoreMvccVersion(Cell a, Cell b) {
1469    // row
1470    boolean res = matchingRows(a, b);
1471    if (!res) return res;
1472
1473    // family
1474    res = matchingColumn(a, b);
1475    if (!res) return res;
1476
1477    // timestamp: later sorts first
1478    if (!matchingTimestamp(a, b)) return false;
1479
1480    // type
1481    int c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
1482    if (c != 0) return false;
1483    else return true;
1484  }
1485
1486  /**************** equals ****************************/
1487
1488  public static boolean equals(Cell a, Cell b) {
1489    return matchingRows(a, b) && matchingFamily(a, b) && matchingQualifier(a, b)
1490      && matchingTimestamp(a, b) && PrivateCellUtil.matchingType(a, b);
1491  }
1492
1493  public static boolean matchingTimestamp(Cell a, Cell b) {
1494    return CellComparator.getInstance().compareTimestamps(a.getTimestamp(), b.getTimestamp()) == 0;
1495  }
1496
1497  /**
1498   * Check whether the type of these two cells are the same
1499   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1500   */
1501  @Deprecated
1502  public static boolean matchingType(Cell a, Cell b) {
1503    return a.getTypeByte() == b.getTypeByte();
1504  }
1505
1506  /** Compares the row of two keyvalues for equality */
1507  public static boolean matchingRows(final Cell left, final Cell right) {
1508    short lrowlength = left.getRowLength();
1509    short rrowlength = right.getRowLength();
1510    return matchingRows(left, lrowlength, right, rrowlength);
1511  }
1512
1513  /** Compares the row of two keyvalues for equality */
1514  public static boolean matchingRows(final Cell left, final short lrowlength, final Cell right,
1515    final short rrowlength) {
1516    if (lrowlength != rrowlength) return false;
1517    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
1518      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
1519        ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength,
1520        ((ByteBufferExtendedCell) right).getRowByteBuffer(),
1521        ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength);
1522    }
1523    if (left instanceof ByteBufferExtendedCell) {
1524      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
1525        ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
1526        right.getRowOffset(), rrowlength);
1527    }
1528    if (right instanceof ByteBufferExtendedCell) {
1529      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getRowByteBuffer(),
1530        ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
1531        left.getRowOffset(), lrowlength);
1532    }
1533    return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
1534      right.getRowOffset(), rrowlength);
1535  }
1536
1537  /** Compares the row and column of two keyvalues for equality */
1538  public static boolean matchingRowColumn(final Cell left, final Cell right) {
1539    short lrowlength = left.getRowLength();
1540    short rrowlength = right.getRowLength();
1541    // match length
1542    if (lrowlength != rrowlength) {
1543      return false;
1544    }
1545
1546    byte lfamlength = left.getFamilyLength();
1547    byte rfamlength = right.getFamilyLength();
1548    if (lfamlength != rfamlength) {
1549      return false;
1550    }
1551
1552    int lqlength = left.getQualifierLength();
1553    int rqlength = right.getQualifierLength();
1554    if (lqlength != rqlength) {
1555      return false;
1556    }
1557
1558    if (!matchingRows(left, lrowlength, right, rrowlength)) {
1559      return false;
1560    }
1561    return matchingColumn(left, lfamlength, lqlength, right, rfamlength, rqlength);
1562  }
1563
1564  /** Compares the row and column of two keyvalues for equality */
1565  public static boolean matchingRowColumnBytes(final Cell left, final Cell right) {
1566    int lrowlength = left.getRowLength();
1567    int rrowlength = right.getRowLength();
1568    int lfamlength = left.getFamilyLength();
1569    int rfamlength = right.getFamilyLength();
1570    int lqlength = left.getQualifierLength();
1571    int rqlength = right.getQualifierLength();
1572
1573    // match length
1574    if ((lrowlength != rrowlength) || (lfamlength != rfamlength) || (lqlength != rqlength)) {
1575      return false;
1576    }
1577
1578    // match row
1579    if (
1580      !Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
1581        right.getRowOffset(), rrowlength)
1582    ) {
1583      return false;
1584    }
1585    // match family
1586    if (
1587      !Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
1588        right.getFamilyArray(), right.getFamilyOffset(), rfamlength)
1589    ) {
1590      return false;
1591    }
1592    // match qualifier
1593    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
1594      right.getQualifierArray(), right.getQualifierOffset(), rqlength);
1595  }
1596
1597  /**
1598   * Compares the cell's qualifier with the given byte[]
1599   * @param left    the cell for which the qualifier has to be compared
1600   * @param right   the byte[] having the qualifier
1601   * @param rOffset the offset of the qualifier
1602   * @param rLength the length of the qualifier
1603   * @return greater than 0 if left cell's qualifier is bigger than byte[], lesser than 0 if left
1604   *         cell's qualifier is lesser than byte[] and 0 otherwise
1605   */
1606  public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
1607    if (left instanceof ByteBufferExtendedCell) {
1608      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
1609        ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(), right,
1610        rOffset, rLength);
1611    }
1612    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
1613      left.getQualifierLength(), right, rOffset, rLength);
1614  }
1615
1616  /**
1617   * Used when a cell needs to be compared with a key byte[] such as cases of finding the index from
1618   * the index block, bloom keys from the bloom blocks This byte[] is expected to be serialized in
1619   * the KeyValue serialization format If the KeyValue (Cell's) serialization format changes this
1620   * method cannot be used.
1621   * @param comparator the cell comparator
1622   * @param left       the cell to be compared
1623   * @param key        the serialized key part of a KeyValue
1624   * @param offset     the offset in the key byte[]
1625   * @param length     the length of the key byte[]
1626   * @return an int greater than 0 if left is greater than right lesser than 0 if left is lesser
1627   *         than right equal to 0 if left is equal to right
1628   * @deprecated As of HBase-2.0. Will be removed in HBase-3.0
1629   */
1630  @InterfaceAudience.Private
1631  @Deprecated
1632  public static final int compare(CellComparator comparator, Cell left, byte[] key, int offset,
1633    int length) {
1634    // row
1635    short rrowlength = Bytes.toShort(key, offset);
1636    int c = comparator.compareRows(left, key, offset + Bytes.SIZEOF_SHORT, rrowlength);
1637    if (c != 0) return c;
1638
1639    // Compare the rest of the two KVs without making any assumptions about
1640    // the common prefix. This function will not compare rows anyway, so we
1641    // don't need to tell it that the common prefix includes the row.
1642    return PrivateCellUtil.compareWithoutRow(comparator, left, key, offset, length, rrowlength);
1643  }
1644
1645  /**
1646   * Compares the cell's family with the given byte[]
1647   * @param left    the cell for which the family has to be compared
1648   * @param right   the byte[] having the family
1649   * @param roffset the offset of the family
1650   * @param rlength the length of the family
1651   * @return greater than 0 if left cell's family is bigger than byte[], lesser than 0 if left
1652   *         cell's family is lesser than byte[] and 0 otherwise
1653   */
1654  public final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
1655    if (left instanceof ByteBufferExtendedCell) {
1656      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
1657        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
1658        rlength);
1659    }
1660    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
1661      right, roffset, rlength);
1662  }
1663
1664  /**
1665   * Compares the cell's column (family and qualifier) with the given byte[]
1666   * @param left     the cell for which the column has to be compared
1667   * @param right    the byte[] having the column
1668   * @param rfoffset the offset of the family
1669   * @param rflength the length of the family
1670   * @param rqoffset the offset of the qualifier
1671   * @param rqlength the length of the qualifier
1672   * @return greater than 0 if left cell's column is bigger than byte[], lesser than 0 if left
1673   *         cell's column is lesser than byte[] and 0 otherwise
1674   */
1675  public final static int compareColumns(Cell left, byte[] right, int rfoffset, int rflength,
1676    int rqoffset, int rqlength) {
1677    int diff = compareFamilies(left, right, rfoffset, rflength);
1678    if (diff != 0) return diff;
1679    return compareQualifiers(left, right, rqoffset, rqlength);
1680  }
1681
1682  public static void cloneIfNecessary(ArrayList<Cell> cells) {
1683    if (cells == null || cells.isEmpty()) {
1684      return;
1685    }
1686    for (int i = 0; i < cells.size(); i++) {
1687      Cell cell = cells.get(i);
1688      cells.set(i, cloneIfNecessary(cell));
1689    }
1690  }
1691
1692  public static Cell cloneIfNecessary(Cell cell) {
1693    return (cell instanceof ByteBufferExtendedCell ? KeyValueUtil.copyToNewKeyValue(cell) : cell);
1694  }
1695}