You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/08/10 01:30:12 UTC

svn commit: r1155970 [2/2] - in /hbase/trunk: ./ src/docbkx/ src/main/java/org/apache/hadoop/hbase/filter/ src/main/java/org/apache/hadoop/hbase/zookeeper/ src/test/java/org/apache/hadoop/hbase/filter/ src/test/java/org/apache/hadoop/hbase/zookeeper/

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java Tue Aug  9 23:30:11 2011
@@ -1,813 +0,0 @@
-/**
- * Copyright 2011 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.filter;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.util.TreeSet;
-import java.util.ArrayList;
-import java.util.Stack;
-import java.util.HashMap;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.filter.ParseConstants;
-
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import java.lang.ArrayIndexOutOfBoundsException;
-import java.lang.ClassCastException;
-import java.util.EmptyStackException;
-
-/**
- * This class allows a user to specify a filter via a string
- * The string is parsed using the methods of this class and
- * a filter object is constructed. This filter object is then wrapped
- * in a scanner object which is then returned
- * <p>
- * This class addresses the HBASE-4168 JIRA. More documentaton on this
- * Filter Language can be found at: https://issues.apache.org/jira/browse/HBASE-4176
- */
-public class ParseFilter {
-
-  private HashMap<ByteBuffer, Filter> filterHashMap;
-  private HashMap<ByteBuffer, Integer> operatorPrecedenceHashMap;
-
-  /**
-   * Constructor
-   * <p>
-   * Sets up the filterHashMap by registering all the filters
-   * Creates the operatorPrecedenceHashMap
-   */
-  public ParseFilter() {
-
-    filterHashMap = new HashMap<ByteBuffer, Filter>();
-    registerFilter("KeyOnlyFilter", new KeyOnlyFilter());
-    registerFilter("FirstKeyOnlyFilter", new FirstKeyOnlyFilter());
-    registerFilter("PrefixFilter", new PrefixFilter());
-    registerFilter("ColumnPrefixFilter", new ColumnPrefixFilter());
-    registerFilter("MultipleColumnPrefixFilter", new MultipleColumnPrefixFilter());
-    registerFilter("ColumnCountGetFilter", new ColumnCountGetFilter());
-    registerFilter("PageFilter", new PageFilter());
-    registerFilter("ColumnPaginationFilter", new ColumnPaginationFilter());
-    registerFilter("InclusiveStopFilter", new InclusiveStopFilter());
-    registerFilter("TimestampsFilter", new TimestampsFilter());
-    registerFilter("RowFilter", new RowFilter());
-    registerFilter("FamilyFilter", new FamilyFilter());
-    registerFilter("QualifierFilter", new QualifierFilter());
-    registerFilter("ValueFilter", new ValueFilter());
-    registerFilter("ColumnRangeFilter", new ColumnRangeFilter());
-    registerFilter("SingleColumnValueFilter", new SingleColumnValueFilter());
-    registerFilter("SingleColumnValueExcludeFilter", new SingleColumnValueExcludeFilter());
-    registerFilter("DependentColumnFilter", new DependentColumnFilter());
-
-    operatorPrecedenceHashMap = new HashMap<ByteBuffer, Integer>();
-    operatorPrecedenceHashMap.put(ParseConstants.SKIP_BUFFER, 1);
-    operatorPrecedenceHashMap.put(ParseConstants.WHILE_BUFFER, 1);
-    operatorPrecedenceHashMap.put(ParseConstants.AND_BUFFER, 2);
-    operatorPrecedenceHashMap.put(ParseConstants.OR_BUFFER, 3);
-  }
-
-  /**
-   * Parses the filterString and constructs a filter using it
-   * <p>
-   * @param filterStringAsByteArray filter string given by the user
-   * @return filter object we constructed
-   */
-  public Filter parseFilterString (byte [] filterStringAsByteArray)
-    throws CharacterCodingException {
-    // stack for the operators and parenthesis
-    Stack <ByteBuffer> operatorStack = new Stack<ByteBuffer>();
-    // stack for the filter objects
-    Stack <Filter> filterStack = new Stack<Filter>();
-
-    Filter filter = null;
-    for (int i=0; i<filterStringAsByteArray.length; i++) {
-      if (filterStringAsByteArray[i] == ParseConstants.LPAREN) {
-        // LPAREN found
-        operatorStack.push(ParseConstants.LPAREN_BUFFER);
-      } else if (filterStringAsByteArray[i] == ParseConstants.WHITESPACE ||
-                 filterStringAsByteArray[i] == ParseConstants.TAB) {
-        // WHITESPACE or TAB found
-        continue;
-      } else if (checkForOr(filterStringAsByteArray, i)) {
-        // OR found
-        i += ParseConstants.OR_ARRAY.length - 1;
-        reduce(operatorStack, filterStack, ParseConstants.OR_BUFFER);
-        operatorStack.push(ParseConstants.OR_BUFFER);
-      } else if (checkForAnd(filterStringAsByteArray, i)) {
-        // AND found
-        i += ParseConstants.AND_ARRAY.length - 1;
-        reduce(operatorStack, filterStack, ParseConstants.AND_BUFFER);
-        operatorStack.push(ParseConstants.AND_BUFFER);
-      } else if (checkForSkip(filterStringAsByteArray, i)) {
-        // SKIP found
-        i += ParseConstants.SKIP_ARRAY.length - 1;
-        reduce(operatorStack, filterStack, ParseConstants.SKIP_BUFFER);
-        operatorStack.push(ParseConstants.SKIP_BUFFER);
-      } else if (checkForWhile(filterStringAsByteArray, i)) {
-        // WHILE found
-        i += ParseConstants.WHILE_ARRAY.length - 1;
-        reduce(operatorStack, filterStack, ParseConstants.WHILE_BUFFER);
-        operatorStack.push(ParseConstants.WHILE_BUFFER);
-      } else if (filterStringAsByteArray[i] == ParseConstants.RPAREN) {
-        // RPAREN found
-        if (operatorStack.empty()) {
-          throw new IllegalArgumentException("Mismatched parenthesis");
-        }
-        ByteBuffer argumentOnTopOfStack = operatorStack.peek();
-        while (!(argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER))) {
-          filterStack.push(popArguments(operatorStack, filterStack));
-          if (operatorStack.empty()) {
-            throw new IllegalArgumentException("Mismatched parenthesis");
-          }
-          argumentOnTopOfStack = operatorStack.pop();
-        }
-      } else {
-        // SimpleFilterExpression found
-        byte [] filterSimpleExpression = extractFilterSimpleExpression(filterStringAsByteArray, i);
-        i+= (filterSimpleExpression.length - 1);
-        filter = parseSimpleFilterExpression(filterSimpleExpression);
-        filterStack.push(filter);
-      }
-    }
-
-    // Finished parsing filterString
-    while (!operatorStack.empty()) {
-      filterStack.push(popArguments(operatorStack, filterStack));
-    }
-    filter = filterStack.pop();
-    if (!filterStack.empty()) {
-      throw new IllegalArgumentException("Incorrect Filter String");
-    }
-    return filter;
-  }
-
-/**
- * Extracts a simple filter expression from the filter string given by the user
- * <p>
- * A simpleFilterExpression is of the form: FilterName('arg', 'arg', 'arg')
- * The user given filter string can have many simpleFilterExpressions combined
- * using operators.
- * <p>
- * This function extracts a simpleFilterExpression from the
- * larger filterString given the start offset of the simpler expression
- * <p>
- * @param filterStringAsByteArray filter string given by the user
- * @param filterExpressionStartOffset start index of the simple filter expression
- * @return byte array containing the simple filter expression
- */
-  public byte [] extractFilterSimpleExpression (byte [] filterStringAsByteArray,
-                                                int filterExpressionStartOffset)
-    throws CharacterCodingException {
-    int quoteCount = 0;
-    for (int i=filterExpressionStartOffset; i<filterStringAsByteArray.length; i++) {
-      if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
-        if (isQuoteUnescaped(filterStringAsByteArray, i)) {
-          quoteCount ++;
-        } else {
-          // To skip the next quote that has been escaped
-          i++;
-        }
-      }
-      if (filterStringAsByteArray[i] == ParseConstants.RPAREN && (quoteCount %2 ) == 0) {
-        byte [] filterSimpleExpression = new byte [i - filterExpressionStartOffset + 1];
-        Bytes.putBytes(filterSimpleExpression, 0, filterStringAsByteArray,
-                       filterExpressionStartOffset, i-filterExpressionStartOffset + 1);
-        return filterSimpleExpression;
-      }
-    }
-    throw new IllegalArgumentException("Incorrect Filter String");
-  }
-
-/**
- * Constructs a filter object given a simple filter expression
- * <p>
- * @param filterStringAsByteArray filter string given by the user
- * @return filter object we constructed
- */
-  public Filter parseSimpleFilterExpression (byte [] filterStringAsByteArray)
-    throws CharacterCodingException {
-
-    byte [] filterName = getFilterName(filterStringAsByteArray);
-    ArrayList<byte []> filterArguments = getFilterArguments(filterStringAsByteArray);
-
-    if (filterHashMap.containsKey(ByteBuffer.wrap(filterName))) {
-      Filter filterSimpleExpression = filterHashMap.get(ByteBuffer.wrap(filterName));
-      return filterSimpleExpression.createFilterFromArguments(filterArguments);
-    } else {
-      throw new IllegalArgumentException("Incorrect filter string " +
-                                         new String(filterStringAsByteArray));
-    }
-  }
-
-  /**
-   * Registers the filter
-   * <p>
-   * Inserts a key-value composed of a ByteBuffer containing the name of the filter
-   * and an object of the filter into the HashMap
-   * <p>
-   * @param filterName the name of the filter we wish to register in the filterHashMap
-   * @param filter an object of that filter
-   */
-  private void registerFilter(String filterName, FilterBase filter) {
-    byte [] filterNameAsByteArray = Bytes.toBytes(filterName);
-    ByteBuffer filterNameAsByteBuffer = ByteBuffer.wrap(filterNameAsByteArray);
-    filterHashMap.put(filterNameAsByteBuffer, filter);
-  }
-
-/**
- * Returns the filter name given a simple filter expression
- * <p>
- * @param filterStringAsByteArray a simple filter expression
- * @return name of filter in the simple filter expression
- */
-  public static byte [] getFilterName (byte [] filterStringAsByteArray) {
-    int filterNameStartIndex = 0;
-    int filterNameEndIndex = 0;
-
-    for (int i=filterNameStartIndex; i<filterStringAsByteArray.length; i++) {
-      if (filterStringAsByteArray[i] == ParseConstants.LPAREN ||
-          filterStringAsByteArray[i] == ParseConstants.WHITESPACE) {
-        filterNameEndIndex = i;
-        break;
-      }
-    }
-
-    if (filterNameEndIndex == 0) {
-      throw new IllegalArgumentException("Incorrect Filter Name");
-    }
-
-    byte [] filterName = new byte[filterNameEndIndex - filterNameStartIndex];
-    Bytes.putBytes(filterName, 0, filterStringAsByteArray, 0,
-                   filterNameEndIndex - filterNameStartIndex);
-    return filterName;
-  }
-
-/**
- * Returns the arguments of the filter from the filter string
- * <p>
- * @param filter_string filter string given by the user
- * @return an ArrayList containing the arguments of the filter in the filter string
- */
-  public static ArrayList<byte []> getFilterArguments (byte [] filterStringAsByteArray) {
-    int argumentListStartIndex = KeyValue.getDelimiter(filterStringAsByteArray, 0,
-                                                       filterStringAsByteArray.length,
-                                                       ParseConstants.LPAREN);
-    if (argumentListStartIndex == -1) {
-      throw new IllegalArgumentException("Incorrect argument list");
-    }
-
-    int argumentStartIndex = 0;
-    int argumentEndIndex = 0;
-    ArrayList<byte []> filterArguments = new ArrayList<byte []>();
-
-    for (int i = argumentListStartIndex + 1; i<filterStringAsByteArray.length; i++) {
-
-      if (filterStringAsByteArray[i] == ParseConstants.WHITESPACE ||
-          filterStringAsByteArray[i] == ParseConstants.COMMA ||
-          filterStringAsByteArray[i] == ParseConstants.RPAREN) {
-        continue;
-      }
-
-      // The argument is in single quotes - for example 'prefix'
-      if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
-        argumentStartIndex = i;
-        for (int j = argumentStartIndex+1; j < filterStringAsByteArray.length; j++) {
-          if (filterStringAsByteArray[j] == ParseConstants.SINGLE_QUOTE) {
-            if (isQuoteUnescaped(filterStringAsByteArray,j)) {
-              argumentEndIndex = j;
-              i = j+1;
-              byte [] filterArgument = createUnescapdArgument(filterStringAsByteArray,
-                                                              argumentStartIndex, argumentEndIndex);
-              filterArguments.add(filterArgument);
-              break;
-            } else {
-              // To jump over the second escaped quote
-              j++;
-            }
-          } else if (j == filterStringAsByteArray.length - 1) {
-            throw new IllegalArgumentException("Incorrect argument list");
-          }
-        }
-      } else {
-        // The argument is an integer, boolean, comparison operator like <, >, != etc
-        argumentStartIndex = i;
-        for (int j = argumentStartIndex; j < filterStringAsByteArray.length; j++) {
-          if (filterStringAsByteArray[j] == ParseConstants.WHITESPACE ||
-              filterStringAsByteArray[j] == ParseConstants.COMMA ||
-              filterStringAsByteArray[j] == ParseConstants.RPAREN) {
-            argumentEndIndex = j - 1;
-            i = j;
-            byte [] filterArgument = new byte [argumentEndIndex - argumentStartIndex + 1];
-            Bytes.putBytes(filterArgument, 0, filterStringAsByteArray,
-                           argumentStartIndex, argumentEndIndex - argumentStartIndex + 1);
-            filterArguments.add(filterArgument);
-            break;
-          } else if (j == filterStringAsByteArray.length - 1) {
-            throw new IllegalArgumentException("Incorrect argument list");
-          }
-        }
-      }
-    }
-    return filterArguments;
-  }
-
-/**
- * This function is called while parsing the filterString and an operator is parsed
- * <p>
- * @param operatorStack the stack containing the operators and parenthesis
- * @param filterStack the stack containing the filters
- * @param operator the operator found while parsing the filterString
- * @return returns the filterStack after evaluating the stack
- */
-  public void reduce(Stack<ByteBuffer> operatorStack,
-                     Stack<Filter> filterStack,
-                     ByteBuffer operator) {
-    while (!operatorStack.empty() &&
-           !(ParseConstants.LPAREN_BUFFER.equals(operatorStack.peek())) &&
-           hasHigherPriority(operatorStack.peek(), operator)) {
-      filterStack.push(popArguments(operatorStack, filterStack));
-    }
-  }
-
-  /**
-   * Pops an argument from the operator stack and the number of arguments required by the operator
-   * from the filterStack and evaluates them
-   * <p>
-   * @param operatorStack the stack containing the operators
-   * @param filterStack the stack containing the filters
-   * @return the evaluated filter
-   */
-  public static Filter popArguments (Stack<ByteBuffer> operatorStack, Stack <Filter> filterStack) {
-    ByteBuffer argumentOnTopOfStack = operatorStack.peek();
-
-    if (argumentOnTopOfStack.equals(ParseConstants.OR_BUFFER)) {
-      // The top of the stack is an OR
-      try {
-        ArrayList<Filter> listOfFilters = new ArrayList<Filter>();
-        while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.OR_BUFFER)) {
-          Filter filter = filterStack.pop();
-          listOfFilters.add(0, filter);
-          operatorStack.pop();
-        }
-        Filter filter = filterStack.pop();
-        listOfFilters.add(0, filter);
-        Filter orFilter = new FilterList(FilterList.Operator.MUST_PASS_ONE, listOfFilters);
-        return orFilter;
-      } catch (EmptyStackException e) {
-        throw new IllegalArgumentException("Incorrect input string - an OR needs two filters");
-      }
-
-    } else if (argumentOnTopOfStack.equals(ParseConstants.AND_BUFFER)) {
-      // The top of the stack is an AND
-      try {
-        ArrayList<Filter> listOfFilters = new ArrayList<Filter>();
-        while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.AND_BUFFER)) {
-          Filter filter = filterStack.pop();
-          listOfFilters.add(0, filter);
-          operatorStack.pop();
-        }
-        Filter filter = filterStack.pop();
-        listOfFilters.add(0, filter);
-        Filter andFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL, listOfFilters);
-        return andFilter;
-      } catch (EmptyStackException e) {
-        throw new IllegalArgumentException("Incorrect input string - an AND needs two filters");
-      }
-
-    } else if (argumentOnTopOfStack.equals(ParseConstants.SKIP_BUFFER)) {
-      // The top of the stack is a SKIP
-      try {
-        Filter wrappedFilter = filterStack.pop();
-        Filter skipFilter = new SkipFilter(wrappedFilter);
-        operatorStack.pop();
-        return skipFilter;
-      } catch (EmptyStackException e) {
-        throw new IllegalArgumentException("Incorrect input string - a SKIP wraps a filter");
-      }
-
-    } else if (argumentOnTopOfStack.equals(ParseConstants.WHILE_BUFFER)) {
-      // The top of the stack is a WHILE
-      try {
-        Filter wrappedFilter = filterStack.pop();
-        Filter whileMatchFilter = new WhileMatchFilter(wrappedFilter);
-        operatorStack.pop();
-        return whileMatchFilter;
-      } catch (EmptyStackException e) {
-        throw new IllegalArgumentException("Incorrect input string - a WHILE wraps a filter");
-      }
-
-    } else if (argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER)) {
-      // The top of the stack is a LPAREN
-      try {
-        Filter filter  = filterStack.pop();
-        operatorStack.pop();
-        return filter;
-      } catch (EmptyStackException e) {
-        throw new IllegalArgumentException("Incorrect Filter String");
-      }
-
-    } else {
-      throw new IllegalArgumentException("Incorrect arguments on operatorStack");
-    }
-  }
-
-/**
- * Returns which operator has higher precedence
- * <p>
- * If a has higher precedence than b, it returns true
- * If they have the same precedence, it returns false
- */
-  public boolean hasHigherPriority(ByteBuffer a, ByteBuffer b) {
-    if ((operatorPrecedenceHashMap.get(a) - operatorPrecedenceHashMap.get(b)) < 0) {
-      return true;
-    }
-    return false;
-  }
-
-/**
- * Removes the single quote escaping a single quote - thus it returns an unescaped argument
- * <p>
- * @param filterStringAsByteArray filter string given by user
- * @param argumentStartIndex start index of the argument
- * @param argumentEndIndex end index of the argument
- * @return returns an unescaped argument
- */
-  public static byte [] createUnescapdArgument (byte [] filterStringAsByteArray,
-                                                int argumentStartIndex, int argumentEndIndex) {
-    int unescapedArgumentLength = 2;
-    for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
-      unescapedArgumentLength ++;
-      if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE &&
-          i != (argumentEndIndex - 1) &&
-          filterStringAsByteArray[i+1] == ParseConstants.SINGLE_QUOTE) {
-        i++;
-        continue;
-      }
-    }
-
-    byte [] unescapedArgument = new byte [unescapedArgumentLength];
-    int count = 1;
-    unescapedArgument[0] = '\'';
-    for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
-      if (filterStringAsByteArray [i] == ParseConstants.SINGLE_QUOTE &&
-          i != (argumentEndIndex - 1) &&
-          filterStringAsByteArray [i+1] == ParseConstants.SINGLE_QUOTE) {
-        unescapedArgument[count++] = filterStringAsByteArray [i+1];
-        i++;
-      }
-      else {
-        unescapedArgument[count++] = filterStringAsByteArray [i];
-      }
-    }
-    unescapedArgument[unescapedArgumentLength - 1] = '\'';
-    return unescapedArgument;
-  }
-
-/**
- * Checks if the current index of filter string we are on is the beginning of the keyword 'OR'
- * <p>
- * @param filterStringAsByteArray filter string given by the user
- * @param indexOfOr index at which an 'O' was read
- * @return true if the keyword 'OR' is at the current index
- */
-  public static boolean checkForOr (byte [] filterStringAsByteArray, int indexOfOr)
-    throws CharacterCodingException, ArrayIndexOutOfBoundsException {
-
-    try {
-      if (filterStringAsByteArray[indexOfOr] == ParseConstants.O &&
-          filterStringAsByteArray[indexOfOr+1] == ParseConstants.R &&
-          (filterStringAsByteArray[indexOfOr-1] == ParseConstants.WHITESPACE ||
-           filterStringAsByteArray[indexOfOr-1] == ParseConstants.RPAREN) &&
-          (filterStringAsByteArray[indexOfOr+2] == ParseConstants.WHITESPACE ||
-           filterStringAsByteArray[indexOfOr+2] == ParseConstants.LPAREN)) {
-        return true;
-      } else {
-        return false;
-      }
-    } catch (ArrayIndexOutOfBoundsException e) {
-      return false;
-    }
-  }
-
-/**
- * Checks if the current index of filter string we are on is the beginning of the keyword 'AND'
- * <p>
- * @param filterStringAsByteArray filter string given by the user
- * @param indexOfAnd index at which an 'A' was read
- * @return true if the keyword 'AND' is at the current index
- */
-  public static boolean checkForAnd (byte [] filterStringAsByteArray, int indexOfAnd)
-    throws CharacterCodingException {
-
-    try {
-      if (filterStringAsByteArray[indexOfAnd] == ParseConstants.A &&
-          filterStringAsByteArray[indexOfAnd+1] == ParseConstants.N &&
-          filterStringAsByteArray[indexOfAnd+2] == ParseConstants.D &&
-          (filterStringAsByteArray[indexOfAnd-1] == ParseConstants.WHITESPACE ||
-           filterStringAsByteArray[indexOfAnd-1] == ParseConstants.RPAREN) &&
-          (filterStringAsByteArray[indexOfAnd+3] == ParseConstants.WHITESPACE ||
-           filterStringAsByteArray[indexOfAnd+3] == ParseConstants.LPAREN)) {
-        return true;
-      } else {
-        return false;
-      }
-    } catch (ArrayIndexOutOfBoundsException e) {
-      return false;
-    }
-  }
-
-/**
- * Checks if the current index of filter string we are on is the beginning of the keyword 'SKIP'
- * <p>
- * @param filterStringAsByteArray filter string given by the user
- * @param indexOfSkip index at which an 'S' was read
- * @return true if the keyword 'SKIP' is at the current index
- */
-  public static boolean checkForSkip (byte [] filterStringAsByteArray, int indexOfSkip)
-    throws CharacterCodingException {
-
-    try {
-      if (filterStringAsByteArray[indexOfSkip] == ParseConstants.S &&
-          filterStringAsByteArray[indexOfSkip+1] == ParseConstants.K &&
-          filterStringAsByteArray[indexOfSkip+2] == ParseConstants.I &&
-          filterStringAsByteArray[indexOfSkip+3] == ParseConstants.P &&
-          (indexOfSkip == 0 ||
-           filterStringAsByteArray[indexOfSkip-1] == ParseConstants.WHITESPACE ||
-           filterStringAsByteArray[indexOfSkip-1] == ParseConstants.RPAREN ||
-           filterStringAsByteArray[indexOfSkip-1] == ParseConstants.LPAREN) &&
-          (filterStringAsByteArray[indexOfSkip+4] == ParseConstants.WHITESPACE ||
-           filterStringAsByteArray[indexOfSkip+4] == ParseConstants.LPAREN)) {
-        return true;
-      } else {
-        return false;
-      }
-    } catch (ArrayIndexOutOfBoundsException e) {
-      return false;
-    }
-  }
-
-/**
- * Checks if the current index of filter string we are on is the beginning of the keyword 'WHILE'
- * <p>
- * @param filterStringAsByteArray filter string given by the user
- * @param indexOfWhile index at which an 'W' was read
- * @return true if the keyword 'WHILE' is at the current index
- */
-  public static boolean checkForWhile (byte [] filterStringAsByteArray, int indexOfWhile)
-    throws CharacterCodingException {
-
-    try {
-      if (filterStringAsByteArray[indexOfWhile] == ParseConstants.W &&
-          filterStringAsByteArray[indexOfWhile+1] == ParseConstants.H &&
-          filterStringAsByteArray[indexOfWhile+2] == ParseConstants.I &&
-          filterStringAsByteArray[indexOfWhile+3] == ParseConstants.L &&
-          filterStringAsByteArray[indexOfWhile+4] == ParseConstants.E &&
-          (indexOfWhile == 0 || filterStringAsByteArray[indexOfWhile-1] == ParseConstants.WHITESPACE
-           || filterStringAsByteArray[indexOfWhile-1] == ParseConstants.RPAREN ||
-           filterStringAsByteArray[indexOfWhile-1] == ParseConstants.LPAREN) &&
-          (filterStringAsByteArray[indexOfWhile+5] == ParseConstants.WHITESPACE ||
-           filterStringAsByteArray[indexOfWhile+5] == ParseConstants.LPAREN)) {
-        return true;
-      } else {
-        return false;
-      }
-    } catch (ArrayIndexOutOfBoundsException e) {
-      return false;
-    }
-  }
-
-/**
- * Returns a boolean indicating whether the quote was escaped or not
- * <p>
- * @param array byte array in which the quote was found
- * @param quoteIndex index of the single quote
- * @return returns true if the quote was unescaped
- */
-  public static boolean isQuoteUnescaped (byte [] array, int quoteIndex) {
-    if (array == null) {
-      throw new IllegalArgumentException("isQuoteUnescaped called with a null array");
-    }
-
-    if (quoteIndex == array.length - 1 || array[quoteIndex+1] != ParseConstants.SINGLE_QUOTE) {
-      return true;
-    }
-    else {
-      return false;
-    }
-  }
-
-/**
- * Takes a quoted byte array and converts it into an unquoted byte array
- * For example: given a byte array representing 'abc', it returns a
- * byte array representing 'abc'
- * <p>
- * @param stringAsByteArray the quoted byte array
- * @return
- */
-  public static byte [] convertByteArrayToString (byte [] stringAsByteArray) {
-    if (stringAsByteArray == null ||
-        stringAsByteArray.length < 2 ||
-        stringAsByteArray[0] != ParseConstants.SINGLE_QUOTE ||
-        stringAsByteArray[stringAsByteArray.length - 1] != ParseConstants.SINGLE_QUOTE) {
-      throw new IllegalArgumentException("convertByteArrayToString needs a quotes byte array");
-    } else {
-      byte [] targetString = new byte [stringAsByteArray.length - 2];
-      Bytes.putBytes(targetString, 0, stringAsByteArray, 1, stringAsByteArray.length - 2);
-      return targetString;
-    }
-  }
-
-/**
- * Converts an int expressed in a byte array to an actual int
- * <p>
- * This doesn't use Bytes.toInt because that assumes
- * that there will be {@link #SIZEOF_INT} bytes available.
- * <p>
- * @param numberAsByteArray the int value expressed as a byte array
- * @return the int value
- */
-  public static int convertByteArrayToInt (byte [] numberAsByteArray) {
-
-    long tempResult = ParseFilter.convertByteArrayToLong(numberAsByteArray);
-
-    if (tempResult > Integer.MAX_VALUE) {
-      throw new IllegalArgumentException("Integer Argument too large");
-    } else if (tempResult < Integer.MIN_VALUE) {
-      throw new IllegalArgumentException("Integer Argument too small");
-    }
-
-    int result = (int) tempResult;
-    return result;
-  }
-
-/**
- * Converts a long expressed in a byte array to an actual long
- * <p>
- * This doesn't use Bytes.toLong because that assumes
- * that there will be {@link #SIZEOF_LONG} bytes available.
- * <p>
- * @param numberAsByteArray the long value expressed as a byte array
- * @return the long value
- */
-  public static long convertByteArrayToLong (byte [] numberAsByteArray) {
-    if (numberAsByteArray == null) {
-      throw new IllegalArgumentException("convertByteArrayToLong called with a null array");
-    }
-
-    int i = 0;
-    long result = 0;
-    boolean isNegative = false;
-
-    if (numberAsByteArray[i] == ParseConstants.MINUS_SIGN) {
-      i++;
-      isNegative = true;
-    }
-
-    while (i != numberAsByteArray.length) {
-      if (numberAsByteArray[i] < ParseConstants.ZERO ||
-          numberAsByteArray[i] > ParseConstants.NINE) {
-        throw new IllegalArgumentException("Byte Array should only contain digits");
-      }
-      result = result*10 + (numberAsByteArray[i] - ParseConstants.ZERO);
-      if (result < 0) {
-        throw new IllegalArgumentException("Long Argument too large");
-      }
-      i++;
-    }
-
-    if (isNegative) {
-      return -result;
-    } else {
-      return result;
-    }
-  }
-
-/**
- * Converts a boolean expressed in a byte array to an actual boolean
- *<p>
- * This doesn't used Bytes.toBoolean because Bytes.toBoolean(byte [])
- * assumes that 1 stands for true and 0 for false.
- * Here, the byte array representing "true" and "false" is parsed
- * <p>
- * @param booleanAsByteArray the boolean value expressed as a byte array
- * @return the boolean value
- */
-  public static boolean convertByteArrayToBoolean (byte [] booleanAsByteArray) {
-    if (booleanAsByteArray == null) {
-      throw new IllegalArgumentException("convertByteArrayToBoolean called with a null array");
-    }
-
-    if (booleanAsByteArray.length == 4 &&
-        (booleanAsByteArray[0] == 't' || booleanAsByteArray[0] == 'T') &&
-        (booleanAsByteArray[1] == 'r' || booleanAsByteArray[1] == 'R') &&
-        (booleanAsByteArray[2] == 'u' || booleanAsByteArray[2] == 'U') &&
-        (booleanAsByteArray[3] == 'e' || booleanAsByteArray[3] == 'E')) {
-      return true;
-    }
-    else if (booleanAsByteArray.length == 5 &&
-             (booleanAsByteArray[0] == 'f' || booleanAsByteArray[0] == 'F') &&
-             (booleanAsByteArray[1] == 'a' || booleanAsByteArray[1] == 'A') &&
-             (booleanAsByteArray[2] == 'l' || booleanAsByteArray[2] == 'L') &&
-             (booleanAsByteArray[3] == 's' || booleanAsByteArray[3] == 'S') &&
-             (booleanAsByteArray[4] == 'e' || booleanAsByteArray[4] == 'E')) {
-      return false;
-    }
-    else {
-      throw new IllegalArgumentException("Incorrect Boolean Expression");
-    }
-  }
-
-/**
- * Takes a compareOperator symbol as a byte array and returns the corresponding CompareOperator
- * <p>
- * @param compareOpAsByteArray the comparatorOperator symbol as a byte array
- * @return the Compare Operator
- */
-  public static CompareFilter.CompareOp createCompareOp (byte [] compareOpAsByteArray) {
-    ByteBuffer compareOp = ByteBuffer.wrap(compareOpAsByteArray);
-    if (compareOp.equals(ParseConstants.LESS_THAN_BUFFER))
-      return CompareOp.LESS;
-    else if (compareOp.equals(ParseConstants.LESS_THAN_OR_EQUAL_TO_BUFFER))
-      return CompareOp.LESS_OR_EQUAL;
-    else if (compareOp.equals(ParseConstants.GREATER_THAN_BUFFER))
-      return CompareOp.GREATER;
-    else if (compareOp.equals(ParseConstants.GREATER_THAN_OR_EQUAL_TO_BUFFER))
-      return CompareOp.GREATER_OR_EQUAL;
-    else if (compareOp.equals(ParseConstants.NOT_EQUAL_TO_BUFFER))
-      return CompareOp.NOT_EQUAL;
-    else if (compareOp.equals(ParseConstants.EQUAL_TO_BUFFER))
-      return CompareOp.EQUAL;
-    else
-      throw new IllegalArgumentException("Invalid compare operator");
-  }
-
-/**
- * Parses a comparator of the form comparatorType:comparatorValue form and returns a comparator
- * <p>
- * @param comparator the comparator in the form comparatorType:comparatorValue
- * @return the parsed comparator
- */
-  public static WritableByteArrayComparable createComparator (byte [] comparator) {
-    if (comparator == null)
-      throw new IllegalArgumentException("Incorrect Comparator");
-    byte [][] parsedComparator = ParseFilter.parseComparator(comparator);
-    byte [] comparatorType = parsedComparator[0];
-    byte [] comparatorValue = parsedComparator[1];
-
-
-    if (Bytes.equals(comparatorType, ParseConstants.binaryType))
-      return new BinaryComparator(comparatorValue);
-    else if (Bytes.equals(comparatorType, ParseConstants.binaryPrefixType))
-      return new BinaryPrefixComparator(comparatorValue);
-    else if (Bytes.equals(comparatorType, ParseConstants.regexStringType))
-      return new RegexStringComparator(new String(comparatorValue));
-    else if (Bytes.equals(comparatorType, ParseConstants.substringType))
-      return new SubstringComparator(new String(comparatorValue));
-    else
-      throw new IllegalArgumentException("Incorrect comparatorType");
-  }
-
-/**
- * Splits a column in comparatorType:comparatorValue form into separate byte arrays
- * <p>
- * @param comparator the comparator
- * @return the parsed arguments of the comparator as a 2D byte array
- */
-  public static byte [][] parseComparator (byte [] comparator) {
-    final int index = KeyValue.getDelimiter(comparator, 0, comparator.length, ParseConstants.COLON);
-    if (index == -1) {
-      throw new IllegalArgumentException("Incorrect comparator");
-    }
-
-    byte [][] result = new byte [2][0];
-    result[0] = new byte [index];
-    System.arraycopy(comparator, 0, result[0], 0, index);
-
-    final int len = comparator.length - (index + 1);
-    result[1] = new byte[len];
-    System.arraycopy(comparator, index + 1, result[1], 0, len);
-
-    return result;
-  }
-}

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java Tue Aug  9 23:30:11 2011
@@ -27,7 +27,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.DataInput;
 import java.util.List;
