You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/01/23 19:34:08 UTC

svn commit: r1234932 [5/5] - in /lucene/dev/branches/solrcloud: ./ dev-tools/idea/lucene/contrib/ dev-tools/maven/ dev-tools/maven/solr/ dev-tools/maven/solr/contrib/analysis-extras/ dev-tools/maven/solr/contrib/clustering/ dev-tools/maven/solr/contrib...

Modified: lucene/dev/branches/solrcloud/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java (original)
+++ lucene/dev/branches/solrcloud/modules/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizer.java Mon Jan 23 18:34:04 2012
@@ -111,7 +111,7 @@ public final class ICUTokenizer extends 
   @Override
   public void end() throws IOException {
     final int finalOffset = (length < 0) ? offset : offset + length;
-    offsetAtt.setOffset(finalOffset, finalOffset);
+    offsetAtt.setOffset(correctOffset(finalOffset), correctOffset(finalOffset));
   }  
 
   /*

Modified: lucene/dev/branches/solrcloud/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java (original)
+++ lucene/dev/branches/solrcloud/modules/analysis/morfologik/src/test/org/apache/lucene/analysis/morfologik/TestMorfologikAnalyzer.java Mon Jan 23 18:34:04 2012
@@ -102,4 +102,9 @@ public class TestMorfologikAnalyzer exte
     assertPOSToken(ts, "list",  "subst:sg:loc.voc:m3");
     assertPOSToken(ts, "lista", "subst:sg:dat.loc:f");
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandom() throws Exception {
+    checkRandomData(random, getTestAnalyzer(), 10000 * RANDOM_MULTIPLIER); 
+  }
 }

Modified: lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java (original)
+++ lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/DoubleMetaphoneFilterTest.java Mon Jan 23 18:34:04 2012
@@ -16,11 +16,17 @@
  */
 package org.apache.lucene.analysis.phonetic;
 
+import java.io.Reader;
 import java.io.StringReader;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.Analyzer.TokenStreamComponents;
 import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+import org.apache.lucene.util._TestUtil;
 
 public class DoubleMetaphoneFilterTest extends BaseTokenStreamTestCase {
 
@@ -65,4 +71,28 @@ public class DoubleMetaphoneFilterTest e
     assertTokenStreamContents(filter, new String[] { "12345", "#$%@#^%&", "HL" });
   }
 
+  public void testRandom() throws Exception {
+    final int codeLen = _TestUtil.nextInt(random, 1, 8);
+    Analyzer a = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new DoubleMetaphoneFilter(tokenizer, codeLen, false));
+      }
+      
+    };
+    checkRandomData(random, a, 1000 * RANDOM_MULTIPLIER);
+    
+    Analyzer b = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+        return new TokenStreamComponents(tokenizer, new DoubleMetaphoneFilter(tokenizer, codeLen, true));
+      }
+      
+    };
+    checkRandomData(random, b, 1000 * RANDOM_MULTIPLIER); 
+  }
 }

Modified: lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java (original)
+++ lucene/dev/branches/solrcloud/modules/analysis/phonetic/src/test/org/apache/lucene/analysis/phonetic/TestPhoneticFilter.java Mon Jan 23 18:34:04 2012
@@ -17,6 +17,8 @@
 
 package org.apache.lucene.analysis.phonetic;
 
+import java.io.IOException;
+import java.io.Reader;
 import java.io.StringReader;
 
 import org.apache.commons.codec.Encoder;
@@ -25,7 +27,9 @@ import org.apache.commons.codec.language
 import org.apache.commons.codec.language.Metaphone;
 import org.apache.commons.codec.language.RefinedSoundex;
 import org.apache.commons.codec.language.Soundex;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.core.WhitespaceTokenizer;
 
@@ -70,4 +74,33 @@ public class TestPhoneticFilter extends 
     PhoneticFilter filter = new PhoneticFilter(tokenizer, encoder, inject);
     assertTokenStreamContents(filter, expected);
   }
