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;
023
024import java.io.IOException;
025import java.nio.ByteBuffer;
026import java.util.ArrayList;
027import java.util.Arrays;
028import java.util.Iterator;
029import java.util.List;
030import java.util.Map.Entry;
031import java.util.NavigableMap;
032import java.util.function.Function;
033import org.apache.hadoop.hbase.util.ByteBufferUtils;
034import org.apache.hadoop.hbase.util.Bytes;
035import org.apache.yetus.audience.InterfaceAudience;
036
037/**
038 * Utility methods helpful for slinging {@link Cell} instances. Some methods below are for internal
039 * use only and are marked InterfaceAudience.Private at the method level. Note that all such methods
040 * have been marked deprecated in HBase-2.0 which will be subsequently removed in HBase-3.0
041 */
042@InterfaceAudience.Public
043public final class CellUtil {
044
045  /**
046   * Private constructor to keep this class from being instantiated.
047   */
048  private CellUtil() {
049  }
050
051  /***************** get individual arrays for tests ************/
052
053  public static byte[] cloneRow(Cell cell) {
054    byte[] output = new byte[cell.getRowLength()];
055    copyRowTo(cell, output, 0);
056    return output;
057  }
058
059  public static byte[] cloneFamily(Cell cell) {
060    byte[] output = new byte[cell.getFamilyLength()];
061    copyFamilyTo(cell, output, 0);
062    return output;
063  }
064
065  public static byte[] cloneQualifier(Cell cell) {
066    byte[] output = new byte[cell.getQualifierLength()];
067    copyQualifierTo(cell, output, 0);
068    return output;
069  }
070
071  public static byte[] cloneValue(Cell cell) {
072    byte[] output = new byte[cell.getValueLength()];
073    copyValueTo(cell, output, 0);
074    return output;
075  }
076
077  /**
078   * Makes a column in family:qualifier form from separate byte arrays.
079   * <p>
080   * Not recommended for usage as this is old-style API.
081   * @return family:qualifier
082   */
083  public static byte[] makeColumn(byte[] family, byte[] qualifier) {
084    return Bytes.add(family, COLUMN_FAMILY_DELIM_ARRAY, qualifier);
085  }
086
087  /**
088   * Splits a column in {@code family:qualifier} form into separate byte arrays. An empty qualifier
089   * (ie, {@code fam:}) is parsed as <code>{ fam, EMPTY_BYTE_ARRAY }</code> while no delimiter (ie,
090   * {@code fam}) is parsed as an array of one element, <code>{ fam }</code>.
091   * <p>
092   * Don't forget, HBase DOES support empty qualifiers. (see HBASE-9549)
093   * </p>
094   * <p>
095   * Not recommend to be used as this is old-style API.
096   * </p>
097   * @param c The column.
098   * @return The parsed column.
099   */
100  public static byte[][] parseColumn(byte[] c) {
101    final int index = getDelimiter(c, 0, c.length, COLUMN_FAMILY_DELIMITER);
102    if (index == -1) {
103      // If no delimiter, return array of size 1
104      return new byte[][] { c };
105    } else if (index == c.length - 1) {
106      // family with empty qualifier, return array size 2
107      byte[] family = new byte[c.length - 1];
108      System.arraycopy(c, 0, family, 0, family.length);
109      return new byte[][] { family, HConstants.EMPTY_BYTE_ARRAY };
110    }
111    // Family and column, return array size 2
112    final byte[][] result = new byte[2][];
113    result[0] = new byte[index];
114    System.arraycopy(c, 0, result[0], 0, index);
115    final int len = c.length - (index + 1);
116    result[1] = new byte[len];
117    System.arraycopy(c, index + 1 /* Skip delimiter */, result[1], 0, len);
118    return result;
119  }
120
121  /******************** copyTo **********************************/
122
123  /**
124   * Copies the row to the given byte[]
125   * @param cell              the cell whose row has to be copied
126   * @param destination       the destination byte[] to which the row has to be copied
127   * @param destinationOffset the offset in the destination byte[]
128   * @return the offset of the byte[] after the copy has happened
129   */
130  public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
131    short rowLen = cell.getRowLength();
132    if (cell instanceof ByteBufferExtendedCell) {
133      ByteBufferUtils.copyFromBufferToArray(destination,
134        ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
135        ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
136    } else {
137      System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
138        rowLen);
139    }
140    return destinationOffset + rowLen;
141  }
142
143  /**
144   * Copies the row to the given bytebuffer
145   * @param cell              cell the cell whose row has to be copied
146   * @param destination       the destination bytebuffer to which the row has to be copied
147   * @param destinationOffset the offset in the destination byte[]
148   * @return the offset of the bytebuffer after the copy has happened
149   */
150  public static int copyRowTo(Cell cell, ByteBuffer destination, int destinationOffset) {
151    short rowLen = cell.getRowLength();
152    if (cell instanceof ByteBufferExtendedCell) {
153      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
154        destination, ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
155    } else {
156      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getRowArray(),
157        cell.getRowOffset(), rowLen);
158    }
159    return destinationOffset + rowLen;
160  }
161
162  /**
163   * Copies the row to a new byte[]
164   * @param cell the cell from which row has to copied
165   * @return the byte[] containing the row
166   */
167  public static byte[] copyRow(Cell cell) {
168    if (cell instanceof ByteBufferExtendedCell) {
169      return ByteBufferUtils.copyOfRange(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
170        ((ByteBufferExtendedCell) cell).getRowPosition(),
171        ((ByteBufferExtendedCell) cell).getRowPosition() + cell.getRowLength());
172    } else {
173      return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
174        cell.getRowOffset() + cell.getRowLength());
175    }
176  }
177
178  /**
179   * Copies the family to the given byte[]
180   * @param cell              the cell whose family has to be copied
181   * @param destination       the destination byte[] to which the family has to be copied
182   * @param destinationOffset the offset in the destination byte[]
183   * @return the offset of the byte[] after the copy has happened
184   */
185  public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
186    byte fLen = cell.getFamilyLength();
187    if (cell instanceof ByteBufferExtendedCell) {
188      ByteBufferUtils.copyFromBufferToArray(destination,
189        ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
190        ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
191    } else {
192      System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
193        destinationOffset, fLen);
194    }
195    return destinationOffset + fLen;
196  }
197
198  /**
199   * Copies the family to the given bytebuffer
200   * @param cell              the cell whose family has to be copied
201   * @param destination       the destination bytebuffer to which the family has to be copied
202   * @param destinationOffset the offset in the destination bytebuffer
203   * @return the offset of the bytebuffer after the copy has happened
204   */
205  public static int copyFamilyTo(Cell cell, ByteBuffer destination, int destinationOffset) {
206    byte fLen = cell.getFamilyLength();
207    if (cell instanceof ByteBufferExtendedCell) {
208      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
209        destination, ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
210    } else {
211      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getFamilyArray(),
212        cell.getFamilyOffset(), fLen);
213    }
214    return destinationOffset + fLen;
215  }
216
217  /**
218   * Copies the qualifier to the given byte[]
219   * @param cell              the cell whose qualifier has to be copied
220   * @param destination       the destination byte[] to which the qualifier has to be copied
221   * @param destinationOffset the offset in the destination byte[]
222   * @return the offset of the byte[] after the copy has happened
223   */
224  public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
225    int qlen = cell.getQualifierLength();
226    if (cell instanceof ByteBufferExtendedCell) {
227      ByteBufferUtils.copyFromBufferToArray(destination,
228        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
229        ((ByteBufferExtendedCell) cell).getQualifierPosition(), destinationOffset, qlen);
230    } else {
231      System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
232        destinationOffset, qlen);
233    }
234    return destinationOffset + qlen;
235  }
236
237  /**
238   * Copies the qualifier to the given bytebuffer
239   * @param cell              the cell whose qualifier has to be copied
240   * @param destination       the destination bytebuffer to which the qualifier has to be copied
241   * @param destinationOffset the offset in the destination bytebuffer
242   * @return the offset of the bytebuffer after the copy has happened
243   */
244  public static int copyQualifierTo(Cell cell, ByteBuffer destination, int destinationOffset) {
245    int qlen = cell.getQualifierLength();
246    if (cell instanceof ByteBufferExtendedCell) {
247      ByteBufferUtils.copyFromBufferToBuffer(
248        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(), destination,
249        ((ByteBufferExtendedCell) cell).getQualifierPosition(), destinationOffset, qlen);
250    } else {
251      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset,
252        cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
253    }
254    return destinationOffset + qlen;
255  }
256
257  /**
258   * Copies the value to the given byte[]
259   * @param cell              the cell whose value has to be copied
260   * @param destination       the destination byte[] to which the value has to be copied
261   * @param destinationOffset the offset in the destination byte[]
262   * @return the offset of the byte[] after the copy has happened
263   */
264  public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
265    int vlen = cell.getValueLength();
266    if (cell instanceof ByteBufferExtendedCell) {
267      ByteBufferUtils.copyFromBufferToArray(destination,
268        ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
269        ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
270    } else {
271      System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
272        vlen);
273    }
274    return destinationOffset + vlen;
275  }
276
277  /**
278   * Copies the value to the given bytebuffer
279   * @param cell              the cell whose value has to be copied
280   * @param destination       the destination bytebuffer to which the value has to be copied
281   * @param destinationOffset the offset in the destination bytebuffer
282   * @return the offset of the bytebuffer after the copy has happened
283   */
284  public static int copyValueTo(Cell cell, ByteBuffer destination, int destinationOffset) {
285    int vlen = cell.getValueLength();
286    if (cell instanceof ByteBufferExtendedCell) {
287      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
288        destination, ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
289    } else {
290      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getValueArray(),
291        cell.getValueOffset(), vlen);
292    }
293    return destinationOffset + vlen;
294  }
295
296  /** Returns CellScanner interface over <code>cellIterables</code> */
297  public static CellScanner
298    createCellScanner(final List<? extends CellScannable> cellScannerables) {
299    return new CellScanner() {
300      private final Iterator<? extends CellScannable> iterator = cellScannerables.iterator();
301      private CellScanner cellScanner = null;
302
303      @Override
304      public Cell current() {
305        return this.cellScanner != null ? this.cellScanner.current() : null;
306      }
307
308      @Override
309      public boolean advance() throws IOException {
310        while (true) {
311          if (this.cellScanner == null) {
312            if (!this.iterator.hasNext()) return false;
313            this.cellScanner = this.iterator.next().cellScanner();
314          }
315          if (this.cellScanner.advance()) return true;
316          this.cellScanner = null;
317        }
318      }
319    };
320  }
321
322  /** Returns CellScanner interface over <code>cellIterable</code> */
323  public static CellScanner createCellScanner(final Iterable<Cell> cellIterable) {
324    if (cellIterable == null) return null;
325    return createCellScanner(cellIterable.iterator());
326  }
327
328  /**
329   * Returns CellScanner interface over <code>cellIterable</code> or null if <code>cells</code> is
330   * null
331   */
332  public static CellScanner createCellScanner(final Iterator<Cell> cells) {
333    if (cells == null) return null;
334    return new CellScanner() {
335      private final Iterator<Cell> iterator = cells;
336      private Cell current = null;
337
338      @Override
339      public Cell current() {
340        return this.current;
341      }
342
343      @Override
344      public boolean advance() {
345        boolean hasNext = this.iterator.hasNext();
346        this.current = hasNext ? this.iterator.next() : null;
347        return hasNext;
348      }
349    };
350  }
351
352  /** Returns CellScanner interface over <code>cellArray</code> */
353  public static CellScanner createCellScanner(final Cell[] cellArray) {
354    return new CellScanner() {
355      private final Cell[] cells = cellArray;
356      private int index = -1;
357
358      @Override
359      public Cell current() {
360        if (cells == null) return null;
361        return (index < 0) ? null : this.cells[index];
362      }
363
364      @Override
365      public boolean advance() {
366        if (cells == null) return false;
367        return ++index < this.cells.length;
368      }
369    };
370  }
371
372  /**
373   * Flatten the map of cells out under the CellScanner
374   * @param map Map of Cell Lists; for example, the map of families to Cells that is used inside
375   *            Put, etc., keeping Cells organized by family.
376   * @return CellScanner interface over <code>cellIterable</code>
377   */
378  public static CellScanner createCellScanner(final NavigableMap<byte[], List<Cell>> map) {
379    return new CellScanner() {
380      private final Iterator<Entry<byte[], List<Cell>>> entries = map.entrySet().iterator();
381      private Iterator<Cell> currentIterator = null;
382      private Cell currentCell;
383
384      @Override
385      public Cell current() {
386        return this.currentCell;
387      }
388
389      @Override
390      public boolean advance() {
391        while (true) {
392          if (this.currentIterator == null) {
393            if (!this.entries.hasNext()) return false;
394            this.currentIterator = this.entries.next().getValue().iterator();
395          }
396          if (this.currentIterator.hasNext()) {
397            this.currentCell = this.currentIterator.next();
398            return true;
399          }
400          this.currentCell = null;
401          this.currentIterator = null;
402        }
403      }
404    };
405  }
406
407  public static boolean matchingRows(final Cell left, final byte[] buf) {
408    if (buf == null) {
409      return left.getRowLength() == 0;
410    }
411    return PrivateCellUtil.matchingRows(left, buf, 0, buf.length);
412  }
413
414  public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
415    final int length) {
416    return PrivateCellUtil.matchingRows(left, buf, offset, length);
417  }
418
419  public static boolean matchingFamily(final Cell left, final Cell right) {
420    byte lfamlength = left.getFamilyLength();
421    byte rfamlength = right.getFamilyLength();
422    return matchingFamily(left, lfamlength, right, rfamlength);
423  }
424
425  public static boolean matchingFamily(final Cell left, final byte lfamlength, final Cell right,
426    final byte rfamlength) {
427    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
428      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
429        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
430        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
431        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength);
432    }
433    if (left instanceof ByteBufferExtendedCell) {
434      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
435        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength, right.getFamilyArray(),
436        right.getFamilyOffset(), rfamlength);
437    }
438    if (right instanceof ByteBufferExtendedCell) {
439      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
440        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength, left.getFamilyArray(),
441        left.getFamilyOffset(), lfamlength);
442    }
443    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
444      right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
445  }
446
447  public static boolean matchingFamily(final Cell left, final byte[] buf) {
448    if (buf == null) {
449      return left.getFamilyLength() == 0;
450    }
451    return PrivateCellUtil.matchingFamily(left, buf, 0, buf.length);
452  }
453
454  public static boolean matchingQualifier(final Cell left, final Cell right) {
455    int lqlength = left.getQualifierLength();
456    int rqlength = right.getQualifierLength();
457    return matchingQualifier(left, lqlength, right, rqlength);
458  }
459
460  private static boolean matchingQualifier(final Cell left, final int lqlength, final Cell right,
461    final int rqlength) {
462    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
463      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
464        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
465        ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
466        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength);
467    }
468    if (left instanceof ByteBufferExtendedCell) {
469      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
470        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength, right.getQualifierArray(),
471        right.getQualifierOffset(), rqlength);
472    }
473    if (right instanceof ByteBufferExtendedCell) {
474      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
475        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength, left.getQualifierArray(),
476        left.getQualifierOffset(), lqlength);
477    }
478    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
479      right.getQualifierArray(), right.getQualifierOffset(), rqlength);
480  }
481
482  /**
483   * Finds if the qualifier part of the cell and the KV serialized byte[] are equal.
484   * @return true if the qualifier matches, false otherwise
485   */
486  public static boolean matchingQualifier(final Cell left, final byte[] buf) {
487    if (buf == null) {
488      return left.getQualifierLength() == 0;
489    }
490    return PrivateCellUtil.matchingQualifier(left, buf, 0, buf.length);
491  }
492
493  public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
494    return matchingFamily(left, fam) && matchingQualifier(left, qual);
495  }
496
497  /** Returns True if matching column family and the qualifier starts with <code>qual</code> */
498  public static boolean matchingColumnFamilyAndQualifierPrefix(final Cell left, final byte[] fam,
499    final byte[] qual) {
500    return matchingFamily(left, fam) && PrivateCellUtil.qualifierStartsWith(left, qual);
501  }
502
503  public static boolean matchingColumn(final Cell left, final Cell right) {
504    if (!matchingFamily(left, right)) return false;
505    return matchingQualifier(left, right);
506  }
507
508  private static boolean matchingColumn(final Cell left, final byte lFamLen, final int lQualLength,
509    final Cell right, final byte rFamLen, final int rQualLength) {
510    if (!matchingFamily(left, lFamLen, right, rFamLen)) {
511      return false;
512    }
513    return matchingQualifier(left, lQualLength, right, rQualLength);
514  }
515
516  public static boolean matchingValue(final Cell left, final Cell right) {
517    return matchingValue(left, right, left.getValueLength(), right.getValueLength());
518  }
519
520  public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
521    int rvlength) {
522    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
523      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
524        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
525        ((ByteBufferExtendedCell) right).getValueByteBuffer(),
526        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
527    }
528    if (left instanceof ByteBufferExtendedCell) {
529      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
530        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
531        right.getValueOffset(), rvlength);
532    }
533    if (right instanceof ByteBufferExtendedCell) {
534      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
535        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
536        left.getValueOffset(), lvlength);
537    }
538    return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
539      right.getValueArray(), right.getValueOffset(), rvlength);
540  }
541
542  public static boolean matchingValue(final Cell left, final byte[] buf) {
543    if (left instanceof ByteBufferExtendedCell) {
544      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getValueByteBuffer(),
545        ((ByteBufferExtendedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
546        buf.length) == 0;
547    }
548    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
549      buf.length);
550  }
551
552  public static boolean matchingTags(final Cell left, final Cell right) {
553    return PrivateCellUtil.matchingTags(left, right, left.getTagsLength(), right.getTagsLength());
554  }
555
556  /**
557   * Return true if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
558   * or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
559   */
560  @SuppressWarnings("deprecation")
561  public static boolean isDelete(final Cell cell) {
562    return PrivateCellUtil.isDelete(cell.getTypeByte());
563  }
564
565  /** Returns True if this cell is a Put. */
566  @SuppressWarnings("deprecation")
567  public static boolean isPut(Cell cell) {
568    return cell.getTypeByte() == KeyValue.Type.Put.getCode();
569  }
570
571  /**
572   * Sets the given timestamp to the cell. Note that this method is a LimitedPrivate API and may
573   * change between minor releases.
574   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
575   */
576  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
577  public static void setTimestamp(Cell cell, long ts) throws IOException {
578    PrivateCellUtil.setTimestamp(cell, ts);
579  }
580
581  /**
582   * Sets the given timestamp to the cell. Note that this method is a LimitedPrivate API and may
583   * change between minor releases.
584   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
585   */
586  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
587  public static void setTimestamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
588    PrivateCellUtil.setTimestamp(cell, Bytes.toLong(ts, tsOffset));
589  }
590
591  /** Returns The Key portion of the passed <code>cell</code> as a String. */
592  public static String getCellKeyAsString(Cell cell) {
593    return getCellKeyAsString(cell,
594      c -> Bytes.toStringBinary(c.getRowArray(), c.getRowOffset(), c.getRowLength()));
595  }
596
597  /**
598   * Return the Key portion of the passed <code>cell</code> as a String.
599   * @param cell         the cell to convert
600   * @param rowConverter used to convert the row of the cell to a string
601   * @return The Key portion of the passed <code>cell</code> as a String.
602   */
603  public static String getCellKeyAsString(Cell cell, Function<Cell, String> rowConverter) {
604    StringBuilder sb = new StringBuilder(rowConverter.apply(cell));
605    sb.append('/');
606    sb.append(cell.getFamilyLength() == 0
607      ? ""
608      : Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
609        cell.getFamilyLength()));
610    // KeyValue only added ':' if family is non-null. Do same.
611    if (cell.getFamilyLength() > 0) sb.append(':');
612    sb.append(cell.getQualifierLength() == 0
613      ? ""
614      : Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
615        cell.getQualifierLength()));
616    sb.append('/');
617    sb.append(KeyValue.humanReadableTimestamp(cell.getTimestamp()));
618    sb.append('/');
619    sb.append(KeyValue.Type.codeToType(cell.getTypeByte()));
620    if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
621      sb.append("/vlen=");
622      sb.append(cell.getValueLength());
623    }
624    sb.append("/seqid=");
625    sb.append(cell.getSequenceId());
626    return sb.toString();
627  }
628
629  /** Returns a string representation of the cell */
630  public static String toString(Cell cell, boolean verbose) {
631    if (cell == null) {
632      return "";
633    }
634    StringBuilder builder = new StringBuilder();
635    String keyStr = getCellKeyAsString(cell);
636
637    String tag = null;
638    String value = null;
639    if (verbose) {
640      // TODO: pretty print tags as well
641      if (cell.getTagsLength() > 0) {
642        tag = Bytes.toStringBinary(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
643      }
644      if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
645        value =
646          Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
647      }
648    }
649
650    builder.append(keyStr);
651    if (tag != null && !tag.isEmpty()) {
652      builder.append("/").append(tag);
653    }
654    if (value != null) {
655      builder.append("/").append(value);
656    }
657
658    return builder.toString();
659  }
660
661  /**************** equals ****************************/
662
663  public static boolean equals(Cell a, Cell b) {
664    return matchingRows(a, b) && matchingFamily(a, b) && matchingQualifier(a, b)
665      && matchingTimestamp(a, b) && PrivateCellUtil.matchingType(a, b);
666  }
667
668  public static boolean matchingTimestamp(Cell a, Cell b) {
669    return CellComparator.getInstance().compareTimestamps(a.getTimestamp(), b.getTimestamp()) == 0;
670  }
671
672  /** Compares the row of two keyvalues for equality */
673  public static boolean matchingRows(final Cell left, final Cell right) {
674    short lrowlength = left.getRowLength();
675    short rrowlength = right.getRowLength();
676    return matchingRows(left, lrowlength, right, rrowlength);
677  }
678
679  /** Compares the row of two keyvalues for equality */
680  public static boolean matchingRows(final Cell left, final short lrowlength, final Cell right,
681    final short rrowlength) {
682    if (lrowlength != rrowlength) return false;
683    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
684      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
685        ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength,
686        ((ByteBufferExtendedCell) right).getRowByteBuffer(),
687        ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength);
688    }
689    if (left instanceof ByteBufferExtendedCell) {
690      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
691        ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
692        right.getRowOffset(), rrowlength);
693    }
694    if (right instanceof ByteBufferExtendedCell) {
695      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getRowByteBuffer(),
696        ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
697        left.getRowOffset(), lrowlength);
698    }
699    return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
700      right.getRowOffset(), rrowlength);
701  }
702
703  /** Compares the row and column of two keyvalues for equality */
704  public static boolean matchingRowColumn(final Cell left, final Cell right) {
705    short lrowlength = left.getRowLength();
706    short rrowlength = right.getRowLength();
707    // match length
708    if (lrowlength != rrowlength) {
709      return false;
710    }
711
712    byte lfamlength = left.getFamilyLength();
713    byte rfamlength = right.getFamilyLength();
714    if (lfamlength != rfamlength) {
715      return false;
716    }
717
718    int lqlength = left.getQualifierLength();
719    int rqlength = right.getQualifierLength();
720    if (lqlength != rqlength) {
721      return false;
722    }
723
724    if (!matchingRows(left, lrowlength, right, rrowlength)) {
725      return false;
726    }
727    return matchingColumn(left, lfamlength, lqlength, right, rfamlength, rqlength);
728  }
729
730  /** Compares the row and column of two keyvalues for equality */
731  public static boolean matchingRowColumnBytes(final Cell left, final Cell right) {
732    int lrowlength = left.getRowLength();
733    int rrowlength = right.getRowLength();
734    int lfamlength = left.getFamilyLength();
735    int rfamlength = right.getFamilyLength();
736    int lqlength = left.getQualifierLength();
737    int rqlength = right.getQualifierLength();
738
739    // match length
740    if ((lrowlength != rrowlength) || (lfamlength != rfamlength) || (lqlength != rqlength)) {
741      return false;
742    }
743
744    // match row
745    if (
746      !Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
747        right.getRowOffset(), rrowlength)
748    ) {
749      return false;
750    }
751    // match family
752    if (
753      !Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
754        right.getFamilyArray(), right.getFamilyOffset(), rfamlength)
755    ) {
756      return false;
757    }
758    // match qualifier
759    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
760      right.getQualifierArray(), right.getQualifierOffset(), rqlength);
761  }
762
763  /**
764   * Compares the cell's qualifier with the given byte[]
765   * @param left    the cell for which the qualifier has to be compared
766   * @param right   the byte[] having the qualifier
767   * @param rOffset the offset of the qualifier
768   * @param rLength the length of the qualifier
769   * @return greater than 0 if left cell's qualifier is bigger than byte[], lesser than 0 if left
770   *         cell's qualifier is lesser than byte[] and 0 otherwise
771   */
772  public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
773    if (left instanceof ByteBufferExtendedCell) {
774      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
775        ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(), right,
776        rOffset, rLength);
777    }
778    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
779      left.getQualifierLength(), right, rOffset, rLength);
780  }
781
782  /**
783   * Compares the cell's family with the given byte[]
784   * @param left    the cell for which the family has to be compared
785   * @param right   the byte[] having the family
786   * @param roffset the offset of the family
787   * @param rlength the length of the family
788   * @return greater than 0 if left cell's family is bigger than byte[], lesser than 0 if left
789   *         cell's family is lesser than byte[] and 0 otherwise
790   */
791  public final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
792    if (left instanceof ByteBufferExtendedCell) {
793      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
794        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
795        rlength);
796    }
797    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
798      right, roffset, rlength);
799  }
800
801  /**
802   * Compares the cell's column (family and qualifier) with the given byte[]
803   * @param left     the cell for which the column has to be compared
804   * @param right    the byte[] having the column
805   * @param rfoffset the offset of the family
806   * @param rflength the length of the family
807   * @param rqoffset the offset of the qualifier
808   * @param rqlength the length of the qualifier
809   * @return greater than 0 if left cell's column is bigger than byte[], lesser than 0 if left
810   *         cell's column is lesser than byte[] and 0 otherwise
811   */
812  public final static int compareColumns(Cell left, byte[] right, int rfoffset, int rflength,
813    int rqoffset, int rqlength) {
814    int diff = compareFamilies(left, right, rfoffset, rflength);
815    if (diff != 0) return diff;
816    return compareQualifiers(left, right, rqoffset, rqlength);
817  }
818
819  public static void cloneIfNecessary(ArrayList<Cell> cells) {
820    if (cells == null || cells.isEmpty()) {
821      return;
822    }
823    for (int i = 0; i < cells.size(); i++) {
824      Cell cell = cells.get(i);
825      cells.set(i, cloneIfNecessary(cell));
826    }
827  }
828
829  public static Cell cloneIfNecessary(Cell cell) {
830    return (cell instanceof ByteBufferExtendedCell ? KeyValueUtil.copyToNewKeyValue(cell) : cell);
831  }
832}