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.lang.reflect.InvocationTargetException;
021import java.lang.reflect.Method;
022import java.nio.ByteBuffer;
023import java.nio.charset.CharacterCodingException;
024import java.nio.charset.StandardCharsets;
025import java.util.ArrayList;
026import java.util.Collections;
027import java.util.EmptyStackException;
028import java.util.HashMap;
029import java.util.Map;
030import java.util.Set;
031import java.util.Stack;
032import java.util.regex.Pattern;
033import org.apache.hadoop.hbase.CompareOperator;
034import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
035import org.apache.hadoop.hbase.util.Bytes;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040/**
041 * This class allows a user to specify a filter via a string The string is parsed using the methods
042 * of this class and a filter object is constructed. This filter object is then wrapped in a scanner
043 * object which is then returned
044 * <p>
045 * This class addresses the HBASE-4168 JIRA. More documentation on this Filter Language can be found
046 * at: https://issues.apache.org/jira/browse/HBASE-4176
047 */
048@InterfaceAudience.Public
049public class ParseFilter {
050  private static final Logger LOG = LoggerFactory.getLogger(ParseFilter.class);
051
052  private static HashMap<ByteBuffer, Integer> operatorPrecedenceHashMap;
053  private static HashMap<String, String> filterHashMap;
054
055  static {
056    // Registers all the filter supported by the Filter Language
057    filterHashMap = new HashMap<>();
058    filterHashMap.put("KeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." + "KeyOnlyFilter");
059    filterHashMap.put("FirstKeyOnlyFilter",
060      ParseConstants.FILTER_PACKAGE + "." + "FirstKeyOnlyFilter");
061    filterHashMap.put("PrefixFilter", ParseConstants.FILTER_PACKAGE + "." + "PrefixFilter");
062    filterHashMap.put("ColumnPrefixFilter",
063      ParseConstants.FILTER_PACKAGE + "." + "ColumnPrefixFilter");
064    filterHashMap.put("MultipleColumnPrefixFilter",
065      ParseConstants.FILTER_PACKAGE + "." + "MultipleColumnPrefixFilter");
066    filterHashMap.put("ColumnCountGetFilter",
067      ParseConstants.FILTER_PACKAGE + "." + "ColumnCountGetFilter");
068    filterHashMap.put("PageFilter", ParseConstants.FILTER_PACKAGE + "." + "PageFilter");
069    filterHashMap.put("ColumnPaginationFilter",
070      ParseConstants.FILTER_PACKAGE + "." + "ColumnPaginationFilter");
071    filterHashMap.put("InclusiveStopFilter",
072      ParseConstants.FILTER_PACKAGE + "." + "InclusiveStopFilter");
073    filterHashMap.put("TimestampsFilter", ParseConstants.FILTER_PACKAGE + "." + "TimestampsFilter");
074    filterHashMap.put("RowFilter", ParseConstants.FILTER_PACKAGE + "." + "RowFilter");
075    filterHashMap.put("FamilyFilter", ParseConstants.FILTER_PACKAGE + "." + "FamilyFilter");
076    filterHashMap.put("QualifierFilter", ParseConstants.FILTER_PACKAGE + "." + "QualifierFilter");
077    filterHashMap.put("ValueFilter", ParseConstants.FILTER_PACKAGE + "." + "ValueFilter");
078    filterHashMap.put("ColumnRangeFilter",
079      ParseConstants.FILTER_PACKAGE + "." + "ColumnRangeFilter");
080    filterHashMap.put("SingleColumnValueFilter",
081      ParseConstants.FILTER_PACKAGE + "." + "SingleColumnValueFilter");
082    filterHashMap.put("SingleColumnValueExcludeFilter",
083      ParseConstants.FILTER_PACKAGE + "." + "SingleColumnValueExcludeFilter");
084    filterHashMap.put("DependentColumnFilter",
085      ParseConstants.FILTER_PACKAGE + "." + "DependentColumnFilter");
086    filterHashMap.put("ColumnValueFilter",
087      ParseConstants.FILTER_PACKAGE + "." + "ColumnValueFilter");
088
089    // Creates the operatorPrecedenceHashMap
090    operatorPrecedenceHashMap = new HashMap<>();
091    operatorPrecedenceHashMap.put(ParseConstants.SKIP_BUFFER, 1);
092    operatorPrecedenceHashMap.put(ParseConstants.WHILE_BUFFER, 1);
093    operatorPrecedenceHashMap.put(ParseConstants.AND_BUFFER, 2);
094    operatorPrecedenceHashMap.put(ParseConstants.OR_BUFFER, 3);
095  }
096
097  /**
098   * Parses the filterString and constructs a filter using it
099   * <p>
100   * @param filterString filter string given by the user
101   * @return filter object we constructed
102   */
103  public Filter parseFilterString(String filterString) throws CharacterCodingException {
104    return parseFilterString(Bytes.toBytes(filterString));
105  }
106
107  /**
108   * Parses the filterString and constructs a filter using it
109   * <p>
110   * @param filterStringAsByteArray filter string given by the user
111   * @return filter object we constructed
112   */
113  public Filter parseFilterString(byte[] filterStringAsByteArray) throws CharacterCodingException {
114    // stack for the operators and parenthesis
115    Stack<ByteBuffer> operatorStack = new Stack<>();
116    // stack for the filter objects
117    Stack<Filter> filterStack = new Stack<>();
118
119    Filter filter = null;
120    for (int i = 0; i < filterStringAsByteArray.length; i++) {
121      if (filterStringAsByteArray[i] == ParseConstants.LPAREN) {
122        // LPAREN found
123        operatorStack.push(ParseConstants.LPAREN_BUFFER);
124      } else if (
125        filterStringAsByteArray[i] == ParseConstants.WHITESPACE
126          || filterStringAsByteArray[i] == ParseConstants.TAB
127      ) {
128        // WHITESPACE or TAB found
129        continue;
130      } else if (checkForOr(filterStringAsByteArray, i)) {
131        // OR found
132        i += ParseConstants.OR_ARRAY.length - 1;
133        reduce(operatorStack, filterStack, ParseConstants.OR_BUFFER);
134        operatorStack.push(ParseConstants.OR_BUFFER);
135      } else if (checkForAnd(filterStringAsByteArray, i)) {
136        // AND found
137        i += ParseConstants.AND_ARRAY.length - 1;
138        reduce(operatorStack, filterStack, ParseConstants.AND_BUFFER);
139        operatorStack.push(ParseConstants.AND_BUFFER);
140      } else if (checkForSkip(filterStringAsByteArray, i)) {
141        // SKIP found
142        i += ParseConstants.SKIP_ARRAY.length - 1;
143        reduce(operatorStack, filterStack, ParseConstants.SKIP_BUFFER);
144        operatorStack.push(ParseConstants.SKIP_BUFFER);
145      } else if (checkForWhile(filterStringAsByteArray, i)) {
146        // WHILE found
147        i += ParseConstants.WHILE_ARRAY.length - 1;
148        reduce(operatorStack, filterStack, ParseConstants.WHILE_BUFFER);
149        operatorStack.push(ParseConstants.WHILE_BUFFER);
150      } else if (filterStringAsByteArray[i] == ParseConstants.RPAREN) {
151        // RPAREN found
152        if (operatorStack.empty()) {
153          throw new IllegalArgumentException("Mismatched parenthesis");
154        }
155        ByteBuffer argumentOnTopOfStack = operatorStack.peek();
156        if (argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER)) {
157          operatorStack.pop();
158          continue;
159        }
160        while (!argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER)) {
161          filterStack.push(popArguments(operatorStack, filterStack));
162          if (operatorStack.empty()) {
163            throw new IllegalArgumentException("Mismatched parenthesis");
164          }
165          argumentOnTopOfStack = operatorStack.pop();
166        }
167      } else {
168        // SimpleFilterExpression found
169        byte[] filterSimpleExpression = extractFilterSimpleExpression(filterStringAsByteArray, i);
170        i += (filterSimpleExpression.length - 1);
171        filter = parseSimpleFilterExpression(filterSimpleExpression);
172        filterStack.push(filter);
173      }
174    }
175
176    // Finished parsing filterString
177    while (!operatorStack.empty()) {
178      filterStack.push(popArguments(operatorStack, filterStack));
179    }
180    if (filterStack.empty()) {
181      throw new IllegalArgumentException("Incorrect Filter String");
182    }
183    filter = filterStack.pop();
184    if (!filterStack.empty()) {
185      throw new IllegalArgumentException("Incorrect Filter String");
186    }
187    return filter;
188  }
189
190  /**
191   * Extracts a simple filter expression from the filter string given by the user
192   * <p>
193   * A simpleFilterExpression is of the form: FilterName('arg', 'arg', 'arg') The user given filter
194   * string can have many simpleFilterExpressions combined using operators.
195   * <p>
196   * This function extracts a simpleFilterExpression from the larger filterString given the start
197   * offset of the simpler expression
198   * <p>
199   * @param filterStringAsByteArray     filter string given by the user
200   * @param filterExpressionStartOffset start index of the simple filter expression
201   * @return byte array containing the simple filter expression
202   */
203  public byte[] extractFilterSimpleExpression(byte[] filterStringAsByteArray,
204    int filterExpressionStartOffset) throws CharacterCodingException {
205    int quoteCount = 0;
206    for (int i = filterExpressionStartOffset; i < filterStringAsByteArray.length; i++) {
207      if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
208        if (isQuoteUnescaped(filterStringAsByteArray, i)) {
209          quoteCount++;
210        } else {
211          // To skip the next quote that has been escaped
212          i++;
213        }
214      }
215      if (filterStringAsByteArray[i] == ParseConstants.RPAREN && (quoteCount % 2) == 0) {
216        byte[] filterSimpleExpression = new byte[i - filterExpressionStartOffset + 1];
217        Bytes.putBytes(filterSimpleExpression, 0, filterStringAsByteArray,
218          filterExpressionStartOffset, i - filterExpressionStartOffset + 1);
219        return filterSimpleExpression;
220      }
221    }
222    throw new IllegalArgumentException("Incorrect Filter String");
223  }
224
225  /**
226   * Constructs a filter object given a simple filter expression
227   * <p>
228   * @param filterStringAsByteArray filter string given by the user
229   * @return filter object we constructed
230   */
231  public Filter parseSimpleFilterExpression(byte[] filterStringAsByteArray)
232    throws CharacterCodingException {
233
234    String filterName = Bytes.toString(getFilterName(filterStringAsByteArray));
235    ArrayList<byte[]> filterArguments = getFilterArguments(filterStringAsByteArray);
236    if (!filterHashMap.containsKey(filterName)) {
237      throw new IllegalArgumentException("Filter Name " + filterName + " not supported");
238    }
239    try {
240      filterName = filterHashMap.get(filterName);
241      Class<?> c = Class.forName(filterName);
242      Class<?>[] argTypes = new Class[] { ArrayList.class };
243      Method m = c.getDeclaredMethod("createFilterFromArguments", argTypes);
244      return (Filter) m.invoke(null, filterArguments);
245    } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException
246      | InvocationTargetException e) {
247      LOG.warn("Exception while invoking createFilterFromArguments", e);
248      throw new IllegalArgumentException(
249        "Incorrect filter string " + new String(filterStringAsByteArray, StandardCharsets.UTF_8));
250    }
251  }
252
253  /**
254   * Returns the filter name given a simple filter expression
255   * <p>
256   * @param filterStringAsByteArray a simple filter expression
257   * @return name of filter in the simple filter expression
258   */
259  public static byte[] getFilterName(byte[] filterStringAsByteArray) {
260    int filterNameStartIndex = 0;
261    int filterNameEndIndex = 0;
262
263    for (int i = filterNameStartIndex; i < filterStringAsByteArray.length; i++) {
264      if (
265        filterStringAsByteArray[i] == ParseConstants.LPAREN
266          || filterStringAsByteArray[i] == ParseConstants.WHITESPACE
267      ) {
268        filterNameEndIndex = i;
269        break;
270      }
271    }
272
273    if (filterNameEndIndex == 0) {
274      throw new IllegalArgumentException("Incorrect Filter Name");
275    }
276
277    byte[] filterName = new byte[filterNameEndIndex - filterNameStartIndex];
278    Bytes.putBytes(filterName, 0, filterStringAsByteArray, 0,
279      filterNameEndIndex - filterNameStartIndex);
280    return filterName;
281  }
282
283  /**
284   * Returns the arguments of the filter from the filter string
285   * <p>
286   * @param filterStringAsByteArray filter string given by the user
287   * @return an ArrayList containing the arguments of the filter in the filter string
288   */
289  public static ArrayList<byte[]> getFilterArguments(byte[] filterStringAsByteArray) {
290    int argumentListStartIndex = Bytes.searchDelimiterIndex(filterStringAsByteArray, 0,
291      filterStringAsByteArray.length, ParseConstants.LPAREN);
292    if (argumentListStartIndex == -1) {
293      throw new IllegalArgumentException("Incorrect argument list");
294    }
295
296    int argumentStartIndex = 0;
297    int argumentEndIndex = 0;
298    ArrayList<byte[]> filterArguments = new ArrayList<>();
299
300    for (int i = argumentListStartIndex + 1; i < filterStringAsByteArray.length; i++) {
301
302      if (
303        filterStringAsByteArray[i] == ParseConstants.WHITESPACE
304          || filterStringAsByteArray[i] == ParseConstants.COMMA
305          || filterStringAsByteArray[i] == ParseConstants.RPAREN
306      ) {
307        continue;
308      }
309
310      // The argument is in single quotes - for example 'prefix'
311      if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
312        argumentStartIndex = i;
313        for (int j = argumentStartIndex + 1; j < filterStringAsByteArray.length; j++) {
314          if (filterStringAsByteArray[j] == ParseConstants.SINGLE_QUOTE) {
315            if (isQuoteUnescaped(filterStringAsByteArray, j)) {
316              argumentEndIndex = j;
317              i = j + 1;
318              byte[] filterArgument = createUnescapdArgument(filterStringAsByteArray,
319                argumentStartIndex, argumentEndIndex);
320              filterArguments.add(filterArgument);
321              break;
322            } else {
323              // To jump over the second escaped quote
324              j++;
325            }
326          } else if (j == filterStringAsByteArray.length - 1) {
327            throw new IllegalArgumentException("Incorrect argument list");
328          }
329        }
330      } else {
331        // The argument is an integer, boolean, comparison operator like <, >, != etc
332        argumentStartIndex = i;
333        for (int j = argumentStartIndex; j < filterStringAsByteArray.length; j++) {
334          if (
335            filterStringAsByteArray[j] == ParseConstants.WHITESPACE
336              || filterStringAsByteArray[j] == ParseConstants.COMMA
337              || filterStringAsByteArray[j] == ParseConstants.RPAREN
338          ) {
339            argumentEndIndex = j - 1;
340            i = j;
341            byte[] filterArgument = new byte[argumentEndIndex - argumentStartIndex + 1];
342            Bytes.putBytes(filterArgument, 0, filterStringAsByteArray, argumentStartIndex,
343              argumentEndIndex - argumentStartIndex + 1);
344            filterArguments.add(filterArgument);
345            break;
346          } else if (j == filterStringAsByteArray.length - 1) {
347            throw new IllegalArgumentException("Incorrect argument list");
348          }
349        }
350      }
351    }
352    return filterArguments;
353  }
354
355  /**
356   * This function is called while parsing the filterString and an operator is parsed
357   * <p>
358   * @param operatorStack the stack containing the operators and parenthesis
359   * @param filterStack   the stack containing the filters
360   * @param operator      the operator found while parsing the filterString
361   */
362  public void reduce(Stack<ByteBuffer> operatorStack, Stack<Filter> filterStack,
363    ByteBuffer operator) {
364    while (
365      !operatorStack.empty() && !ParseConstants.LPAREN_BUFFER.equals(operatorStack.peek())
366        && hasHigherPriority(operatorStack.peek(), operator)
367    ) {
368      filterStack.push(popArguments(operatorStack, filterStack));
369    }
370  }
371
372  /**
373   * Pops an argument from the operator stack and the number of arguments required by the operator
374   * from the filterStack and evaluates them
375   * <p>
376   * @param operatorStack the stack containing the operators
377   * @param filterStack   the stack containing the filters
378   * @return the evaluated filter
379   */
380  public static Filter popArguments(Stack<ByteBuffer> operatorStack, Stack<Filter> filterStack) {
381    ByteBuffer argumentOnTopOfStack = operatorStack.peek();
382
383    if (argumentOnTopOfStack.equals(ParseConstants.OR_BUFFER)) {
384      // The top of the stack is an OR
385      try {
386        ArrayList<Filter> listOfFilters = new ArrayList<>();
387        while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.OR_BUFFER)) {
388          Filter filter = filterStack.pop();
389          listOfFilters.add(0, filter);
390          operatorStack.pop();
391        }
392        Filter filter = filterStack.pop();
393        listOfFilters.add(0, filter);
394        Filter orFilter = new FilterList(FilterList.Operator.MUST_PASS_ONE, listOfFilters);
395        return orFilter;
396      } catch (EmptyStackException e) {
397        throw new IllegalArgumentException("Incorrect input string - an OR needs two filters");
398      }
399
400    } else if (argumentOnTopOfStack.equals(ParseConstants.AND_BUFFER)) {
401      // The top of the stack is an AND
402      try {
403        ArrayList<Filter> listOfFilters = new ArrayList<>();
404        while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.AND_BUFFER)) {
405          Filter filter = filterStack.pop();
406          listOfFilters.add(0, filter);
407          operatorStack.pop();
408        }
409        Filter filter = filterStack.pop();
410        listOfFilters.add(0, filter);
411        Filter andFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL, listOfFilters);
412        return andFilter;
413      } catch (EmptyStackException e) {
414        throw new IllegalArgumentException("Incorrect input string - an AND needs two filters");
415      }
416
417    } else if (argumentOnTopOfStack.equals(ParseConstants.SKIP_BUFFER)) {
418      // The top of the stack is a SKIP
419      try {
420        Filter wrappedFilter = filterStack.pop();
421        Filter skipFilter = new SkipFilter(wrappedFilter);
422        operatorStack.pop();
423        return skipFilter;
424      } catch (EmptyStackException e) {
425        throw new IllegalArgumentException("Incorrect input string - a SKIP wraps a filter");
426      }
427
428    } else if (argumentOnTopOfStack.equals(ParseConstants.WHILE_BUFFER)) {
429      // The top of the stack is a WHILE
430      try {
431        Filter wrappedFilter = filterStack.pop();
432        Filter whileMatchFilter = new WhileMatchFilter(wrappedFilter);
433        operatorStack.pop();
434        return whileMatchFilter;
435      } catch (EmptyStackException e) {
436        throw new IllegalArgumentException("Incorrect input string - a WHILE wraps a filter");
437      }
438
439    } else if (argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER)) {
440      // The top of the stack is a LPAREN
441      try {
442        Filter filter = filterStack.pop();
443        operatorStack.pop();
444        return filter;
445      } catch (EmptyStackException e) {
446        throw new IllegalArgumentException("Incorrect Filter String");
447      }
448
449    } else {
450      throw new IllegalArgumentException("Incorrect arguments on operatorStack");
451    }
452  }
453
454  /**
455   * Returns which operator has higher precedence
456   * <p>
457   * If a has higher precedence than b, it returns true If they have the same precedence, it returns
458   * false
459   */
460  public boolean hasHigherPriority(ByteBuffer a, ByteBuffer b) {
461    if ((operatorPrecedenceHashMap.get(a) - operatorPrecedenceHashMap.get(b)) < 0) {
462      return true;
463    }
464    return false;
465  }
466
467  /**
468   * Removes the single quote escaping a single quote - thus it returns an unescaped argument
469   * <p>
470   * @param filterStringAsByteArray filter string given by user
471   * @param argumentStartIndex      start index of the argument
472   * @param argumentEndIndex        end index of the argument
473   * @return returns an unescaped argument
474   */
475  public static byte[] createUnescapdArgument(byte[] filterStringAsByteArray,
476    int argumentStartIndex, int argumentEndIndex) {
477    int unescapedArgumentLength = 2;
478    for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
479      unescapedArgumentLength++;
480      if (
481        filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE && i != (argumentEndIndex - 1)
482          && filterStringAsByteArray[i + 1] == ParseConstants.SINGLE_QUOTE
483      ) {
484        i++;
485        continue;
486      }
487    }
488
489    byte[] unescapedArgument = new byte[unescapedArgumentLength];
490    int count = 1;
491    unescapedArgument[0] = '\'';
492    for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
493      if (
494        filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE && i != (argumentEndIndex - 1)
495          && filterStringAsByteArray[i + 1] == ParseConstants.SINGLE_QUOTE
496      ) {
497        unescapedArgument[count++] = filterStringAsByteArray[i + 1];
498        i++;
499      } else {
500        unescapedArgument[count++] = filterStringAsByteArray[i];
501      }
502    }
503    unescapedArgument[unescapedArgumentLength - 1] = '\'';
504    return unescapedArgument;
505  }
506
507  /**
508   * Checks if the current index of filter string we are on is the beginning of the keyword 'OR'
509   * <p>
510   * @param filterStringAsByteArray filter string given by the user
511   * @param indexOfOr               index at which an 'O' was read
512   * @return true if the keyword 'OR' is at the current index
513   */
514  public static boolean checkForOr(byte[] filterStringAsByteArray, int indexOfOr)
515    throws CharacterCodingException, ArrayIndexOutOfBoundsException {
516
517    try {
518      if (
519        filterStringAsByteArray[indexOfOr] == ParseConstants.O
520          && filterStringAsByteArray[indexOfOr + 1] == ParseConstants.R
521          && (filterStringAsByteArray[indexOfOr - 1] == ParseConstants.WHITESPACE
522            || filterStringAsByteArray[indexOfOr - 1] == ParseConstants.RPAREN)
523          && (filterStringAsByteArray[indexOfOr + 2] == ParseConstants.WHITESPACE
524            || filterStringAsByteArray[indexOfOr + 2] == ParseConstants.LPAREN)
525      ) {
526        return true;
527      } else {
528        return false;
529      }
530    } catch (ArrayIndexOutOfBoundsException e) {
531      return false;
532    }
533  }
534
535  /**
536   * Checks if the current index of filter string we are on is the beginning of the keyword 'AND'
537   * <p>
538   * @param filterStringAsByteArray filter string given by the user
539   * @param indexOfAnd              index at which an 'A' was read
540   * @return true if the keyword 'AND' is at the current index
541   */
542  public static boolean checkForAnd(byte[] filterStringAsByteArray, int indexOfAnd)
543    throws CharacterCodingException {
544
545    try {
546      if (
547        filterStringAsByteArray[indexOfAnd] == ParseConstants.A
548          && filterStringAsByteArray[indexOfAnd + 1] == ParseConstants.N
549          && filterStringAsByteArray[indexOfAnd + 2] == ParseConstants.D
550          && (filterStringAsByteArray[indexOfAnd - 1] == ParseConstants.WHITESPACE
551            || filterStringAsByteArray[indexOfAnd - 1] == ParseConstants.RPAREN)
552          && (filterStringAsByteArray[indexOfAnd + 3] == ParseConstants.WHITESPACE
553            || filterStringAsByteArray[indexOfAnd + 3] == ParseConstants.LPAREN)
554      ) {
555        return true;
556      } else {
557        return false;
558      }
559    } catch (ArrayIndexOutOfBoundsException e) {
560      return false;
561    }
562  }
563
564  /**
565   * Checks if the current index of filter string we are on is the beginning of the keyword 'SKIP'
566   * <p>
567   * @param filterStringAsByteArray filter string given by the user
568   * @param indexOfSkip             index at which an 'S' was read
569   * @return true if the keyword 'SKIP' is at the current index
570   */
571  public static boolean checkForSkip(byte[] filterStringAsByteArray, int indexOfSkip)
572    throws CharacterCodingException {
573
574    try {
575      if (
576        filterStringAsByteArray[indexOfSkip] == ParseConstants.S
577          && filterStringAsByteArray[indexOfSkip + 1] == ParseConstants.K
578          && filterStringAsByteArray[indexOfSkip + 2] == ParseConstants.I
579          && filterStringAsByteArray[indexOfSkip + 3] == ParseConstants.P
580          && (indexOfSkip == 0
581            || filterStringAsByteArray[indexOfSkip - 1] == ParseConstants.WHITESPACE
582            || filterStringAsByteArray[indexOfSkip - 1] == ParseConstants.RPAREN
583            || filterStringAsByteArray[indexOfSkip - 1] == ParseConstants.LPAREN)
584          && (filterStringAsByteArray[indexOfSkip + 4] == ParseConstants.WHITESPACE
585            || filterStringAsByteArray[indexOfSkip + 4] == ParseConstants.LPAREN)
586      ) {
587        return true;
588      } else {
589        return false;
590      }
591    } catch (ArrayIndexOutOfBoundsException e) {
592      return false;
593    }
594  }
595
596  /**
597   * Checks if the current index of filter string we are on is the beginning of the keyword 'WHILE'
598   * <p>
599   * @param filterStringAsByteArray filter string given by the user
600   * @param indexOfWhile            index at which an 'W' was read
601   * @return true if the keyword 'WHILE' is at the current index
602   */
603  public static boolean checkForWhile(byte[] filterStringAsByteArray, int indexOfWhile)
604    throws CharacterCodingException {
605
606    try {
607      if (
608        filterStringAsByteArray[indexOfWhile] == ParseConstants.W
609          && filterStringAsByteArray[indexOfWhile + 1] == ParseConstants.H
610          && filterStringAsByteArray[indexOfWhile + 2] == ParseConstants.I
611          && filterStringAsByteArray[indexOfWhile + 3] == ParseConstants.L
612          && filterStringAsByteArray[indexOfWhile + 4] == ParseConstants.E
613          && (indexOfWhile == 0
614            || filterStringAsByteArray[indexOfWhile - 1] == ParseConstants.WHITESPACE
615            || filterStringAsByteArray[indexOfWhile - 1] == ParseConstants.RPAREN
616            || filterStringAsByteArray[indexOfWhile - 1] == ParseConstants.LPAREN)
617          && (filterStringAsByteArray[indexOfWhile + 5] == ParseConstants.WHITESPACE
618            || filterStringAsByteArray[indexOfWhile + 5] == ParseConstants.LPAREN)
619      ) {
620        return true;
621      } else {
622        return false;
623      }
624    } catch (ArrayIndexOutOfBoundsException e) {
625      return false;
626    }
627  }
628
629  /**
630   * Returns a boolean indicating whether the quote was escaped or not
631   * <p>
632   * @param array      byte array in which the quote was found
633   * @param quoteIndex index of the single quote
634   * @return returns true if the quote was unescaped
635   */
636  public static boolean isQuoteUnescaped(byte[] array, int quoteIndex) {
637    if (array == null) {
638      throw new IllegalArgumentException("isQuoteUnescaped called with a null array");
639    }
640
641    if (quoteIndex == array.length - 1 || array[quoteIndex + 1] != ParseConstants.SINGLE_QUOTE) {
642      return true;
643    } else {
644      return false;
645    }
646  }
647
648  /**
649   * Takes a quoted byte array and converts it into an unquoted byte array For example: given a byte
650   * array representing 'abc', it returns a byte array representing abc
651   * <p>
652   * @param quotedByteArray the quoted byte array
653   * @return Unquoted byte array
654   */
655  public static byte[] removeQuotesFromByteArray(byte[] quotedByteArray) {
656    if (
657      quotedByteArray == null || quotedByteArray.length < 2
658        || quotedByteArray[0] != ParseConstants.SINGLE_QUOTE
659        || quotedByteArray[quotedByteArray.length - 1] != ParseConstants.SINGLE_QUOTE
660    ) {
661      throw new IllegalArgumentException("removeQuotesFromByteArray needs a quoted byte array");
662    } else {
663      byte[] targetString = new byte[quotedByteArray.length - 2];
664      Bytes.putBytes(targetString, 0, quotedByteArray, 1, quotedByteArray.length - 2);
665      return targetString;
666    }
667  }
668
669  /**
670   * Converts an int expressed in a byte array to an actual int
671   * <p>
672   * This doesn't use Bytes.toInt because that assumes that there will be {@link Bytes#SIZEOF_INT}
673   * bytes available.
674   * <p>
675   * @param numberAsByteArray the int value expressed as a byte array
676   * @return the int value
677   */
678  public static int convertByteArrayToInt(byte[] numberAsByteArray) {
679
680    long tempResult = ParseFilter.convertByteArrayToLong(numberAsByteArray);
681
682    if (tempResult > Integer.MAX_VALUE) {
683      throw new IllegalArgumentException("Integer Argument too large");
684    } else if (tempResult < Integer.MIN_VALUE) {
685      throw new IllegalArgumentException("Integer Argument too small");
686    }
687
688    int result = (int) tempResult;
689    return result;
690  }
691
692  /**
693   * Converts a long expressed in a byte array to an actual long
694   * <p>
695   * This doesn't use Bytes.toLong because that assumes that there will be {@link Bytes#SIZEOF_INT}
696   * bytes available.
697   * <p>
698   * @param numberAsByteArray the long value expressed as a byte array
699   * @return the long value
700   */
701  public static long convertByteArrayToLong(byte[] numberAsByteArray) {
702    if (numberAsByteArray == null) {
703      throw new IllegalArgumentException("convertByteArrayToLong called with a null array");
704    }
705
706    int i = 0;
707    long result = 0;
708    boolean isNegative = false;
709
710    if (numberAsByteArray[i] == ParseConstants.MINUS_SIGN) {
711      i++;
712      isNegative = true;
713    }
714
715    while (i != numberAsByteArray.length) {
716      if (
717        numberAsByteArray[i] < ParseConstants.ZERO || numberAsByteArray[i] > ParseConstants.NINE
718      ) {
719        throw new IllegalArgumentException("Byte Array should only contain digits");
720      }
721      result = result * 10 + (numberAsByteArray[i] - ParseConstants.ZERO);
722      if (result < 0) {
723        throw new IllegalArgumentException("Long Argument too large");
724      }
725      i++;
726    }
727
728    if (isNegative) {
729      return -result;
730    } else {
731      return result;
732    }
733  }
734
735  /**
736   * Converts a boolean expressed in a byte array to an actual boolean
737   * <p>
738   * This doesn't used Bytes.toBoolean because Bytes.toBoolean(byte []) assumes that 1 stands for
739   * true and 0 for false. Here, the byte array representing "true" and "false" is parsed
740   * <p>
741   * @param booleanAsByteArray the boolean value expressed as a byte array
742   * @return the boolean value
743   */
744  public static boolean convertByteArrayToBoolean(byte[] booleanAsByteArray) {
745    if (booleanAsByteArray == null) {
746      throw new IllegalArgumentException("convertByteArrayToBoolean called with a null array");
747    }
748
749    if (
750      booleanAsByteArray.length == 4
751        && (booleanAsByteArray[0] == 't' || booleanAsByteArray[0] == 'T')
752        && (booleanAsByteArray[1] == 'r' || booleanAsByteArray[1] == 'R')
753        && (booleanAsByteArray[2] == 'u' || booleanAsByteArray[2] == 'U')
754        && (booleanAsByteArray[3] == 'e' || booleanAsByteArray[3] == 'E')
755    ) {
756      return true;
757    } else if (
758      booleanAsByteArray.length == 5
759        && (booleanAsByteArray[0] == 'f' || booleanAsByteArray[0] == 'F')
760        && (booleanAsByteArray[1] == 'a' || booleanAsByteArray[1] == 'A')
761        && (booleanAsByteArray[2] == 'l' || booleanAsByteArray[2] == 'L')
762        && (booleanAsByteArray[3] == 's' || booleanAsByteArray[3] == 'S')
763        && (booleanAsByteArray[4] == 'e' || booleanAsByteArray[4] == 'E')
764    ) {
765      return false;
766    } else {
767      throw new IllegalArgumentException("Incorrect Boolean Expression");
768    }
769  }
770
771  /**
772   * Takes a compareOperator symbol as a byte array and returns the corresponding CompareOperator
773   * @param compareOpAsByteArray the comparatorOperator symbol as a byte array
774   * @return the Compare Operator
775   */
776  public static CompareOperator createCompareOperator(byte[] compareOpAsByteArray) {
777    ByteBuffer compareOp = ByteBuffer.wrap(compareOpAsByteArray);
778    if (compareOp.equals(ParseConstants.LESS_THAN_BUFFER)) return CompareOperator.LESS;
779    else if (compareOp.equals(ParseConstants.LESS_THAN_OR_EQUAL_TO_BUFFER))
780      return CompareOperator.LESS_OR_EQUAL;
781    else if (compareOp.equals(ParseConstants.GREATER_THAN_BUFFER)) return CompareOperator.GREATER;
782    else if (compareOp.equals(ParseConstants.GREATER_THAN_OR_EQUAL_TO_BUFFER))
783      return CompareOperator.GREATER_OR_EQUAL;
784    else if (compareOp.equals(ParseConstants.NOT_EQUAL_TO_BUFFER)) return CompareOperator.NOT_EQUAL;
785    else if (compareOp.equals(ParseConstants.EQUAL_TO_BUFFER)) return CompareOperator.EQUAL;
786    else throw new IllegalArgumentException("Invalid compare operator");
787  }
788
789  /**
790   * Takes a compareOperator symbol as a byte array and returns the corresponding CompareOperator
791   * @deprecated Since 2.0
792   *             <p>
793   * @param compareOpAsByteArray the comparatorOperator symbol as a byte array
794   * @return the Compare Operator
795   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #createCompareOperator(byte [])}
796   */
797  @Deprecated
798  public static CompareFilter.CompareOp createCompareOp(byte[] compareOpAsByteArray) {
799    ByteBuffer compareOp = ByteBuffer.wrap(compareOpAsByteArray);
800    if (compareOp.equals(ParseConstants.LESS_THAN_BUFFER)) return CompareOp.LESS;
801    else if (compareOp.equals(ParseConstants.LESS_THAN_OR_EQUAL_TO_BUFFER))
802      return CompareOp.LESS_OR_EQUAL;
803    else if (compareOp.equals(ParseConstants.GREATER_THAN_BUFFER)) return CompareOp.GREATER;
804    else if (compareOp.equals(ParseConstants.GREATER_THAN_OR_EQUAL_TO_BUFFER))
805      return CompareOp.GREATER_OR_EQUAL;
806    else if (compareOp.equals(ParseConstants.NOT_EQUAL_TO_BUFFER)) return CompareOp.NOT_EQUAL;
807    else if (compareOp.equals(ParseConstants.EQUAL_TO_BUFFER)) return CompareOp.EQUAL;
808    else throw new IllegalArgumentException("Invalid compare operator");
809  }
810
811  /**
812   * Parses a comparator of the form comparatorType:comparatorValue form and returns a comparator
813   * <p>
814   * @param comparator the comparator in the form comparatorType:comparatorValue
815   * @return the parsed comparator
816   */
817  public static ByteArrayComparable createComparator(byte[] comparator) {
818    if (comparator == null) throw new IllegalArgumentException("Incorrect Comparator");
819    byte[][] parsedComparator = ParseFilter.parseComparator(comparator);
820    byte[] comparatorType = parsedComparator[0];
821    byte[] comparatorValue = parsedComparator[1];
822
823    if (Bytes.equals(comparatorType, ParseConstants.binaryType))
824      return new BinaryComparator(comparatorValue);
825    else if (Bytes.equals(comparatorType, ParseConstants.binaryPrefixType))
826      return new BinaryPrefixComparator(comparatorValue);
827    else if (Bytes.equals(comparatorType, ParseConstants.regexStringType))
828      return new RegexStringComparator(new String(comparatorValue, StandardCharsets.UTF_8));
829    else if (Bytes.equals(comparatorType, ParseConstants.regexStringNoCaseType))
830      return new RegexStringComparator(new String(comparatorValue, StandardCharsets.UTF_8),
831        Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
832    else if (Bytes.equals(comparatorType, ParseConstants.substringType))
833      return new SubstringComparator(new String(comparatorValue, StandardCharsets.UTF_8));
834    else throw new IllegalArgumentException("Incorrect comparatorType");
835  }
836
837  /**
838   * Splits a column in comparatorType:comparatorValue form into separate byte arrays
839   * <p>
840   * @param comparator the comparator
841   * @return the parsed arguments of the comparator as a 2D byte array
842   */
843  public static byte[][] parseComparator(byte[] comparator) {
844    final int index =
845      Bytes.searchDelimiterIndex(comparator, 0, comparator.length, ParseConstants.COLON);
846    if (index == -1) {
847      throw new IllegalArgumentException("Incorrect comparator");
848    }
849
850    byte[][] result = new byte[2][0];
851    result[0] = new byte[index];
852    System.arraycopy(comparator, 0, result[0], 0, index);
853
854    final int len = comparator.length - (index + 1);
855    result[1] = new byte[len];
856    System.arraycopy(comparator, index + 1, result[1], 0, len);
857
858    return result;
859  }
860
861  /**
862   * Return a Set of filters supported by the Filter Language
863   */
864  public Set<String> getSupportedFilters() {
865    return filterHashMap.keySet();
866  }
867
868  /**
869   * Returns all known filters
870   * @return an unmodifiable map of filters
871   */
872  public static Map<String, String> getAllFilters() {
873    return Collections.unmodifiableMap(filterHashMap);
874  }
875
876  /**
877   * Register a new filter with the parser. If the filter is already registered, an
878   * IllegalArgumentException will be thrown.
879   * @param name        a name for the filter
880   * @param filterClass fully qualified class name
881   */
882  public static void registerFilter(String name, String filterClass) {
883    if (LOG.isInfoEnabled()) LOG.info("Registering new filter " + name);
884
885    filterHashMap.put(name, filterClass);
886  }
887}