+  
+  /** blast some random strings through the analyzer */
+  public void testRandomStrings() throws IOException {
+    Encoder encoders[] = new Encoder[] {
+      new Metaphone(), new DoubleMetaphone(), new Soundex(), new RefinedSoundex(), new Caverphone()
+    };
+    
+    for (final Encoder e : encoders) {
+      Analyzer a = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+          return new TokenStreamComponents(tokenizer, new PhoneticFilter(tokenizer, e, false));
+        }   
+      };
+      
+      checkRandomData(random, a, 1000*RANDOM_MULTIPLIER);
+      
+      Analyzer b = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+          return new TokenStreamComponents(tokenizer, new PhoneticFilter(tokenizer, e, false));
+        }   
+      };
+      
+      checkRandomData(random, b, 1000*RANDOM_MULTIPLIER);
+    }
+  }
 }

Modified: lucene/dev/branches/solrcloud/modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java (original)
+++ lucene/dev/branches/solrcloud/modules/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java Mon Jan 23 18:34:04 2012
@@ -4,6 +4,7 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.logging.Level;
@@ -100,6 +101,9 @@ public class DirectoryTaxonomyReader imp
 
   private volatile boolean closed = false;
   
+  // set refCount to 1 at start
+  private final AtomicInteger refCount = new AtomicInteger(1);
+  
   /**
    * Open for reading a taxonomy stored in a given {@link Directory}.
    * @param directory
@@ -130,7 +134,7 @@ public class DirectoryTaxonomyReader imp
    * @throws AlreadyClosedException if this IndexReader is closed
    */
   protected final void ensureOpen() throws AlreadyClosedException {
-    if (indexReader.getRefCount() <= 0) {
+    if (getRefCount() <= 0) {
       throw new AlreadyClosedException("this TaxonomyReader is closed");
     }
   }
@@ -415,8 +419,12 @@ public class DirectoryTaxonomyReader imp
 
   public void close() throws IOException {
     if (!closed) {
-      decRef();
-      closed = true;
+      synchronized (this) {
+        if (!closed) {
+          decRef();
+          closed = true;
+        }
+      }
     }
   }
   
@@ -555,27 +563,31 @@ public class DirectoryTaxonomyReader imp
   }
 
   /**
-   * Expert: decreases the refCount of this TaxonomyReader instance. 
-   * If the refCount drops to 0, then pending changes (if any) are 
-   * committed to the taxonomy index and this reader is closed. 
-   * @throws IOException 
+   * Expert: decreases the refCount of this TaxonomyReader instance. If the
+   * refCount drops to 0, then this reader is closed.
    */
   public void decRef() throws IOException {
     ensureOpen();
-    if (indexReader.getRefCount() == 1) {
-      // Do not decRef the indexReader - doClose does it by calling reader.close()
-      doClose();
-    } else {
-      indexReader.decRef();
+    final int rc = refCount.decrementAndGet();
+    if (rc == 0) {
+      boolean success = false;
+      try {
+        doClose();
+        success = true;
+      } finally {
+        if (!success) {
+          // Put reference back on failure
+          refCount.incrementAndGet();
+        }
+      }
+    } else if (rc < 0) {
+      throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
     }
   }
   
-  /**
-   * Expert: returns the current refCount for this taxonomy reader
-   */
+  /** Expert: returns the current refCount for this taxonomy reader */
   public int getRefCount() {
-    ensureOpen();
-    return this.indexReader.getRefCount();
+    return refCount.get();
   }
   
   /**
@@ -587,6 +599,6 @@ public class DirectoryTaxonomyReader imp
    */
   public void incRef() {
     ensureOpen();
-    this.indexReader.incRef();
+    refCount.incrementAndGet();
   }
 }

