You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ry...@apache.org on 2011/05/06 05:10:03 UTC

svn commit: r1100033 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/backwards/ lucene/contrib/analyzers/common/src/java/org/apache/lucene/analysis/path/ lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/ solr/ solr/src/ja...

Author: ryan
Date: Fri May  6 03:10:02 2011
New Revision: 1100033

URL: http://svn.apache.org/viewvc?rev=1100033&view=rev
Log:
LUCENE-3071:  Add ReversePathHierarchyTokenizer and enable skip on PathHierarchyTokenizer

Added:
    lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestReversePathHierarchyTokenizer.java   (with props)
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/backwards/   (props changed)
    lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java
    lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestPathHierarchyTokenizer.java
    lucene/dev/branches/branch_3x/solr/   (props changed)
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/analysis/PathHierarchyTokenizerFactory.java

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1100033&r1=1100032&r2=1100033&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Fri May  6 03:10:02 2011
@@ -18,6 +18,9 @@ New features
   segments, which means docIDs no longer necessarily stay "in order".
   (Mike McCandless, Shai Erera)
 
+* LUCENE-3071: Adding ReversePathHierarchyTokenizer, added skip parameter to 
+  PathHierarchyTokenizer (Olivier Favre via ryan)
+
 API Changes
 
 * LUCENE-3061: IndexWriter's getNextMerge() and merge(OneMerge) are now public

