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.Serializable;
22  import java.util.Comparator;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.hbase.KeyValue.Type;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.classification.InterfaceStability;
29  import org.apache.hadoop.hbase.filter.ByteArrayComparable;
30  import org.apache.hadoop.hbase.util.ByteBufferUtils;
31  import org.apache.hadoop.hbase.util.Bytes;
32  
33  import com.google.common.primitives.Longs;
34  
35  /**
36   * Compare two HBase cells.  Do not use this method comparing <code>-ROOT-</code> or
37   * <code>hbase:meta</code> cells.  Cells from these tables need a specialized comparator, one that
38   * takes account of the special formatting of the row where we have commas to delimit table from
39   * regionname, from row.  See KeyValue for how it has a special comparator to do hbase:meta cells
40   * and yet another for -ROOT-.
41   * While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells
42   * format should be taken into consideration, for which the instance of this comparator
43   * should be used.  In all other cases the static APIs in this comparator would be enough
44   */
45  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
46      value="UNKNOWN",
47      justification="Findbugs doesn't like the way we are negating the result of a compare in below")
48  @InterfaceAudience.Private
49  @InterfaceStability.Evolving
50  public class CellComparator implements Comparator<Cell>, Serializable {
51    static final Log LOG = LogFactory.getLog(CellComparator.class);
52    private static final long serialVersionUID = -8760041766259623329L;
53  
54    /**
55     * Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion
56     * of KeyValue only.
57     */
58    public static final CellComparator COMPARATOR = new CellComparator();
59    /**
60     * A {@link CellComparator} for <code>hbase:meta</code> catalog table
61     * {@link KeyValue}s.
62     */
63    public static final CellComparator META_COMPARATOR = new MetaCellComparator();
64  
65    @Override
66    public int compare(Cell a, Cell b) {
67      return compare(a, b, false);
68    }
69  
70    /**
71     * Compares only the key portion of a cell. It does not include the sequence id/mvcc of the
72     * cell 
73     * @param left
74     * @param right
75     * @return an int greater than 0 if left &gt; than right
76     *                lesser than 0 if left &lt; than right
77     *                equal to 0 if left is equal to right
78     */
79    public final int compareKeyIgnoresMvcc(Cell left, Cell right) {
80      return compare(left, right, true);
81    }
82  
83    /**
84     * Used when a cell needs to be compared with a key byte[] such as cases of
85     * finding the index from the index block, bloom keys from the bloom blocks
86     * This byte[] is expected to be serialized in the KeyValue serialization format
87     * If the KeyValue (Cell's) serialization format changes this method cannot be used.
88     * @param left the cell to be compared
89     * @param key the serialized key part of a KeyValue
90     * @param offset the offset in the key byte[]
91     * @param length the length of the key byte[]
92     * @return an int greater than 0 if left is greater than right
93     *                lesser than 0 if left is lesser than right
94     *                equal to 0 if left is equal to right
95     */
96    public final int compare(Cell left, byte[] key, int offset, int length) {
97      // row
98      short rrowlength = Bytes.toShort(key, offset);
99      int c = compareRows(left, key, offset + Bytes.SIZEOF_SHORT, rrowlength);
100     if (c != 0) return c;
101 
102     // Compare the rest of the two KVs without making any assumptions about
103     // the common prefix. This function will not compare rows anyway, so we
104     // don't need to tell it that the common prefix includes the row.
105     return compareWithoutRow(left, key, offset, length, rrowlength);
106   }
107 
108   /**
109    * Compare cells.
110    * @param a
111    * @param b
112    * @param ignoreSequenceid True if we are to compare the key portion only and ignore
113    * the sequenceid. Set to false to compare key and consider sequenceid.
114    * @return 0 if equal, -1 if a &lt; b, and +1 if a &gt; b.
115    */
116   private final int compare(final Cell a, final Cell b, boolean ignoreSequenceid) {
117     // row
118     int c = compareRows(a, b);
119     if (c != 0) return c;
120 
121     c = compareWithoutRow(a, b);
122     if(c != 0) return c;
123 
124     if (!ignoreSequenceid) {
125       // Negate following comparisons so later edits show up first
126       // mvccVersion: later sorts first
127       return Longs.compare(b.getSequenceId(), a.getSequenceId());
128     } else {
129       return c;
130     }
131   }
132 
133   /**
134    * Compares the family and qualifier part of the cell
135    * @param left the left cell
136    * @param right the right cell
137    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
138    */
139   public final static int compareColumns(final Cell left, final Cell right) {
140     int diff = compareFamilies(left, right);
141     if (diff != 0) {
142       return diff;
143     }
144     return compareQualifiers(left, right);
145   }
146 
147   private final static int compareColumns(Cell left, byte[] right, int rfoffset, int rflength,
148       int rqoffset, int rqlength) {
149     int diff = compareFamilies(left, right, rfoffset, rflength);
150     if (diff != 0)
151       return diff;
152     return compareQualifiers(left, right, rqoffset, rqlength);
153   }
154 
155   /**
156    * Compare the families of left and right cell
157    * @param left
158    * @param right
159    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
160    */
161   public final static int compareFamilies(Cell left, Cell right) {
162     if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
163       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
164           ((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(),
165           ((ByteBufferedCell) right).getFamilyByteBuffer(),
166           ((ByteBufferedCell) right).getFamilyPosition(), right.getFamilyLength());
167     }
168     if (left instanceof ByteBufferedCell) {
169       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
170           ((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(),
171           right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
172     }
173     if (right instanceof ByteBufferedCell) {
174       return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getFamilyByteBuffer(),
175           ((ByteBufferedCell) right).getFamilyPosition(), right.getFamilyLength(),
176           left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength()));
177     }
178     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
179         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
180   }
181 
182   private final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
183     if (left instanceof ByteBufferedCell) {
184       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
185           ((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(), right,
186           roffset, rlength);
187     }
188     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
189         right, roffset, rlength);
190   }
191 
192   /**
193    * Compare the qualifiers part of the left and right cells.
194    * @param left
195    * @param right
196    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
197    */
198   public final static int compareQualifiers(Cell left, Cell right) {
199     if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
200       return ByteBufferUtils
201           .compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
202               ((ByteBufferedCell) left).getQualifierPosition(),
203               left.getQualifierLength(), ((ByteBufferedCell) right).getQualifierByteBuffer(),
204               ((ByteBufferedCell) right).getQualifierPosition(),
205               right.getQualifierLength());
206     }
207     if (left instanceof ByteBufferedCell) {
208       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
209           ((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
210           right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
211     }
212     if (right instanceof ByteBufferedCell) {
213       return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getQualifierByteBuffer(),
214           ((ByteBufferedCell) right).getQualifierPosition(),
215           right.getQualifierLength(), left.getQualifierArray(), left.getQualifierOffset(),
216           left.getQualifierLength()));
217     }
218     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
219         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
220         right.getQualifierLength());
221   }
222 
223   public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
224     if (left instanceof ByteBufferedCell) {
225       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
226           ((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
227           right, rOffset, rLength);
228     }
229     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
230         left.getQualifierLength(), right, rOffset, rLength);
231   }
232 
233   /**
234    * Compare columnFamily, qualifier, timestamp, and key type (everything
235    * except the row). This method is used both in the normal comparator and
236    * the "same-prefix" comparator. Note that we are assuming that row portions
237    * of both KVs have already been parsed and found identical, and we don't
238    * validate that assumption here.
239    * @param commonPrefix
240    *          the length of the common prefix of the two key-values being
241    *          compared, including row length and row
242    */
243   private final int compareWithoutRow(Cell left,
244       byte[] right, int roffset, int rlength, short rowlength) {
245     /***
246      * KeyValue Format and commonLength:
247      * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
248      * ------------------|-------commonLength--------|--------------
249      */
250     int commonLength = KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE + rowlength;
251 
252     // commonLength + TIMESTAMP_TYPE_SIZE
253     int commonLengthWithTSAndType = KeyValue.TIMESTAMP_TYPE_SIZE + commonLength;
254     // ColumnFamily + Qualifier length.
255     int lcolumnlength = left.getFamilyLength() + left.getQualifierLength();
256     int rcolumnlength = rlength - commonLengthWithTSAndType;
257 
258     byte ltype = left.getTypeByte();
259     byte rtype = right[roffset + (rlength - 1)];
260 
261     // If the column is not specified, the "minimum" key type appears the
262     // latest in the sorted order, regardless of the timestamp. This is used
263     // for specifying the last key/value in a given row, because there is no
264     // "lexicographically last column" (it would be infinitely long). The
265     // "maximum" key type does not need this behavior.
266     if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
267       // left is "bigger", i.e. it appears later in the sorted order
268       return 1;
269     }
270     if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
271       return -1;
272     }
273 
274     int rfamilyoffset = commonLength + roffset;
275 
276     // Column family length.
277     int lfamilylength = left.getFamilyLength();
278     int rfamilylength = right[rfamilyoffset - 1];
279     // If left family size is not equal to right family size, we need not
280     // compare the qualifiers.
281     boolean sameFamilySize = (lfamilylength == rfamilylength);
282     if (!sameFamilySize) {
283       // comparing column family is enough.
284       return compareFamilies(left, right, rfamilyoffset, rfamilylength);
285     }
286     // Compare family & qualifier together.
287     // Families are same. Compare on qualifiers.
288     int comparison = compareColumns(left, right, rfamilyoffset, rfamilylength, rfamilyoffset
289         + rfamilylength, (rcolumnlength - rfamilylength));
290     if (comparison != 0) {
291       return comparison;
292     }
293 
294     // //
295     // Next compare timestamps.
296     long rtimestamp = Bytes.toLong(right, roffset + (rlength - KeyValue.TIMESTAMP_TYPE_SIZE));
297     int compare = compareTimestamps(left.getTimestamp(), rtimestamp);
298     if (compare != 0) {
299       return compare;
300     }
301 
302     // Compare types. Let the delete types sort ahead of puts; i.e. types
303     // of higher numbers sort before those of lesser numbers. Maximum (255)
304     // appears ahead of everything, and minimum (0) appears after
305     // everything.
306     return (0xff & rtype) - (0xff & ltype);
307   }
308 
309   /**
310    * Compares the rows of the left and right cell.
311    * For the hbase:meta case this method is overridden such that it can handle hbase:meta cells.
312    * The caller should ensure using the appropriate comparator for hbase:meta.
313    * @param left
314    * @param right
315    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
316    */
317   public int compareRows(final Cell left, final Cell right) {
318     if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
319       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
320           ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(),
321           ((ByteBufferedCell) right).getRowByteBuffer(),
322           ((ByteBufferedCell) right).getRowPosition(), right.getRowLength());
323     }
324     if (left instanceof ByteBufferedCell) {
325       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
326           ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(),
327           right.getRowArray(), right.getRowOffset(), right.getRowLength());
328     }
329     if (right instanceof ByteBufferedCell) {
330       return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getRowByteBuffer(),
331           ((ByteBufferedCell) right).getRowPosition(), right.getRowLength(),
332           left.getRowArray(), left.getRowOffset(), left.getRowLength()));
333     }
334     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
335         right.getRowArray(), right.getRowOffset(), right.getRowLength());
336   }
337 
338   /**
339    * Compares the row part of the cell with a simple plain byte[] like the
340    * stopRow in Scan. This should be used with context where for hbase:meta
341    * cells the {{@link #META_COMPARATOR} should be used
342    *
343    * @param left
344    *          the cell to be compared
345    * @param right
346    *          the kv serialized byte[] to be compared with
347    * @param roffset
348    *          the offset in the byte[]
349    * @param rlength
350    *          the length in the byte[]
351    * @return 0 if both cell and the byte[] are equal, 1 if the cell is bigger
352    *         than byte[], -1 otherwise
353    */
354   public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
355     if (left instanceof ByteBufferedCell) {
356       return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
357           ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(), right,
358           roffset, rlength);
359     }
360     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
361         roffset, rlength);
362   }
363 
364   private static int compareWithoutRow(final Cell left, final Cell right) {
365     // If the column is not specified, the "minimum" key type appears the
366     // latest in the sorted order, regardless of the timestamp. This is used
367     // for specifying the last key/value in a given row, because there is no
368     // "lexicographically last column" (it would be infinitely long). The
369     // "maximum" key type does not need this behavior.
370     // Copied from KeyValue. This is bad in that we can't do memcmp w/ special rules like this.
371     int lFamLength = left.getFamilyLength();
372     int rFamLength = right.getFamilyLength();
373     int lQualLength = left.getQualifierLength();
374     int rQualLength = right.getQualifierLength();
375     if (lFamLength + lQualLength == 0
376           && left.getTypeByte() == Type.Minimum.getCode()) {
377       // left is "bigger", i.e. it appears later in the sorted order
378       return 1;
379     }
380     if (rFamLength + rQualLength == 0
381         && right.getTypeByte() == Type.Minimum.getCode()) {
382       return -1;
383     }
384     if (lFamLength != rFamLength) {
385       // comparing column family is enough.
386       return compareFamilies(left, right);
387     }
388     // Compare cf:qualifier
389     int diff = compareColumns(left, right);
390     if (diff != 0) return diff;
391 
392     diff = compareTimestamps(left, right);
393     if (diff != 0) return diff;
394 
395     // Compare types. Let the delete types sort ahead of puts; i.e. types
396     // of higher numbers sort before those of lesser numbers. Maximum (255)
397     // appears ahead of everything, and minimum (0) appears after
398     // everything.
399     return (0xff & right.getTypeByte()) - (0xff & left.getTypeByte());
400   }
401 
402   /**
403    * Compares cell's timestamps in DESCENDING order.
404    * The below older timestamps sorting ahead of newer timestamps looks
405    * wrong but it is intentional. This way, newer timestamps are first
406    * found when we iterate over a memstore and newer versions are the
407    * first we trip over when reading from a store file.
408    * @return 1 if left's timestamp &lt; right's timestamp
409    *         -1 if left's timestamp &gt; right's timestamp
410    *         0 if both timestamps are equal
411    */
412   public static int compareTimestamps(final Cell left, final Cell right) {
413     return compareTimestamps(left.getTimestamp(), right.getTimestamp());
414   }
415 
416   /**
417    * Used to compare two cells based on the column hint provided. This is specifically
418    * used when we need to optimize the seeks based on the next indexed key. This is an
419    * advance usage API specifically needed for some optimizations.
420    * @param nextIndexedCell the next indexed cell 
421    * @param currentCell the cell to be compared
422    * @param foff the family offset of the currentCell
423    * @param flen the family length of the currentCell
424    * @param colHint the column hint provided - could be null
425    * @param coff the offset of the column hint if provided, if not offset of the currentCell's
426    * qualifier
427    * @param clen the length of the column hint if provided, if not length of the currentCell's
428    * qualifier
429    * @param ts the timestamp to be seeked
430    * @param type the type to be seeked
431    * @return an int based on the given column hint
432    * TODO : To be moved out of here because this is a special API used in scan
433    * optimization.
434    */
435   // compare a key against row/fam/qual/ts/type
436   public final int compareKeyBasedOnColHint(Cell nextIndexedCell, Cell currentCell, int foff,
437       int flen, byte[] colHint, int coff, int clen, long ts, byte type) {
438     int compare = compareRows(nextIndexedCell, currentCell);
439     if (compare != 0) {
440       return compare;
441     }
442     // If the column is not specified, the "minimum" key type appears the
443     // latest in the sorted order, regardless of the timestamp. This is used
444     // for specifying the last key/value in a given row, because there is no
445     // "lexicographically last column" (it would be infinitely long). The
446     // "maximum" key type does not need this behavior.
447     if (nextIndexedCell.getFamilyLength() + nextIndexedCell.getQualifierLength() == 0
448         && nextIndexedCell.getTypeByte() == Type.Minimum.getCode()) {
449       // left is "bigger", i.e. it appears later in the sorted order
450       return 1;
451     }
452     if (flen + clen == 0 && type == Type.Minimum.getCode()) {
453       return -1;
454     }
455 
456     compare = compareFamilies(nextIndexedCell, currentCell);
457     if (compare != 0) {
458       return compare;
459     }
460     if (colHint == null) {
461       compare = compareQualifiers(nextIndexedCell, currentCell);
462     } else {
463       compare = compareQualifiers(nextIndexedCell, colHint, coff, clen);
464     }
465     if (compare != 0) {
466       return compare;
467     }
468     // Next compare timestamps.
469     compare = compareTimestamps(nextIndexedCell.getTimestamp(), ts);
470     if (compare != 0) {
471       return compare;
472     }
473 
474     // Compare types. Let the delete types sort ahead of puts; i.e. types
475     // of higher numbers sort before those of lesser numbers. Maximum (255)
476     // appears ahead of everything, and minimum (0) appears after
477     // everything.
478     return (0xff & type) - (0xff & nextIndexedCell.getTypeByte());
479   }
480 
481   /**
482    * Compares timestamps in DESCENDING order.
483    * The below older timestamps sorting ahead of newer timestamps looks
484    * wrong but it is intentional. This way, newer timestamps are first
485    * found when we iterate over a memstore and newer versions are the
486    * first we trip over when reading from a store file.
487    * @return 1 if left timestamp &lt; right timestamp
488    *         -1 if left timestamp &gt; right timestamp
489    *         0 if both timestamps are equal
490    */
491   public static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
492     if (ltimestamp < rtimestamp) {
493       return 1;
494     } else if (ltimestamp > rtimestamp) {
495       return -1;
496     }
497     return 0;
498   }
499 
500   /**
501    * Compare cell's row against given comparator
502    * @param cell
503    * @param comparator
504    * @return result comparing cell's row
505    */
506   public static int compareRow(Cell cell, ByteArrayComparable comparator) {
507     if (cell instanceof ByteBufferedCell) {
508       return comparator.compareTo(((ByteBufferedCell) cell).getRowByteBuffer(),
509           ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength());
510     }
511     return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
512   }
513 
514   /**
515    * Compare cell's column family against given comparator
516    * @param cell
517    * @param comparator
518    * @return result comparing cell's column family
519    */
520   public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
521     if (cell instanceof ByteBufferedCell) {
522       return comparator.compareTo(((ByteBufferedCell) cell).getFamilyByteBuffer(),
523           ((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength());
524     }
525     return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
526         cell.getFamilyLength());
527   }
528 
529   /**
530    * Compare cell's qualifier against given comparator
531    * @param cell
532    * @param comparator
533    * @return result comparing cell's qualifier
534    */
535   public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
536     if (cell instanceof ByteBufferedCell) {
537       return comparator.compareTo(((ByteBufferedCell) cell).getQualifierByteBuffer(),
538           ((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength());
539     }
540     return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
541         cell.getQualifierLength());
542   }
543 
544   /**
545    * Compare cell's value against given comparator
546    * @param cell
547    * @param comparator
548    * @return result comparing cell's value
549    */
550   public static int compareValue(Cell cell, ByteArrayComparable comparator) {
551     if (cell instanceof ByteBufferedCell) {
552       return comparator.compareTo(((ByteBufferedCell) cell).getValueByteBuffer(),
553           ((ByteBufferedCell) cell).getValuePosition(), cell.getValueLength());
554     }
555     return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
556   }
557 
558   /**
559    * Comparator that compares row component only of a Cell
560    */
561   public static class RowComparator extends CellComparator {
562     @Override
563     public int compare(Cell a, Cell b) {
564       return compareRows(a, b);
565     }
566   }
567 
568   /**
569    * A {@link CellComparator} for <code>hbase:meta</code> catalog table
570    * {@link KeyValue}s.
571    */
572   public static class MetaCellComparator extends CellComparator {
573 
574     @Override
575     public int compareRows(final Cell left, final Cell right) {
576       return compareRows(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
577           right.getRowArray(), right.getRowOffset(), right.getRowLength());
578     }
579 
580     @Override
581     public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
582       return compareRows(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
583           roffset, rlength);
584     }
585 
586     private int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
587         int rlength) {
588       int leftDelimiter = Bytes.searchDelimiterIndex(left, loffset, llength, HConstants.DELIMITER);
589       int rightDelimiter = Bytes
590           .searchDelimiterIndex(right, roffset, rlength, HConstants.DELIMITER);
591       // Compare up to the delimiter
592       int lpart = (leftDelimiter < 0 ? llength : leftDelimiter - loffset);
593       int rpart = (rightDelimiter < 0 ? rlength : rightDelimiter - roffset);
594       int result = Bytes.compareTo(left, loffset, lpart, right, roffset, rpart);
595       if (result != 0) {
596         return result;
597       } else {
598         if (leftDelimiter < 0 && rightDelimiter >= 0) {
599           return -1;
600         } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
601           return 1;
602         } else if (leftDelimiter < 0 && rightDelimiter < 0) {
603           return 0;
604         }
605       }
606       // Compare middle bit of the row.
607       // Move past delimiter
608       leftDelimiter++;
609       rightDelimiter++;
610       int leftFarDelimiter = Bytes.searchDelimiterIndexInReverse(left, leftDelimiter, llength
611           - (leftDelimiter - loffset), HConstants.DELIMITER);
612       int rightFarDelimiter = Bytes.searchDelimiterIndexInReverse(right, rightDelimiter, rlength
613           - (rightDelimiter - roffset), HConstants.DELIMITER);
614       // Now compare middlesection of row.
615       lpart = (leftFarDelimiter < 0 ? llength + loffset : leftFarDelimiter) - leftDelimiter;
616       rpart = (rightFarDelimiter < 0 ? rlength + roffset : rightFarDelimiter) - rightDelimiter;
617       result = Bytes.compareTo(left, leftDelimiter, lpart, right, rightDelimiter, rpart);
618       if (result != 0) {
619         return result;
620       } else {
621         if (leftDelimiter < 0 && rightDelimiter >= 0) {
622           return -1;
623         } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
624           return 1;
625         } else if (leftDelimiter < 0 && rightDelimiter < 0) {
626           return 0;
627         }
628       }
629       // Compare last part of row, the rowid.
630       leftFarDelimiter++;
631       rightFarDelimiter++;
632       result = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
633           right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
634       return result;
635     }
636   }
637 }