Modified: lucene/dev/branches/solrcloud/modules/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/modules/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/modules/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java (original)
+++ lucene/dev/branches/solrcloud/modules/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java Mon Jan 23 18:34:04 2012
@@ -11,6 +11,7 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
@@ -178,4 +179,28 @@ public class TestDirectoryTaxonomyReader
     }
   }
   
+  @Test
+  public void testRefreshAndRefCount() throws Exception {
+    Directory dir = new RAMDirectory(); // no need for random directories here
+
+    DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(dir);
+    taxoWriter.addCategory(new CategoryPath("a"));
+    taxoWriter.commit();
+
+    DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(dir);
+    assertEquals("wrong refCount", 1, taxoReader.getRefCount());
+
+    taxoReader.incRef();
+    assertEquals("wrong refCount", 2, taxoReader.getRefCount());
+
+    taxoWriter.addCategory(new CategoryPath("a", "b"));
+    taxoWriter.commit();
+    taxoReader.refresh();
+    assertEquals("wrong refCount", 2, taxoReader.getRefCount());
+
+    taxoWriter.close();
+    taxoReader.close();
+    dir.close();
+  }
+
 }

Modified: lucene/dev/branches/solrcloud/modules/join/src/java/org/apache/lucene/search/join/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/modules/join/src/java/org/apache/lucene/search/join/package.html?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/modules/join/src/java/org/apache/lucene/search/join/package.html (original)
+++ lucene/dev/branches/solrcloud/modules/join/src/java/org/apache/lucene/search/join/package.html Mon Jan 23 18:34:04 2012
@@ -42,7 +42,7 @@
 <h2>Search-time joins</h2>
 
 <p>
-  The query time joining is terms based and implemented as two pass search. The first pass collects all the terms from a fromField
+  The query time joining is index term based and implemented as two pass search. The first pass collects all the terms from a fromField
   that match the fromQuery. The second pass returns all documents that have matching terms in a toField to the terms
   collected in the first pass.
 </p>
@@ -62,7 +62,7 @@
 <pre class="prettyprint">
   String fromField = "from"; // Name of the from field
   boolean multipleValuesPerDocument = false; // Set only yo true in the case when your fromField has multiple values per document in your index
-  String fromField = "to"; // Name of the to field
+  String toField = "to"; // Name of the to field
   Query fromQuery = new TermQuery(new Term("content", searchTerm)); // Query executed to collect from values to join to the to values
 
   MultiTermQuery joinQuery = JoinUtil.createJoinQuery(fromField, multipleValuesPerDocument, toField, fromQuery, fromSearcher);

Modified: lucene/dev/branches/solrcloud/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/CHANGES.txt?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/CHANGES.txt (original)
+++ lucene/dev/branches/solrcloud/solr/CHANGES.txt Mon Jan 23 18:34:04 2012
@@ -369,10 +369,7 @@ Other Changes
 * SOLR-2588: Moved VelocityResponseWriter back to contrib module in order to 
   remove it as a mandatory core dependency.  (ehatcher)
   
-* SOLR-2718: Add ability to lazy load response writers, defined with startup="lazy".
-  (ehatcher)
-
-* SOLR-2862: More explicit lexical resources location logged if Carrot2 clustering 
+* SOLR-2862: More explicit lexical resources location logged if Carrot2 clustering
   extension is used. Fixed solr. impl. of IResource and IResourceLookup. (Dawid Weiss)
 
 * SOLR-1123: Changed JSONResponseWriter to now use application/json as its Content-Type
@@ -404,6 +401,14 @@ Upgrading from Solr 3.5
 * As doGet() methods in SimplePostTool was changed to static, the client applications of this
   class need to be recompiled.
 
+* In Solr version 3.5 and earlier, HTMLStripCharFilter had known bugs in the
+  character offsets it provided, triggering e.g. exceptions in highlighting.
+  HTMLStripCharFilter has been re-implemented, addressing this and other
+  issues.  See the entry for LUCENE-3690 in the Bug Fixes section below for a
+  detailed list of changes.  For people who depend on the behavior of
+  HTMLStripCharFilter in Solr version 3.5 and earlier: the old implementation
+  (bugs and all) is preserved as LegacyHTMLStripCharFilter.
+
 New Features
 ----------------------
 * SOLR-2904: BinaryUpdateRequestHandler should be able to accept multiple update requests from