-import java.util.ArrayList;
 
 /**
  * Pass results that have same row prefix.
@@ -68,17 +67,6 @@ public class PrefixFilter extends Filter
     return passedPrefix;
   }
 
-  @Override
-  public Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
-    if (filterArguments.size() != 1) {
-      throw new IllegalArgumentException("Incorrect arguments passed to PrefixFilter. " +
-                                         "Expected: 1 but got: " + filterArguments.size());
-    }
-
-    byte [] prefix = ParseFilter.convertByteArrayToString(filterArguments.get(0));
-    return new PrefixFilter(prefix);
-  }
-
   public void write(DataOutput out) throws IOException {
     Bytes.writeByteArray(out, this.prefix);
   }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java Tue Aug  9 23:30:11 2011
@@ -23,8 +23,6 @@ package org.apache.hadoop.hbase.filter;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Get;
 
-import java.util.ArrayList;
-
 /**
  * This filter is used to filter based on the column qualifier. It takes an
  * operator (equal, greater, not equal, etc) and a byte [] comparator for the
@@ -67,9 +65,4 @@ public class QualifierFilter extends Com
     }
     return ReturnCode.INCLUDE;
   }
-
-  @Override
-  public Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
-    return super.createFilterFromArguments(filterArguments);
-  }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java Tue Aug  9 23:30:11 2011
@@ -21,9 +21,9 @@
 package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
 
 import java.util.List;
-import java.util.ArrayList;
 
 /**
  * This filter is used to filter based on the key. It takes an operator
@@ -83,9 +83,4 @@ public class RowFilter extends CompareFi
   public boolean filterRow() {
     return this.filterOutRow;
   }
-
-  @Override
-  public Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
-    return super.createFilterFromArguments(filterArguments);
-  }
 }
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java Tue Aug  9 23:30:11 2011
@@ -23,8 +23,6 @@ package org.apache.hadoop.hbase.filter;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 
-import java.util.ArrayList;
-
 /**
  * A {@link Filter} that checks a single column value, but does not emit the
  * tested column. This will enable a performance boost over
@@ -87,20 +85,4 @@ public class SingleColumnValueExcludeFil
     }
     return superRetCode;
   }
-
-  @Override
-  public Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
-    SingleColumnValueFilter singleColumnValueFilter = new SingleColumnValueFilter();
-    SingleColumnValueFilter tempFilter = (SingleColumnValueFilter)
-      singleColumnValueFilter.createFilterFromArguments(filterArguments);
-    SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter (
-      tempFilter.getFamily(), tempFilter.getQualifier(),
-      tempFilter.getOperator(), tempFilter.getComparator());
-
-    if (filterArguments.size() == 6) {
-      filter.setFilterIfMissing(tempFilter.getFilterIfMissing());
-      filter.setLatestVersionOnly(tempFilter.getLatestVersionOnly());
-    }
-    return filter;
-  }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java Tue Aug  9 23:30:11 2011
@@ -33,7 +33,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
-import java.util.ArrayList;
 
 /**
  * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
@@ -248,39 +247,6 @@ public class SingleColumnValueFilter ext
     this.latestVersionOnly = latestVersionOnly;
   }
 
-  @Override
-  public Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
-    if (filterArguments.size() != 4 && filterArguments.size() != 6)
-      throw new IllegalArgumentException("Incorrect Arguments passed to SingleColumnValueFilter. " +
-                                         "Expected: 4 or 6 but got: " + filterArguments.size());
-
-    byte [] family = ParseFilter.convertByteArrayToString(filterArguments.get(0));
-    byte [] qualifier = ParseFilter.convertByteArrayToString(filterArguments.get(1));
-    CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(2));
-    WritableByteArrayComparable comparator = ParseFilter.createComparator(
-      ParseFilter.convertByteArrayToString(filterArguments.get(3)));
-
-    if (comparator instanceof RegexStringComparator ||
-        comparator instanceof SubstringComparator) {
-      if (compareOp != CompareOp.EQUAL &&
-          compareOp != CompareOp.NOT_EQUAL) {
-        throw new IllegalArgumentException ("A regexstring comparator and substring comparator " +
-                                            "can only be used with EQUAL and NOT_EQUAL");
-      }
-    }
-
-    SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier,
-                                                                 compareOp, comparator);
-
-    if (filterArguments.size() == 6) {
-      boolean filterIfMissing = ParseFilter.convertByteArrayToBoolean(filterArguments.get(4));
-      boolean latestVersionOnly = ParseFilter.convertByteArrayToBoolean(filterArguments.get(5));
-      filter.setFilterIfMissing(filterIfMissing);
-      filter.setLatestVersionOnly(latestVersionOnly);
-    }
-    return filter;
-  }
-
   public void readFields(final DataInput in) throws IOException {
     this.columnFamily = Bytes.readByteArray(in);
     if(this.columnFamily.length == 0) {

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java Tue Aug  9 23:30:11 2011
@@ -6,7 +6,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.TreeSet;
-import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.KeyValue;
 
@@ -42,11 +41,6 @@ public class TimestampsFilter extends Fi
    * @param timestamps
    */
   public TimestampsFilter(List<Long> timestamps) {
-    for (Long timestamp : timestamps) {
-      if (timestamp < 0) {
-        throw new IllegalArgumentException("Timestamps must not be negative");
-      }
-    }
     this.timestamps = new TreeSet<Long>(timestamps);
     init();
   }
