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