@@ -445,6 +450,10 @@ New Features
 * SOLR-1709: Distributed support for Date and Numeric Range Faceting
   (Peter Sturge, David Smiley, hossman, Simon Willnauer)
 
+* SOLR-3054, LUCENE-3671: Add TypeTokenFilterFactory that creates TypeTokenFilter
+  that filters tokens based on their TypeAttribute.  (Tommaso Teofili via
+  Uwe Schindler)
+
 Optimizations
 ----------------------
 * SOLR-1931: Speedup for LukeRequestHandler and admin/schema browser. New parameter
@@ -486,6 +495,50 @@ Bug Fixes
 
 * SOLR-2970: CSV ResponseWriter returns fields defined as stored=false in schema (janhoy)
 
+* LUCENE-3690, LUCENE-2208, SOLR-882, SOLR-42: Re-implemented
+  HTMLStripCharFilter as a JFlex-generated scanner.  See below for a list
+  of bug fixes and other changes.  To get the same behavior as
+  HTMLStripCharFilter in Solr version 3.5 and earlier (including the bugs),
+  use LegacyHTMLStripCharFilter, which is the previous implementation.
+
+  Behavior changes from the previous version:
+
+  - Known offset bugs are fixed.
+  - The "Mark invalid" exceptions reported in SOLR-1283 are no longer
+    triggered (the bug is still present in LegacyHTMLStripCharFilter).
+  - The character entity "&apos;" is now always properly decoded.
+  - More cases of <script> tags are now properly stripped.
+  - CDATA sections are now handled properly.
+  - Valid tag name characters now include the supplementary Unicode characters
+    from Unicode character classes [:ID_Start:] and [:ID_Continue:].
+  - Uppercase character entities "&QUOT;", "&COPY;", "&GT;", "&LT;", "&REG;",
+    and "&AMP;" are now recognized and handled as if they were in lowercase.
+  - The REPLACEMENT CHARACTER U+FFFD is now used to replace numeric character 
+    entities for unpaired UTF-16 low and high surrogates (in the range
+    [U+D800-U+DFFF]).
+  - Properly paired numeric character entities for UTF-16 surrogates are now
+    converted to the corresponding code units.
+  - Opening tags with unbalanced quotation marks are now properly stripped.
+  - Literal "<" and ">" characters in opening tags, regardless of whether they
+    appear inside quotation marks, now inhibit recognition (and stripping) of
+    the tags.  The only exception to this is for values of event-handler
+    attributes, e.g. "onClick", "onLoad", "onSelect".
+  - A newline '\n' is substituted instead of a space for stripped HTML markup.
+  - Nothing is substituted for opening and closing inline tags - they are
+    simply removed.  The list of inline tags is (case insensitively): <a>,
+    <abbr>, <acronym>, <b>, <basefont>, <bdo>, <big>, <cite>, <code>, <dfn>,
+    <em>, <font>, <i>, <img>, <input>, <kbd>, <label>, <q>, <s>, <samp>,
+    <select>, <small>, <span>, <strike>, <strong>, <sub>, <sup>, <textarea>,
+    <tt>, <u>, and <var>.
+  - HTMLStripCharFilterFactory now handles HTMLStripCharFilter's "escapedTags"
+    feature: opening and closing tags with the given names, including any
+    attributes and their values, are left intact in the output.
+  (Steve Rowe)
+
+* LUCENE-3717: Fixed offset bugs in TrimFilter, WordDelimiterFilter, and
+  HyphenatedWordsFilter where they would create invalid offsets in
+  some situations, leading to problems in highlighting.  (Robert Muir)
+
 Other Changes
 ----------------------
 * SOLR-2922: Upgrade commons-io and commons-lang to 2.1 and 2.6, respectively. (koji)