Modified: lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java?rev=1100033&r1=1100032&r2=1100033&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/java/org/apache/lucene/analysis/path/PathHierarchyTokenizer.java Fri May  6 03:10:02 2011
@@ -25,57 +25,71 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 
 /**
- * 
+ *
  * Take something like:
- * 
+ *
  * <pre>
- *  /soemthing/something/else
+ *  /something/something/else
  * </pre>
- * 
+ *
  * and make:
- *  
+ *
  * <pre>
- *  /soemthing
- *  /soemthing/something
- *  /soemthing/something/else
+ *  /something
+ *  /something/something
+ *  /something/something/else
  * </pre>
- * 
  */
 public class PathHierarchyTokenizer extends Tokenizer {
 
   public PathHierarchyTokenizer(Reader input) {
-    this(input, DEFAULT_BUFFER_SIZE, DEFAULT_DELIMITER);
+    this(input, DEFAULT_BUFFER_SIZE, DEFAULT_DELIMITER, DEFAULT_DELIMITER, DEFAULT_SKIP);
+  }
+
+  public PathHierarchyTokenizer(Reader input, int skip) {
+    this(input, DEFAULT_BUFFER_SIZE, DEFAULT_DELIMITER, DEFAULT_DELIMITER, skip);
   }
 
   public PathHierarchyTokenizer(Reader input, int bufferSize, char delimiter) {
-    this(input, bufferSize, delimiter, delimiter);
+    this(input, bufferSize, delimiter, delimiter, DEFAULT_SKIP);
   }
 
   public PathHierarchyTokenizer(Reader input, char delimiter, char replacement) {
-    this(input, DEFAULT_BUFFER_SIZE, delimiter, replacement);
+    this(input, DEFAULT_BUFFER_SIZE, delimiter, replacement, DEFAULT_SKIP);
   }
 
-  public PathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement) {
+  public PathHierarchyTokenizer(Reader input, char delimiter, char replacement, int skip) {
+    this(input, DEFAULT_BUFFER_SIZE, delimiter, replacement, skip);
+  }
+
+  public PathHierarchyTokenizer(Reader input, int bufferSize, char delimiter, char replacement, int skip) {
     super(input);
     termAtt.resizeBuffer(bufferSize);
+
     this.delimiter = delimiter;
     this.replacement = replacement;
-    endDelimiter = false;
+    this.skip = skip;
     resultToken = new StringBuilder(bufferSize);
   }
-  
+
   private static final int DEFAULT_BUFFER_SIZE = 1024;
   public static final char DEFAULT_DELIMITER = '/';
+  public static final int DEFAULT_SKIP = 0;
+
   private final char delimiter;
   private final char replacement;
-  
+  private final int skip;
+
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
   private final PositionIncrementAttribute posAtt = addAttribute(PositionIncrementAttribute.class);
+  private int startPosition = 0;
   private int finalOffset = 0;
-  private boolean endDelimiter;
+  private int skipped = 0;
+  private boolean endDelimiter = false;
   private StringBuilder resultToken;
 
+
   @Override
   public final boolean incrementToken() throws IOException {
     clearAttributes();
@@ -97,43 +111,69 @@ public class PathHierarchyTokenizer exte
 
     while (true) {
       int c = input.read();
-      if( c < 0 ) {
-        length += resultToken.length();
-        termAtt.setLength(length);
-        finalOffset = correctOffset(length);
-        offsetAtt.setOffset(correctOffset(0), finalOffset);
-        if( added ){
-          resultToken.setLength(0);
-          resultToken.append(termAtt.buffer(), 0, length);
-        }
-        return added;
-      }
-      added = true;
-      if( c == delimiter ) {
-        if( length > 0 ){
-          endDelimiter = true;
-          break;
+      if( c < 0 ){
+        if( skipped > skip ) {
+          length += resultToken.length();
+          termAtt.setLength(length);
+          finalOffset = correctOffset(startPosition + length);
+          offsetAtt.setOffset(correctOffset(startPosition), finalOffset);
+          if( added ){
+            resultToken.setLength(0);
+            resultToken.append(termAtt.buffer(), 0, length);
+          }
+          return added;
         }
         else{
-          termAtt.append(replacement);
+          finalOffset = correctOffset(startPosition + length);
+          return false;
+        }
+      }
+      if( !added ){
+        added = true;
+        skipped++;
+        if( skipped > skip ){
+          termAtt.append(c == delimiter ? replacement : (char)c);
           length++;
         }
+        else {
+          startPosition++;
+        }
       }
       else {
-        termAtt.append((char)c);
-        length++;
+        if( c == delimiter ){
+          if( skipped > skip ){
+            endDelimiter = true;
+            break;
+          }
+          skipped++;
+          if( skipped > skip ){
+            termAtt.append(replacement);
+            length++;
+          }
+          else {
+            startPosition++;
+          }
+        }
+        else {
+          if( skipped > skip ){
+            termAtt.append((char)c);
+            length++;
+          }
+          else {
+            startPosition++;
+          }
+        }
       }
     }
-
     length += resultToken.length();
     termAtt.setLength(length);
-    finalOffset = correctOffset(length);
-    offsetAtt.setOffset(correctOffset(0), finalOffset);
+    finalOffset = correctOffset(startPosition + length);
+    offsetAtt.setOffset(correctOffset(startPosition), finalOffset);
     resultToken.setLength(0);
     resultToken.append(termAtt.buffer(), 0, length);
     return true;
   }
-  
+
   @Override
   public final void end() {
     // set final offset
@@ -146,5 +186,6 @@ public class PathHierarchyTokenizer exte
     resultToken.setLength(0);
     finalOffset = 0;
     endDelimiter = false;
+    skipped = 0;
   }
 }

Modified: lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestPathHierarchyTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestPathHierarchyTokenizer.java?rev=1100033&r1=1100032&r2=1100033&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestPathHierarchyTokenizer.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestPathHierarchyTokenizer.java Fri May  6 03:10:02 2011
@@ -33,7 +33,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"/a", "/a/b", "/a/b/c"},
         new int[]{0, 0, 0},
         new int[]{2, 4, 6},
-        new int[]{1, 0, 0});
+        new int[]{1, 0, 0},
+        path.length());
   }
 
   public void testEndOfDelimiter() throws Exception {
@@ -43,7 +44,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"/a", "/a/b", "/a/b/c", "/a/b/c/"},
         new int[]{0, 0, 0, 0},
         new int[]{2, 4, 6, 7},
-        new int[]{1, 0, 0, 0});
+        new int[]{1, 0, 0, 0},
+        path.length());
   }
 
   public void testStartOfChar() throws Exception {
@@ -53,7 +55,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"a", "a/b", "a/b/c"},
         new int[]{0, 0, 0},
         new int[]{1, 3, 5},
-        new int[]{1, 0, 0});
+        new int[]{1, 0, 0},
+        path.length());
   }
 
   public void testStartOfCharEndOfDelimiter() throws Exception {
@@ -63,7 +66,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"a", "a/b", "a/b/c", "a/b/c/"},
         new int[]{0, 0, 0, 0},
         new int[]{1, 3, 5, 6},
-        new int[]{1, 0, 0, 0});
+        new int[]{1, 0, 0, 0},
+        path.length());
   }
 
   public void testOnlyDelimiter() throws Exception {
@@ -73,7 +77,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"/"},
         new int[]{0},
         new int[]{1},
-        new int[]{1});
+        new int[]{1},
+        path.length());
   }
 
   public void testOnlyDelimiters() throws Exception {
@@ -83,7 +88,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"/", "//"},
         new int[]{0, 0},
         new int[]{1, 2},
