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