You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2016/07/12 14:36:46 UTC

[2/3] lucene-solr:master: LUCENE-7355: Add Analyzer#normalize() and use it in query parsers.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
index cdfa477..fbe08a9 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
@@ -16,14 +16,13 @@
  */
 package org.apache.lucene.queryparser.classic;
 
-import java.io.IOException;
 import java.io.StringReader;
 import java.text.DateFormat;
 import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryparser.classic.QueryParser.Operator;
@@ -32,6 +31,7 @@ import org.apache.lucene.search.*;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery.TooManyClauses;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.QueryBuilder;
 import org.apache.lucene.util.automaton.RegExp;
 
@@ -41,9 +41,6 @@ import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZ
  * and acts to separate the majority of the Java code from the .jj grammar file. 
  */
 public abstract class QueryParserBase extends QueryBuilder implements CommonQueryParserConfiguration {
-  
-  /** Do not catch this exception in your code, it means you are using methods that you should no longer use. */
-  public static class MethodRemovedUseAnother extends Throwable {}
 
   static final int CONJ_NONE   = 0;
   static final int CONJ_AND    = 1;
@@ -63,7 +60,6 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
   /** The actual operator that parser uses to combine query terms */
   Operator operator = OR_OPERATOR;
 
-  boolean lowercaseExpandedTerms = true;
   MultiTermQuery.RewriteMethod multiTermRewriteMethod = MultiTermQuery.CONSTANT_SCORE_REWRITE;
   boolean allowLeadingWildcard = false;
 
@@ -79,10 +75,6 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
   // maps field names to date resolutions
   Map<String,DateTools.Resolution> fieldToDateResolution = null;
 
-  //Whether or not to analyze range terms when constructing RangeQuerys
-  // (For example, analyzing terms into collation keys for locale-sensitive RangeQuery)
-  boolean analyzeRangeTerms = false;
-
   boolean autoGeneratePhraseQueries;
   int maxDeterminizedStates = DEFAULT_MAX_DETERMINIZED_STATES;
 
@@ -253,24 +245,7 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     return operator;
   }
 
-
-  /**
-   * Whether terms of wildcard, prefix, fuzzy and range queries are to be automatically
-   * lower-cased or not.  Default is <code>true</code>.
-   */
-  @Override
-  public void setLowercaseExpandedTerms(boolean lowercaseExpandedTerms) {
-    this.lowercaseExpandedTerms = lowercaseExpandedTerms;
-  }
-
-  /**
-   * @see #setLowercaseExpandedTerms(boolean)
-   */
-  @Override
-  public boolean getLowercaseExpandedTerms() {
-    return lowercaseExpandedTerms;
-  }
-
+  
   /**
    * By default QueryParser uses {@link org.apache.lucene.search.MultiTermQuery#CONSTANT_SCORE_REWRITE}
    * when creating a {@link PrefixQuery}, {@link WildcardQuery} or {@link TermRangeQuery}. This implementation is generally preferable because it
@@ -379,24 +354,6 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
   }
 
   /**
-   * Set whether or not to analyze range terms when constructing {@link TermRangeQuery}s.
-   * For example, setting this to true can enable analyzing terms into 
-   * collation keys for locale-sensitive {@link TermRangeQuery}.
-   * 
-   * @param analyzeRangeTerms whether or not terms should be analyzed for RangeQuerys
-   */
-  public void setAnalyzeRangeTerms(boolean analyzeRangeTerms) {
-    this.analyzeRangeTerms = analyzeRangeTerms;
-  }
-
-  /**
-   * @return whether or not to analyze range terms when constructing {@link TermRangeQuery}s.
-   */
-  public boolean getAnalyzeRangeTerms() {
-    return analyzeRangeTerms;
-  }
-
-  /**
    * @param maxDeterminizedStates the maximum number of states that
    *   determinizing a regexp query can result in.  If the query results in any
    *   more states a TooComplexToDeterminizeException is thrown.
@@ -558,12 +515,6 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
                                 boolean startInclusive,
                                 boolean endInclusive) throws ParseException
   {
-    if (lowercaseExpandedTerms) {
-      part1 = part1==null ? null : part1.toLowerCase(locale);
-      part2 = part2==null ? null : part2.toLowerCase(locale);
-    }
-
-
     DateFormat df = DateFormat.getDateInstance(DateFormat.SHORT, locale);
     df.setLenient(true);
     DateTools.Resolution resolution = getDateResolution(field);
@@ -640,31 +591,6 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     return new FuzzyQuery(term,numEdits,prefixLength);
   }
 
-  // TODO: Should this be protected instead?
-  private BytesRef analyzeMultitermTerm(String field, String part) {
-    return analyzeMultitermTerm(field, part, getAnalyzer());
-  }
-
-  protected BytesRef analyzeMultitermTerm(String field, String part, Analyzer analyzerIn) {
-    if (analyzerIn == null) analyzerIn = getAnalyzer();
-
-    try (TokenStream source = analyzerIn.tokenStream(field, part)) {
-      source.reset();
-      
-      TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
-
-      if (!source.incrementToken())
-        throw new IllegalArgumentException("analyzer returned no terms for multiTerm term: " + part);
-      BytesRef bytes = BytesRef.deepCopyOf(termAtt.getBytesRef());
-      if (source.incrementToken())
-        throw new IllegalArgumentException("analyzer returned too many terms for multiTerm term: " + part);
-      source.end();
-      return bytes;
-    } catch (IOException e) {
-      throw new RuntimeException("Error analyzing multiTerm term: " + part, e);
-    }
-  }
-
   /**
    * Builds a new {@link TermRangeQuery} instance
    * @param field Field
@@ -681,13 +607,13 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     if (part1 == null) {
       start = null;
     } else {
-      start = analyzeRangeTerms ? analyzeMultitermTerm(field, part1) : new BytesRef(part1);
+      start = getAnalyzer().normalize(field, part1);
     }
      
     if (part2 == null) {
       end = null;
     } else {
-      end = analyzeRangeTerms ? analyzeMultitermTerm(field, part2) : new BytesRef(part2);
+      end = getAnalyzer().normalize(field, part2);
     }
       
     final TermRangeQuery query = new TermRangeQuery(field, start, end, startInclusive, endInclusive);
@@ -767,13 +693,38 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     }
     if (!allowLeadingWildcard && (termStr.startsWith("*") || termStr.startsWith("?")))
       throw new ParseException("'*' or '?' not allowed as first character in WildcardQuery");
-    if (lowercaseExpandedTerms) {
-      termStr = termStr.toLowerCase(locale);
-    }
-    Term t = new Term(field, termStr);
+
+    Term t = new Term(field, analyzeWildcard(field, termStr));
     return newWildcardQuery(t);
   }
 
+  private static final Pattern WILDCARD_PATTERN = Pattern.compile("(\\\\.)|([?*]+)");
+
+  private BytesRef analyzeWildcard(String field, String termStr) {
+    // best effort to not pass the wildcard characters and escaped characters through #normalize
+    Matcher wildcardMatcher = WILDCARD_PATTERN.matcher(termStr);
+    BytesRefBuilder sb = new BytesRefBuilder();
+    int last = 0;
+
+    while (wildcardMatcher.find()){
+      if (wildcardMatcher.start() > 0) {
+        String chunk = termStr.substring(last, wildcardMatcher.start());
+        BytesRef normalized = getAnalyzer().normalize(field, chunk);
+        sb.append(normalized);
+      }
+      //append the matched group - without normalizing
+      sb.append(new BytesRef(wildcardMatcher.group()));
+
+      last = wildcardMatcher.end();
+    }
+    if (last < termStr.length()){
+      String chunk = termStr.substring(last);
+      BytesRef normalized = getAnalyzer().normalize(field, chunk);
+      sb.append(normalized);
+    }
+    return sb.toBytesRef();
+  }
+
   /**
    * Factory method for generating a query. Called when parser
    * parses an input term token that contains a regular expression
@@ -796,10 +747,11 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
    */
   protected Query getRegexpQuery(String field, String termStr) throws ParseException
   {
-    if (lowercaseExpandedTerms) {
-      termStr = termStr.toLowerCase(locale);
-    }
-    Term t = new Term(field, termStr);
+    // We need to pass the whole string to #normalize, which will not work with
+    // custom attribute factories for the binary term impl, and may not work
+    // with some analyzers
+    BytesRef term = getAnalyzer().normalize(field, termStr);
+    Term t = new Term(field, term);
     return newRegexpQuery(t);
   }
 
@@ -830,10 +782,8 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
   {
     if (!allowLeadingWildcard && termStr.startsWith("*"))
       throw new ParseException("'*' not allowed as first character in PrefixQuery");
-    if (lowercaseExpandedTerms) {
-      termStr = termStr.toLowerCase(locale);
-    }
-    Term t = new Term(field, termStr);
+    BytesRef term = getAnalyzer().normalize(field, termStr);
+    Term t = new Term(field, term);
     return newPrefixQuery(t);
   }
 
