001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase;
019
020import java.util.Comparator;
021import org.apache.hadoop.hbase.util.ByteBufferUtils;
022import org.apache.hadoop.hbase.util.Bytes;
023import org.apache.yetus.audience.InterfaceAudience;
024import org.apache.yetus.audience.InterfaceStability;
025
026/**
027 * Compare two HBase cells. Do not use this method comparing <code>-ROOT-</code> or
028 * <code>hbase:meta</code> cells. Cells from these tables need a specialized comparator, one that
029 * takes account of the special formatting of the row where we have commas to delimit table from
030 * regionname, from row. See KeyValue for how it has a special comparator to do hbase:meta cells and
031 * yet another for -ROOT-.
032 * <p>
033 * While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells
034 * format should be taken into consideration, for which the instance of this comparator should be
035 * used. In all other cases the static APIs in this comparator would be enough
036 * <p>
037 * HOT methods. We spend a good portion of CPU comparing. Anything that makes the compare faster
038 * will likely manifest at the macro level.
039 * </p>
040 */
041@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UNKNOWN",
042    justification = "Findbugs doesn't like the way we are negating the result of"
043      + " a compare in below")
044@InterfaceAudience.Private
045@InterfaceStability.Evolving
046public class CellComparatorImpl implements CellComparator {
047
048  private static final long serialVersionUID = 8186411895799094989L;
049
050  /**
051   * Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion of
052   * KeyValue only.
053   */
054  public static final CellComparatorImpl COMPARATOR = new CellComparatorImpl();
055
056  @Override
057  public final int compare(final Cell a, final Cell b) {
058    return compare(a, b, false);
059  }
060
061  @Override
062  public int compare(final Cell l, final Cell r, boolean ignoreSequenceid) {
063    int diff = 0;
064    // "Peel off" the most common path.
065    if (l instanceof KeyValue && r instanceof KeyValue) {
066      diff = compareKeyValues((KeyValue) l, (KeyValue) r);
067      if (diff != 0) {
068        return diff;
069      }
070    } else if (l instanceof KeyValue && r instanceof ByteBufferKeyValue) {
071      diff = compareKVVsBBKV((KeyValue) l, (ByteBufferKeyValue) r);
072      if (diff != 0) {
073        return diff;
074      }
075    } else if (l instanceof ByteBufferKeyValue && r instanceof KeyValue) {
076      diff = compareKVVsBBKV((KeyValue) r, (ByteBufferKeyValue) l);
077      if (diff != 0) {
078        // negate- Findbugs will complain?
079        return -diff;
080      }
081    } else if (l instanceof ByteBufferKeyValue && r instanceof ByteBufferKeyValue) {
082      diff = compareBBKV((ByteBufferKeyValue) l, (ByteBufferKeyValue) r);
083      if (diff != 0) {
084        return diff;
085      }
086    } else {
087      int leftRowLength = l.getRowLength();
088      int rightRowLength = r.getRowLength();
089      diff = compareRows(l, leftRowLength, r, rightRowLength);
090      if (diff != 0) {
091        return diff;
092      }
093
094      diff = compareWithoutRow(l, r);
095      if (diff != 0) {
096        return diff;
097      }
098    }
099
100    if (ignoreSequenceid) {
101      return diff;
102    }
103    // Negate following comparisons so later edits show up first mvccVersion: later sorts first
104    return Long.compare(PrivateCellUtil.getSequenceId(r), PrivateCellUtil.getSequenceId(l));
105  }
106
107  private int compareKeyValues(final KeyValue left, final KeyValue right) {
108    int diff;
109    // Compare Rows. Cache row length.
110    int leftRowLength = left.getRowLength();
111    int rightRowLength = right.getRowLength();
112    diff = Bytes.compareTo(left.getRowArray(), left.getRowOffset(), leftRowLength,
113      right.getRowArray(), right.getRowOffset(), rightRowLength);
114    if (diff != 0) {
115      return diff;
116    }
117
118    // If the column is not specified, the "minimum" key type appears as latest in the sorted
119    // order, regardless of the timestamp. This is used for specifying the last key/value in a
120    // given row, because there is no "lexicographically last column" (it would be infinitely
121    // long).
122    // The "maximum" key type does not need this behavior. Copied from KeyValue. This is bad in
123    // that
124    // we can't do memcmp w/ special rules like this.
125    // TODO: Is there a test for this behavior?
126    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
127    int leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
128    int leftKeyLength = left.getKeyLength();
129    int leftQualifierLength =
130      left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
131
132    // No need of left row length below here.
133
134    byte leftType = left.getTypeByte(leftKeyLength);
135    if (
136      leftType == KeyValue.Type.Minimum.getCode() && leftFamilyLength + leftQualifierLength == 0
137    ) {
138      // left is "bigger", i.e. it appears later in the sorted order
139      return 1;
140    }
141
142    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
143    int rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
144    int rightKeyLength = right.getKeyLength();
145    int rightQualifierLength =
146      right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
147
148    // No need of right row length below here.
149
150    byte rightType = right.getTypeByte(rightKeyLength);
151    if (
152      rightType == KeyValue.Type.Minimum.getCode() && rightFamilyLength + rightQualifierLength == 0
153    ) {
154      return -1;
155    }
156
157    // Compare families.
158    int leftFamilyPosition = left.getFamilyOffset(leftFamilyLengthPosition);
159    int rightFamilyPosition = right.getFamilyOffset(rightFamilyLengthPosition);
160    diff = compareFamilies(left, leftFamilyPosition, leftFamilyLength, right, rightFamilyPosition,
161      rightFamilyLength);
162    if (diff != 0) {
163      return diff;
164    }
165
166    // Compare qualifiers
167    diff = Bytes.compareTo(left.getQualifierArray(),
168      left.getQualifierOffset(leftFamilyPosition, leftFamilyLength), leftQualifierLength,
169      right.getQualifierArray(), right.getQualifierOffset(rightFamilyPosition, rightFamilyLength),
170      rightQualifierLength);
171    if (diff != 0) {
172      return diff;
173    }
174
175    // Timestamps.
176    // Swap order we pass into compare so we get DESCENDING order.
177    // TODO : Ensure we read the bytes and do the compare instead of the value.
178    diff = Long.compare(right.getTimestamp(rightKeyLength), left.getTimestamp(leftKeyLength));
179    if (diff != 0) {
180      return diff;
181    }
182
183    // Compare types. Let the delete types sort ahead of puts; i.e. types
184    // of higher numbers sort before those of lesser numbers. Maximum (255)
185    // appears ahead of everything, and minimum (0) appears after
186    // everything.
187    return (0xff & rightType) - (0xff & leftType);
188  }
189
190  private int compareBBKV(final ByteBufferKeyValue left, final ByteBufferKeyValue right) {
191    int diff;
192    // Compare Rows. Cache row length.
193    int leftRowLength = left.getRowLength();
194    int rightRowLength = right.getRowLength();
195    diff = ByteBufferUtils.compareTo(left.getRowByteBuffer(), left.getRowPosition(), leftRowLength,
196      right.getRowByteBuffer(), right.getRowPosition(), rightRowLength);
197    if (diff != 0) {
198      return diff;
199    }
200
201    // If the column is not specified, the "minimum" key type appears as latest in the sorted
202    // order, regardless of the timestamp. This is used for specifying the last key/value in a
203    // given row, because there is no "lexicographically last column" (it would be infinitely
204    // long).
205    // The "maximum" key type does not need this behavior. Copied from KeyValue. This is bad in
206    // that
207    // we can't do memcmp w/ special rules like this.
208    // TODO: Is there a test for this behavior?
209    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
210    int leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
211    int leftKeyLength = left.getKeyLength();
212    int leftQualifierLength =
213      left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
214
215    // No need of left row length below here.
216
217    byte leftType = left.getTypeByte(leftKeyLength);
218    if (
219      leftType == KeyValue.Type.Minimum.getCode() && leftFamilyLength + leftQualifierLength == 0
220    ) {
221      // left is "bigger", i.e. it appears later in the sorted order
222      return 1;
223    }
224
225    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
226    int rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
227    int rightKeyLength = right.getKeyLength();
228    int rightQualifierLength =
229      right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
230
231    // No need of right row length below here.
232
233    byte rightType = right.getTypeByte(rightKeyLength);
234    if (
235      rightType == KeyValue.Type.Minimum.getCode() && rightFamilyLength + rightQualifierLength == 0
236    ) {
237      return -1;
238    }
239
240    // Compare families.
241    int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
242    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
243    diff = compareFamilies(left, leftFamilyPosition, leftFamilyLength, right, rightFamilyPosition,
244      rightFamilyLength);
245    if (diff != 0) {
246      return diff;
247    }
248
249    // Compare qualifiers
250    diff = ByteBufferUtils.compareTo(left.getQualifierByteBuffer(),
251      left.getQualifierPosition(leftFamilyPosition, leftFamilyLength), leftQualifierLength,
252      right.getQualifierByteBuffer(),
253      right.getQualifierPosition(rightFamilyPosition, rightFamilyLength), rightQualifierLength);
254    if (diff != 0) {
255      return diff;
256    }
257
258    // Timestamps.
259    // Swap order we pass into compare so we get DESCENDING order.
260    diff = Long.compare(right.getTimestamp(rightKeyLength), left.getTimestamp(leftKeyLength));
261    if (diff != 0) {
262      return diff;
263    }
264
265    // Compare types. Let the delete types sort ahead of puts; i.e. types
266    // of higher numbers sort before those of lesser numbers. Maximum (255)
267    // appears ahead of everything, and minimum (0) appears after
268    // everything.
269    return (0xff & rightType) - (0xff & leftType);
270  }
271
272  private int compareKVVsBBKV(final KeyValue left, final ByteBufferKeyValue right) {
273    int diff;
274    // Compare Rows. Cache row length.
275    int leftRowLength = left.getRowLength();
276    int rightRowLength = right.getRowLength();
277    diff = ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset(), leftRowLength,
278      right.getRowByteBuffer(), right.getRowPosition(), rightRowLength);
279    if (diff != 0) {
280      return diff;
281    }
282
283    // If the column is not specified, the "minimum" key type appears as latest in the sorted
284    // order, regardless of the timestamp. This is used for specifying the last key/value in a
285    // given row, because there is no "lexicographically last column" (it would be infinitely
286    // long).
287    // The "maximum" key type does not need this behavior. Copied from KeyValue. This is bad in
288    // that
289    // we can't do memcmp w/ special rules like this.
290    // TODO: Is there a test for this behavior?
291    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
292    int leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
293    int leftKeyLength = left.getKeyLength();
294    int leftQualifierLength =
295      left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
296
297    // No need of left row length below here.
298
299    byte leftType = left.getTypeByte(leftKeyLength);
300    if (
301      leftType == KeyValue.Type.Minimum.getCode() && leftFamilyLength + leftQualifierLength == 0
302    ) {
303      // left is "bigger", i.e. it appears later in the sorted order
304      return 1;
305    }
306
307    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
308    int rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
309    int rightKeyLength = right.getKeyLength();
310    int rightQualifierLength =
311      right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
312
313    // No need of right row length below here.
314
315    byte rightType = right.getTypeByte(rightKeyLength);
316    if (
317      rightType == KeyValue.Type.Minimum.getCode() && rightFamilyLength + rightQualifierLength == 0
318    ) {
319      return -1;
320    }
321
322    // Compare families.
323    int leftFamilyPosition = left.getFamilyOffset(leftFamilyLengthPosition);
324    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
325    diff = compareFamilies(left, leftFamilyPosition, leftFamilyLength, right, rightFamilyPosition,
326      rightFamilyLength);
327    if (diff != 0) {
328      return diff;
329    }
330
331    // Compare qualifiers
332    diff = ByteBufferUtils.compareTo(left.getQualifierArray(),
333      left.getQualifierOffset(leftFamilyPosition, leftFamilyLength), leftQualifierLength,
334      right.getQualifierByteBuffer(),
335      right.getQualifierPosition(rightFamilyPosition, rightFamilyLength), rightQualifierLength);
336    if (diff != 0) {
337      return diff;
338    }
339
340    // Timestamps.
341    // Swap order we pass into compare so we get DESCENDING order.
342    diff = Long.compare(right.getTimestamp(rightKeyLength), left.getTimestamp(leftKeyLength));
343    if (diff != 0) {
344      return diff;
345    }
346
347    // Compare types. Let the delete types sort ahead of puts; i.e. types
348    // of higher numbers sort before those of lesser numbers. Maximum (255)
349    // appears ahead of everything, and minimum (0) appears after
350    // everything.
351    return (0xff & rightType) - (0xff & leftType);
352  }
353
354  /**
355   * Compares the family and qualifier part of the cell
356   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
357   */
358  public final int compareColumns(final Cell left, final Cell right) {
359    int diff = compareFamilies(left, right);
360    if (diff != 0) {
361      return diff;
362    }
363    return compareQualifiers(left, right);
364  }
365
366  private int compareColumns(final Cell left, final int leftFamLen, final int leftQualLen,
367    final Cell right, final int rightFamLen, final int rightQualLen) {
368    int diff = compareFamilies(left, leftFamLen, right, rightFamLen);
369    if (diff != 0) {
370      return diff;
371    }
372    return compareQualifiers(left, leftQualLen, right, rightQualLen);
373  }
374
375  /**
376   * This method will be overridden when we compare cells inner store to bypass family comparing.
377   */
378  protected int compareFamilies(Cell left, int leftFamLen, Cell right, int rightFamLen) {
379    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
380      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
381        ((ByteBufferExtendedCell) left).getFamilyPosition(), leftFamLen,
382        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
383        ((ByteBufferExtendedCell) right).getFamilyPosition(), rightFamLen);
384    }
385    if (left instanceof ByteBufferExtendedCell) {
386      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
387        ((ByteBufferExtendedCell) left).getFamilyPosition(), leftFamLen, right.getFamilyArray(),
388        right.getFamilyOffset(), rightFamLen);
389    }
390    if (right instanceof ByteBufferExtendedCell) {
391      // Notice how we flip the order of the compare here. We used to negate the return value but
392      // see what FindBugs says
393      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
394      // It suggest flipping the order to get same effect and 'safer'.
395      return ByteBufferUtils.compareTo(left.getFamilyArray(), left.getFamilyOffset(), leftFamLen,
396        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
397        ((ByteBufferExtendedCell) right).getFamilyPosition(), rightFamLen);
398    }
399    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), leftFamLen,
400      right.getFamilyArray(), right.getFamilyOffset(), rightFamLen);
401  }
402
403  private final int compareQualifiers(Cell left, int leftQualLen, Cell right, int rightQualLen) {
404    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
405      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
406        ((ByteBufferExtendedCell) left).getQualifierPosition(), leftQualLen,
407        ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
408        ((ByteBufferExtendedCell) right).getQualifierPosition(), rightQualLen);
409    }
410    if (left instanceof ByteBufferExtendedCell) {
411      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
412        ((ByteBufferExtendedCell) left).getQualifierPosition(), leftQualLen,
413        right.getQualifierArray(), right.getQualifierOffset(), rightQualLen);
414    }
415    if (right instanceof ByteBufferExtendedCell) {
416      // Notice how we flip the order of the compare here. We used to negate the return value but
417      // see what FindBugs says
418      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
419      // It suggest flipping the order to get same effect and 'safer'.
420      return ByteBufferUtils.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
421        leftQualLen, ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
422        ((ByteBufferExtendedCell) right).getQualifierPosition(), rightQualLen);
423    }
424    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(), leftQualLen,
425      right.getQualifierArray(), right.getQualifierOffset(), rightQualLen);
426  }
427
428  /**
429   * Compare the families of left and right cell
430   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
431   */
432  @Override
433  public final int compareFamilies(Cell left, Cell right) {
434    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
435      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
436        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
437        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
438        ((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
439    }
440    if (left instanceof ByteBufferExtendedCell) {
441      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
442        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
443        right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
444    }
445    if (right instanceof ByteBufferExtendedCell) {
446      // Notice how we flip the order of the compare here. We used to negate the return value but
447      // see what FindBugs says
448      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
449      // It suggest flipping the order to get same effect and 'safer'.
450      return ByteBufferUtils.compareTo(left.getFamilyArray(), left.getFamilyOffset(),
451        left.getFamilyLength(), ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
452        ((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
453    }
454    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
455      right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
456  }
457
458  /**
459   * This method will be overridden when we compare cells inner store to bypass family comparing.
460   */
461  protected int compareFamilies(KeyValue left, int leftFamilyPosition, int leftFamilyLength,
462    KeyValue right, int rightFamilyPosition, int rightFamilyLength) {
463    return Bytes.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
464      right.getFamilyArray(), rightFamilyPosition, rightFamilyLength);
465  }
466
467  /**
468   * This method will be overridden when we compare cells inner store to bypass family comparing.
469   */
470  protected int compareFamilies(ByteBufferKeyValue left, int leftFamilyPosition,
471    int leftFamilyLength, ByteBufferKeyValue right, int rightFamilyPosition,
472    int rightFamilyLength) {
473    return ByteBufferUtils.compareTo(left.getFamilyByteBuffer(), leftFamilyPosition,
474      leftFamilyLength, right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
475  }
476
477  /**
478   * This method will be overridden when we compare cells inner store to bypass family comparing.
479   */
480  protected int compareFamilies(KeyValue left, int leftFamilyPosition, int leftFamilyLength,
481    ByteBufferKeyValue right, int rightFamilyPosition, int rightFamilyLength) {
482    return ByteBufferUtils.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
483      right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
484  }
485
486  static int compareQualifiers(KeyValue left, KeyValue right) {
487    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
488    // sharing gets us a few percent more throughput in compares. If changes here or there, make
489    // sure done in both places.
490    // Compare Rows. Cache row length.
491    int leftRowLength = left.getRowLength();
492    int rightRowLength = right.getRowLength();
493
494    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
495    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
496    int leftKeyLength = left.getKeyLength();
497    int leftQualifierLength =
498      left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
499
500    // No need of left row length below here.
501
502    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
503    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
504    int rightKeyLength = right.getKeyLength();
505    int rightQualifierLength =
506      right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
507
508    // Compare families.
509    int leftFamilyOffset = left.getFamilyOffset(leftFamilyLengthPosition);
510    int rightFamilyOffset = right.getFamilyOffset(rightFamilyLengthPosition);
511
512    // Compare qualifiers
513    return Bytes.compareTo(left.getQualifierArray(), leftFamilyOffset + leftFamilyLength,
514      leftQualifierLength, right.getQualifierArray(), rightFamilyOffset + rightFamilyLength,
515      rightQualifierLength);
516  }
517
518  static int compareQualifiers(KeyValue left, ByteBufferKeyValue right) {
519    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
520    // sharing gets us a few percent more throughput in compares. If changes here or there, make
521    // sure done in both places.
522    // Compare Rows. Cache row length.
523    int leftRowLength = left.getRowLength();
524    int rightRowLength = right.getRowLength();
525
526    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
527    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
528    int leftKeyLength = left.getKeyLength();
529    int leftQualifierLength =
530      left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
531
532    // No need of left row length below here.
533
534    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
535    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
536    int rightKeyLength = right.getKeyLength();
537    int rightQualifierLength =
538      right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
539
540    // Compare families.
541    int leftFamilyOffset = left.getFamilyOffset(leftFamilyLengthPosition);
542    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
543
544    // Compare qualifiers
545    return ByteBufferUtils.compareTo(left.getQualifierArray(), leftFamilyOffset + leftFamilyLength,
546      leftQualifierLength, right.getQualifierByteBuffer(), rightFamilyPosition + rightFamilyLength,
547      rightQualifierLength);
548  }
549
550  static int compareQualifiers(ByteBufferKeyValue left, KeyValue right) {
551    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
552    // sharing gets us a few percent more throughput in compares. If changes here or there, make
553    // sure done in both places.
554    // Compare Rows. Cache row length.
555    int leftRowLength = left.getRowLength();
556    int rightRowLength = right.getRowLength();
557
558    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
559    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
560    int leftKeyLength = left.getKeyLength();
561    int leftQualifierLength =
562      left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
563
564    // No need of left row length below here.
565
566    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
567    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
568    int rightKeyLength = right.getKeyLength();
569    int rightQualifierLength =
570      right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
571
572    // Compare families.
573    int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
574    int rightFamilyOffset = right.getFamilyOffset(rightFamilyLengthPosition);
575
576    // Compare qualifiers
577    return ByteBufferUtils.compareTo(left.getQualifierByteBuffer(),
578      leftFamilyPosition + leftFamilyLength, leftQualifierLength, right.getQualifierArray(),
579      rightFamilyOffset + rightFamilyLength, rightQualifierLength);
580  }
581
582  static int compareQualifiers(ByteBufferKeyValue left, ByteBufferKeyValue right) {
583    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
584    // sharing gets us a few percent more throughput in compares. If changes here or there, make
585    // sure done in both places.
586    // Compare Rows. Cache row length.
587    int leftRowLength = left.getRowLength();
588    int rightRowLength = right.getRowLength();
589
590    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
591    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
592    int leftKeyLength = left.getKeyLength();
593    int leftQualifierLength =
594      left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
595
596    // No need of left row length below here.
597
598    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
599    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
600    int rightKeyLength = right.getKeyLength();
601    int rightQualifierLength =
602      right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
603
604    // Compare families.
605    int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
606    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
607
608    // Compare qualifiers
609    return ByteBufferUtils.compareTo(left.getQualifierByteBuffer(),
610      leftFamilyPosition + leftFamilyLength, leftQualifierLength, right.getQualifierByteBuffer(),
611      rightFamilyPosition + rightFamilyLength, rightQualifierLength);
612  }
613
614  /**
615   * Compare the qualifiers part of the left and right cells.
616   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
617   */
618  @Override
619  public final int compareQualifiers(Cell left, Cell right) {
620    if ((left instanceof ByteBufferKeyValue) && (right instanceof ByteBufferKeyValue)) {
621      return compareQualifiers((ByteBufferKeyValue) left, (ByteBufferKeyValue) right);
622    } else if ((left instanceof KeyValue) && (right instanceof KeyValue)) {
623      return compareQualifiers((KeyValue) left, (KeyValue) right);
624    } else if ((left instanceof KeyValue) && (right instanceof ByteBufferKeyValue)) {
625      return compareQualifiers((KeyValue) left, (ByteBufferKeyValue) right);
626    } else if ((left instanceof ByteBufferKeyValue) && (right instanceof KeyValue)) {
627      return compareQualifiers((ByteBufferKeyValue) left, (KeyValue) right);
628    } else {
629      if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
630        return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
631          ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
632          ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
633          ((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength());
634      }
635      if (left instanceof ByteBufferExtendedCell) {
636        return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
637          ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
638          right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
639      }
640      if (right instanceof ByteBufferExtendedCell) {
641        // Notice how we flip the order of the compare here. We used to negate the return value but
642        // see what FindBugs says
643        // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
644        // It suggest flipping the order to get same effect and 'safer'.
645        return ByteBufferUtils.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
646          left.getQualifierLength(), ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
647          ((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength());
648      }
649      return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
650        left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
651        right.getQualifierLength());
652    }
653
654  }
655
656  /**
657   * Compares the rows of the left and right cell. For the hbase:meta case this method is overridden
658   * such that it can handle hbase:meta cells. The caller should ensure using the appropriate
659   * comparator for hbase:meta.
660   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
661   */
662  @Override
663  public int compareRows(final Cell left, final Cell right) {
664    return compareRows(left, left.getRowLength(), right, right.getRowLength());
665  }
666
667  static int compareRows(final Cell left, int leftRowLength, final Cell right, int rightRowLength) {
668    // left and right can be exactly the same at the beginning of a row
669    if (left == right) {
670      return 0;
671    }
672    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
673      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
674        ((ByteBufferExtendedCell) left).getRowPosition(), leftRowLength,
675        ((ByteBufferExtendedCell) right).getRowByteBuffer(),
676        ((ByteBufferExtendedCell) right).getRowPosition(), rightRowLength);
677    }
678    if (left instanceof ByteBufferExtendedCell) {
679      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
680        ((ByteBufferExtendedCell) left).getRowPosition(), leftRowLength, right.getRowArray(),
681        right.getRowOffset(), rightRowLength);
682    }
683    if (right instanceof ByteBufferExtendedCell) {
684      // Notice how we flip the order of the compare here. We used to negate the return value but
685      // see what FindBugs says
686      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
687      // It suggest flipping the order to get same effect and 'safer'.
688      return ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset(), leftRowLength,
689        ((ByteBufferExtendedCell) right).getRowByteBuffer(),
690        ((ByteBufferExtendedCell) right).getRowPosition(), rightRowLength);
691    }
692    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), leftRowLength,
693      right.getRowArray(), right.getRowOffset(), rightRowLength);
694  }
695
696  /**
697   * Compares the row part of the cell with a simple plain byte[] like the stopRow in Scan. This
698   * should be used with context where for hbase:meta cells the
699   * {{@link MetaCellComparator#META_COMPARATOR} should be used the cell to be compared the kv
700   * serialized byte[] to be compared with the offset in the byte[] the length in the byte[]
701   * @return 0 if both cell and the byte[] are equal, 1 if the cell is bigger than byte[], -1
702   *         otherwise
703   */
704  @Override
705  public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
706    if (left instanceof ByteBufferExtendedCell) {
707      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
708        ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), right, roffset,
709        rlength);
710    }
711    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
712      roffset, rlength);
713  }
714
715  @Override
716  public final int compareWithoutRow(final Cell left, final Cell right) {
717    // If the column is not specified, the "minimum" key type appears the
718    // latest in the sorted order, regardless of the timestamp. This is used
719    // for specifying the last key/value in a given row, because there is no
720    // "lexicographically last column" (it would be infinitely long). The
721    // "maximum" key type does not need this behavior.
722    // Copied from KeyValue. This is bad in that we can't do memcmp w/ special rules like this.
723    int lFamLength = left.getFamilyLength();
724    int rFamLength = right.getFamilyLength();
725    int lQualLength = left.getQualifierLength();
726    int rQualLength = right.getQualifierLength();
727    byte leftType = PrivateCellUtil.getTypeByte(left);
728    byte rightType = PrivateCellUtil.getTypeByte(right);
729    if (lFamLength + lQualLength == 0 && leftType == KeyValue.Type.Minimum.getCode()) {
730      // left is "bigger", i.e. it appears later in the sorted order
731      return 1;
732    }
733    if (rFamLength + rQualLength == 0 && rightType == KeyValue.Type.Minimum.getCode()) {
734      return -1;
735    }
736    if (lFamLength != rFamLength) {
737      // comparing column family is enough.
738      return compareFamilies(left, lFamLength, right, rFamLength);
739    }
740    // Compare cf:qualifier
741    int diff = compareColumns(left, lFamLength, lQualLength, right, rFamLength, rQualLength);
742    if (diff != 0) {
743      return diff;
744    }
745
746    diff = compareTimestamps(left.getTimestamp(), right.getTimestamp());
747    if (diff != 0) {
748      return diff;
749    }
750
751    // Compare types. Let the delete types sort ahead of puts; i.e. types
752    // of higher numbers sort before those of lesser numbers. Maximum (255)
753    // appears ahead of everything, and minimum (0) appears after
754    // everything.
755    return (0xff & rightType) - (0xff & leftType);
756  }
757
758  @Override
759  public int compareTimestamps(final Cell left, final Cell right) {
760    return compareTimestamps(left.getTimestamp(), right.getTimestamp());
761  }
762
763  @Override
764  public int compareTimestamps(final long ltimestamp, final long rtimestamp) {
765    // Swap order we pass into compare so we get DESCENDING order.
766    return Long.compare(rtimestamp, ltimestamp);
767  }
768
769  @Override
770  public Comparator<Cell> getSimpleComparator() {
771    return this;
772  }
773
774  /**
775   * Utility method that makes a guess at comparator to use based off passed tableName. Use in
776   * extreme when no comparator specified.
777   * @return CellComparator to use going off the {@code tableName} passed.
778   */
779  public static CellComparator getCellComparator(TableName tableName) {
780    return getCellComparator(tableName.toBytes());
781  }
782
783  /**
784   * Utility method that makes a guess at comparator to use based off passed tableName. Use in
785   * extreme when no comparator specified.
786   * @return CellComparator to use going off the {@code tableName} passed.
787   */
788  public static CellComparator getCellComparator(byte[] tableName) {
789    // FYI, TableName.toBytes does not create an array; just returns existing array pointer.
790    return Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes())
791      ? MetaCellComparator.META_COMPARATOR
792      : CellComparatorImpl.COMPARATOR;
793  }
794}