-        new int[]{1, 0});
+        new int[]{1, 0},
+        path.length());
   }
 
   public void testReplace() throws Exception {
@@ -93,7 +99,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"\\a", "\\a\\b", "\\a\\b\\c"},
         new int[]{0, 0, 0},
         new int[]{2, 4, 6},
-        new int[]{1, 0, 0});
+        new int[]{1, 0, 0},
+        path.length());
   }
 
   public void testWindowsPath() throws Exception {
@@ -103,7 +110,8 @@ public class TestPathHierarchyTokenizer 
         new String[]{"c:", "c:\\a", "c:\\a\\b", "c:\\a\\b\\c"},
         new int[]{0, 0, 0, 0},
         new int[]{2, 4, 6, 8},
-        new int[]{1, 0, 0, 0});
+        new int[]{1, 0, 0, 0},
+        path.length());
   }
 
   public void testNormalizeWinDelimToLinuxDelim() throws Exception {
@@ -116,6 +124,73 @@ public class TestPathHierarchyTokenizer 
         new String[]{"c:", "c:/a", "c:/a/b", "c:/a/b/c"},
         new int[]{0, 0, 0, 0},
         new int[]{2, 4, 6, 8},
-        new int[]{1, 0, 0, 0});
+        new int[]{1, 0, 0, 0},
+        path.length());
+  }
+
+  public void testBasicSkip() throws Exception {
+    String path = "/a/b/c";
+    PathHierarchyTokenizer t = new PathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"/b", "/b/c"},
+        new int[]{2, 2},
+        new int[]{4, 6},
+        new int[]{1, 0},
+        path.length());
+  }
+
+  public void testEndOfDelimiterSkip() throws Exception {
+    String path = "/a/b/c/";
+    PathHierarchyTokenizer t = new PathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"/b", "/b/c", "/b/c/"},
+        new int[]{2, 2, 2},
+        new int[]{4, 6, 7},
+        new int[]{1, 0, 0},
+        path.length());
+  }
+
+  public void testStartOfCharSkip() throws Exception {
+    String path = "a/b/c";
+    PathHierarchyTokenizer t = new PathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"/b", "/b/c"},
+        new int[]{1, 1},
+        new int[]{3, 5},
+        new int[]{1, 0},
+        path.length());
+  }
+
+  public void testStartOfCharEndOfDelimiterSkip() throws Exception {
+    String path = "a/b/c/";
+    PathHierarchyTokenizer t = new PathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"/b", "/b/c", "/b/c/"},
+        new int[]{1, 1, 1},
+        new int[]{3, 5, 6},
+        new int[]{1, 0, 0},
+        path.length());
+  }
+
+  public void testOnlyDelimiterSkip() throws Exception {
+    String path = "/";
+    PathHierarchyTokenizer t = new PathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{},
+        new int[]{},
+        new int[]{},
+        new int[]{},
+        path.length());
+  }
+
+  public void testOnlyDelimitersSkip() throws Exception {
+    String path = "//";
+    PathHierarchyTokenizer t = new PathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"/"},
+        new int[]{1},
+        new int[]{2},
+        new int[]{1},
+        path.length());
   }
 }