@@ -498,6 +551,9 @@ Other Changes
   structure will disappear in 4.0. Errors should be caught and logged at the
   top-most level or logged and NOT propagated up the chain. (Erick Erickson)
 
+* SOLR-2718: Add ability to lazy load response writers, defined with startup="lazy".
+  (ehatcher)
+
 Build
 ----------------------
 * SOLR-2487: Add build target to package war without slf4j jars (janhoy)

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/analysis/HTMLStripCharFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/analysis/HTMLStripCharFilterFactory.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/analysis/HTMLStripCharFilterFactory.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/analysis/HTMLStripCharFilterFactory.java Mon Jan 23 18:34:04 2012
@@ -21,21 +21,50 @@ package org.apache.solr.analysis;
 import org.apache.lucene.analysis.CharStream;
 import org.apache.lucene.analysis.charfilter.HTMLStripCharFilter;
 
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 /**
 * Factory for {@link HTMLStripCharFilter}. 
  * <pre class="prettyprint" >
  * &lt;fieldType name="text_html" class="solr.TextField" positionIncrementGap="100"&gt;
  *   &lt;analyzer&gt;
- *     &lt;charFilter class="solr.HTMLStripCharFilterFactory"/&gt;
+ *     &lt;charFilter class="solr.HTMLStripCharFilterFactory" escapedTags="a, title" /&gt;
  *     &lt;tokenizer class="solr.WhitespaceTokenizerFactory"/&gt;
  *   &lt;/analyzer&gt;
  * &lt;/fieldType&gt;</pre
  *
  */
  public class HTMLStripCharFilterFactory extends BaseCharFilterFactory {
+  
+  Set<String> escapedTags = null;
+  Pattern TAG_NAME_PATTERN = Pattern.compile("[^\\s,]+");
 
   public HTMLStripCharFilter create(CharStream input) {
-    return new HTMLStripCharFilter(input);
+    HTMLStripCharFilter charFilter;
+    if (null == escapedTags) {
+      charFilter = new HTMLStripCharFilter(input);
+    } else {
+      charFilter = new HTMLStripCharFilter(input, escapedTags);
+    }
+    return charFilter;
+  }
+  
+  @Override
+  public void init(Map<String,String> args) {
+    super.init(args);
+    String escapedTagsArg = args.get("escapedTags");
+    if (null != escapedTagsArg) {
+      Matcher matcher = TAG_NAME_PATTERN.matcher(escapedTagsArg);
+      while (matcher.find()) {
+        if (null == escapedTags) {
+          escapedTags = new HashSet<String>();
+        }
+        escapedTags.add(matcher.group(0));
+      }
+    }
   }
-
 }

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java?rev=1234932&r1=1234931&r2=1234932&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java Mon Jan 23 18:34:04 2012
@@ -326,8 +326,8 @@ public class FieldAnalysisRequestHandler
     NamedList indexPart = textType.get("index");
     assertNotNull("expecting an index token analysis for field type 'charfilthtmlmap'", indexPart);
     
-    assertEquals("  whátëvêr  ", indexPart.get("org.apache.lucene.analysis.charfilter.HTMLStripCharFilter"));
-    assertEquals("  whatever  ", indexPart.get("org.apache.lucene.analysis.charfilter.MappingCharFilter"));
+    assertEquals("\n\nwhátëvêr\n\n", indexPart.get("org.apache.lucene.analysis.charfilter.HTMLStripCharFilter"));
+    assertEquals("\n\nwhatever\n\n", indexPart.get("org.apache.lucene.analysis.charfilter.MappingCharFilter"));
 
     List<NamedList> tokenList = (List<NamedList>)indexPart.get(MockTokenizer.class.getName());
     assertNotNull("Expecting MockTokenizer analysis breakdown", tokenList);