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.filter;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Arrays;
023import java.util.Comparator;
024import java.util.List;
025import java.util.Objects;
026import java.util.PriorityQueue;
027import org.apache.hadoop.hbase.Cell;
028import org.apache.hadoop.hbase.CellComparator;
029import org.apache.hadoop.hbase.PrivateCellUtil;
030import org.apache.hadoop.hbase.exceptions.DeserializationException;
031import org.apache.hadoop.hbase.unsafe.HBasePlatformDependent;
032import org.apache.hadoop.hbase.util.Bytes;
033import org.apache.hadoop.hbase.util.Pair;
034import org.apache.yetus.audience.InterfaceAudience;
035
036import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
037import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
038
039import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
040import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair;
041
042/**
043 * This is optimized version of a standard FuzzyRowFilter Filters data based on fuzzy row key.
044 * Performs fast-forwards during scanning. It takes pairs (row key, fuzzy info) to match row keys.
045 * Where fuzzy info is a byte array with 0 or 1 as its values:
046 * <ul>
047 * <li>0 - means that this byte in provided row key is fixed, i.e. row key's byte at same position
048 * must match</li>
049 * <li>1 - means that this byte in provided row key is NOT fixed, i.e. row key's byte at this
050 * position can be different from the one in provided row key</li>
051 * </ul>
052 * Example:
053 * <p/>
054 * Let's assume row key format is userId_actionId_year_month. Length of userId is fixed and is 4,
055 * length of actionId is 2 and year and month are 4 and 2 bytes long respectively.
056 * <p/>
057 * Let's assume that we need to fetch all users that performed certain action (encoded as "99") in
058 * Jan of any year. Then the pair (row key, fuzzy info) would be the following:
059 *
060 * <pre>
061 * row key = "????_99_????_01" (one can use any value instead of "?")
062 * fuzzy info = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00"
063 * </pre>
064 *
065 * I.e. fuzzy info tells the matching mask is "????_99_????_01", where at ? can be any value.
066 */
067@InterfaceAudience.Public
068public class FuzzyRowFilter extends FilterBase {
069
070  private static final boolean UNSAFE_UNALIGNED = HBasePlatformDependent.unaligned();
071
072  // the wildcard byte is 1 on the user side. but the filter converts it internally
073  // in preprocessMask. This was changed in HBASE-15676 due to a bug with using 0.
074  // in v1, the 1 byte gets converted to 0
075  // in v2, the 1 byte gets converted to 2.
076  // we support both here to ensure backwards compatibility between client and server
077  static final byte V1_PROCESSED_WILDCARD_MASK = 0;
078  static final byte V2_PROCESSED_WILDCARD_MASK = 2;
079
080  private final byte processedWildcardMask;
081  private List<Pair<byte[], byte[]>> fuzzyKeysData;
082  // Used to record whether we want to skip the current row.
083  // Usually we should use filterRowKey here but in the current scan implementation, if filterRowKey
084  // returns true, we will just skip to next row, instead of calling getNextCellHint to determine
085  // the actual next row, so we need to implement filterCell and return SEEK_NEXT_USING_HINT to let
086  // upper layer call getNextCellHint.
087  // And if we do not implement filterRow, sometimes we will get incorrect result when using
088  // FuzzyRowFilter together with other filters, please see the description for HBASE-26967 for more
089  // details.
090  private boolean filterRow;
091  private boolean done = false;
092
093  /**
094   * The index of a last successfully found matching fuzzy string (in fuzzyKeysData). We will start
095   * matching next KV with this one. If they do not match then we will return back to the one-by-one
096   * iteration over fuzzyKeysData.
097   */
098  private int lastFoundIndex = -1;
099
100  /**
101   * Row tracker (keeps all next rows after SEEK_NEXT_USING_HINT was returned)
102   */
103  private RowTracker tracker;
104
105  // this client side constructor ensures that all client-constructed
106  // FuzzyRowFilters use the new v2 mask.
107  public FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData) {
108    this(fuzzyKeysData, V2_PROCESSED_WILDCARD_MASK);
109  }
110
111  // This constructor is only used internally here, when parsing from protos on the server side.
112  // It exists to enable seamless migration from v1 to v2.
113  // Additionally used in tests, but never used on client side.
114  FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData, byte processedWildcardMask) {
115    this.processedWildcardMask = processedWildcardMask;
116
117    List<Pair<byte[], byte[]>> fuzzyKeyDataCopy = new ArrayList<>(fuzzyKeysData.size());
118
119    for (Pair<byte[], byte[]> aFuzzyKeysData : fuzzyKeysData) {
120      if (aFuzzyKeysData.getFirst().length != aFuzzyKeysData.getSecond().length) {
121        Pair<String, String> readable = new Pair<>(Bytes.toStringBinary(aFuzzyKeysData.getFirst()),
122          Bytes.toStringBinary(aFuzzyKeysData.getSecond()));
123        throw new IllegalArgumentException("Fuzzy pair lengths do not match: " + readable);
124      }
125
126      Pair<byte[], byte[]> p = new Pair<>();
127      // create a copy of pair bytes so that they are not modified by the filter.
128      p.setFirst(Arrays.copyOf(aFuzzyKeysData.getFirst(), aFuzzyKeysData.getFirst().length));
129      p.setSecond(Arrays.copyOf(aFuzzyKeysData.getSecond(), aFuzzyKeysData.getSecond().length));
130
131      // update mask ( 0 -> -1 (0xff), 1 -> [0 or 2 depending on processedWildcardMask value])
132      p.setSecond(preprocessMask(p.getSecond()));
133      preprocessSearchKey(p);
134
135      fuzzyKeyDataCopy.add(p);
136    }
137    this.fuzzyKeysData = fuzzyKeyDataCopy;
138    this.tracker = new RowTracker();
139  }
140
141  private void preprocessSearchKey(Pair<byte[], byte[]> p) {
142    if (!UNSAFE_UNALIGNED) {
143      // do nothing
144      return;
145    }
146    byte[] key = p.getFirst();
147    byte[] mask = p.getSecond();
148    for (int i = 0; i < mask.length; i++) {
149      // set non-fixed part of a search key to 0.
150      if (mask[i] == processedWildcardMask) {
151        key[i] = 0;
152      }
153    }
154  }
155
156  /**
157   * We need to preprocess mask array, as since we treat 2's as unfixed positions and -1 (0xff) as
158   * fixed positions
159   * @return mask array
160   */
161  private byte[] preprocessMask(byte[] mask) {
162    if (!UNSAFE_UNALIGNED) {
163      // do nothing
164      return mask;
165    }
166    if (isPreprocessedMask(mask)) return mask;
167    for (int i = 0; i < mask.length; i++) {
168      if (mask[i] == 0) {
169        mask[i] = -1; // 0 -> -1
170      } else if (mask[i] == 1) {
171        mask[i] = processedWildcardMask;// 1 -> 0 or 2 depending on mask version
172      }
173    }
174    return mask;
175  }
176
177  private boolean isPreprocessedMask(byte[] mask) {
178    for (int i = 0; i < mask.length; i++) {
179      if (mask[i] != -1 && mask[i] != processedWildcardMask) {
180        return false;
181      }
182    }
183    return true;
184  }
185
186  @Deprecated
187  @Override
188  public ReturnCode filterKeyValue(final Cell c) {
189    return filterCell(c);
190  }
191
192  @Override
193  public void reset() throws IOException {
194    filterRow = false;
195  }
196
197  @Override
198  public boolean filterRow() throws IOException {
199    return filterRow;
200  }
201
202  @Override
203  public ReturnCode filterCell(final Cell c) {
204    final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0;
205    final int size = fuzzyKeysData.size();
206    for (int i = startIndex; i < size + startIndex; i++) {
207      final int index = i % size;
208      Pair<byte[], byte[]> fuzzyData = fuzzyKeysData.get(index);
209      idempotentMaskShift(fuzzyData.getSecond());
210      SatisfiesCode satisfiesCode = satisfies(isReversed(), c.getRowArray(), c.getRowOffset(),
211        c.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond());
212      if (satisfiesCode == SatisfiesCode.YES) {
213        lastFoundIndex = index;
214        return ReturnCode.INCLUDE;
215      }
216    }
217    // NOT FOUND -> seek next using hint
218    lastFoundIndex = -1;
219    filterRow = true;
220    return ReturnCode.SEEK_NEXT_USING_HINT;
221  }
222
223  static void idempotentMaskShift(byte[] mask) {
224    // This shift is idempotent - always end up with 0 and -1 as mask values.
225    // This works regardless of mask version, because both 0 >> 2 and 2 >> 2
226    // result in 0.
227    for (int j = 0; j < mask.length; j++) {
228      mask[j] >>= 2;
229    }
230  }
231
232  @Override
233  public Cell getNextCellHint(Cell currentCell) {
234    boolean result = tracker.updateTracker(currentCell);
235    if (result == false) {
236      done = true;
237      return null;
238    }
239    byte[] nextRowKey = tracker.nextRow();
240    return PrivateCellUtil.createFirstOnRow(nextRowKey, 0, (short) nextRowKey.length);
241  }
242
243  /**
244   * If we have multiple fuzzy keys, row tracker should improve overall performance. It calculates
245   * all next rows (one per every fuzzy key) and put them (the fuzzy key is bundled) into a priority
246   * queue so that the smallest row key always appears at queue head, which helps to decide the
247   * "Next Cell Hint". As scanning going on, the number of candidate rows in the RowTracker will
248   * remain the size of fuzzy keys until some of the fuzzy keys won't possibly have matches any
249   * more.
250   */
251  private class RowTracker {
252    private final PriorityQueue<Pair<byte[], Pair<byte[], byte[]>>> nextRows;
253    private boolean initialized = false;
254
255    RowTracker() {
256      nextRows = new PriorityQueue<>(fuzzyKeysData.size(),
257        new Comparator<Pair<byte[], Pair<byte[], byte[]>>>() {
258          @Override
259          public int compare(Pair<byte[], Pair<byte[], byte[]>> o1,
260            Pair<byte[], Pair<byte[], byte[]>> o2) {
261            return isReversed()
262              ? Bytes.compareTo(o2.getFirst(), o1.getFirst())
263              : Bytes.compareTo(o1.getFirst(), o2.getFirst());
264          }
265        });
266    }
267
268    byte[] nextRow() {
269      if (nextRows.isEmpty()) {
270        throw new IllegalStateException("NextRows should not be empty, "
271          + "make sure to call nextRow() after updateTracker() return true");
272      } else {
273        return nextRows.peek().getFirst();
274      }
275    }
276
277    boolean updateTracker(Cell currentCell) {
278      if (!initialized) {
279        for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
280          updateWith(currentCell, fuzzyData);
281        }
282        initialized = true;
283      } else {
284        while (!nextRows.isEmpty() && !lessThan(currentCell, nextRows.peek().getFirst())) {
285          Pair<byte[], Pair<byte[], byte[]>> head = nextRows.poll();
286          Pair<byte[], byte[]> fuzzyData = head.getSecond();
287          updateWith(currentCell, fuzzyData);
288        }
289      }
290      return !nextRows.isEmpty();
291    }
292
293    boolean lessThan(Cell currentCell, byte[] nextRowKey) {
294      int compareResult =
295        CellComparator.getInstance().compareRows(currentCell, nextRowKey, 0, nextRowKey.length);
296      return (!isReversed() && compareResult < 0) || (isReversed() && compareResult > 0);
297    }
298
299    void updateWith(Cell currentCell, Pair<byte[], byte[]> fuzzyData) {
300      byte[] nextRowKeyCandidate =
301        getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), currentCell.getRowOffset(),
302          currentCell.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond());
303      if (nextRowKeyCandidate != null) {
304        nextRows.add(new Pair<>(nextRowKeyCandidate, fuzzyData));
305      }
306    }
307
308  }
309
310  @Override
311  public boolean filterAllRemaining() {
312    return done;
313  }
314
315  /** Returns The filter serialized using pb */
316  @Override
317  public byte[] toByteArray() {
318    FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter.newBuilder()
319      .setIsMaskV2(processedWildcardMask == V2_PROCESSED_WILDCARD_MASK);
320    for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
321      BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
322      bbpBuilder.setFirst(UnsafeByteOperations.unsafeWrap(fuzzyData.getFirst()));
323      bbpBuilder.setSecond(UnsafeByteOperations.unsafeWrap(fuzzyData.getSecond()));
324      builder.addFuzzyKeysData(bbpBuilder);
325    }
326    return builder.build().toByteArray();
327  }
328
329  /**
330   * Parse a serialized representation of {@link FuzzyRowFilter}
331   * @param pbBytes A pb serialized {@link FuzzyRowFilter} instance
332   * @return An instance of {@link FuzzyRowFilter} made from <code>bytes</code>
333   * @throws DeserializationException if an error occurred
334   * @see #toByteArray
335   */
336  public static FuzzyRowFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
337    FilterProtos.FuzzyRowFilter proto;
338    try {
339      proto = FilterProtos.FuzzyRowFilter.parseFrom(pbBytes);
340    } catch (InvalidProtocolBufferException e) {
341      throw new DeserializationException(e);
342    }
343    int count = proto.getFuzzyKeysDataCount();
344    ArrayList<Pair<byte[], byte[]>> fuzzyKeysData = new ArrayList<>(count);
345    for (int i = 0; i < count; ++i) {
346      BytesBytesPair current = proto.getFuzzyKeysData(i);
347      byte[] keyBytes = current.getFirst().toByteArray();
348      byte[] keyMeta = current.getSecond().toByteArray();
349      fuzzyKeysData.add(new Pair<>(keyBytes, keyMeta));
350    }
351    byte processedWildcardMask = proto.hasIsMaskV2() && proto.getIsMaskV2()
352      ? V2_PROCESSED_WILDCARD_MASK
353      : V1_PROCESSED_WILDCARD_MASK;
354    return new FuzzyRowFilter(fuzzyKeysData, processedWildcardMask);
355  }
356
357  @Override
358  public String toString() {
359    final StringBuilder sb = new StringBuilder();
360    sb.append("FuzzyRowFilter");
361    sb.append("{fuzzyKeysData=");
362    for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
363      sb.append('{').append(Bytes.toStringBinary(fuzzyData.getFirst())).append(":");
364      sb.append(Bytes.toStringBinary(fuzzyData.getSecond())).append('}');
365    }
366    sb.append("}, ");
367    return sb.toString();
368  }
369
370  // Utility methods
371
372  static enum SatisfiesCode {
373    /** row satisfies fuzzy rule */
374    YES,
375    /** row doesn't satisfy fuzzy rule, but there's possible greater row that does */
376    NEXT_EXISTS,
377    /** row doesn't satisfy fuzzy rule and there's no greater row that does */
378    NO_NEXT
379  }
380
381  @InterfaceAudience.Private
382  static SatisfiesCode satisfies(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
383    return satisfies(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
384  }
385
386  @InterfaceAudience.Private
387  static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
388    byte[] fuzzyKeyMeta) {
389    return satisfies(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
390  }
391
392  static SatisfiesCode satisfies(boolean reverse, byte[] row, int offset, int length,
393    byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
394
395    if (!UNSAFE_UNALIGNED) {
396      return satisfiesNoUnsafe(reverse, row, offset, length, fuzzyKeyBytes, fuzzyKeyMeta);
397    }
398
399    if (row == null) {
400      // do nothing, let scan to proceed
401      return SatisfiesCode.YES;
402    }
403    length = Math.min(length, fuzzyKeyBytes.length);
404    int numWords = length / Bytes.SIZEOF_LONG;
405
406    int j = numWords << 3; // numWords * SIZEOF_LONG;
407
408    for (int i = 0; i < j; i += Bytes.SIZEOF_LONG) {
409      long fuzzyBytes = Bytes.toLong(fuzzyKeyBytes, i);
410      long fuzzyMeta = Bytes.toLong(fuzzyKeyMeta, i);
411      long rowValue = Bytes.toLong(row, offset + i);
412      if ((rowValue & fuzzyMeta) != fuzzyBytes) {
413        // We always return NEXT_EXISTS
414        return SatisfiesCode.NEXT_EXISTS;
415      }
416    }
417
418    int off = j;
419
420    if (length - off >= Bytes.SIZEOF_INT) {
421      int fuzzyBytes = Bytes.toInt(fuzzyKeyBytes, off);
422      int fuzzyMeta = Bytes.toInt(fuzzyKeyMeta, off);
423      int rowValue = Bytes.toInt(row, offset + off);
424      if ((rowValue & fuzzyMeta) != fuzzyBytes) {
425        // We always return NEXT_EXISTS
426        return SatisfiesCode.NEXT_EXISTS;
427      }
428      off += Bytes.SIZEOF_INT;
429    }
430
431    if (length - off >= Bytes.SIZEOF_SHORT) {
432      short fuzzyBytes = Bytes.toShort(fuzzyKeyBytes, off);
433      short fuzzyMeta = Bytes.toShort(fuzzyKeyMeta, off);
434      short rowValue = Bytes.toShort(row, offset + off);
435      if ((rowValue & fuzzyMeta) != fuzzyBytes) {
436        // We always return NEXT_EXISTS
437        // even if it does not (in this case getNextForFuzzyRule
438        // will return null)
439        return SatisfiesCode.NEXT_EXISTS;
440      }
441      off += Bytes.SIZEOF_SHORT;
442    }
443
444    if (length - off >= Bytes.SIZEOF_BYTE) {
445      int fuzzyBytes = fuzzyKeyBytes[off] & 0xff;
446      int fuzzyMeta = fuzzyKeyMeta[off] & 0xff;
447      int rowValue = row[offset + off] & 0xff;
448      if ((rowValue & fuzzyMeta) != fuzzyBytes) {
449        // We always return NEXT_EXISTS
450        return SatisfiesCode.NEXT_EXISTS;
451      }
452    }
453    return SatisfiesCode.YES;
454  }
455
456  static SatisfiesCode satisfiesNoUnsafe(boolean reverse, byte[] row, int offset, int length,
457    byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
458    if (row == null) {
459      // do nothing, let scan to proceed
460      return SatisfiesCode.YES;
461    }
462
463    Order order = Order.orderFor(reverse);
464    boolean nextRowKeyCandidateExists = false;
465
466    for (int i = 0; i < fuzzyKeyMeta.length && i < length; i++) {
467      // First, checking if this position is fixed and not equals the given one
468      boolean byteAtPositionFixed = fuzzyKeyMeta[i] == 0;
469      boolean fixedByteIncorrect = byteAtPositionFixed && fuzzyKeyBytes[i] != row[i + offset];
470      if (fixedByteIncorrect) {
471        // in this case there's another row that satisfies fuzzy rule and bigger than this row
472        if (nextRowKeyCandidateExists) {
473          return SatisfiesCode.NEXT_EXISTS;
474        }
475
476        // If this row byte is less than fixed then there's a byte array bigger than
477        // this row and which satisfies the fuzzy rule. Otherwise there's no such byte array:
478        // this row is simply bigger than any byte array that satisfies the fuzzy rule
479        boolean rowByteLessThanFixed = (row[i + offset] & 0xFF) < (fuzzyKeyBytes[i] & 0xFF);
480        if (rowByteLessThanFixed && !reverse) {
481          return SatisfiesCode.NEXT_EXISTS;
482        } else if (!rowByteLessThanFixed && reverse) {
483          return SatisfiesCode.NEXT_EXISTS;
484        } else {
485          return SatisfiesCode.NO_NEXT;
486        }
487      }
488
489      // Second, checking if this position is not fixed and byte value is not the biggest. In this
490      // case there's a byte array bigger than this row and which satisfies the fuzzy rule. To get
491      // bigger byte array that satisfies the rule we need to just increase this byte
492      // (see the code of getNextForFuzzyRule below) by one.
493      // Note: if non-fixed byte is already at biggest value, this doesn't allow us to say there's
494      // bigger one that satisfies the rule as it can't be increased.
495      if (fuzzyKeyMeta[i] == 1 && !order.isMax(fuzzyKeyBytes[i])) {
496        nextRowKeyCandidateExists = true;
497      }
498    }
499    return SatisfiesCode.YES;
500  }
501
502  @InterfaceAudience.Private
503  static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
504    return getNextForFuzzyRule(false, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
505  }
506
507  @InterfaceAudience.Private
508  static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
509    byte[] fuzzyKeyMeta) {
510    return getNextForFuzzyRule(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
511  }
512
513  /** Abstracts directional comparisons based on scan direction. */
514  private enum Order {
515    ASC {
516      @Override
517      public boolean lt(int lhs, int rhs) {
518        return lhs < rhs;
519      }
520
521      @Override
522      public boolean gt(int lhs, int rhs) {
523        return lhs > rhs;
524      }
525
526      @Override
527      public byte inc(byte val) {
528        // TODO: what about over/underflow?
529        return (byte) (val + 1);
530      }
531
532      @Override
533      public boolean isMax(byte val) {
534        return val == (byte) 0xff;
535      }
536
537      @Override
538      public byte min() {
539        return 0;
540      }
541    },
542    DESC {
543      @Override
544      public boolean lt(int lhs, int rhs) {
545        return lhs > rhs;
546      }
547
548      @Override
549      public boolean gt(int lhs, int rhs) {
550        return lhs < rhs;
551      }
552
553      @Override
554      public byte inc(byte val) {
555        // TODO: what about over/underflow?
556        return (byte) (val - 1);
557      }
558
559      @Override
560      public boolean isMax(byte val) {
561        return val == 0;
562      }
563
564      @Override
565      public byte min() {
566        return (byte) 0xFF;
567      }
568    };
569
570    public static Order orderFor(boolean reverse) {
571      return reverse ? DESC : ASC;
572    }
573
574    /** Returns true when {@code lhs < rhs}. */
575    public abstract boolean lt(int lhs, int rhs);
576
577    /** Returns true when {@code lhs > rhs}. */
578    public abstract boolean gt(int lhs, int rhs);
579
580    /** Returns {@code val} incremented by 1. */
581    public abstract byte inc(byte val);
582
583    /** Return true when {@code val} is the maximum value */
584    public abstract boolean isMax(byte val);
585
586    /** Return the minimum value according to this ordering scheme. */
587    public abstract byte min();
588  }
589
590  /**
591   * @return greater byte array than given (row) which satisfies the fuzzy rule if it exists, null
592   *         otherwise
593   */
594  @InterfaceAudience.Private
595  static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, int offset, int length,
596    byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
597    // To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than
598    // the given one we do the following:
599    // 1. setting values on all "fixed" positions to the values from fuzzyKeyBytes
600    // 2. if during the first step given row did not increase, then we increase the value at
601    // the first "non-fixed" position (where it is not maximum already)
602
603    // It is easier to perform this by using fuzzyKeyBytes copy and setting "non-fixed" position
604    // values than otherwise.
605    byte[] result =
606      Arrays.copyOf(fuzzyKeyBytes, length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length);
607    if (reverse && length > fuzzyKeyBytes.length) {
608      // we need trailing 0xff's instead of trailing 0x00's
609      for (int i = fuzzyKeyBytes.length; i < result.length; i++) {
610        result[i] = (byte) 0xFF;
611      }
612    }
613    int toInc = -1;
614    final Order order = Order.orderFor(reverse);
615
616    boolean increased = false;
617    for (int i = 0; i < result.length; i++) {
618      if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 0 /* non-fixed */) {
619        result[i] = row[offset + i];
620        if (!order.isMax(row[offset + i])) {
621          // this is "non-fixed" position and is not at max value, hence we can increase it
622          toInc = i;
623        }
624      } else if (i < fuzzyKeyMeta.length && fuzzyKeyMeta[i] == -1 /* fixed */) {
625        if (order.lt((row[i + offset] & 0xFF), (fuzzyKeyBytes[i] & 0xFF))) {
626          // if setting value for any fixed position increased the original array,
627          // we are OK
628          increased = true;
629          break;
630        }
631
632        if (order.gt((row[i + offset] & 0xFF), (fuzzyKeyBytes[i] & 0xFF))) {
633          // if setting value for any fixed position makes array "smaller", then just stop:
634          // in case we found some non-fixed position to increase we will do it, otherwise
635          // there's no "next" row key that satisfies fuzzy rule and "greater" than given row
636          break;
637        }
638      }
639    }
640
641    if (!increased) {
642      if (toInc < 0) {
643        return null;
644      }
645      result[toInc] = order.inc(result[toInc]);
646
647      // Setting all "non-fixed" positions to zeroes to the right of the one we increased so
648      // that found "next" row key is the smallest possible
649      for (int i = toInc + 1; i < result.length; i++) {
650        if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 0 /* non-fixed */) {
651          result[i] = order.min();
652        }
653      }
654    }
655
656    return reverse ? result : trimTrailingZeroes(result, fuzzyKeyMeta, toInc);
657  }
658
659  /**
660   * For forward scanner, next cell hint should not contain any trailing zeroes unless they are part
661   * of fuzzyKeyMeta hint = '\x01\x01\x01\x00\x00' will skip valid row '\x01\x01\x01'
662   * @param toInc - position of incremented byte
663   * @return trimmed version of result
664   */
665
666  private static byte[] trimTrailingZeroes(byte[] result, byte[] fuzzyKeyMeta, int toInc) {
667    int off = fuzzyKeyMeta.length >= result.length ? result.length - 1 : fuzzyKeyMeta.length - 1;
668    for (; off >= 0; off--) {
669      if (fuzzyKeyMeta[off] != 0) break;
670    }
671    if (off < toInc) off = toInc;
672    byte[] retValue = new byte[off + 1];
673    System.arraycopy(result, 0, retValue, 0, retValue.length);
674    return retValue;
675  }
676
677  /**
678   * Returns true if and only if the fields of the filter that are serialized are equal to the
679   * corresponding fields in other. Used for testing.
680   */
681  @Override
682  boolean areSerializedFieldsEqual(Filter o) {
683    if (o == this) {
684      return true;
685    }
686    if (!(o instanceof FuzzyRowFilter)) {
687      return false;
688    }
689    FuzzyRowFilter other = (FuzzyRowFilter) o;
690    if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) return false;
691    for (int i = 0; i < fuzzyKeysData.size(); ++i) {
692      Pair<byte[], byte[]> thisData = this.fuzzyKeysData.get(i);
693      Pair<byte[], byte[]> otherData = other.fuzzyKeysData.get(i);
694      if (
695        !(Bytes.equals(thisData.getFirst(), otherData.getFirst())
696          && Bytes.equals(thisData.getSecond(), otherData.getSecond()))
697      ) {
698        return false;
699      }
700    }
701    return true;
702  }
703
704  @Override
705  public boolean equals(Object obj) {
706    return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
707  }
708
709  @Override
710  public int hashCode() {
711    return Objects.hash(this.fuzzyKeysData);
712  }
713}