Added: lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestReversePathHierarchyTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestReversePathHierarchyTokenizer.java?rev=1100033&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestReversePathHierarchyTokenizer.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/analyzers/common/src/test/org/apache/lucene/analysis/path/TestReversePathHierarchyTokenizer.java Fri May  6 03:10:02 2011
@@ -0,0 +1,157 @@
+package org.apache.lucene.analysis.path;
+
+/**
+ * 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.
+ */
+
+import java.io.StringReader;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+
+public class TestReversePathHierarchyTokenizer extends BaseTokenStreamTestCase {
+
+  public void testBasicReverse() throws Exception {
+    String path = "/a/b/c";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path) );
+    assertTokenStreamContents(t,
+        new String[]{"/a/b/c", "a/b/c", "b/c", "c"},
+        new int[]{0, 1, 3, 5},
+        new int[]{6, 6, 6, 6},
+        new int[]{1, 0, 0, 0},
+        path.length());
+  }
+
+  public void testEndOfDelimiterReverse() throws Exception {
+    String path = "/a/b/c/";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path) );
+    assertTokenStreamContents(t,
+        new String[]{"/a/b/c/", "a/b/c/", "b/c/", "c/"},
+        new int[]{0, 1, 3, 5},
+        new int[]{7, 7, 7, 7},
+        new int[]{1, 0, 0, 0},
+        path.length());
+  }
+
+  public void testStartOfCharReverse() throws Exception {
+    String path = "a/b/c";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path) );
+    assertTokenStreamContents(t,
+        new String[]{"a/b/c", "b/c", "c"},
+        new int[]{0, 2, 4},
+        new int[]{5, 5, 5},
+        new int[]{1, 0, 0},
+        path.length());
+  }
+
+  public void testStartOfCharEndOfDelimiterReverse() throws Exception {
+    String path = "a/b/c/";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path) );
+    assertTokenStreamContents(t,
+        new String[]{"a/b/c/", "b/c/", "c/"},
+        new int[]{0, 2, 4},
+        new int[]{6, 6, 6},
+        new int[]{1, 0, 0},
+        path.length());
+  }
+
+  public void testOnlyDelimiterReverse() throws Exception {
+    String path = "/";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path) );
+    assertTokenStreamContents(t,
+        new String[]{"/"},
+        new int[]{0},
+        new int[]{1},
+        new int[]{1},
+        path.length());
+  }
+
+  public void testOnlyDelimitersReverse() throws Exception {
+    String path = "//";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path) );
+    assertTokenStreamContents(t,
+        new String[]{"//", "/"},
+        new int[]{0, 1},
+        new int[]{2, 2},
+        new int[]{1, 0},
+        path.length());
+  }
+
+  public void testEndOfDelimiterReverseSkip() throws Exception {
+    String path = "/a/b/c/";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"/a/b/", "a/b/", "b/"},
+        new int[]{0, 1, 3},
+        new int[]{5, 5, 5},
+        new int[]{1, 0, 0},
+        path.length());
+  }
+
+  public void testStartOfCharReverseSkip() throws Exception {
+    String path = "a/b/c";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"a/b/", "b/"},
+        new int[]{0, 2},
+        new int[]{4, 4},
+        new int[]{1, 0},
+        path.length());
+  }
+
+  public void testStartOfCharEndOfDelimiterReverseSkip() throws Exception {
+    String path = "a/b/c/";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"a/b/", "b/"},
+        new int[]{0, 2},
+        new int[]{4, 4},
+        new int[]{1, 0},
+        path.length());
+  }
+
+  public void testOnlyDelimiterReverseSkip() throws Exception {
+    String path = "/";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{},
+        new int[]{},
+        new int[]{},
+        new int[]{},
+        path.length());
+  }
+
+  public void testOnlyDelimitersReverseSkip() throws Exception {
+    String path = "//";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path), 1 );
+    assertTokenStreamContents(t,
+        new String[]{"/"},
+        new int[]{0},
+        new int[]{1},
+        new int[]{1},
+        path.length());
+  }
+
+  public void testReverseSkip2() throws Exception {
+    String path = "/a/b/c/";
+    ReversePathHierarchyTokenizer t = new ReversePathHierarchyTokenizer( new StringReader(path), 2 );
+    assertTokenStreamContents(t,
+        new String[]{"/a/", "a/"},
+        new int[]{0, 1},
+        new int[]{3, 3},
+        new int[]{1, 0},
+        path.length());
+  }
+}

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/analysis/PathHierarchyTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/analysis/PathHierarchyTokenizerFactory.java?rev=1100033&r1=1100032&r2=1100033&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/analysis/PathHierarchyTokenizerFactory.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/analysis/PathHierarchyTokenizerFactory.java Fri May  6 03:10:02 2011
@@ -21,6 +21,7 @@ import java.util.Map;
 
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.path.PathHierarchyTokenizer;
+import org.apache.lucene.analysis.path.ReversePathHierarchyTokenizer;
 
 
 /**
@@ -37,6 +38,8 @@ public class PathHierarchyTokenizerFacto
   
   private char delimiter;
   private char replacement;
+  private boolean reverse = false;
+  private int skip =  PathHierarchyTokenizer.DEFAULT_SKIP;
   
   /**
    * Require a configured pattern
@@ -70,10 +73,23 @@ public class PathHierarchyTokenizerFacto
     else{
       replacement = delimiter;
     }
+    
+    v = args.get( "reverse" );
+    if( v != null ){
+      reverse = "true".equals( v );
+    }
+
+    v = args.get( "skip" );
+    if( v != null ){
+      skip = Integer.parseInt( v );
+    }
   }
 
   public Tokenizer create(Reader input) {
-    return new PathHierarchyTokenizer(input, delimiter, replacement);
+    if( reverse ) {
+      return new ReversePathHierarchyTokenizer(input, delimiter, replacement, skip);
+    }
+    return new PathHierarchyTokenizer(input, delimiter, replacement, skip);
   }
 }