@@ -54,8 +48,10 @@ public class TimestampsFilter extends Fi
   /**
    * @return the list of timestamps
    */
-  public TreeSet<Long> getTimestamps() {
-    return this.timestamps;
+  public List<Long> getTimestamps() {
+    List<Long> list = new ArrayList<Long>(timestamps.size());
+    list.addAll(timestamps);
+    return list;
   }
 
   private void init() {
@@ -85,16 +81,6 @@ public class TimestampsFilter extends Fi
   }
 
   @Override
-  public Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
-    ArrayList<Long> timestamps = new ArrayList<Long>();
-    for (int i = 0; i<filterArguments.size(); i++) {
-      long timestamp = ParseFilter.convertByteArrayToLong(filterArguments.get(i));
-      timestamps.add(timestamp);
-    }
-    return new TimestampsFilter(timestamps);
-  }
-
-  @Override
   public void readFields(DataInput in) throws IOException {
     int numTimestamps = in.readInt();
     this.timestamps = new TreeSet<Long>();

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java Tue Aug  9 23:30:11 2011
@@ -22,8 +22,6 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.hbase.KeyValue;
 
-import java.util.ArrayList;
-
 /**
  * This filter is used to filter based on column value. It takes an
  * operator (equal, greater, not equal, etc) and a byte [] comparator for the
@@ -63,9 +61,4 @@ public class ValueFilter extends Compare
     }
     return ReturnCode.INCLUDE;
   }
-
-  @Override
-  public Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
-    return super.createFilterFromArguments(filterArguments);
-  }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java Tue Aug  9 23:30:11 2011
@@ -47,7 +47,8 @@ public class ZKConfig {
 
   private static final String ZK_CFG_PROPERTY = "hbase.zookeeper.property.";
   private static final int ZK_CFG_PROPERTY_SIZE = ZK_CFG_PROPERTY.length();
-  private static final String ZK_CLIENT_PORT_KEY = "clientPort";
+  private static final String ZK_CLIENT_PORT_KEY = ZK_CFG_PROPERTY
+      + "clientPort";
 
   /**
    * Make a Properties object holding ZooKeeper config equivalent to zoo.cfg.

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java Tue Aug  9 23:30:11 2011
@@ -1,754 +0,0 @@
-/**
- * Copyright 2011 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.filter;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueTestUtil;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * This class tests ParseFilter.java
- * It tests the entire work flow from when a string is given by the user
- * and how it is parsed to construct the corresponding Filter object
- */
-public class TestParseFilter {
-
-  ParseFilter f;
-  Filter filter;
-
-  @Before
-  public void setUp() throws Exception {
-    f = new ParseFilter();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    // Nothing to do.
-  }
-
-  @Test
-  public void testKeyOnlyFilter() throws IOException {
-    String filterString = " KeyOnlyFilter( ) ";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof KeyOnlyFilter);
-
-    String filterString2 = "KeyOnlyFilter ('') ";
-    byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray2);
-      assertTrue(filter instanceof KeyOnlyFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println(e.getMessage());
-    }
-  }
-
-  @Test
-  public void testFirstKeyOnlyFilter() throws IOException {
-    String filterString = " FirstKeyOnlyFilter( ) ";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof FirstKeyOnlyFilter);
-
-    String filterString2 = " FirstKeyOnlyFilter ('') ";
-    byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray2);
-      assertTrue(filter instanceof FirstKeyOnlyFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println(e.getMessage());
-    }
-  }
-
-  @Test
-  public void testPrefixFilter() throws IOException {
-    String filterString = " PrefixFilter('row' ) ";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof PrefixFilter);
-
-    PrefixFilter prefixFilter = (PrefixFilter) filter;
-    byte [] prefix = prefixFilter.getPrefix();
-    assertEquals(new String(prefix), "row");
-
-    filterString = " PrefixFilter(row)";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof PrefixFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("PrefixFilter needs a quoted string");
-    }
-
-  }
-
-  @Test
-  public void testColumnPrefixFilter() throws IOException {
-    String filterString = " ColumnPrefixFilter('qualifier' ) ";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof ColumnPrefixFilter);
-    byte [] columnPrefix = ((ColumnPrefixFilter)filter).getPrefix();
-    assertEquals(new String(columnPrefix), "qualifier");
-  }
-
-  @Test
-  public void testMultipleColumnPrefixFilter() throws IOException {
-    String filterString = " MultipleColumnPrefixFilter('qualifier1', 'qualifier2' ) ";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof MultipleColumnPrefixFilter);
-    byte [][] prefixes = ((MultipleColumnPrefixFilter)filter).getPrefix();
-    assertEquals(new String(prefixes[0]), "qualifier1");
-    assertEquals(new String(prefixes[1]), "qualifier2");
-  }
-
-  @Test
-  public void testColumnCountGetFilter() throws IOException {
-    String filterString = " ColumnCountGetFilter(4)";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof ColumnCountGetFilter);
-    int limit = ((ColumnCountGetFilter)filter).getLimit();
-    assertEquals(limit, 4);
-
-    filterString = " ColumnCountGetFilter('abc')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof ColumnCountGetFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("ColumnCountGetFilter needs an int as an argument");
-    }
-
-    filterString = " ColumnCountGetFilter(2147483648)";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof ColumnCountGetFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("Integer argument too large");
-    }
-  }
-
-  @Test
-  public void testPageFilter() throws IOException {
-    String filterString = " PageFilter(4)";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof PageFilter);
-
-    long pageSize = ((PageFilter)filter).getPageSize();
-    assertEquals(pageSize, 4);
-
-    filterString = " PageFilter('123')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof PageFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("PageFilter needs an int as an argument");
-    }
-  }
-
-  @Test
-  public void testColumnPaginationFilter() throws IOException {
-    String filterString = "ColumnPaginationFilter(4, 6)";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof ColumnPaginationFilter);
-
-    int limit = ((ColumnPaginationFilter)filter).getLimit();
-    assertEquals(limit, 4);
-    int offset = ((ColumnPaginationFilter)filter).getOffset();
-    assertEquals(offset, 6);
-
-    filterString = " ColumnPaginationFilter('124')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof ColumnPaginationFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("ColumnPaginationFilter needs two arguments");
-    }
-
-    filterString = " ColumnPaginationFilter('4' , '123a')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof ColumnPaginationFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("ColumnPaginationFilter needs two ints as arguments");
-    }
-
-    filterString = " ColumnPaginationFilter('4' , '-123')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof ColumnPaginationFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("ColumnPaginationFilter arguments should not be negative");
-    }
-  }
-
-  @Test
-  public void testInclusiveStopFilter() throws IOException {
-    String filterString = "InclusiveStopFilter ('row 3')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof InclusiveStopFilter);
-
-    byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey();
-    assertEquals(new String(stopRowKey), "row 3");
-  }
-
-
-  @Test
-  public void testTimestampsFilter() throws IOException {
-    String filterString = "TimestampsFilter(9223372036854775806, 6)";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof TimestampsFilter);
-
-    TreeSet<Long> timestamps = ((TimestampsFilter)filter).getTimestamps();
-    assertEquals(timestamps.size(), 2);
-    assertTrue(timestamps.contains(new Long(6)));
-
-    filterString = "TimestampsFilter()";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof TimestampsFilter);
-
-    timestamps = ((TimestampsFilter)filter).getTimestamps();
-    assertEquals(timestamps.size(), 0);
-
-    filterString = "TimestampsFilter(9223372036854775808, 6)";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof TimestampsFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("Long Argument was too large");
-    }
-
-    filterString = "TimestampsFilter(-45, 6)";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof TimestampsFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("Timestamp Arguments should not be negative");
-    }
-  }
-
-  @Test
-  public void testRowFilter() throws IOException {
-    String filterString = "RowFilter ( =,   'binary:regionse')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof RowFilter);
-
-    RowFilter rowFilter = (RowFilter)filter;
-    assertEquals(CompareFilter.CompareOp.EQUAL, rowFilter.getOperator());
-    assertTrue(rowFilter.getComparator() instanceof BinaryComparator);
-    BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator();
-    assertEquals("regionse", new String(binaryComparator.getValue()));
-  }
-
-  @Test
-  public void testFamilyFilter() throws IOException {
-    String filterString = "FamilyFilter(>=, 'binaryprefix:pre')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof FamilyFilter);
-
-    FamilyFilter familyFilter = (FamilyFilter)filter;
-    assertEquals(CompareFilter.CompareOp.GREATER_OR_EQUAL, familyFilter.getOperator());
-    assertTrue(familyFilter.getComparator() instanceof BinaryPrefixComparator);
-    BinaryPrefixComparator binaryPrefixComparator =
-      (BinaryPrefixComparator) familyFilter.getComparator();
-    assertEquals("pre", new String(binaryPrefixComparator.getValue()));
-  }
-
-  @Test
-  public void testQualifierFilter() throws IOException {
-    String filterString = "QualifierFilter(=, 'regexstring:pre*')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof QualifierFilter);
-
-    QualifierFilter qualifierFilter = (QualifierFilter) filter;
-    assertEquals(CompareFilter.CompareOp.EQUAL, qualifierFilter.getOperator());
-    assertTrue(qualifierFilter.getComparator() instanceof RegexStringComparator);
-    RegexStringComparator regexStringComparator =
-      (RegexStringComparator) qualifierFilter.getComparator();
-    assertEquals("pre*", new String(regexStringComparator.getValue()));
-  }
-
-  @Test
-  public void testValueFilter() throws IOException {
-    String filterString = "ValueFilter(!=, 'substring:pre')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof ValueFilter);
-
-    ValueFilter valueFilter = (ValueFilter) filter;
-    assertEquals(CompareFilter.CompareOp.NOT_EQUAL, valueFilter.getOperator());
-    assertTrue(valueFilter.getComparator() instanceof SubstringComparator);
-    SubstringComparator substringComparator =
-      (SubstringComparator) valueFilter.getComparator();
-    assertEquals("pre", new String(substringComparator.getValue()));
-  }
-
-  @Test
-  public void testColumnRangeFilter() throws IOException {
-    String filterString = "ColumnRangeFilter('abc', true, 'xyz', false)";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof ColumnRangeFilter);
-
-    ColumnRangeFilter columnRangeFilter = (ColumnRangeFilter) filter;
-    assertEquals("abc", new String(columnRangeFilter.getMinColumn()));
-    assertEquals("xyz", new String(columnRangeFilter.getMaxColumn()));
-    assertTrue(columnRangeFilter.isMinColumnInclusive());
-    assertFalse(columnRangeFilter.isMaxColumnInclusive());
-  }
-
-  @Test
-  public void testDependentColumnFilter() throws IOException {
-    String filterString = "DependentColumnFilter('family', 'qualifier', true, =, 'binary:abc')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof DependentColumnFilter);
-
-    DependentColumnFilter dependentColumnFilter = (DependentColumnFilter) filter;
-    assertEquals("family", new String(dependentColumnFilter.getFamily()));
-    assertEquals("qualifier", new String(dependentColumnFilter.getQualifier()));
-    assertTrue(dependentColumnFilter.getDropDependentColumn());
-    assertEquals(CompareFilter.CompareOp.EQUAL, dependentColumnFilter.getOperator());
-    assertTrue(dependentColumnFilter.getComparator() instanceof BinaryComparator);
-    BinaryComparator binaryComparator = (BinaryComparator)dependentColumnFilter.getComparator();
-    assertEquals("abc", new String(binaryComparator.getValue()));
-  }
-
-  @Test
-  public void testSingleColumnValueFilter() throws IOException {
-
-    String filterString = "SingleColumnValueFilter " +
-      "('family', 'qualifier', >=, 'binary:a', true, false)";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-
-    assertTrue(filter instanceof SingleColumnValueFilter);
-
-    SingleColumnValueFilter singleColumnValueFilter = (SingleColumnValueFilter)filter;
-    assertEquals("family", new String(singleColumnValueFilter.getFamily()));
-    assertEquals("qualifier", new String(singleColumnValueFilter.getQualifier()));
-    assertEquals(singleColumnValueFilter.getOperator(), CompareFilter.CompareOp.GREATER_OR_EQUAL);
-    assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryComparator);
-    BinaryComparator binaryComparator = (BinaryComparator) singleColumnValueFilter.getComparator();
-    assertEquals(new String(binaryComparator.getValue()), "a");
-    assertTrue(singleColumnValueFilter.getFilterIfMissing());
-    assertFalse(singleColumnValueFilter.getLatestVersionOnly());
-
-
-    filterString = "SingleColumnValueFilter ('family', 'qualifier', >, 'binaryprefix:a')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-
-    assertTrue(filter instanceof SingleColumnValueFilter);
-
-    singleColumnValueFilter = (SingleColumnValueFilter)filter;
-    assertEquals("family", new String(singleColumnValueFilter.getFamily()));
-    assertEquals("qualifier", new String(singleColumnValueFilter.getQualifier()));
-    assertEquals(singleColumnValueFilter.getOperator(), CompareFilter.CompareOp.GREATER);
-    assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryPrefixComparator);
-    BinaryPrefixComparator binaryPrefixComparator =
-      (BinaryPrefixComparator) singleColumnValueFilter.getComparator();
-    assertEquals(new String(binaryPrefixComparator.getValue()), "a");
-    assertFalse(singleColumnValueFilter.getFilterIfMissing());
-    assertTrue(singleColumnValueFilter.getLatestVersionOnly());
-  }
-
-  @Test
-  public void testSingleColumnValueExcludeFilter() throws IOException {
-
-    String filterString =
-      "SingleColumnValueExcludeFilter ('family', 'qualifier', <, 'binaryprefix:a')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof SingleColumnValueExcludeFilter);
-
-    SingleColumnValueExcludeFilter singleColumnValueExcludeFilter =
-      (SingleColumnValueExcludeFilter)filter;
-    assertEquals(singleColumnValueExcludeFilter.getOperator(), CompareFilter.CompareOp.LESS);
-    assertEquals("family", new String(singleColumnValueExcludeFilter.getFamily()));
-    assertEquals("qualifier", new String(singleColumnValueExcludeFilter.getQualifier()));
-    assertEquals(new String(singleColumnValueExcludeFilter.getComparator().getValue()), "a");
-    assertFalse(singleColumnValueExcludeFilter.getFilterIfMissing());
-    assertTrue(singleColumnValueExcludeFilter.getLatestVersionOnly());
-
-    filterString = "SingleColumnValueExcludeFilter " +
-      "('family', 'qualifier', <=, 'binaryprefix:a', true, false)";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof SingleColumnValueExcludeFilter);
-
-    singleColumnValueExcludeFilter = (SingleColumnValueExcludeFilter)filter;
-    assertEquals("family", new String(singleColumnValueExcludeFilter.getFamily()));
-    assertEquals("qualifier", new String(singleColumnValueExcludeFilter.getQualifier()));
-    assertEquals(singleColumnValueExcludeFilter.getOperator(),
-                 CompareFilter.CompareOp.LESS_OR_EQUAL);
-    assertTrue(singleColumnValueExcludeFilter.getComparator() instanceof BinaryPrefixComparator);
-    BinaryPrefixComparator binaryPrefixComparator =
-      (BinaryPrefixComparator) singleColumnValueExcludeFilter.getComparator();
-    assertEquals(new String(binaryPrefixComparator.getValue()), "a");
-    assertTrue(singleColumnValueExcludeFilter.getFilterIfMissing());
-    assertFalse(singleColumnValueExcludeFilter.getLatestVersionOnly());
-  }
-
-  @Test
-  public void testSkipFilter() throws IOException {
-    String filterString = "SKIP ValueFilter( =,  'binary:0')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof SkipFilter);
-    SkipFilter skipFilter = (SkipFilter) filter;
-    assertTrue(skipFilter.getFilter() instanceof ValueFilter);
-    ValueFilter valueFilter = (ValueFilter) skipFilter.getFilter();
-
-    assertEquals(CompareFilter.CompareOp.EQUAL, valueFilter.getOperator());
-    assertTrue(valueFilter.getComparator() instanceof BinaryComparator);
-    BinaryComparator binaryComparator = (BinaryComparator) valueFilter.getComparator();
-    assertEquals("0", new String(binaryComparator.getValue()));
-  }
-
-  @Test
-  public void testWhileFilter() throws IOException {
-    String filterString = " WHILE   RowFilter ( !=, 'binary:row1')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof WhileMatchFilter);
-    WhileMatchFilter whileMatchFilter = (WhileMatchFilter) filter;
-    assertTrue(whileMatchFilter.getFilter() instanceof RowFilter);
-    RowFilter rowFilter = (RowFilter) whileMatchFilter.getFilter();
-
-    assertEquals(CompareFilter.CompareOp.NOT_EQUAL, rowFilter.getOperator());
-    assertTrue(rowFilter.getComparator() instanceof BinaryComparator);
-    BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator();
-    assertEquals("row1", new String(binaryComparator.getValue()));
-  }
-
-  @Test
-  public void testCompoundFilter1() throws IOException {
-    String filterString = " (PrefixFilter ('realtime')AND  FirstKeyOnlyFilter())";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    FilterList filterList = (FilterList)filter;
-    ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
-
-    assertTrue(filters.get(0) instanceof PrefixFilter);
-    assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
-
-    PrefixFilter PrefixFilter = (PrefixFilter) filters.get(0);
-    byte [] prefix = PrefixFilter.getPrefix();
-    assertEquals(new String(prefix), "realtime");
-    FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
-  }
-
-  @Test
-  public void testCompoundFilter2() throws IOException {
-    String filterString = "(PrefixFilter('realtime') AND QualifierFilter (>=, 'binary:e'))" +
-      "OR FamilyFilter (=, 'binary:qualifier') ";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-
-    FilterList filterList = (FilterList)filter;
-    ArrayList<Filter> filterListFilters = (ArrayList<Filter>) filterList.getFilters();
-    assertTrue(filterListFilters.get(0) instanceof FilterList);
-    assertTrue(filterListFilters.get(1) instanceof FamilyFilter);
-    assertEquals(filterList.getOperator(), FilterList.Operator.MUST_PASS_ONE);
-
-    filterList = (FilterList) filterListFilters.get(0);
-    FamilyFilter familyFilter = (FamilyFilter) filterListFilters.get(1);
-
-    filterListFilters = (ArrayList<Filter>)filterList.getFilters();
-    assertTrue(filterListFilters.get(0) instanceof PrefixFilter);
-    assertTrue(filterListFilters.get(1) instanceof QualifierFilter);
-    assertEquals(filterList.getOperator(), FilterList.Operator.MUST_PASS_ALL);
-
-    assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
-    assertTrue(familyFilter.getComparator() instanceof BinaryComparator);
-    BinaryComparator binaryComparator = (BinaryComparator) familyFilter.getComparator();
-    assertEquals("qualifier", new String(binaryComparator.getValue()));
-
-    PrefixFilter prefixFilter = (PrefixFilter) filterListFilters.get(0);
-    byte [] prefix = prefixFilter.getPrefix();
-    assertEquals(new String(prefix), "realtime");
-
-    QualifierFilter qualifierFilter = (QualifierFilter) filterListFilters.get(1);
-    assertEquals(CompareFilter.CompareOp.GREATER_OR_EQUAL, qualifierFilter.getOperator());
-    assertTrue(qualifierFilter.getComparator() instanceof BinaryComparator);
-    binaryComparator = (BinaryComparator) qualifierFilter.getComparator();
-    assertEquals("e", new String(binaryComparator.getValue()));
-  }
-
-  @Test
-  public void testCompoundFilter3() throws IOException {
-    String filterString = " ColumnPrefixFilter ('realtime')AND  " +
-      "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    FilterList filterList = (FilterList)filter;
-    ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
-
-    assertTrue(filters.get(0) instanceof FilterList);
-    assertTrue(filters.get(1) instanceof SkipFilter);
-
-    filterList = (FilterList) filters.get(0);
-    SkipFilter skipFilter = (SkipFilter) filters.get(1);
-
-    filters = (ArrayList<Filter>) filterList.getFilters();
-    assertTrue(filters.get(0) instanceof ColumnPrefixFilter);
-    assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
-
-    ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0);
-    byte [] columnPrefix = columnPrefixFilter.getPrefix();
-    assertEquals(new String(columnPrefix), "realtime");
-
-    FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
-
-    assertTrue(skipFilter.getFilter() instanceof FamilyFilter);
-    FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter();
-
-    assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
-    assertTrue(familyFilter.getComparator() instanceof SubstringComparator);
-    SubstringComparator substringComparator =
-      (SubstringComparator) familyFilter.getComparator();
-    assertEquals("hihi", new String(substringComparator.getValue()));
-  }
-
-  @Test
-  public void testCompoundFilter4() throws IOException {
-    String filterString = " ColumnPrefixFilter ('realtime') OR " +
-      "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    FilterList filterList = (FilterList)filter;
-    ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
-
-    assertTrue(filters.get(0) instanceof ColumnPrefixFilter);
-    assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
-    assertTrue(filters.get(2) instanceof SkipFilter);
-
-    ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0);
-    FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
-    SkipFilter skipFilter = (SkipFilter) filters.get(2);
-
-    byte [] columnPrefix = columnPrefixFilter.getPrefix();
-    assertEquals(new String(columnPrefix), "realtime");
-
-    assertTrue(skipFilter.getFilter() instanceof FamilyFilter);
-    FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter();
-
-    assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
-    assertTrue(familyFilter.getComparator() instanceof SubstringComparator);
-    SubstringComparator substringComparator =
-      (SubstringComparator) familyFilter.getComparator();
-    assertEquals("hihi", new String(substringComparator.getValue()));
-  }
-
-  @Test
-  public void testIncorrectCompareOperator() throws IOException {
-    String filterString = "RowFilter ('>>' , 'binary:region')";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof RowFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("Incorrect compare operator >>");
-    }
-  }
-
-  @Test
-  public void testIncorrectComparatorType () throws IOException {
-    String  filterString = "RowFilter ('>=' , 'binaryoperator:region')";
-    byte []  filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof RowFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("Incorrect comparator type: binaryoperator");
-    }
-
-    filterString = "RowFilter ('>=' 'regexstring:pre*')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof RowFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("RegexStringComparator can only be used with EQUAL or NOT_EQUAL");
-    }
-
-    filterString = "SingleColumnValueFilter" +
-      " ('family', 'qualifier', '>=', 'substring:a', 'true', 'false')')";
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-      assertTrue(filter instanceof RowFilter);
-    } catch (IllegalArgumentException e) {
-      System.out.println("SubtringComparator can only be used with EQUAL or NOT_EQUAL");
-    }
-  }
-
-  @Test
-  public void testPrecedence1() throws IOException {
-    String filterString = " (PrefixFilter ('realtime')AND  FirstKeyOnlyFilter()" +
-      " OR KeyOnlyFilter())";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    FilterList filterList = (FilterList)filter;
-    ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
-
-    assertTrue(filters.get(0) instanceof FilterList);
-    assertTrue(filters.get(1) instanceof KeyOnlyFilter);
-
-    filterList = (FilterList) filters.get(0);
-    filters = (ArrayList<Filter>) filterList.getFilters();
-
-    assertTrue(filters.get(0) instanceof PrefixFilter);
-    assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
-
-    PrefixFilter prefixFilter = (PrefixFilter)filters.get(0);
-    byte [] prefix = prefixFilter.getPrefix();
-    assertEquals(new String(prefix), "realtime");
-  }
-
-  @Test
-  public void testPrecedence2() throws IOException {
-    String filterString = " PrefixFilter ('realtime')AND  SKIP FirstKeyOnlyFilter()" +
-      "OR KeyOnlyFilter()";
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    FilterList filterList = (FilterList)filter;
-    ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
-
-    assertTrue(filters.get(0) instanceof FilterList);
-    assertTrue(filters.get(1) instanceof KeyOnlyFilter);
-
-    filterList = (FilterList) filters.get(0);
-    filters = (ArrayList<Filter>) filterList.getFilters();
-
-    assertTrue(filters.get(0) instanceof PrefixFilter);
-    assertTrue(filters.get(1) instanceof SkipFilter);
-
-    PrefixFilter prefixFilter = (PrefixFilter)filters.get(0);
-    byte [] prefix = prefixFilter.getPrefix();
-    assertEquals(new String(prefix), "realtime");
-
-    SkipFilter skipFilter = (SkipFilter)filters.get(1);
-    assertTrue(skipFilter.getFilter() instanceof FirstKeyOnlyFilter);
-  }
-
-  @Test
-  public void testUnescapedQuote1 () throws IOException {
-    String  filterString = "InclusiveStopFilter ('row''3')";
-    System.out.println(filterString);
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof InclusiveStopFilter);
-
-    byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey();
-    assertEquals(new String(stopRowKey), "row'3");
-  }
-
-  @Test
-  public void testUnescapedQuote2 () throws IOException {
-    String  filterString = "InclusiveStopFilter ('row''3''')";
-    System.out.println(filterString);
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof InclusiveStopFilter);
-
-    byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey();
-    assertEquals(new String(stopRowKey), "row'3'");
-  }
-
-  @Test
-  public void testUnescapedQuote3 () throws IOException {
-    String  filterString = "	InclusiveStopFilter ('''')";
-    System.out.println(filterString);
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof InclusiveStopFilter);
-
-    byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey();
-    assertEquals(new String(stopRowKey), "'");
-  }
-
-  @Test
-  public void testIncorrectFilterString () throws IOException {
-    String  filterString = "()";
-    System.out.println(filterString);
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    try {
-      filter = f.parseFilterString(filterStringAsByteArray);
-    } catch (IllegalArgumentException e) {
-      System.out.println(e.getMessage());
-    }
-
-    filterString = " OR KeyOnlyFilter() FirstKeyOnlyFilter()";
-    System.out.println(filterString);
-    filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-  }
-
-  @Test
-  public void testCorrectFilterString () throws IOException {
-    String  filterString = "(FirstKeyOnlyFilter())";
-    System.out.println(filterString);
-    byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
-    filter = f.parseFilterString(filterStringAsByteArray);
-    assertTrue(filter instanceof FirstKeyOnlyFilter);
-  }
-}

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java?rev=1155970&r1=1155969&r2=1155970&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/zookeeper/TestHQuorumPeer.java Tue Aug  9 23:30:11 2011
@@ -132,11 +132,14 @@ public class TestHQuorumPeer {
     assertEquals("foo.bar", server.addr.getHostName());
   }
 
+  /**
+   * Test Case for HBASE-2305
+   */
   @Test public void testShouldAssignDefaultZookeeperClientPort() {
     Configuration config = HBaseConfiguration.create();
     config.clear();
     Properties p = ZKConfig.makeZKProps(config);
     assertNotNull(p);
-    assertEquals(2181, p.get("clientPort"));
+    assertEquals(2181, p.get("hbase.zookeeper.property.clientPort"));
   }
 }