View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase;
20  
21  import java.io.DataOutputStream;
22  import java.io.IOException;
23  import java.nio.ByteBuffer;
24  import java.util.Iterator;
25  import java.util.List;
26  import java.util.Map.Entry;
27  import java.util.NavigableMap;
28  
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.classification.InterfaceStability;
31  import org.apache.hadoop.hbase.KeyValue.Type;
32  import org.apache.hadoop.hbase.io.HeapSize;
33  import org.apache.hadoop.hbase.util.ByteBufferUtils;
34  import org.apache.hadoop.hbase.util.ByteRange;
35  import org.apache.hadoop.hbase.util.Bytes;
36  
37  /**
38   * Utility methods helpful slinging {@link Cell} instances.
39   */
40  @InterfaceAudience.Public
41  @InterfaceStability.Evolving
42  public final class CellUtil {
43  
44    /******************* ByteRange *******************************/
45  
46    public static ByteRange fillRowRange(Cell cell, ByteRange range) {
47      return range.set(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
48    }
49  
50    public static ByteRange fillFamilyRange(Cell cell, ByteRange range) {
51      return range.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
52    }
53  
54    public static ByteRange fillQualifierRange(Cell cell, ByteRange range) {
55      return range.set(cell.getQualifierArray(), cell.getQualifierOffset(),
56        cell.getQualifierLength());
57    }
58  
59    public static ByteRange fillValueRange(Cell cell, ByteRange range) {
60      return range.set(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
61    }
62  
63    public static ByteRange fillTagRange(Cell cell, ByteRange range) {
64      return range.set(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
65    }
66  
67    /***************** get individual arrays for tests ************/
68  
69    public static byte[] cloneRow(Cell cell){
70      byte[] output = new byte[cell.getRowLength()];
71      copyRowTo(cell, output, 0);
72      return output;
73    }
74  
75    public static byte[] cloneFamily(Cell cell){
76      byte[] output = new byte[cell.getFamilyLength()];
77      copyFamilyTo(cell, output, 0);
78      return output;
79    }
80  
81    public static byte[] cloneQualifier(Cell cell){
82      byte[] output = new byte[cell.getQualifierLength()];
83      copyQualifierTo(cell, output, 0);
84      return output;
85    }
86  
87    public static byte[] cloneValue(Cell cell){
88      byte[] output = new byte[cell.getValueLength()];
89      copyValueTo(cell, output, 0);
90      return output;
91    }
92  
93    /**
94     * Returns tag value in a new byte array. If server-side, use
95     * {@link Tag#getBuffer()} with appropriate {@link Tag#getTagOffset()} and
96     * {@link Tag#getTagLength()} instead to save on allocations.
97     * @param cell
98     * @return tag value in a new byte array.
99     */
100   public static byte[] getTagArray(Cell cell){
101     byte[] output = new byte[cell.getTagsLength()];
102     copyTagTo(cell, output, 0);
103     return output;
104   }
105 
106 
107   /******************** copyTo **********************************/
108 
109   public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
110     System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
111       cell.getRowLength());
112     return destinationOffset + cell.getRowLength();
113   }
114 
115   public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
116     System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination, destinationOffset,
117       cell.getFamilyLength());
118     return destinationOffset + cell.getFamilyLength();
119   }
120 
121   public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
122     System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
123       destinationOffset, cell.getQualifierLength());
124     return destinationOffset + cell.getQualifierLength();
125   }
126 
127   public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
128     System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
129         cell.getValueLength());
130     return destinationOffset + cell.getValueLength();
131   }
132 
133   /**
134    * Copies the tags info into the tag portion of the cell
135    * @param cell
136    * @param destination
137    * @param destinationOffset
138    * @return position after tags
139    */
140   public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
141     System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
142         cell.getTagsLength());
143     return destinationOffset + cell.getTagsLength();
144   }
145 
146   /********************* misc *************************************/
147 
148   public static byte getRowByte(Cell cell, int index) {
149     return cell.getRowArray()[cell.getRowOffset() + index];
150   }
151 
152   public static ByteBuffer getValueBufferShallowCopy(Cell cell) {
153     ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(),
154       cell.getValueLength());
155     return buffer;
156   }
157 
158   public static ByteBuffer getQualifierBufferShallowCopy(Cell cell) {
159     ByteBuffer buffer = ByteBuffer.wrap(cell.getQualifierArray(), cell.getQualifierOffset(),
160         cell.getQualifierLength());
161     return buffer;
162   }
163 
164   public static Cell createCell(final byte [] row, final byte [] family, final byte [] qualifier,
165       final long timestamp, final byte type, final byte [] value) {
166     // I need a Cell Factory here.  Using KeyValue for now. TODO.
167     // TODO: Make a new Cell implementation that just carries these
168     // byte arrays.
169     // TODO: Call factory to create Cell
170     return new KeyValue(row, family, qualifier, timestamp, KeyValue.Type.codeToType(type), value);
171   }
172 
173   public static Cell createCell(final byte [] rowArray, final int rowOffset, final int rowLength,
174       final byte [] familyArray, final int familyOffset, final int familyLength,
175       final byte [] qualifierArray, final int qualifierOffset, final int qualifierLength) {
176     // See createCell(final byte [] row, final byte [] value) for why we default Maximum type.
177     return new KeyValue(rowArray, rowOffset, rowLength,
178         familyArray, familyOffset, familyLength,
179         qualifierArray, qualifierOffset, qualifierLength,
180         HConstants.LATEST_TIMESTAMP,
181         KeyValue.Type.Maximum,
182         HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length);
183   }
184 
185   public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
186       final long timestamp, final byte type, final byte[] value, final long memstoreTS) {
187     KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp,
188         KeyValue.Type.codeToType(type), value);
189     keyValue.setSequenceId(memstoreTS);
190     return keyValue;
191   }
192 
193   public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
194       final long timestamp, final byte type, final byte[] value, byte[] tags, final long memstoreTS) {
195     KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp,
196         KeyValue.Type.codeToType(type), value, tags);
197     keyValue.setSequenceId(memstoreTS);
198     return keyValue;
199   }
200 
201   public static Cell createCell(final byte[] row, final byte[] family, final byte[] qualifier,
202       final long timestamp, Type type, final byte[] value, byte[] tags) {
203     KeyValue keyValue = new KeyValue(row, family, qualifier, timestamp, type, value, tags);
204     return keyValue;
205   }
206 
207   /**
208    * Create a Cell with specific row.  Other fields defaulted.
209    * @param row
210    * @return Cell with passed row but all other fields are arbitrary
211    */
212   public static Cell createCell(final byte [] row) {
213     return createCell(row, HConstants.EMPTY_BYTE_ARRAY);
214   }
215 
216   /**
217    * Create a Cell with specific row and value.  Other fields are defaulted.
218    * @param row
219    * @param value
220    * @return Cell with passed row and value but all other fields are arbitrary
221    */
222   public static Cell createCell(final byte [] row, final byte [] value) {
223     // An empty family + empty qualifier + Type.Minimum is used as flag to indicate last on row.
224     // See the CellComparator and KeyValue comparator.  Search for compareWithoutRow.
225     // Lets not make a last-on-row key as default but at same time, if you are making a key
226     // without specifying type, etc., flag it as weird by setting type to be Maximum.
227     return createCell(row, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
228       HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum.getCode(), value);
229   }
230 
231   /**
232    * Create a Cell with specific row.  Other fields defaulted.
233    * @param row
234    * @param family
235    * @param qualifier
236    * @return Cell with passed row but all other fields are arbitrary
237    */
238   public static Cell createCell(final byte [] row, final byte [] family, final byte [] qualifier) {
239     // See above in createCell(final byte [] row, final byte [] value) why we set type to Maximum.
240     return createCell(row, family, qualifier,
241         HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum.getCode(), HConstants.EMPTY_BYTE_ARRAY);
242   }
243 
244   /**
245    * @param cellScannerables
246    * @return CellScanner interface over <code>cellIterables</code>
247    */
248   public static CellScanner createCellScanner(final List<? extends CellScannable> cellScannerables) {
249     return new CellScanner() {
250       private final Iterator<? extends CellScannable> iterator = cellScannerables.iterator();
251       private CellScanner cellScanner = null;
252 
253       @Override
254       public Cell current() {
255         return this.cellScanner != null? this.cellScanner.current(): null;
256       }
257 
258       @Override
259       public boolean advance() throws IOException {
260         while (true) {
261           if (this.cellScanner == null) {
262             if (!this.iterator.hasNext()) return false;
263             this.cellScanner = this.iterator.next().cellScanner();
264           }
265           if (this.cellScanner.advance()) return true;
266           this.cellScanner = null;
267         }
268       }
269     };
270   }
271 
272   /**
273    * @param cellIterable
274    * @return CellScanner interface over <code>cellIterable</code>
275    */
276   public static CellScanner createCellScanner(final Iterable<Cell> cellIterable) {
277     if (cellIterable == null) return null;
278     return createCellScanner(cellIterable.iterator());
279   }
280 
281   /**
282    * @param cells
283    * @return CellScanner interface over <code>cellIterable</code> or null if <code>cells</code> is
284    * null
285    */
286   public static CellScanner createCellScanner(final Iterator<Cell> cells) {
287     if (cells == null) return null;
288     return new CellScanner() {
289       private final Iterator<Cell> iterator = cells;
290       private Cell current = null;
291 
292       @Override
293       public Cell current() {
294         return this.current;
295       }
296 
297       @Override
298       public boolean advance() {
299         boolean hasNext = this.iterator.hasNext();
300         this.current = hasNext? this.iterator.next(): null;
301         return hasNext;
302       }
303     };
304   }
305 
306   /**
307    * @param cellArray
308    * @return CellScanner interface over <code>cellArray</code>
309    */
310   public static CellScanner createCellScanner(final Cell[] cellArray) {
311     return new CellScanner() {
312       private final Cell [] cells = cellArray;
313       private int index = -1;
314 
315       @Override
316       public Cell current() {
317         if (cells == null) return null;
318         return (index < 0)? null: this.cells[index];
319       }
320 
321       @Override
322       public boolean advance() {
323         if (cells == null) return false;
324         return ++index < this.cells.length;
325       }
326     };
327   }
328 
329   /**
330    * Flatten the map of cells out under the CellScanner
331    * @param map Map of Cell Lists; for example, the map of families to Cells that is used
332    * inside Put, etc., keeping Cells organized by family.
333    * @return CellScanner interface over <code>cellIterable</code>
334    */
335   public static CellScanner createCellScanner(final NavigableMap<byte [], List<Cell>> map) {
336     return new CellScanner() {
337       private final Iterator<Entry<byte[], List<Cell>>> entries = map.entrySet().iterator();
338       private Iterator<Cell> currentIterator = null;
339       private Cell currentCell;
340 
341       @Override
342       public Cell current() {
343         return this.currentCell;
344       }
345 
346       @Override
347       public boolean advance() {
348         while(true) {
349           if (this.currentIterator == null) {
350             if (!this.entries.hasNext()) return false;
351             this.currentIterator = this.entries.next().getValue().iterator();
352           }
353           if (this.currentIterator.hasNext()) {
354             this.currentCell = this.currentIterator.next();
355             return true;
356           }
357           this.currentCell = null;
358           this.currentIterator = null;
359         }
360       }
361     };
362   }
363 
364   /**
365    * @param left
366    * @param right
367    * @return True if the rows in <code>left</code> and <code>right</code> Cells match
368    */
369   public static boolean matchingRow(final Cell left, final Cell right) {
370     return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
371         right.getRowArray(), right.getRowOffset(), right.getRowLength());
372   }
373 
374   public static boolean matchingRow(final Cell left, final byte[] buf) {
375     return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, 0,
376         buf.length);
377   }
378 
379   public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
380       final int length) {
381     return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
382         length);
383   }
384 
385   public static boolean matchingFamily(final Cell left, final Cell right) {
386     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
387         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
388   }
389 
390   public static boolean matchingFamily(final Cell left, final byte[] buf) {
391     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
392         0, buf.length);
393   }
394 
395   public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
396       final int length) {
397     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
398         offset, length);
399   }
400 
401   public static boolean matchingQualifier(final Cell left, final Cell right) {
402     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
403         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
404         right.getQualifierLength());
405   }
406 
407   public static boolean matchingQualifier(final Cell left, final byte[] buf) {
408     if (buf == null) {
409       return left.getQualifierLength() == 0;
410     }
411     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
412         left.getQualifierLength(), buf, 0, buf.length);
413   }
414 
415   public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
416       final int length) {
417     if (buf == null) {
418       return left.getQualifierLength() == 0;
419     }
420     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
421         left.getQualifierLength(), buf, offset, length);
422   }
423 
424   public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
425     if (!matchingFamily(left, fam))
426       return false;
427     return matchingQualifier(left, qual);
428   }
429 
430   public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
431       final int flength, final byte[] qual, final int qoffset, final int qlength) {
432     if (!matchingFamily(left, fam, foffset, flength))
433       return false;
434     return matchingQualifier(left, qual, qoffset, qlength);
435   }
436 
437   public static boolean matchingColumn(final Cell left, final Cell right) {
438     if (!matchingFamily(left, right))
439       return false;
440     return matchingQualifier(left, right);
441   }
442 
443   public static boolean matchingValue(final Cell left, final Cell right) {
444     return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
445         right.getValueArray(), right.getValueOffset(), right.getValueLength());
446   }
447 
448   public static boolean matchingValue(final Cell left, final byte[] buf) {
449     return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
450         buf.length);
451   }
452 
453   /**
454    * @return True if a delete type, a {@link KeyValue.Type#Delete} or a
455    *         {KeyValue.Type#DeleteFamily} or a
456    *         {@link KeyValue.Type#DeleteColumn} KeyValue type.
457    */
458   public static boolean isDelete(final Cell cell) {
459     return isDelete(cell.getTypeByte());
460   }
461 
462   /**
463    * @return True if a delete type, a {@link KeyValue.Type#Delete} or a
464    *         {KeyValue.Type#DeleteFamily} or a
465    *         {@link KeyValue.Type#DeleteColumn} KeyValue type.
466    */
467   public static boolean isDelete(final byte type) {
468     return Type.Delete.getCode() <= type
469         && type <= Type.DeleteFamily.getCode();
470   }
471 
472   /**
473    * @return True if this cell is a {@link KeyValue.Type#Delete} type.
474    */
475   public static boolean isDeleteType(Cell cell) {
476     return cell.getTypeByte() == Type.Delete.getCode();
477   }
478 
479   public static boolean isDeleteFamily(final Cell cell) {
480     return cell.getTypeByte() == Type.DeleteFamily.getCode();
481   }
482 
483   public static boolean isDeleteFamilyVersion(final Cell cell) {
484     return cell.getTypeByte() == Type.DeleteFamilyVersion.getCode();
485   }
486 
487   public static boolean isDeleteColumns(final Cell cell) {
488     return cell.getTypeByte() == Type.DeleteColumn.getCode();
489   }
490 
491   public static boolean isDeleteColumnVersion(final Cell cell) {
492     return cell.getTypeByte() == Type.Delete.getCode();
493   }
494 
495   /**
496    *
497    * @return True if this cell is a delete family or column type.
498    */
499   public static boolean isDeleteColumnOrFamily(Cell cell) {
500     int t = cell.getTypeByte();
501     return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
502   }
503 
504   /**
505    * @param cell
506    * @return Estimate of the <code>cell</code> size in bytes.
507    * @deprecated please use estimatedSerializedSizeOf(Cell)
508    */
509   @Deprecated
510   public static int estimatedSizeOf(final Cell cell) {
511     return estimatedSerializedSizeOf(cell);
512   }
513 
514   /**
515    * @param cell
516    * @return Estimate of the <code>cell</code> size in bytes.
517    */
518   public static int estimatedSerializedSizeOf(final Cell cell) {
519     // If a KeyValue, we can give a good estimate of size.
520     if (cell instanceof KeyValue) {
521       return ((KeyValue)cell).getLength() + Bytes.SIZEOF_INT;
522     }
523     // TODO: Should we add to Cell a sizeOf?  Would it help? Does it make sense if Cell is
524     // prefix encoded or compressed?
525     return getSumOfCellElementLengths(cell) +
526       // Use the KeyValue's infrastructure size presuming that another implementation would have
527       // same basic cost.
528       KeyValue.KEY_INFRASTRUCTURE_SIZE +
529       // Serialization is probably preceded by a length (it is in the KeyValueCodec at least).
530       Bytes.SIZEOF_INT;
531   }
532 
533   /**
534    * @param cell
535    * @return Sum of the lengths of all the elements in a Cell; does not count in any infrastructure
536    */
537   private static int getSumOfCellElementLengths(final Cell cell) {
538     return getSumOfCellKeyElementLengths(cell) + cell.getValueLength() + cell.getTagsLength();
539   }
540 
541   /**
542    * @param cell
543    * @return Sum of all elements that make up a key; does not include infrastructure, tags or
544    * values.
545    */
546   private static int getSumOfCellKeyElementLengths(final Cell cell) {
547     return cell.getRowLength() + cell.getFamilyLength() +
548     cell.getQualifierLength() +
549     KeyValue.TIMESTAMP_TYPE_SIZE;
550   }
551 
552   public static int estimatedSerializedSizeOfKey(final Cell cell) {
553     if (cell instanceof KeyValue) return ((KeyValue)cell).getKeyLength();
554     // This will be a low estimate.  Will do for now.
555     return getSumOfCellKeyElementLengths(cell);
556   }
557 
558   /**
559    * This is an estimate of the heap space occupied by a cell. When the cell is of type
560    * {@link HeapSize} we call {@link HeapSize#heapSize()} so cell can give a correct value. In other
561    * cases we just consider the byte occupied by the cell components ie. row, CF, qualifier,
562    * timestamp, type, value and tags.
563    * @param cell
564    * @return estimate of the heap space
565    */
566   public static long estimatedHeapSizeOf(final Cell cell) {
567     if (cell instanceof HeapSize) {
568       return ((HeapSize) cell).heapSize();
569     }
570     // TODO: Add sizing of references that hold the row, family, etc., arrays.
571     return estimatedSerializedSizeOf(cell);
572   }
573 
574   /**
575    * This is a hack that should be removed once we don't care about matching
576    * up client- and server-side estimations of cell size. It needed to be
577    * backwards compatible with estimations done by older clients. We need to
578    * pretend that tags never exist and cells aren't serialized with tag
579    * length included. See HBASE-13262 and HBASE-13303
580    */
581   @Deprecated
582   public static long estimatedHeapSizeOfWithoutTags(final Cell cell) {
583     if (cell instanceof KeyValue) {
584       return ((KeyValue)cell).heapSizeWithoutTags();
585     }
586     return getSumOfCellKeyElementLengths(cell) + cell.getValueLength();
587   }
588 
589   /********************* tags *************************************/
590   /**
591    * Util method to iterate through the tags
592    *
593    * @param tags
594    * @param offset
595    * @param length
596    * @return iterator for the tags
597    */
598   public static Iterator<Tag> tagsIterator(final byte[] tags, final int offset, final int length) {
599     return new Iterator<Tag>() {
600       private int pos = offset;
601       private int endOffset = offset + length - 1;
602 
603       @Override
604       public boolean hasNext() {
605         return this.pos < endOffset;
606       }
607 
608       @Override
609       public Tag next() {
610         if (hasNext()) {
611           int curTagLen = Bytes.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
612           Tag tag = new Tag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
613           this.pos += Bytes.SIZEOF_SHORT + curTagLen;
614           return tag;
615         }
616         return null;
617       }
618 
619       @Override
620       public void remove() {
621         throw new UnsupportedOperationException();
622       }
623     };
624   }
625 
626   /**
627    * Returns true if the first range start1...end1 overlaps with the second range
628    * start2...end2, assuming the byte arrays represent row keys
629    */
630   public static boolean overlappingKeys(final byte[] start1, final byte[] end1,
631       final byte[] start2, final byte[] end2) {
632     return (end2.length == 0 || start1.length == 0 || Bytes.compareTo(start1,
633         end2) < 0)
634         && (end1.length == 0 || start2.length == 0 || Bytes.compareTo(start2,
635             end1) < 0);
636   }
637 
638   /**
639    * Sets the given seqId to the cell.
640    * @param cell
641    * @param seqId
642    * @throws IOException when the passed cell is not of type {@link SettableSequenceId}
643    */
644   public static void setSequenceId(Cell cell, long seqId) throws IOException {
645     if (cell instanceof SettableSequenceId) {
646       ((SettableSequenceId) cell).setSequenceId(seqId);
647     } else {
648       throw new IOException(new UnsupportedOperationException("Cell is not of type "
649           + SettableSequenceId.class.getName()));
650     }
651   }
652 
653   /**
654    * Sets the given timestamp to the cell.
655    * @param cell
656    * @param ts
657    * @throws IOException when the passed cell is not of type {@link SettableTimestamp}
658    */
659   public static void setTimestamp(Cell cell, long ts) throws IOException {
660     if (cell instanceof SettableTimestamp) {
661       ((SettableTimestamp) cell).setTimestamp(ts);
662     } else {
663       throw new IOException(new UnsupportedOperationException("Cell is not of type "
664           + SettableTimestamp.class.getName()));
665     }
666   }
667 
668   /**
669    * Sets the given timestamp to the cell.
670    * @param cell
671    * @param ts buffer containing the timestamp value
672    * @param tsOffset offset to the new timestamp
673    * @throws IOException when the passed cell is not of type {@link SettableTimestamp}
674    */
675   public static void setTimestamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
676     if (cell instanceof SettableTimestamp) {
677       ((SettableTimestamp) cell).setTimestamp(ts, tsOffset);
678     } else {
679       throw new IOException(new UnsupportedOperationException("Cell is not of type "
680           + SettableTimestamp.class.getName()));
681     }
682   }
683 
684   /**
685    * Sets the given timestamp to the cell iff current timestamp is
686    * {@link HConstants#LATEST_TIMESTAMP}.
687    * @param cell
688    * @param ts
689    * @return True if cell timestamp is modified.
690    * @throws IOException when the passed cell is not of type {@link SettableTimestamp}
691    */
692   public static boolean updateLatestStamp(Cell cell, long ts) throws IOException {
693     if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
694       setTimestamp(cell, ts);
695       return true;
696     }
697     return false;
698   }
699 
700   /**
701    * Sets the given timestamp to the cell iff current timestamp is
702    * {@link HConstants#LATEST_TIMESTAMP}.
703    * @param cell
704    * @param ts buffer containing the timestamp value
705    * @param tsOffset offset to the new timestamp
706    * @return True if cell timestamp is modified.
707    * @throws IOException when the passed cell is not of type {@link SettableTimestamp}
708    */
709   public static boolean updateLatestStamp(Cell cell, byte[] ts, int tsOffset) throws IOException {
710     if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP) {
711       setTimestamp(cell, ts, tsOffset);
712       return true;
713     }
714     return false;
715   }
716 
717   /**
718    * Writes the Cell's key part as it would have serialized in a KeyValue. The format is &lt;2 bytes
719    * rk len&gt;&lt;rk&gt;&lt;1 byte cf len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes
720    * timestamp&gt;&lt;1 byte type&gt;
721    * @param cell
722    * @param out
723    * @throws IOException
724    */
725   public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException {
726     short rowLen = cell.getRowLength();
727     out.writeShort(rowLen);
728     out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
729     byte fLen = cell.getFamilyLength();
730     out.writeByte(fLen);
731     out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
732     out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
733     out.writeLong(cell.getTimestamp());
734     out.writeByte(cell.getTypeByte());
735   }
736 
737   /**
738    * @param cell
739    * @return The Key portion of the passed <code>cell</code> as a String.
740    */
741   public static String getCellKeyAsString(Cell cell) {
742     StringBuilder sb = new StringBuilder(Bytes.toStringBinary(
743       cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
744     sb.append('/');
745     sb.append(cell.getFamilyLength() == 0? "":
746       Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()));
747     // KeyValue only added ':' if family is non-null.  Do same.
748     if (cell.getFamilyLength() > 0) sb.append(':');
749     sb.append(cell.getQualifierLength() == 0? "":
750       Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
751         cell.getQualifierLength()));
752     sb.append('/');
753     sb.append(KeyValue.humanReadableTimestamp(cell.getTimestamp()));
754     sb.append('/');
755     sb.append(Type.codeToType(cell.getTypeByte()));
756     sb.append("/vlen=");
757     sb.append(cell.getValueLength());
758     sb.append("/seqid=");
759     sb.append(cell.getSequenceId());
760     return sb.toString();
761   }
762 
763   /**
764    * This method exists just to encapsulate how we serialize keys.  To be replaced by a factory
765    * that we query to figure what the Cell implementation is and then, what serialization engine
766    * to use and further, how to serialize the key for inclusion in hfile index. TODO.
767    * @param cell
768    * @return The key portion of the Cell serialized in the old-school KeyValue way or null if
769    * passed a null <code>cell</code>
770    */
771   public static byte [] getCellKeySerializedAsKeyValueKey(final Cell cell) {
772     if (cell == null) return null;
773     byte [] b = new byte[KeyValueUtil.keyLength(cell)];
774     KeyValueUtil.appendKeyTo(cell, b, 0);
775     return b;
776   }
777 
778   /**
779    * Write rowkey excluding the common part.
780    * @param cell
781    * @param rLen
782    * @param commonPrefix
783    * @param out
784    * @throws IOException
785    */
786   public static void writeRowKeyExcludingCommon(Cell cell, short rLen, int commonPrefix,
787       DataOutputStream out) throws IOException {
788     if (commonPrefix == 0) {
789       out.writeShort(rLen);
790     } else if (commonPrefix == 1) {
791       out.writeByte((byte) rLen);
792       commonPrefix--;
793     } else {
794       commonPrefix -= KeyValue.ROW_LENGTH_SIZE;
795     }
796     if (rLen > commonPrefix) {
797       out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rLen - commonPrefix);
798     }
799   }
800 
801   /**
802    * Find length of common prefix in keys of the cells, considering key as byte[] if serialized in
803    * {@link KeyValue}. The key format is &lt;2 bytes rk len&gt;&lt;rk&gt;&lt;1 byte cf
804    * len&gt;&lt;cf&gt;&lt;qualifier&gt;&lt;8 bytes timestamp&gt;&lt;1 byte type&gt;
805    * @param c1
806    *          the cell
807    * @param c2
808    *          the cell
809    * @param bypassFamilyCheck
810    *          when true assume the family bytes same in both cells. Pass it as true when dealing
811    *          with Cells in same CF so as to avoid some checks
812    * @param withTsType
813    *          when true check timestamp and type bytes also.
814    * @return length of common prefix
815    */
816   public static int findCommonPrefixInFlatKey(Cell c1, Cell c2, boolean bypassFamilyCheck,
817       boolean withTsType) {
818     // Compare the 2 bytes in RK length part
819     short rLen1 = c1.getRowLength();
820     short rLen2 = c2.getRowLength();
821     int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
822     if (rLen1 != rLen2) {
823       // early out when the RK length itself is not matching
824       return ByteBufferUtils.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE,
825           Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
826     }
827     // Compare the RKs
828     int rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(),
829         rLen1, c2.getRowArray(), c2.getRowOffset(), rLen2);
830     commonPrefix += rkCommonPrefix;
831     if (rkCommonPrefix != rLen1) {
832       // Early out when RK is not fully matching.
833       return commonPrefix;
834     }
835     // Compare 1 byte CF length part
836     byte fLen1 = c1.getFamilyLength();
837     if (bypassFamilyCheck) {
838       // This flag will be true when caller is sure that the family will be same for both the cells
839       // Just make commonPrefix to increment by the family part
840       commonPrefix += KeyValue.FAMILY_LENGTH_SIZE + fLen1;
841     } else {
842       byte fLen2 = c2.getFamilyLength();
843       if (fLen1 != fLen2) {
844         // early out when the CF length itself is not matching
845         return commonPrefix;
846       }
847       // CF lengths are same so there is one more byte common in key part
848       commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
849       // Compare the CF names
850       int fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(),
851           c1.getFamilyOffset(), fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
852       commonPrefix += fCommonPrefix;
853       if (fCommonPrefix != fLen1) {
854         return commonPrefix;
855       }
856     }
857     // Compare the Qualifiers
858     int qLen1 = c1.getQualifierLength();
859     int qLen2 = c2.getQualifierLength();
860     int qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
861         qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
862     commonPrefix += qCommon;
863     if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
864       return commonPrefix;
865     }
866     // Compare the timestamp parts
867     int tsCommonPrefix = ByteBufferUtils.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0,
868         KeyValue.TIMESTAMP_SIZE, Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
869     commonPrefix += tsCommonPrefix;
870     if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
871       return commonPrefix;
872     }
873     // Compare the type
874     if (c1.getTypeByte() == c2.getTypeByte()) {
875       commonPrefix += KeyValue.TYPE_SIZE;
876     }
877     return commonPrefix;
878   }
879 
880   /** Returns a string representation of the cell */
881   public static String toString(Cell cell, boolean verbose) {
882     if (cell == null) {
883       return "";
884     }
885     StringBuilder builder = new StringBuilder();
886     String keyStr = getCellKeyAsString(cell);
887 
888     String tag = null;
889     String value = null;
890     if (verbose) {
891       // TODO: pretty print tags as well
892       tag = Bytes.toStringBinary(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
893       value = Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
894         cell.getValueLength());
895     }
896 
897     builder
898       .append(keyStr);
899     if (tag != null && !tag.isEmpty()) {
900       builder.append("/").append(tag);
901     }
902     if (value != null) {
903       builder.append("/").append(value);
904     }
905 
906     return builder.toString();
907   }
908 
909   /**************** equals ****************************/
910 
911   public static boolean equals(Cell a, Cell b) {
912     return matchingRow(a, b) && matchingFamily(a, b) && matchingQualifier(a, b)
913         && matchingTimestamp(a, b) && matchingType(a, b);
914   }
915 
916   public static boolean matchingType(Cell a, Cell b) {
917     return a.getTypeByte() == b.getTypeByte();
918   }
919 
920   public static boolean matchingTimestamp(Cell a, Cell b) {
921     return CellComparator.compareTimestamps(a, b) == 0;
922   }
923 }