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