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