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()) {
313              return false;
314            }
315            this.cellScanner = this.iterator.next().cellScanner();
316          }
317          if (this.cellScanner.advance()) {
318            return true;
319          }
320          this.cellScanner = null;
321        }
322      }
323    };
324  }
325
326  /** Returns CellScanner interface over <code>cellIterable</code> */
327  public static CellScanner createCellScanner(final Iterable<Cell> cellIterable) {
328    if (cellIterable == null) return null;
329    return createCellScanner(cellIterable.iterator());
330  }
331
332  /**
333   * Returns CellScanner interface over <code>cellIterable</code> or null if <code>cells</code> is
334   * null
335   */
336  public static CellScanner createCellScanner(final Iterator<Cell> cells) {
337    if (cells == null) return null;
338    return new CellScanner() {
339      private final Iterator<Cell> iterator = cells;
340      private Cell current = null;
341
342      @Override
343      public Cell current() {
344        return this.current;
345      }
346
347      @Override
348      public boolean advance() {
349        boolean hasNext = this.iterator.hasNext();
350        this.current = hasNext ? this.iterator.next() : null;
351        return hasNext;
352      }
353    };
354  }
355
356  /** Returns CellScanner interface over <code>cellArray</code> */
357  public static CellScanner createCellScanner(final Cell[] cellArray) {
358    return new CellScanner() {
359      private final Cell[] cells = cellArray;
360      private int index = -1;
361
362      @Override
363      public Cell current() {
364        if (cells == null) {
365          return null;
366        }
367        return (index < 0) ? null : this.cells[index];
368      }
369
370      @Override
371      public boolean advance() {
372        if (cells == null) {
373          return false;
374        }
375        return ++index < this.cells.length;
376      }
377    };
378  }
379
380  /**
381   * Flatten the map of cells out under the CellScanner
382   * @param map Map of Cell Lists; for example, the map of families to Cells that is used inside
383   *            Put, etc., keeping Cells organized by family.
384   * @return CellScanner interface over <code>cellIterable</code>
385   */
386  public static CellScanner createCellScanner(final NavigableMap<byte[], List<Cell>> map) {
387    return new CellScanner() {
388      private final Iterator<Entry<byte[], List<Cell>>> entries = map.entrySet().iterator();
389      private Iterator<Cell> currentIterator = null;
390      private Cell currentCell;
391
392      @Override
393      public Cell current() {
394        return this.currentCell;
395      }
396
397      @Override
398      public boolean advance() {
399        while (true) {
400          if (this.currentIterator == null) {
401            if (!this.entries.hasNext()) return false;
402            this.currentIterator = this.entries.next().getValue().iterator();
403          }
404          if (this.currentIterator.hasNext()) {
405            this.currentCell = this.currentIterator.next();
406            return true;
407          }
408          this.currentCell = null;
409          this.currentIterator = null;
410        }
411      }
412    };
413  }
414
415  public static boolean matchingRows(final Cell left, final byte[] buf) {
416    if (buf == null) {
417      return left.getRowLength() == 0;
418    }
419    return PrivateCellUtil.matchingRows(left, buf, 0, buf.length);
420  }
421
422  public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
423    final int length) {
424    return PrivateCellUtil.matchingRows(left, buf, offset, length);
425  }
426
427  public static boolean matchingFamily(final Cell left, final Cell right) {
428    byte lfamlength = left.getFamilyLength();
429    byte rfamlength = right.getFamilyLength();
430    return matchingFamily(left, lfamlength, right, rfamlength);
431  }
432
433  public static boolean matchingFamily(final Cell left, final byte lfamlength, final Cell right,
434    final byte rfamlength) {
435    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
436      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
437        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
438        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
439        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength);
440    }
441    if (left instanceof ByteBufferExtendedCell) {
442      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
443        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength, right.getFamilyArray(),
444        right.getFamilyOffset(), rfamlength);
445    }
446    if (right instanceof ByteBufferExtendedCell) {
447      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
448        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength, left.getFamilyArray(),
449        left.getFamilyOffset(), lfamlength);
450    }
451    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
452      right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
453  }
454
455  public static boolean matchingFamily(final Cell left, final byte[] buf) {
456    if (buf == null) {
457      return left.getFamilyLength() == 0;
458    }
459    return PrivateCellUtil.matchingFamily(left, buf, 0, buf.length);
460  }
461
462  public static boolean matchingQualifier(final Cell left, final Cell right) {
463    int lqlength = left.getQualifierLength();
464    int rqlength = right.getQualifierLength();
465    return matchingQualifier(left, lqlength, right, rqlength);
466  }
467
468  private static boolean matchingQualifier(final Cell left, final int lqlength, final Cell right,
469    final int rqlength) {
470    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
471      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
472        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
473        ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
474        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength);
475    }
476    if (left instanceof ByteBufferExtendedCell) {
477      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
478        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength, right.getQualifierArray(),
479        right.getQualifierOffset(), rqlength);
480    }
481    if (right instanceof ByteBufferExtendedCell) {
482      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
483        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength, left.getQualifierArray(),
484        left.getQualifierOffset(), lqlength);
485    }
486    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
487      right.getQualifierArray(), right.getQualifierOffset(), rqlength);
488  }
489
490  /**
491   * Finds if the qualifier part of the cell and the KV serialized byte[] are equal.
492   * @return true if the qualifier matches, false otherwise
493   */
494  public static boolean matchingQualifier(final Cell left, final byte[] buf) {
495    if (buf == null) {
496      return left.getQualifierLength() == 0;
497    }
498    return PrivateCellUtil.matchingQualifier(left, buf, 0, buf.length);
499  }
500
501  public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
502    return matchingFamily(left, fam) && matchingQualifier(left, qual);
503  }
504
505  /** Returns True if matching column family and the qualifier starts with <code>qual</code> */
506  public static boolean matchingColumnFamilyAndQualifierPrefix(final Cell left, final byte[] fam,
507    final byte[] qual) {
508    return matchingFamily(left, fam) && PrivateCellUtil.qualifierStartsWith(left, qual);
509  }
510
511  public static boolean matchingColumn(final Cell left, final Cell right) {
512    if (!matchingFamily(left, right)) return false;
513    return matchingQualifier(left, right);
514  }
515
516  private static boolean matchingColumn(final Cell left, final byte lFamLen, final int lQualLength,
517    final Cell right, final byte rFamLen, final int rQualLength) {
518    if (!matchingFamily(left, lFamLen, right, rFamLen)) {
519      return false;
520    }
521    return matchingQualifier(left, lQualLength, right, rQualLength);
522  }
523
524  public static boolean matchingValue(final Cell left, final Cell right) {
525    return matchingValue(left, right, left.getValueLength(), right.getValueLength());
526  }
527
528  public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
529    int rvlength) {
530    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
531      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
532        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
533        ((ByteBufferExtendedCell) right).getValueByteBuffer(),
534        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
535    }
536    if (left instanceof ByteBufferExtendedCell) {
537      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
538        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
539        right.getValueOffset(), rvlength);
540    }
541    if (right instanceof ByteBufferExtendedCell) {
542      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
543        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
544        left.getValueOffset(), lvlength);
545    }
546    return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
547      right.getValueArray(), right.getValueOffset(), rvlength);
548  }
549
550  public static boolean matchingValue(final Cell left, final byte[] buf) {
551    if (left instanceof ByteBufferExtendedCell) {
552      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getValueByteBuffer(),
553        ((ByteBufferExtendedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
554        buf.length) == 0;
555    }
556    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
557      buf.length);
558  }
559
560  /**
561   * @deprecated Since 3.0.0, will be removed in 4.0.0. Tags are now internal only, you should not
562   *             try to check it through the {@link Cell} interface.
563   */
564  @Deprecated
565  public static boolean matchingTags(final Cell left, final Cell right) {
566    return PrivateCellUtil.matchingTags((ExtendedCell) left, (ExtendedCell) right);
567  }
568
569  /**
570   * Return true if a delete type, a {@link KeyValue.Type#Delete} or a {KeyValue.Type#DeleteFamily}
571   * or a {@link KeyValue.Type#DeleteColumn} KeyValue type.
572   */
573  public static boolean isDelete(final Cell cell) {
574    return PrivateCellUtil.isDelete(PrivateCellUtil.getTypeByte(cell));
575  }
576
577  /** Returns True if this cell is a Put. */
578  public static boolean isPut(Cell cell) {
579    return PrivateCellUtil.getTypeByte(cell) == KeyValue.Type.Put.getCode();
580  }
581
582  /**
583   * Sets the given timestamp to the cell. Note that this method is a LimitedPrivate API and may
584   * change between minor releases.
585   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
586   */
587  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
588  public static void setTimestamp(Cell cell, long ts) throws IOException {
589    PrivateCellUtil.setTimestamp(cell, ts);
590  }
591
592  /**
593   * Sets the given timestamp to the cell. Note that this method is a LimitedPrivate API and may
594   * change between minor releases.
595   * @throws IOException when the passed cell is not of type {@link ExtendedCell}
596   */
597  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
598  public static void setTimestamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
599    PrivateCellUtil.setTimestamp(cell, Bytes.toLong(ts, tsOffset));
600  }
601
602  /** Returns The Key portion of the passed <code>cell</code> as a String. */
603  public static String getCellKeyAsString(Cell cell) {
604    return getCellKeyAsString(cell,
605      c -> Bytes.toStringBinary(c.getRowArray(), c.getRowOffset(), c.getRowLength()));
606  }
607
608  /**
609   * Return the Key portion of the passed <code>cell</code> as a String.
610   * @param cell         the cell to convert
611   * @param rowConverter used to convert the row of the cell to a string
612   * @return The Key portion of the passed <code>cell</code> as a String.
613   */
614  public static String getCellKeyAsString(Cell cell, Function<Cell, String> rowConverter) {
615    StringBuilder sb = new StringBuilder(rowConverter.apply(cell));
616    sb.append('/');
617    sb.append(cell.getFamilyLength() == 0
618      ? ""
619      : Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
620        cell.getFamilyLength()));
621    // KeyValue only added ':' if family is non-null. Do same.
622    if (cell.getFamilyLength() > 0) sb.append(':');
623    sb.append(cell.getQualifierLength() == 0
624      ? ""
625      : Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
626        cell.getQualifierLength()));
627    sb.append('/');
628    sb.append(KeyValue.humanReadableTimestamp(cell.getTimestamp()));
629    sb.append('/');
630    if (cell instanceof ExtendedCell) {
631      sb.append(KeyValue.Type.codeToType(((ExtendedCell) cell).getTypeByte()));
632    } else {
633      sb.append(cell.getType());
634    }
635
636    if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
637      sb.append("/vlen=");
638      sb.append(cell.getValueLength());
639    }
640    if (cell instanceof ExtendedCell) {
641      sb.append("/seqid=");
642      sb.append(((ExtendedCell) cell).getSequenceId());
643    }
644
645    return sb.toString();
646  }
647
648  /** Returns a string representation of the cell */
649  public static String toString(Cell cell, boolean verbose) {
650    if (cell == null) {
651      return "";
652    }
653    StringBuilder builder = new StringBuilder();
654    String keyStr = getCellKeyAsString(cell);
655
656    String tag = null;
657    String value = null;
658    if (verbose) {
659      // TODO: pretty print tags as well
660      if (cell instanceof RawCell) {
661        RawCell rawCell = (RawCell) cell;
662        if (rawCell.getTagsLength() > 0) {
663          tag = Bytes.toStringBinary(rawCell.getTagsArray(), rawCell.getTagsOffset(),
664            rawCell.getTagsLength());
665        }
666      }
667      if (!(cell instanceof KeyValue.KeyOnlyKeyValue)) {
668        value =
669          Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
670      }
671    }
672
673    builder.append(keyStr);
674    if (tag != null && !tag.isEmpty()) {
675      builder.append("/").append(tag);
676    }
677    if (value != null) {
678      builder.append("/").append(value);
679    }
680
681    return builder.toString();
682  }
683
684  /**************** equals ****************************/
685
686  public static boolean equals(Cell a, Cell b) {
687    return matchingRows(a, b) && matchingFamily(a, b) && matchingQualifier(a, b)
688      && matchingTimestamp(a, b)
689      && PrivateCellUtil.getTypeByte(a) == PrivateCellUtil.getTypeByte(b);
690  }
691
692  public static boolean matchingTimestamp(Cell a, Cell b) {
693    return CellComparator.getInstance().compareTimestamps(a.getTimestamp(), b.getTimestamp()) == 0;
694  }
695
696  /** Compares the row of two keyvalues for equality */
697  public static boolean matchingRows(final Cell left, final Cell right) {
698    short lrowlength = left.getRowLength();
699    short rrowlength = right.getRowLength();
700    return matchingRows(left, lrowlength, right, rrowlength);
701  }
702
703  /** Compares the row of two keyvalues for equality */
704  public static boolean matchingRows(final Cell left, final short lrowlength, final Cell right,
705    final short rrowlength) {
706    if (lrowlength != rrowlength) return false;
707    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
708      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
709        ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength,
710        ((ByteBufferExtendedCell) right).getRowByteBuffer(),
711        ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength);
712    }
713    if (left instanceof ByteBufferExtendedCell) {
714      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
715        ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
716        right.getRowOffset(), rrowlength);
717    }
718    if (right instanceof ByteBufferExtendedCell) {
719      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getRowByteBuffer(),
720        ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
721        left.getRowOffset(), lrowlength);
722    }
723    return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
724      right.getRowOffset(), rrowlength);
725  }
726
727  /** Compares the row and column of two keyvalues for equality */
728  public static boolean matchingRowColumn(final Cell left, final Cell right) {
729    short lrowlength = left.getRowLength();
730    short rrowlength = right.getRowLength();
731    // match length
732    if (lrowlength != rrowlength) {
733      return false;
734    }
735
736    byte lfamlength = left.getFamilyLength();
737    byte rfamlength = right.getFamilyLength();
738    if (lfamlength != rfamlength) {
739      return false;
740    }
741
742    int lqlength = left.getQualifierLength();
743    int rqlength = right.getQualifierLength();
744    if (lqlength != rqlength) {
745      return false;
746    }
747
748    if (!matchingRows(left, lrowlength, right, rrowlength)) {
749      return false;
750    }
751    return matchingColumn(left, lfamlength, lqlength, right, rfamlength, rqlength);
752  }
753
754  /** Compares the row and column of two keyvalues for equality */
755  public static boolean matchingRowColumnBytes(final Cell left, final Cell right) {
756    int lrowlength = left.getRowLength();
757    int rrowlength = right.getRowLength();
758    int lfamlength = left.getFamilyLength();
759    int rfamlength = right.getFamilyLength();
760    int lqlength = left.getQualifierLength();
761    int rqlength = right.getQualifierLength();
762
763    // match length
764    if ((lrowlength != rrowlength) || (lfamlength != rfamlength) || (lqlength != rqlength)) {
765      return false;
766    }
767
768    // match row
769    if (
770      !Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
771        right.getRowOffset(), rrowlength)
772    ) {
773      return false;
774    }
775    // match family
776    if (
777      !Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
778        right.getFamilyArray(), right.getFamilyOffset(), rfamlength)
779    ) {
780      return false;
781    }
782    // match qualifier
783    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
784      right.getQualifierArray(), right.getQualifierOffset(), rqlength);
785  }
786
787  /**
788   * Compares the cell's qualifier with the given byte[]
789   * @param left    the cell for which the qualifier has to be compared
790   * @param right   the byte[] having the qualifier
791   * @param rOffset the offset of the qualifier
792   * @param rLength the length of the qualifier
793   * @return greater than 0 if left cell's qualifier is bigger than byte[], lesser than 0 if left
794   *         cell's qualifier is lesser than byte[] and 0 otherwise
795   */
796  public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
797    if (left instanceof ByteBufferExtendedCell) {
798      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
799        ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(), right,
800        rOffset, rLength);
801    }
802    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
803      left.getQualifierLength(), right, rOffset, rLength);
804  }
805
806  /**
807   * Compares the cell's family with the given byte[]
808   * @param left    the cell for which the family has to be compared
809   * @param right   the byte[] having the family
810   * @param roffset the offset of the family
811   * @param rlength the length of the family
812   * @return greater than 0 if left cell's family is bigger than byte[], lesser than 0 if left
813   *         cell's family is lesser than byte[] and 0 otherwise
814   */
815  public final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
816    if (left instanceof ByteBufferExtendedCell) {
817      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
818        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
819        rlength);
820    }
821    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
822      right, roffset, rlength);
823  }
824
825  /**
826   * Compares the cell's column (family and qualifier) with the given byte[]
827   * @param left     the cell for which the column has to be compared
828   * @param right    the byte[] having the column
829   * @param rfoffset the offset of the family
830   * @param rflength the length of the family
831   * @param rqoffset the offset of the qualifier
832   * @param rqlength the length of the qualifier
833   * @return greater than 0 if left cell's column is bigger than byte[], lesser than 0 if left
834   *         cell's column is lesser than byte[] and 0 otherwise
835   */
836  public final static int compareColumns(Cell left, byte[] right, int rfoffset, int rflength,
837    int rqoffset, int rqlength) {
838    int diff = compareFamilies(left, right, rfoffset, rflength);
839    if (diff != 0) return diff;
840    return compareQualifiers(left, right, rqoffset, rqlength);
841  }
842
843  public static void cloneIfNecessary(ArrayList<Cell> cells) {
844    if (cells == null || cells.isEmpty()) {
845      return;
846    }
847    for (int i = 0; i < cells.size(); i++) {
848      Cell cell = cells.get(i);
849      cells.set(i, cloneIfNecessary(cell));
850    }
851  }
852
853  public static Cell cloneIfNecessary(Cell cell) {
854    return (cell instanceof ByteBufferExtendedCell
855      ? KeyValueUtil.copyToNewKeyValue((ExtendedCell) cell)
856      : cell);
857  }
858}