@@ -850,10 +800,8 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
    */
   protected Query getFuzzyQuery(String field, String termStr, float minSimilarity) throws ParseException
   {
-    if (lowercaseExpandedTerms) {
-      termStr = termStr.toLowerCase(locale);
-    }
-    Term t = new Term(field, termStr);
+    BytesRef term = getAnalyzer().normalize(field, termStr);
+    Term t = new Term(field, term);
     return newFuzzyQuery(t, minSimilarity, fuzzyPrefixLength);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
index ac808d7..1a7e5e1 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java
@@ -33,9 +33,9 @@ import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.MultiTermQuery.RewriteMethod;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.search.spans.SpanBoostQuery;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanNotQuery;
@@ -186,14 +186,15 @@ public class ComplexPhraseQueryParser extends QueryParser {
   @Override
   protected Query newRangeQuery(String field, String part1, String part2,
       boolean startInclusive, boolean endInclusive) {
-    if (isPass2ResolvingPhrases) {
-      // Must use old-style RangeQuery in order to produce a BooleanQuery
-      // that can be turned into SpanOr clause
-      TermRangeQuery rangeQuery = TermRangeQuery.newStringRange(field, part1, part2, startInclusive, endInclusive);
-      rangeQuery.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);
-      return rangeQuery;
+    RewriteMethod originalRewriteMethod = getMultiTermRewriteMethod();
+    try {
+      if (isPass2ResolvingPhrases) {
+        setMultiTermRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);
+      }
+      return super.newRangeQuery(field, part1, part2, startInclusive, endInclusive);
+    } finally {
+      setMultiTermRewriteMethod(originalRewriteMethod);
     }
-    return super.newRangeQuery(field, part1, part2, startInclusive, endInclusive);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java
index 55e43cd..c44e9e0 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java
@@ -32,18 +32,6 @@ import org.apache.lucene.search.MultiTermQuery;
 public interface CommonQueryParserConfiguration {
   
   /**
-   * Whether terms of multi-term queries (e.g., wildcard,
-   * prefix, fuzzy and range) should be automatically
-   * lower-cased or not.  Default is <code>true</code>.
-   */
-  public void setLowercaseExpandedTerms(boolean lowercaseExpandedTerms);
-  
-  /**
-   * @see #setLowercaseExpandedTerms(boolean)
-   */
-  public boolean getLowercaseExpandedTerms();
-  
-  /**
    * Set to <code>true</code> to allow leading wildcard characters.
    * <p>
    * When set, <code>*</code> or <code>?</code> are allowed as the first

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
index 2cd8084..32cbd02 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
@@ -190,36 +190,6 @@ public class StandardQueryParser extends QueryParserHelper implements CommonQuer
    * Default: false.
    */
   @Override
-  public void setLowercaseExpandedTerms(boolean lowercaseExpandedTerms) {
-    getQueryConfigHandler().set(ConfigurationKeys.LOWERCASE_EXPANDED_TERMS, lowercaseExpandedTerms);
-  }
-  
-  /**
-   * @see #setLowercaseExpandedTerms(boolean)
-   */
-  @Override
-  public boolean getLowercaseExpandedTerms() {
-    Boolean lowercaseExpandedTerms = getQueryConfigHandler().get(ConfigurationKeys.LOWERCASE_EXPANDED_TERMS);
-    
-    if (lowercaseExpandedTerms == null) {
-      return true;
-      
-    } else {
-      return lowercaseExpandedTerms;
-    }
-    
-  }
-  
-  /**
-   * Set to <code>true</code> to allow leading wildcard characters.
-   * <p>
-   * When set, <code>*</code> or <code>?</code> are allowed as the first
-   * character of a PrefixQuery and WildcardQuery. Note that this can produce
-   * very slow queries on big indexes.
-   * <p>
-   * Default: false.
-   */
-  @Override
   public void setAllowLeadingWildcard(boolean allowLeadingWildcard) {
     getQueryConfigHandler().set(ConfigurationKeys.ALLOW_LEADING_WILDCARD, allowLeadingWildcard);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
index bba95ee..5c53d02 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
@@ -55,14 +55,6 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
      * @see StandardQueryParser#getEnablePositionIncrements()
      */
     final public static ConfigurationKey<Boolean> ENABLE_POSITION_INCREMENTS = ConfigurationKey.newInstance();
-    
-    /**
-     * Key used to set whether expanded terms should be lower-cased
-     * 
-     * @see StandardQueryParser#setLowercaseExpandedTerms(boolean)
-     * @see StandardQueryParser#getLowercaseExpandedTerms()
-     */
-    final public static ConfigurationKey<Boolean> LOWERCASE_EXPANDED_TERMS = ConfigurationKey.newInstance();
 
     /**
      * Key used to set whether leading wildcards are supported
@@ -223,7 +215,6 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
     set(ConfigurationKeys.ANALYZER, null); //default value 2.4
     set(ConfigurationKeys.DEFAULT_OPERATOR, Operator.OR);
     set(ConfigurationKeys.PHRASE_SLOP, 0); //default value 2.4
-    set(ConfigurationKeys.LOWERCASE_EXPANDED_TERMS, true); //default value 2.4
     set(ConfigurationKeys.ENABLE_POSITION_INCREMENTS, false); //default value 2.4
     set(ConfigurationKeys.FIELD_BOOST_MAP, new LinkedHashMap<String, Float>());
     set(ConfigurationKeys.FUZZY_CONFIG, new FuzzyConfig());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/FuzzyQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/FuzzyQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/FuzzyQueryNodeProcessor.java
index 0b8a9a7..9479fcf 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/FuzzyQueryNodeProcessor.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/FuzzyQueryNodeProcessor.java
@@ -18,6 +18,7 @@ package org.apache.lucene.queryparser.flexible.standard.processors;
 
 import java.util.List;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
 import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
 import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
@@ -55,9 +56,17 @@ public class FuzzyQueryNodeProcessor extends QueryNodeProcessorImpl {
       FuzzyQueryNode fuzzyNode = (FuzzyQueryNode) node;
       QueryConfigHandler config = getQueryConfigHandler();
 
+      Analyzer analyzer = getQueryConfigHandler().get(ConfigurationKeys.ANALYZER);
+      if (analyzer != null) {
+        // because we call utf8ToString, this will only work with the default TermToBytesRefAttribute
+        String text = fuzzyNode.getTextAsString();
+        text = analyzer.normalize(fuzzyNode.getFieldAsString(), text).utf8ToString();
+        fuzzyNode.setText(text);
+      }
+
       FuzzyConfig fuzzyConfig = null;
       
-      if (config != null && (fuzzyConfig = config.get(ConfigurationKeys.FUZZY_CONFIG)) != null) {
+      if ((fuzzyConfig = config.get(ConfigurationKeys.FUZZY_CONFIG)) != null) {
         fuzzyNode.setPrefixLength(fuzzyConfig.getPrefixLength());
 
         if (fuzzyNode.getSimilarity() < 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LowercaseExpandedTermsQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LowercaseExpandedTermsQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LowercaseExpandedTermsQueryNodeProcessor.java
deleted file mode 100644
index 3bb2075..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LowercaseExpandedTermsQueryNodeProcessor.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.lucene.queryparser.flexible.standard.processors;
-
-import java.util.List;
-import java.util.Locale;
-
-import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
-import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
-import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.TextableQueryNode;
-import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
-import org.apache.lucene.queryparser.flexible.core.util.UnescapedCharSequence;
-import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
-import org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
-import org.apache.lucene.queryparser.flexible.standard.nodes.WildcardQueryNode;
-
-/**
- * This processor verifies if 
- * {@link ConfigurationKeys#LOWERCASE_EXPANDED_TERMS} is defined in the
- * {@link QueryConfigHandler}. If it is and the expanded terms should be
- * lower-cased, it looks for every {@link WildcardQueryNode},
- * {@link FuzzyQueryNode} and children of a {@link RangeQueryNode} and lower-case its
- * term.
- * 
- * @see ConfigurationKeys#LOWERCASE_EXPANDED_TERMS
- */
-public class LowercaseExpandedTermsQueryNodeProcessor extends
-    QueryNodeProcessorImpl {
-
-  public LowercaseExpandedTermsQueryNodeProcessor() {
-  }
-
-  @Override
-  public QueryNode process(QueryNode queryTree) throws QueryNodeException {
-    Boolean lowercaseExpandedTerms = getQueryConfigHandler().get(ConfigurationKeys.LOWERCASE_EXPANDED_TERMS);
-
-    if (lowercaseExpandedTerms != null && lowercaseExpandedTerms) {
-      return super.process(queryTree);
-    }
-
-    return queryTree;
-
-  }
-
-  @Override
-  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
-    
-    Locale locale = getQueryConfigHandler().get(ConfigurationKeys.LOCALE);
-    if (locale == null) {
-      locale = Locale.getDefault();
-    }
-
-    if (node instanceof WildcardQueryNode
-        || node instanceof FuzzyQueryNode
-        || (node instanceof FieldQueryNode && node.getParent() instanceof RangeQueryNode)
-        || node instanceof RegexpQueryNode) {
-
-      TextableQueryNode txtNode = (TextableQueryNode) node;
-      CharSequence text = txtNode.getText();
-      txtNode.setText(text != null ? UnescapedCharSequence.toLowerCase(text, locale) : null);
-    }
-
-    return node;
-
-  }
-
-  @Override
-  protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
-
-    return node;
-
-  }
-
-  @Override
-  protected List<QueryNode> setChildrenOrder(List<QueryNode> children)
-      throws QueryNodeException {
-
-    return children;
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/RegexpQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/RegexpQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/RegexpQueryNodeProcessor.java
new file mode 100644
index 0000000..652de87
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/RegexpQueryNodeProcessor.java
@@ -0,0 +1,56 @@
+/*
+ * 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.lucene.queryparser.flexible.standard.processors;
+
+import java.util.List;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+import org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
+
+/** Processor for Regexp queries. */
+public class RegexpQueryNodeProcessor extends QueryNodeProcessorImpl {
+
+  @Override
+  protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
+    return node;
+  }
+
+  @Override
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+    if (node instanceof RegexpQueryNode) {
+      RegexpQueryNode regexpNode = (RegexpQueryNode) node;
+      Analyzer analyzer = getQueryConfigHandler().get(ConfigurationKeys.ANALYZER);
+      if (analyzer != null) {
+        String text = regexpNode.getText().toString();
+        // because we call utf8ToString, this will only work with the default TermToBytesRefAttribute
+        text = analyzer.normalize(regexpNode.getFieldAsString(), text).utf8ToString();
+        regexpNode.setText(text);
+      }
+    }
+    return node;
+  }
+
+  @Override
+  protected List<QueryNode> setChildrenOrder(List<QueryNode> children) throws QueryNodeException {
+    return children;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
index 38a9a47..5b681b4 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
@@ -47,16 +47,16 @@ public class StandardQueryNodeProcessorPipeline extends
   public StandardQueryNodeProcessorPipeline(QueryConfigHandler queryConfig) {
     super(queryConfig);
 
-    add(new WildcardQueryNodeProcessor());    
+    add(new WildcardQueryNodeProcessor());   
     add(new MultiFieldQueryNodeProcessor());
     add(new FuzzyQueryNodeProcessor());
+    add(new RegexpQueryNodeProcessor());
     add(new MatchAllDocsQueryNodeProcessor());
     add(new OpenRangeQueryNodeProcessor());
     add(new LegacyNumericQueryNodeProcessor());
     add(new LegacyNumericRangeQueryNodeProcessor());
     add(new PointQueryNodeProcessor());
     add(new PointRangeQueryNodeProcessor());
-    add(new LowercaseExpandedTermsQueryNodeProcessor());
     add(new TermRangeQueryNodeProcessor());
     add(new AllowLeadingWildcardProcessor());    
     add(new AnalyzerQueryNodeProcessor());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/TermRangeQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/TermRangeQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/TermRangeQueryNodeProcessor.java
index f9a4583..557c605 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/TermRangeQueryNodeProcessor.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/TermRangeQueryNodeProcessor.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.TimeZone;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.document.DateTools.Resolution;
 import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
@@ -134,7 +135,15 @@ public class TermRangeQueryNodeProcessor extends QueryNodeProcessorImpl {
         }
         
       } catch (Exception e) {
-        // do nothing
+        // not a date
+        Analyzer analyzer = getQueryConfigHandler().get(ConfigurationKeys.ANALYZER);
+        if (analyzer != null) {
+          // because we call utf8ToString, this will only work with the default TermToBytesRefAttribute
+          part1 = analyzer.normalize(lower.getFieldAsString(), part1).utf8ToString();
+          part2 = analyzer.normalize(lower.getFieldAsString(), part2).utf8ToString();
+          lower.setText(part1);
+          upper.setText(part2);
+        }
       }
       
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/WildcardQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/WildcardQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/WildcardQueryNodeProcessor.java
index 7182575..39eb0df 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/WildcardQueryNodeProcessor.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/WildcardQueryNodeProcessor.java
@@ -17,7 +17,10 @@
 package org.apache.lucene.queryparser.flexible.standard.processors;
 
 import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
 import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
@@ -25,11 +28,13 @@ import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.QuotedFieldQueryNode;
 import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
 import org.apache.lucene.queryparser.flexible.core.util.UnescapedCharSequence;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
 import org.apache.lucene.queryparser.flexible.standard.nodes.PrefixWildcardQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.TermRangeQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.WildcardQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.parser.StandardSyntaxParser;
 import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * The {@link StandardSyntaxParser} creates {@link PrefixWildcardQueryNode} nodes which
@@ -43,6 +48,39 @@ import org.apache.lucene.search.PrefixQuery;
  */
 public class WildcardQueryNodeProcessor extends QueryNodeProcessorImpl {
 
+  private static final Pattern WILDCARD_PATTERN = Pattern.compile("(\\.)|([?*]+)");
+
+  // because we call utf8ToString, this will only work with the default TermToBytesRefAttribute
+  private static String analyzeWildcard(Analyzer a, String field, String wildcard) {
+    // best effort to not pass the wildcard characters through #normalize
+    Matcher wildcardMatcher = WILDCARD_PATTERN.matcher(wildcard);
+    StringBuilder sb = new StringBuilder();
+    int last = 0;
+
+    while (wildcardMatcher.find()){
+      // continue if escaped char
+      if (wildcardMatcher.group(1) != null){
+        continue;
+      }
+
+      if (wildcardMatcher.start() > 0){
+        String chunk = wildcard.substring(last, wildcardMatcher.start());
+        BytesRef normalized = a.normalize(field, chunk);
+        sb.append(normalized.utf8ToString());
+      }
+      //append the wildcard character
+      sb.append(wildcardMatcher.group(2));
+
+      last = wildcardMatcher.end();
+    }
+    if (last < wildcard.length()){
+      String chunk = wildcard.substring(last);
+      BytesRef normalized = a.normalize(field, chunk);
+      sb.append(normalized.utf8ToString());
+    }
+    return sb.toString();
+  }
+
   public WildcardQueryNodeProcessor() {
     // empty constructor
   }
@@ -67,15 +105,19 @@ public class WildcardQueryNodeProcessor extends QueryNodeProcessorImpl {
       
       // Code below simulates the old lucene parser behavior for wildcards
       
-      if (isPrefixWildcard(text)) {        
-        PrefixWildcardQueryNode prefixWildcardQN = new PrefixWildcardQueryNode(fqn);
-        return prefixWildcardQN;
-        
-      } else if (isWildcard(text)){
-        WildcardQueryNode wildcardQN = new WildcardQueryNode(fqn);
-        return wildcardQN;
+      
+      if (isWildcard(text)) {
+        Analyzer analyzer = getQueryConfigHandler().get(ConfigurationKeys.ANALYZER);
+        if (analyzer != null) {
+          text = analyzeWildcard(analyzer, fqn.getFieldAsString(), text.toString());
+        }
+        if (isPrefixWildcard(text)) {
+          return new PrefixWildcardQueryNode(fqn.getField(), text, fqn.getBegin(), fqn.getEnd());
+        } else {
+          return new WildcardQueryNode(fqn.getField(), text, fqn.getBegin(), fqn.getEnd());
+        }
       }
-             
+
     }
 
     return node;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
index 3f9d9a4..a417d1b 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
@@ -26,6 +26,7 @@ import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.QueryBuilder;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
@@ -548,7 +549,9 @@ public class SimpleQueryParser extends QueryBuilder {
   protected Query newFuzzyQuery(String text, int fuzziness) {
     BooleanQuery.Builder bq = new BooleanQuery.Builder();
     for (Map.Entry<String,Float> entry : weights.entrySet()) {
-      Query q = new FuzzyQuery(new Term(entry.getKey(), text), fuzziness);
+      final String fieldName = entry.getKey();
+      final BytesRef term = getAnalyzer().normalize(fieldName, text);
+      Query q = new FuzzyQuery(new Term(fieldName, term), fuzziness);
       float boost = entry.getValue();
       if (boost != 1f) {
         q = new BoostQuery(q, boost);
@@ -582,7 +585,9 @@ public class SimpleQueryParser extends QueryBuilder {
   protected Query newPrefixQuery(String text) {
     BooleanQuery.Builder bq = new BooleanQuery.Builder();
     for (Map.Entry<String,Float> entry : weights.entrySet()) {
-      Query q = new PrefixQuery(new Term(entry.getKey(), text));
+      final String fieldName = entry.getKey();
+      final BytesRef term = getAnalyzer().normalize(fieldName, text);
+      Query q = new PrefixQuery(new Term(fieldName, term));
       float boost = entry.getValue();
       if (boost != 1f) {
         q = new BoostQuery(q, boost);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java
deleted file mode 100644
index bf5f69f..0000000
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/analyzing/TestAnalyzingQueryParser.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * 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.lucene.queryparser.analyzing;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.analysis.MockBytesAnalyzer;
-import org.apache.lucene.analysis.MockTokenFilter;
-import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.analysis.TokenFilter;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.queryparser.classic.ParseException;
-import org.apache.lucene.queryparser.classic.QueryParser;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-
-/**
- */
-public class TestAnalyzingQueryParser extends LuceneTestCase {
-  private final static String FIELD = "field";
-   
-  private Analyzer a;
-
-  private String[] wildcardInput;
-  private String[] wildcardExpected;
-  private String[] prefixInput;
-  private String[] prefixExpected;
-  private String[] rangeInput;
-  private String[] rangeExpected;
-  private String[] fuzzyInput;
-  private String[] fuzzyExpected;
-
-  private Map<String, String> wildcardEscapeHits = new TreeMap<>();
-  private Map<String, String> wildcardEscapeMisses = new TreeMap<>();
-
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    wildcardInput = new String[] { "*bersetzung �ber*ung",
-        "M�tley Cr\u00fce M�tl?* Cr�?", "Ren�e Zellweger Ren?? Zellw?ger" };
-    wildcardExpected = new String[] { "*bersetzung uber*ung", "motley crue motl?* cru?",
-        "renee zellweger ren?? zellw?ger" };
-
-    prefixInput = new String[] { "�bersetzung �bersetz*",
-        "M�tley Cr�e M�tl* cr�*", "Ren�? Zellw*" };
-    prefixExpected = new String[] { "ubersetzung ubersetz*", "motley crue motl* cru*",
-        "rene? zellw*" };
-
-    rangeInput = new String[] { "[aa TO bb]", "{Ana�s TO Zo�}" };
-    rangeExpected = new String[] { "[aa TO bb]", "{anais TO zoe}" };
-
-    fuzzyInput = new String[] { "�bersetzung �bersetzung~0.9",
-        "M�tley Cr�e M�tley~0.75 Cr�e~0.5",
-        "Ren�e Zellweger Ren�e~0.9 Zellweger~" };
-    fuzzyExpected = new String[] { "ubersetzung ubersetzung~1",
-        "motley crue motley~1 crue~2", "renee zellweger renee~0 zellweger~2" };
-
-    wildcardEscapeHits.put("m�*tley", "moatley");
-
-    // need to have at least one genuine wildcard to trigger the wildcard analysis
-    // hence the * before the y
-    wildcardEscapeHits.put("m�\\*tl*y", "mo*tley");
-
-    // escaped backslash then true wildcard
-    wildcardEscapeHits.put("m�\\\\*tley", "mo\\atley");
-    
-    // escaped wildcard then true wildcard
-    wildcardEscapeHits.put("m�\\??ley", "mo?tley");
-
-    // the first is an escaped * which should yield a miss
-    wildcardEscapeMisses.put("m�\\*tl*y", "moatley");
-      
-    a = new ASCIIAnalyzer();
-  }
-
-  public void testSingleChunkExceptions() {
-    String termStr = "the*tre";
-      
-    Analyzer stopsAnalyzer = new MockAnalyzer
-        (random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET);
-
-    ParseException expected = expectThrows(ParseException.class, () -> {
-      parseWithAnalyzingQueryParser(termStr, stopsAnalyzer, true);
-    });
-    assertTrue(expected.getMessage().contains("returned nothing"));
-     
-    AnalyzingQueryParser qp = new AnalyzingQueryParser(FIELD, a);
-    expected = expectThrows(ParseException.class, () -> {
-      qp.analyzeSingleChunk(FIELD, "", "not a single chunk");
-    });
-    assertTrue(expected.getMessage().contains("multiple terms"));
-  }
-   
-  public void testWildcardAlone() throws ParseException {
-    //seems like crazy edge case, but can be useful in concordance 
-    expectThrows(ParseException.class, () -> {
-      getAnalyzedQuery("*", a, false);
-    });
-      
-    String qString = parseWithAnalyzingQueryParser("*", a, true);
-    assertEquals("Every word", "*", qString);
-  }
-  public void testWildCardEscapes() throws ParseException, IOException {
-
-    for (Map.Entry<String, String> entry : wildcardEscapeHits.entrySet()){
-      Query q = getAnalyzedQuery(entry.getKey(), a, false);
-      assertEquals("WildcardEscapeHits: " + entry.getKey(), true, isAHit(q, entry.getValue(), a));
-    }
-    for (Map.Entry<String, String> entry : wildcardEscapeMisses.entrySet()){
-      Query q = getAnalyzedQuery(entry.getKey(), a, false);
-      assertEquals("WildcardEscapeMisses: " + entry.getKey(), false, isAHit(q, entry.getValue(), a));
-    }
-
-  }
-  public void testWildCardQueryNoLeadingAllowed() {
-    expectThrows(ParseException.class, () -> {
-      parseWithAnalyzingQueryParser(wildcardInput[0], a, false);
-    });
-  }
-
-  public void testWildCardQuery() throws ParseException {
-    for (int i = 0; i < wildcardInput.length; i++) {
-      assertEquals("Testing wildcards with analyzer " + a.getClass() + ", input string: "
-          + wildcardInput[i], wildcardExpected[i], parseWithAnalyzingQueryParser(wildcardInput[i], a, true));
-    }
-  }
-
-
-  public void testPrefixQuery() throws ParseException {
-    for (int i = 0; i < prefixInput.length; i++) {
-      assertEquals("Testing prefixes with analyzer " + a.getClass() + ", input string: "
-          + prefixInput[i], prefixExpected[i], parseWithAnalyzingQueryParser(prefixInput[i], a, false));
-    }
-  }
-
-  public void testRangeQuery() throws ParseException {
-    for (int i = 0; i < rangeInput.length; i++) {
-      assertEquals("Testing ranges with analyzer " + a.getClass() + ", input string: "
-          + rangeInput[i], rangeExpected[i], parseWithAnalyzingQueryParser(rangeInput[i], a, false));
-    }
-  }
-
-  public void testFuzzyQuery() throws ParseException {
-    for (int i = 0; i < fuzzyInput.length; i++) {
-      assertEquals("Testing fuzzys with analyzer " + a.getClass() + ", input string: "
-          + fuzzyInput[i], fuzzyExpected[i], parseWithAnalyzingQueryParser(fuzzyInput[i], a, false));
-    }
-  }
-
-
-  private String parseWithAnalyzingQueryParser(String s, Analyzer a, boolean allowLeadingWildcard) throws ParseException {
-    Query q = getAnalyzedQuery(s, a, allowLeadingWildcard);
-    return q.toString(FIELD);
-  }
-
-  private Query getAnalyzedQuery(String s, Analyzer a, boolean allowLeadingWildcard) throws ParseException {
-    AnalyzingQueryParser qp = new AnalyzingQueryParser(FIELD, a);
-    qp.setAllowLeadingWildcard(allowLeadingWildcard);
-    org.apache.lucene.search.Query q = qp.parse(s);
-    return q;
-  }
-
-  final static class FoldingFilter extends TokenFilter {
-    final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
-
-    public FoldingFilter(TokenStream input) {
-      super(input);
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (input.incrementToken()) {
-        char term[] = termAtt.buffer();
-        for (int i = 0; i < term.length; i++)
-          switch(term[i]) {
-            case '�':
-              term[i] = 'u'; 
-              break;
-            case '�': 
-              term[i] = 'o'; 
-              break;
-            case '�': 
-              term[i] = 'e'; 
-              break;
-            case '�': 
-              term[i] = 'i'; 
-              break;
-          }
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-  final static class ASCIIAnalyzer extends Analyzer {
-    @Override
-    public TokenStreamComponents createComponents(String fieldName) {
-      Tokenizer result = new MockTokenizer(MockTokenizer.WHITESPACE, true);
-      return new TokenStreamComponents(result, new FoldingFilter(result));
-    }
-  }
-   
-
-  // LUCENE-4176
-  public void testByteTerms() throws Exception {
-    String s = "\u0e40\u0e02";
-    Analyzer analyzer = new MockBytesAnalyzer();
-    QueryParser qp = new AnalyzingQueryParser(FIELD, analyzer);
-    Query q = qp.parse("[\u0e40\u0e02 TO \u0e40\u0e02]");
-    assertEquals(true, isAHit(q, s, analyzer));
-  }
-   
-  
-  private boolean isAHit(Query q, String content, Analyzer analyzer) throws IOException{
-    Directory ramDir = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), ramDir, analyzer);
-    Document doc = new Document();
-    FieldType fieldType = new FieldType();
-    fieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-    fieldType.setTokenized(true);
-    fieldType.setStored(true);
-    Field field = new Field(FIELD, content, fieldType);
-    doc.add(field);
-    writer.addDocument(doc);
-    writer.close();
-    DirectoryReader ir = DirectoryReader.open(ramDir);
-    IndexSearcher is = new IndexSearcher(ir);
-      
-    int hits = is.search(q, 10).totalHits;
-    ir.close();
-    ramDir.close();
-    if (hits == 1){
-      return true;
-    } else {
-      return false;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
index 7a98800..de90e29 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
@@ -18,6 +18,8 @@ package org.apache.lucene.queryparser.classic;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockBytesAnalyzer;
+import org.apache.lucene.analysis.MockLowerCaseFilter;
 import org.apache.lucene.analysis.MockSynonymAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenFilter;
@@ -25,7 +27,13 @@ import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.DateTools.Resolution;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryparser.classic.QueryParser.Operator;
 import org.apache.lucene.queryparser.flexible.standard.CommonQueryParserConfiguration;
@@ -33,11 +41,14 @@ import org.apache.lucene.queryparser.util.QueryParserTestBase;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MultiPhraseQuery;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SynonymQuery;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.automaton.TooComplexToDeterminizeException;
 
 import java.io.IOException;
@@ -48,6 +59,7 @@ import java.io.IOException;
 public class TestQueryParser extends QueryParserTestBase {
 
   protected boolean splitOnWhitespace = QueryParser.DEFAULT_SPLIT_ON_WHITESPACE;
+  private static final String FIELD = "field";
 
   public static class QPTestParser extends QueryParser {
     public QPTestParser(String f, Analyzer a) {
@@ -115,14 +127,6 @@ public class TestQueryParser extends QueryParserTestBase {
   }
   
   @Override
-  public void setAnalyzeRangeTerms(CommonQueryParserConfiguration cqpC,
-      boolean value) {
-    assert (cqpC instanceof QueryParser);
-    QueryParser qp = (QueryParser) cqpC;
-    qp.setAnalyzeRangeTerms(value);
-  }
-  
-  @Override
   public void setAutoGeneratePhraseQueries(CommonQueryParserConfiguration cqpC,
       boolean value) {
     assert (cqpC instanceof QueryParser);
@@ -200,7 +204,7 @@ public class TestQueryParser extends QueryParserTestBase {
   @Override
   public void testStarParsing() throws Exception {
     final int[] type = new int[1];
-    QueryParser qp = new QueryParser("field",
+    QueryParser qp = new QueryParser(FIELD,
         new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)) {
       @Override
       protected Query getWildcardQuery(String field, String termStr) {
@@ -285,7 +289,7 @@ public class TestQueryParser extends QueryParserTestBase {
     Analyzer morePrecise = new Analyzer2();
     
     public SmartQueryParser() {
-      super("field", new Analyzer1());
+      super(FIELD, new Analyzer1());
     }
     
     @Override
@@ -299,9 +303,9 @@ public class TestQueryParser extends QueryParserTestBase {
   @Override
   public void testNewFieldQuery() throws Exception {
     /** ordinary behavior, synonyms form uncoordinated boolean query */
-    QueryParser dumb = new QueryParser("field",
+    QueryParser dumb = new QueryParser(FIELD,
         new Analyzer1());
-    Query expanded = new SynonymQuery(new Term("field", "dogs"), new Term("field", "dog"));
+    Query expanded = new SynonymQuery(new Term(FIELD, "dogs"), new Term(FIELD, "dog"));
     assertEquals(expanded, dumb.parse("\"dogs\""));
     /** even with the phrase operator the behavior is the same */
     assertEquals(expanded, dumb.parse("dogs"));
@@ -312,14 +316,14 @@ public class TestQueryParser extends QueryParserTestBase {
     QueryParser smart = new SmartQueryParser();
     assertEquals(expanded, smart.parse("dogs"));
     
-    Query unexpanded = new TermQuery(new Term("field", "dogs"));
+    Query unexpanded = new TermQuery(new Term(FIELD, "dogs"));
     assertEquals(unexpanded, smart.parse("\"dogs\""));
   }
 
   /** simple synonyms test */
   public void testSynonyms() throws Exception {
-    Query expected = new SynonymQuery(new Term("field", "dogs"), new Term("field", "dog"));
-    QueryParser qp = new QueryParser("field", new MockSynonymAnalyzer());
+    Query expected = new SynonymQuery(new Term(FIELD, "dogs"), new Term(FIELD, "dog"));
+    QueryParser qp = new QueryParser(FIELD, new MockSynonymAnalyzer());
     assertEquals(expected, qp.parse("dogs"));
     assertEquals(expected, qp.parse("\"dogs\""));
     qp.setDefaultOperator(Operator.AND);
@@ -333,9 +337,9 @@ public class TestQueryParser extends QueryParserTestBase {
   /** forms multiphrase query */
   public void testSynonymsPhrase() throws Exception {
     MultiPhraseQuery.Builder expectedQBuilder = new MultiPhraseQuery.Builder();
-    expectedQBuilder.add(new Term("field", "old"));
-    expectedQBuilder.add(new Term[] { new Term("field", "dogs"), new Term("field", "dog") });
-    QueryParser qp = new QueryParser("field", new MockSynonymAnalyzer());
+    expectedQBuilder.add(new Term(FIELD, "old"));
+    expectedQBuilder.add(new Term[] { new Term(FIELD, "dogs"), new Term(FIELD, "dog") });
+    QueryParser qp = new QueryParser(FIELD, new MockSynonymAnalyzer());
     assertEquals(expectedQBuilder.build(), qp.parse("\"old dogs\""));
     qp.setDefaultOperator(Operator.AND);
     assertEquals(expectedQBuilder.build(), qp.parse("\"old dogs\""));
@@ -387,8 +391,8 @@ public class TestQueryParser extends QueryParserTestBase {
   
   /** simple CJK synonym test */
   public void testCJKSynonym() throws Exception {
-    Query expected = new SynonymQuery(new Term("field", "\u56fd"), new Term("field", "\u570b"));
-    QueryParser qp = new QueryParser("field", new MockCJKSynonymAnalyzer());
+    Query expected = new SynonymQuery(new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b"));
+    QueryParser qp = new QueryParser(FIELD, new MockCJKSynonymAnalyzer());
     assertEquals(expected, qp.parse("\u56fd"));
     qp.setDefaultOperator(Operator.AND);
     assertEquals(expected, qp.parse("\u56fd"));
@@ -399,11 +403,11 @@ public class TestQueryParser extends QueryParserTestBase {
   /** synonyms with default OR operator */
   public void testCJKSynonymsOR() throws Exception {
     BooleanQuery.Builder expectedB = new BooleanQuery.Builder();
-    expectedB.add(new TermQuery(new Term("field", "\u4e2d")), BooleanClause.Occur.SHOULD);
-    Query inner = new SynonymQuery(new Term("field", "\u56fd"), new Term("field", "\u570b"));
+    expectedB.add(new TermQuery(new Term(FIELD, "\u4e2d")), BooleanClause.Occur.SHOULD);
+    Query inner = new SynonymQuery(new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b"));
     expectedB.add(inner, BooleanClause.Occur.SHOULD);
     Query expected = expectedB.build();
-    QueryParser qp = new QueryParser("field", new MockCJKSynonymAnalyzer());
+    QueryParser qp = new QueryParser(FIELD, new MockCJKSynonymAnalyzer());
     assertEquals(expected, qp.parse("\u4e2d\u56fd"));
     expected = new BoostQuery(expected, 2f);
     assertEquals(expected, qp.parse("\u4e2d\u56fd^2"));
@@ -412,13 +416,13 @@ public class TestQueryParser extends QueryParserTestBase {
   /** more complex synonyms with default OR operator */
   public void testCJKSynonymsOR2() throws Exception {
     BooleanQuery.Builder expectedB = new BooleanQuery.Builder();
-    expectedB.add(new TermQuery(new Term("field", "\u4e2d")), BooleanClause.Occur.SHOULD);
-    SynonymQuery inner = new SynonymQuery(new Term("field", "\u56fd"), new Term("field", "\u570b"));
+    expectedB.add(new TermQuery(new Term(FIELD, "\u4e2d")), BooleanClause.Occur.SHOULD);
+    SynonymQuery inner = new SynonymQuery(new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b"));
     expectedB.add(inner, BooleanClause.Occur.SHOULD);
-    SynonymQuery inner2 = new SynonymQuery(new Term("field", "\u56fd"), new Term("field", "\u570b"));
+    SynonymQuery inner2 = new SynonymQuery(new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b"));
     expectedB.add(inner2, BooleanClause.Occur.SHOULD);
     Query expected = expectedB.build();
-    QueryParser qp = new QueryParser("field", new MockCJKSynonymAnalyzer());
+    QueryParser qp = new QueryParser(FIELD, new MockCJKSynonymAnalyzer());
     assertEquals(expected, qp.parse("\u4e2d\u56fd\u56fd"));
     expected = new BoostQuery(expected, 2f);
     assertEquals(expected, qp.parse("\u4e2d\u56fd\u56fd^2"));
@@ -427,11 +431,11 @@ public class TestQueryParser extends QueryParserTestBase {
   /** synonyms with default AND operator */
   public void testCJKSynonymsAND() throws Exception {
     BooleanQuery.Builder expectedB = new BooleanQuery.Builder();
-    expectedB.add(new TermQuery(new Term("field", "\u4e2d")), BooleanClause.Occur.MUST);
-    Query inner = new SynonymQuery(new Term("field", "\u56fd"), new Term("field", "\u570b"));
+    expectedB.add(new TermQuery(new Term(FIELD, "\u4e2d")), BooleanClause.Occur.MUST);
+    Query inner = new SynonymQuery(new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b"));
     expectedB.add(inner, BooleanClause.Occur.MUST);
     Query expected = expectedB.build();
-    QueryParser qp = new QueryParser("field", new MockCJKSynonymAnalyzer());
+    QueryParser qp = new QueryParser(FIELD, new MockCJKSynonymAnalyzer());
     qp.setDefaultOperator(Operator.AND);
     assertEquals(expected, qp.parse("\u4e2d\u56fd"));
     expected = new BoostQuery(expected, 2f);
@@ -441,13 +445,13 @@ public class TestQueryParser extends QueryParserTestBase {
   /** more complex synonyms with default AND operator */
   public void testCJKSynonymsAND2() throws Exception {
     BooleanQuery.Builder expectedB = new BooleanQuery.Builder();
-    expectedB.add(new TermQuery(new Term("field", "\u4e2d")), BooleanClause.Occur.MUST);
-    Query inner = new SynonymQuery(new Term("field", "\u56fd"), new Term("field", "\u570b"));
+    expectedB.add(new TermQuery(new Term(FIELD, "\u4e2d")), BooleanClause.Occur.MUST);
+    Query inner = new SynonymQuery(new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b"));
     expectedB.add(inner, BooleanClause.Occur.MUST);
-    Query inner2 = new SynonymQuery(new Term("field", "\u56fd"), new Term("field", "\u570b"));
+    Query inner2 = new SynonymQuery(new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b"));
     expectedB.add(inner2, BooleanClause.Occur.MUST);
     Query expected = expectedB.build();
-    QueryParser qp = new QueryParser("field", new MockCJKSynonymAnalyzer());
+    QueryParser qp = new QueryParser(FIELD, new MockCJKSynonymAnalyzer());
     qp.setDefaultOperator(Operator.AND);
     assertEquals(expected, qp.parse("\u4e2d\u56fd\u56fd"));
     expected = new BoostQuery(expected, 2f);
@@ -457,9 +461,9 @@ public class TestQueryParser extends QueryParserTestBase {
   /** forms multiphrase query */
   public void testCJKSynonymsPhrase() throws Exception {
     MultiPhraseQuery.Builder expectedQBuilder = new MultiPhraseQuery.Builder();
-    expectedQBuilder.add(new Term("field", "\u4e2d"));
-    expectedQBuilder.add(new Term[] { new Term("field", "\u56fd"), new Term("field", "\u570b")});
-    QueryParser qp = new QueryParser("field", new MockCJKSynonymAnalyzer());
+    expectedQBuilder.add(new Term(FIELD, "\u4e2d"));
+    expectedQBuilder.add(new Term[] { new Term(FIELD, "\u56fd"), new Term(FIELD, "\u570b")});
+    QueryParser qp = new QueryParser(FIELD, new MockCJKSynonymAnalyzer());
     qp.setDefaultOperator(Operator.AND);
     assertEquals(expectedQBuilder.build(), qp.parse("\"\u4e2d\u56fd\""));
     Query expected = new BoostQuery(expectedQBuilder.build(), 2f);
@@ -471,7 +475,7 @@ public class TestQueryParser extends QueryParserTestBase {
 
   /** LUCENE-6677: make sure wildcard query respects maxDeterminizedStates. */
   public void testWildcardMaxDeterminizedStates() throws Exception {
-    QueryParser qp = new QueryParser("field", new MockAnalyzer(random()));
+    QueryParser qp = new QueryParser(FIELD, new MockAnalyzer(random()));
     qp.setMaxDeterminizedStates(10);
     expectThrows(TooComplexToDeterminizeException.class, () -> {
       qp.parse("a*aaaaaaa");
@@ -703,4 +707,163 @@ public class TestQueryParser extends QueryParserTestBase {
     assertQueryEquals("guinea pig", new MockSynonymAnalyzer(), "Synonym(cavy guinea) pig");
     splitOnWhitespace = oldSplitOnWhitespace;
   }
-}
\ No newline at end of file
+   
+  public void testWildcardAlone() throws ParseException {
+    //seems like crazy edge case, but can be useful in concordance 
+    QueryParser parser = new QueryParser(FIELD, new ASCIIAnalyzer());
+    parser.setAllowLeadingWildcard(false);
+    expectThrows(ParseException.class, () -> {
+      parser.parse("*");
+    });
+
+    QueryParser parser2 = new QueryParser("*", new ASCIIAnalyzer());
+    parser2.setAllowLeadingWildcard(false);
+    assertEquals(new MatchAllDocsQuery(), parser2.parse("*"));
+  }
+
+  public void testWildCardEscapes() throws ParseException, IOException {
+    Analyzer a = new ASCIIAnalyzer();
+    QueryParser parser = new QueryParser(FIELD, a);
+    assertTrue(isAHit(parser.parse("m�*tley"), "moatley", a));
+    // need to have at least one genuine wildcard to trigger the wildcard analysis
+    // hence the * before the y
+    assertTrue(isAHit(parser.parse("m�\\*tl*y"), "mo*tley", a));
+    // escaped backslash then true wildcard
+    assertTrue(isAHit(parser.parse("m�\\\\*tley"), "mo\\atley", a));
+    // escaped wildcard then true wildcard
+    assertTrue(isAHit(parser.parse("m�\\??ley"), "mo?tley", a));
+
+    // the first is an escaped * which should yield a miss
+    assertFalse(isAHit(parser.parse("m�\\*tl*y"), "moatley", a));
+  }
+
+  public void testWildcardDoesNotNormalizeEscapedChars() throws Exception {
+    Analyzer asciiAnalyzer = new ASCIIAnalyzer();
+    Analyzer keywordAnalyzer = new MockAnalyzer(random());
+    QueryParser parser = new QueryParser(FIELD, asciiAnalyzer);
+
+    assertTrue(isAHit(parser.parse("e*e"), "�tude", asciiAnalyzer));
+    assertTrue(isAHit(parser.parse("�*e"), "etude", asciiAnalyzer));
+    assertFalse(isAHit(parser.parse("\\�*e"), "etude", asciiAnalyzer));
+    assertTrue(isAHit(parser.parse("\\�*e"), "�tude", keywordAnalyzer));
+  }
+
+  public void testWildCardQuery() throws ParseException {
+    Analyzer a = new ASCIIAnalyzer();
+    QueryParser parser = new QueryParser(FIELD, a);
+    parser.setAllowLeadingWildcard(true);
+    assertEquals("*bersetzung uber*ung", parser.parse("*bersetzung �ber*ung").toString(FIELD));
+    parser.setAllowLeadingWildcard(false);
+    assertEquals("motley crue motl?* cru?", parser.parse("M�tley Cr\u00fce M�tl?* Cr�?").toString(FIELD));
+    assertEquals("renee zellweger ren?? zellw?ger", parser.parse("Ren�e Zellweger Ren?? Zellw?ger").toString(FIELD));
+  }
+
+
+  public void testPrefixQuery() throws ParseException {
+    Analyzer a = new ASCIIAnalyzer();
+    QueryParser parser = new QueryParser(FIELD, a);
+    assertEquals("ubersetzung ubersetz*", parser.parse("�bersetzung �bersetz*").toString(FIELD));
+    assertEquals("motley crue motl* cru*", parser.parse("M�tley Cr�e M�tl* cr�*").toString(FIELD));
+    assertEquals("rene? zellw*", parser.parse("Ren�? Zellw*").toString(FIELD));
+  }
+
+  public void testRangeQuery() throws ParseException {
+    Analyzer a = new ASCIIAnalyzer();
+    QueryParser parser = new QueryParser(FIELD, a);
+    assertEquals("[aa TO bb]", parser.parse("[aa TO bb]").toString(FIELD));
+    assertEquals("{anais TO zoe}", parser.parse("{Ana�s TO Zo�}").toString(FIELD));
+  }
+
+  public void testFuzzyQuery() throws ParseException {
+    Analyzer a = new ASCIIAnalyzer();
+    QueryParser parser = new QueryParser(FIELD, a);
+    assertEquals("ubersetzung ubersetzung~1", parser.parse("�bersetzung �bersetzung~0.9").toString(FIELD));
+    assertEquals("motley crue motley~1 crue~2", parser.parse("M�tley Cr�e M�tley~0.75 Cr�e~0.5").toString(FIELD));
+    assertEquals("renee zellweger renee~0 zellweger~2", parser.parse("Ren�e Zellweger Ren�e~0.9 Zellweger~").toString(FIELD));
+  }
+
+  final static class FoldingFilter extends TokenFilter {
+    final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+    public FoldingFilter(TokenStream input) {
+      super(input);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (input.incrementToken()) {
+        char term[] = termAtt.buffer();
+        for (int i = 0; i < term.length; i++)
+          switch(term[i]) {
+            case '�':
+              term[i] = 'u'; 
+              break;
+            case '�': 
+              term[i] = 'o'; 
+              break;
+            case '�': 
+              term[i] = 'e'; 
+              break;
+            case '�': 
+              term[i] = 'i'; 
+              break;
+          }
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  final static class ASCIIAnalyzer extends Analyzer {
+    @Override
+    public TokenStreamComponents createComponents(String fieldName) {
+      Tokenizer result = new MockTokenizer(MockTokenizer.WHITESPACE, true);
+      return new TokenStreamComponents(result, new FoldingFilter(result));
+    }
+    @Override
+    protected TokenStream normalize(String fieldName, TokenStream in) {
+      return new FoldingFilter(new MockLowerCaseFilter(in));
+    }
+  }
+
+  // LUCENE-4176
+  public void testByteTerms() throws Exception {
+    String s = "\u0e40\u0e02";
+    Analyzer analyzer = new MockBytesAnalyzer();
+    QueryParser qp = new QueryParser(FIELD, analyzer);
+
+    assertTrue(isAHit(qp.parse("[\u0e40\u0e02 TO \u0e40\u0e02]"), s, analyzer));
+    assertTrue(isAHit(qp.parse("\u0e40\u0e02~1"), s, analyzer));
+    assertTrue(isAHit(qp.parse("\u0e40\u0e02*"), s, analyzer));
+    assertTrue(isAHit(qp.parse("\u0e40*"), s, analyzer));
+    assertTrue(isAHit(qp.parse("\u0e40??"), s, analyzer));
+  }
+   
+  
+  private boolean isAHit(Query q, String content, Analyzer analyzer) throws IOException{
+    Directory ramDir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), ramDir, analyzer);
+    Document doc = new Document();
+    FieldType fieldType = new FieldType();
+    fieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+    fieldType.setTokenized(true);
+    fieldType.setStored(true);
+    Field field = new Field(FIELD, content, fieldType);
+    doc.add(field);
+    writer.addDocument(doc);
+    writer.close();
+    DirectoryReader ir = DirectoryReader.open(ramDir);
+    IndexSearcher is = new IndexSearcher(ir);
+      
+    int hits = is.search(q, 10).totalHits;
+    ir.close();
+    ramDir.close();
+    if (hits == 1){
+      return true;
+    } else {
+      return false;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
index 88e8b9b..d2deaa6 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
@@ -181,20 +181,7 @@ public class TestPrecedenceQueryParser extends LuceneTestCase {
     }
   }
 
-  public void assertWildcardQueryEquals(String query, boolean lowercase,
-      String result) throws Exception {
-    PrecedenceQueryParser qp = getParser(null);
-    qp.setLowercaseExpandedTerms(lowercase);
-    Query q = qp.parse(query, "field");
-    String s = q.toString("field");
-    if (!s.equals(result)) {
-      fail("WildcardQuery /" + query + "/ yielded /" + s + "/, expecting /"
-          + result + "/");
-    }
-  }
-
-  public void assertWildcardQueryEquals(String query, String result)
-      throws Exception {
+  public void assertWildcardQueryEquals(String query, String result) throws Exception {
     PrecedenceQueryParser qp = getParser(null);
     Query q = qp.parse(query, "field");
     String s = q.toString("field");
@@ -339,36 +326,23 @@ public class TestPrecedenceQueryParser extends LuceneTestCase {
      */
     // First prefix queries:
     // by default, convert to lowercase:
-    assertWildcardQueryEquals("Term*", true, "term*");
+    assertWildcardQueryEquals("Term*", "term*");
     // explicitly set lowercase:
-    assertWildcardQueryEquals("term*", true, "term*");
-    assertWildcardQueryEquals("Term*", true, "term*");
-    assertWildcardQueryEquals("TERM*", true, "term*");
-    // explicitly disable lowercase conversion:
-    assertWildcardQueryEquals("term*", false, "term*");
-    assertWildcardQueryEquals("Term*", false, "Term*");
-    assertWildcardQueryEquals("TERM*", false, "TERM*");
+    assertWildcardQueryEquals("term*", "term*");
+    assertWildcardQueryEquals("Term*", "term*");
+    assertWildcardQueryEquals("TERM*", "term*");
     // Then 'full' wildcard queries:
     // by default, convert to lowercase:
     assertWildcardQueryEquals("Te?m", "te?m");
     // explicitly set lowercase:
-    assertWildcardQueryEquals("te?m", true, "te?m");
-    assertWildcardQueryEquals("Te?m", true, "te?m");
-    assertWildcardQueryEquals("TE?M", true, "te?m");
-    assertWildcardQueryEquals("Te?m*gerM", true, "te?m*germ");
-    // explicitly disable lowercase conversion:
-    assertWildcardQueryEquals("te?m", false, "te?m");
-    assertWildcardQueryEquals("Te?m", false, "Te?m");
-    assertWildcardQueryEquals("TE?M", false, "TE?M");
-    assertWildcardQueryEquals("Te?m*gerM", false, "Te?m*gerM");
+    assertWildcardQueryEquals("te?m", "te?m");
+    assertWildcardQueryEquals("Te?m", "te?m");
+    assertWildcardQueryEquals("TE?M", "te?m");
+    assertWildcardQueryEquals("Te?m*gerM", "te?m*germ");
     // Fuzzy queries:
     assertWildcardQueryEquals("Term~", "term~2");
-    assertWildcardQueryEquals("Term~", true, "term~2");
-    assertWildcardQueryEquals("Term~", false, "Term~2");
     // Range queries:
     assertWildcardQueryEquals("[A TO C]", "[a TO c]");
-    assertWildcardQueryEquals("[A TO C]", true, "[a TO c]");
-    assertWildcardQueryEquals("[A TO C]", false, "[A TO C]");
   }
 
   public void testQPA() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
index 91b799d..2d5ee43 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
@@ -288,10 +288,9 @@ public class TestQPHelper extends LuceneTestCase {
     }
   }
 
-  public void assertWildcardQueryEquals(String query, boolean lowercase,
+  public void assertWildcardQueryEquals(String query,
       String result, boolean allowLeadingWildcard) throws Exception {
     StandardQueryParser qp = getParser(null);
-    qp.setLowercaseExpandedTerms(lowercase);
     qp.setAllowLeadingWildcard(allowLeadingWildcard);
     Query q = qp.parse(query, "field");
     String s = q.toString("field");
@@ -301,20 +300,9 @@ public class TestQPHelper extends LuceneTestCase {
     }
   }
 
-  public void assertWildcardQueryEquals(String query, boolean lowercase,
+  public void assertWildcardQueryEquals(String query,
       String result) throws Exception {
-    assertWildcardQueryEquals(query, lowercase, result, false);
-  }
-
-  public void assertWildcardQueryEquals(String query, String result)
-      throws Exception {
-    StandardQueryParser qp = getParser(null);
-    Query q = qp.parse(query, "field");
-    String s = q.toString("field");
-    if (!s.equals(result)) {
-      fail("WildcardQuery /" + query + "/ yielded /" + s + "/, expecting /"
-          + result + "/");
-    }
+    assertWildcardQueryEquals(query, result, false);
   }
 
   public Query getQueryDOA(String query, Analyzer a) throws Exception {
@@ -597,32 +585,21 @@ public class TestQPHelper extends LuceneTestCase {
      */
     // First prefix queries:
     // by default, convert to lowercase:
-    assertWildcardQueryEquals("Term*", true, "term*");
+    assertWildcardQueryEquals("Term*", "term*");
     // explicitly set lowercase:
-    assertWildcardQueryEquals("term*", true, "term*");
-    assertWildcardQueryEquals("Term*", true, "term*");
-    assertWildcardQueryEquals("TERM*", true, "term*");
-    // explicitly disable lowercase conversion:
-    assertWildcardQueryEquals("term*", false, "term*");
-    assertWildcardQueryEquals("Term*", false, "Term*");
-    assertWildcardQueryEquals("TERM*", false, "TERM*");
+    assertWildcardQueryEquals("term*", "term*");
+    assertWildcardQueryEquals("Term*", "term*");
+    assertWildcardQueryEquals("TERM*", "term*");
     // Then 'full' wildcard queries:
     // by default, convert to lowercase:
     assertWildcardQueryEquals("Te?m", "te?m");
     // explicitly set lowercase:
-    assertWildcardQueryEquals("te?m", true, "te?m");
-    assertWildcardQueryEquals("Te?m", true, "te?m");
-    assertWildcardQueryEquals("TE?M", true, "te?m");
-    assertWildcardQueryEquals("Te?m*gerM", true, "te?m*germ");
-    // explicitly disable lowercase conversion:
-    assertWildcardQueryEquals("te?m", false, "te?m");
-    assertWildcardQueryEquals("Te?m", false, "Te?m");
-    assertWildcardQueryEquals("TE?M", false, "TE?M");
-    assertWildcardQueryEquals("Te?m*gerM", false, "Te?m*gerM");
+    assertWildcardQueryEquals("te?m", "te?m");
+    assertWildcardQueryEquals("Te?m", "te?m");
+    assertWildcardQueryEquals("TE?M", "te?m");
+    assertWildcardQueryEquals("Te?m*gerM", "te?m*germ");
     // Fuzzy queries:
     assertWildcardQueryEquals("Term~", "term~2");
-    assertWildcardQueryEquals("Term~", true, "term~2");
-    assertWildcardQueryEquals("Term~", false, "Term~2");
     // Range queries:
 
     // TODO: implement this on QueryParser
@@ -630,20 +607,18 @@ public class TestQPHelper extends LuceneTestCase {
     // C]': Lexical error at line 1, column 1. Encountered: "[" (91), after
     // : ""
     assertWildcardQueryEquals("[A TO C]", "[a TO c]");
-    assertWildcardQueryEquals("[A TO C]", true, "[a TO c]");
-    assertWildcardQueryEquals("[A TO C]", false, "[A TO C]");
     // Test suffix queries: first disallow
     expectThrows(QueryNodeException.class, () -> {
-      assertWildcardQueryEquals("*Term", true, "*term");
+      assertWildcardQueryEquals("*Term", "*term");
     });
 
     expectThrows(QueryNodeException.class, () -> {
-      assertWildcardQueryEquals("?Term", true, "?term");
+      assertWildcardQueryEquals("?Term", "?term");
     });
 
     // Test suffix queries: then allow
-    assertWildcardQueryEquals("*Term", true, "*term", true);
-    assertWildcardQueryEquals("?Term", true, "?term", true);
+    assertWildcardQueryEquals("*Term", "*term", true);
+    assertWildcardQueryEquals("?Term", "?term", true);
   }
 
   public void testLeadingWildcardType() throws Exception {
@@ -1159,10 +1134,10 @@ public class TestQPHelper extends LuceneTestCase {
   
   public void testRegexps() throws Exception {
     StandardQueryParser qp = new StandardQueryParser();
+    qp.setAnalyzer(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true));
     final String df = "field" ;
     RegexpQuery q = new RegexpQuery(new Term("field", "[a-z][123]"));
     assertEquals(q, qp.parse("/[a-z][123]/", df));
-    qp.setLowercaseExpandedTerms(true);
     assertEquals(q, qp.parse("/[A-Z][123]/", df));
     assertEquals(new BoostQuery(q, 0.5f), qp.parse("/[A-Z][123]/^0.5", df));
     qp.setMultiTermRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java
index f678796..7e50eeb 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java
@@ -88,12 +88,6 @@ public class TestStandardQP extends QueryParserTestBase {
   }
   
   @Override
-  public void setAnalyzeRangeTerms(CommonQueryParserConfiguration cqpC,
-      boolean value) {
-    throw new UnsupportedOperationException();
-  }
-  
-  @Override
   public void setAutoGeneratePhraseQueries(CommonQueryParserConfiguration cqpC,
       boolean value) {
     throw new UnsupportedOperationException();
@@ -149,15 +143,6 @@ public class TestStandardQP extends QueryParserTestBase {
     WildcardQuery q = new WildcardQuery(new Term("field", "foo?ba?r"));//TODO not correct!!
     assertEquals(q, getQuery("foo\\?ba?r", qp));
   }
-
-  
-  @Override
-  public void testCollatedRange() throws Exception {
-    expectThrows(UnsupportedOperationException.class, () -> {
-      setAnalyzeRangeTerms(getParser(null), true);
-      super.testCollatedRange();
-    });
-  }
   
   @Override
   public void testAutoGeneratePhraseQueriesOn() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
index d58f660..2170193 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
@@ -145,8 +145,6 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
 
   public abstract void setDefaultOperatorAND(CommonQueryParserConfiguration cqpC);
 
-  public abstract void setAnalyzeRangeTerms(CommonQueryParserConfiguration cqpC, boolean value);
-
   public abstract void setAutoGeneratePhraseQueries(CommonQueryParserConfiguration cqpC, boolean value);
 
   public abstract void setDateResolution(CommonQueryParserConfiguration cqpC, CharSequence field, DateTools.Resolution value);
@@ -203,10 +201,9 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
     }
   }
 
-  public void assertWildcardQueryEquals(String query, boolean lowercase, String result, boolean allowLeadingWildcard)
+  public void assertWildcardQueryEquals(String query, String result, boolean allowLeadingWildcard)
     throws Exception {
     CommonQueryParserConfiguration cqpC = getParserConfig(null);
-    cqpC.setLowercaseExpandedTerms(lowercase);
     cqpC.setAllowLeadingWildcard(allowLeadingWildcard);
     Query q = getQuery(query, cqpC);
     String s = q.toString("field");
@@ -216,18 +213,9 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
     }
   }
 
-  public void assertWildcardQueryEquals(String query, boolean lowercase, String result)
+  public void assertWildcardQueryEquals(String query, String result)
     throws Exception {
-    assertWildcardQueryEquals(query, lowercase, result, false);
-  }
-
-  public void assertWildcardQueryEquals(String query, String result) throws Exception {
-    Query q = getQuery(query);
-    String s = q.toString("field");
-    if (!s.equals(result)) {
-      fail("WildcardQuery /" + query + "/ yielded /" + s + "/, expecting /"
-          + result + "/");
-    }
+    assertWildcardQueryEquals(query, result, false);
   }
 
   public Query getQueryDOA(String query, Analyzer a)
@@ -473,39 +461,26 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
    */
 // First prefix queries:
     // by default, convert to lowercase:
-    assertWildcardQueryEquals("Term*", true, "term*");
+    assertWildcardQueryEquals("Term*", "term*");
     // explicitly set lowercase:
-    assertWildcardQueryEquals("term*", true, "term*");
-    assertWildcardQueryEquals("Term*", true, "term*");
-    assertWildcardQueryEquals("TERM*", true, "term*");
-    // explicitly disable lowercase conversion:
-    assertWildcardQueryEquals("term*", false, "term*");
-    assertWildcardQueryEquals("Term*", false, "Term*");
-    assertWildcardQueryEquals("TERM*", false, "TERM*");
+    assertWildcardQueryEquals("term*", "term*");
+    assertWildcardQueryEquals("Term*", "term*");
+    assertWildcardQueryEquals("TERM*", "term*");
 // Then 'full' wildcard queries:
     // by default, convert to lowercase:
     assertWildcardQueryEquals("Te?m", "te?m");
     // explicitly set lowercase:
-    assertWildcardQueryEquals("te?m", true, "te?m");
-    assertWildcardQueryEquals("Te?m", true, "te?m");
-    assertWildcardQueryEquals("TE?M", true, "te?m");
-    assertWildcardQueryEquals("Te?m*gerM", true, "te?m*germ");
-    // explicitly disable lowercase conversion:
-    assertWildcardQueryEquals("te?m", false, "te?m");
-    assertWildcardQueryEquals("Te?m", false, "Te?m");
-    assertWildcardQueryEquals("TE?M", false, "TE?M");
-    assertWildcardQueryEquals("Te?m*gerM", false, "Te?m*gerM");
+    assertWildcardQueryEquals("te?m", "te?m");
+    assertWildcardQueryEquals("Te?m", "te?m");
+    assertWildcardQueryEquals("TE?M", "te?m");
+    assertWildcardQueryEquals("Te?m*gerM", "te?m*germ");
 //  Fuzzy queries:
     assertWildcardQueryEquals("Term~", "term~2");
-    assertWildcardQueryEquals("Term~", true, "term~2");
-    assertWildcardQueryEquals("Term~", false, "Term~2");
 //  Range queries:
     assertWildcardQueryEquals("[A TO C]", "[a TO c]");
-    assertWildcardQueryEquals("[A TO C]", true, "[a TO c]");
-    assertWildcardQueryEquals("[A TO C]", false, "[A TO C]");
     // Test suffix queries: first disallow
     try {
-      assertWildcardQueryEquals("*Term", true, "*term");
+      assertWildcardQueryEquals("*Term", "*term", false);
     } catch(Exception pe) {
       // expected exception
       if(!isQueryParserException(pe)){
@@ -513,7 +488,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
       }
     }
     try {
-      assertWildcardQueryEquals("?Term", true, "?term");
+      assertWildcardQueryEquals("?Term", "?term");
       fail();
     } catch(Exception pe) {
       // expected exception
@@ -522,8 +497,8 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
       }
     }
     // Test suffix queries: then allow
-    assertWildcardQueryEquals("*Term", true, "*term", true);
-    assertWildcardQueryEquals("?Term", true, "?term", true);
+    assertWildcardQueryEquals("*Term", "*term", true);
+    assertWildcardQueryEquals("?Term", "?term", true);
   }
   
   public void testLeadingWildcardType() throws Exception {
@@ -982,10 +957,9 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
   }
   
   public void testRegexps() throws Exception {
-    CommonQueryParserConfiguration qp = getParserConfig( new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false));
+    CommonQueryParserConfiguration qp = getParserConfig( new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true));
     RegexpQuery q = new RegexpQuery(new Term("field", "[a-z][123]"));
     assertEquals(q, getQuery("/[a-z][123]/",qp));
-    qp.setLowercaseExpandedTerms(true);
     assertEquals(q, getQuery("/[A-Z][123]/",qp));
     assertEquals(new BoostQuery(q, 0.5f), getQuery("/[A-Z][123]/^0.5",qp));
     qp.setMultiTermRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_REWRITE);
@@ -1169,11 +1143,14 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
       Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true);
       return new TokenStreamComponents(tokenizer, new MockCollationFilter(tokenizer));
     }
+    @Override
+    protected TokenStream normalize(String fieldName, TokenStream in) {
+      return new MockCollationFilter(new LowerCaseFilter(in));
+    }
   }
   
   public void testCollatedRange() throws Exception {
     CommonQueryParserConfiguration qp = getParserConfig(new MockCollationAnalyzer());
-    setAnalyzeRangeTerms(qp, true);
     Query expected = TermRangeQuery.newStringRange(getDefaultField(), "collatedabc", "collateddef", true, true);
     Query actual = getQuery("[abc TO def]", qp);
     assertEquals(expected, actual);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
index c57a8bc..0bb623f 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
@@ -883,7 +883,10 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
       assertTokenStreamContents(ts, 
                                 tokens.toArray(new String[tokens.size()]));
     }
-    
+
+    a.normalize("dummy", text);
+    // TODO: what can we do besides testing that the above method does not throw?
+
     if (field != null) {
       reader = new StringReader(text);
       random = new Random(seed);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e92a38af/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java b/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java
index e87bf45..bbeffe9 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/analysis/MockAnalyzer.java
@@ -92,7 +92,16 @@ public final class MockAnalyzer extends Analyzer {
     MockTokenFilter filt = new MockTokenFilter(tokenizer, filter);
     return new TokenStreamComponents(tokenizer, maybePayload(filt, fieldName));
   }
-  
+
+  @Override
+  protected TokenStream normalize(String fieldName, TokenStream in) {
+    TokenStream result = in;
+    if (lowerCase) {
+      result = new MockLowerCaseFilter(result);
+    }
+    return result;
+  }
+
   private synchronized TokenFilter maybePayload(TokenFilter stream, String fieldName) {
     Integer val = previousMappings.get(fieldName);
     if (val == null) {