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