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