You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/04/01 06:52:28 UTC

svn commit: r1583531 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/analysis/ lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/ lucene/analysis/common/src/java/org/apache/lucene/analysis/sinks/ lucene/analysis/common/src...

Author: rmuir
Date: Tue Apr  1 04:52:27 2014
New Revision: 1583531

URL: http://svn.apache.org/r1583531
Log:
LUCENE-5559: Add missing checks to TokenFilters with numeric arguments

Added:
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java
      - copied, changed from r1583530, lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt
    lucene/dev/branches/branch_4x/lucene/analysis/   (props changed)
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenPositionFilter.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/sinks/TokenRangeSinkFilter.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilter.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilterFactory.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilterFactory.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TokenRangeSinkTokenizerTest.java
    lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizerFactory.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Tue Apr  1 04:52:27 2014
@@ -162,6 +162,9 @@ Bug fixes
 
 * LUCENE-5555: Fix SortedInputIterator to correctly encode/decode contexts in presence of payload (Areek Zillur)
 
+* LUCENE-5559: Add missing argument checks to tokenfilters taking
+  numeric arguments.  (Ahmet Arslan via Robert Muir)
+
 Test Framework
 
 * LUCENE-5449: Rename _TestUtil and _TestHelper to remove the leading _.

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LengthFilter.java Tue Apr  1 04:52:27 2014
@@ -32,13 +32,19 @@ public final class LengthFilter extends 
 
   private final int min;
   private final int max;
-  
+
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
 
   /** @deprecated enablePositionIncrements=false is not supported anymore as of Lucene 4.4. */
   @Deprecated
   public LengthFilter(Version version, boolean enablePositionIncrements, TokenStream in, int min, int max) {
     super(version, enablePositionIncrements, in);
+    if (min < 0) {
+      throw new IllegalArgumentException("minimum length must be greater than or equal to zero");
+    }
+    if (min > max) {
+      throw new IllegalArgumentException("maximum length must not be greater than minimum length");
+    }
     this.min = min;
     this.max = max;
   }
@@ -54,6 +60,12 @@ public final class LengthFilter extends 
    */
   public LengthFilter(Version version, TokenStream in, int min, int max) {
     super(version, in);
+    if (min < 0) {
+      throw new IllegalArgumentException("minimum length must be greater than or equal to zero");
+    }
+    if (min > max) {
+      throw new IllegalArgumentException("maximum length must not be greater than minimum length");
+    }
     this.min = min;
     this.max = max;
   }

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilter.java Tue Apr  1 04:52:27 2014
@@ -61,6 +61,9 @@ public final class LimitTokenCountFilter
    */
   public LimitTokenCountFilter(TokenStream in, int maxTokenCount, boolean consumeAllTokens) {
     super(in);
+    if (maxTokenCount < 1) {
+      throw new IllegalArgumentException("maxTokenCount must be greater than zero");
+    }
     this.maxTokenCount = maxTokenCount;
     this.consumeAllTokens = consumeAllTokens;
   }

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenPositionFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenPositionFilter.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenPositionFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenPositionFilter.java Tue Apr  1 04:52:27 2014
@@ -67,6 +67,9 @@ public final class LimitTokenPositionFil
    */
   public LimitTokenPositionFilter(TokenStream in, int maxTokenPosition, boolean consumeAllTokens) {
     super(in);
+    if (maxTokenPosition < 1) {
+      throw new IllegalArgumentException("maxTokenPosition must be greater than zero");
+    }
     this.maxTokenPosition = maxTokenPosition;
     this.consumeAllTokens = consumeAllTokens;
   }

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/sinks/TokenRangeSinkFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/sinks/TokenRangeSinkFilter.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/sinks/TokenRangeSinkFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/sinks/TokenRangeSinkFilter.java Tue Apr  1 04:52:27 2014
@@ -31,6 +31,12 @@ public class TokenRangeSinkFilter extend
   private int count;
 
   public TokenRangeSinkFilter(int lower, int upper) {
+    if (lower < 1) {
+      throw new IllegalArgumentException("lower must be greater than zero");
+    }
+    if (lower > upper) {
+      throw new IllegalArgumentException("lower must not be greater than upper");
+    }
     this.lower = lower;
     this.upper = upper;
   }

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/ClassicTokenizer.java Tue Apr  1 04:52:27 2014
@@ -83,6 +83,9 @@ public final class ClassicTokenizer exte
   /** Set the max allowed token length.  Any token longer
    *  than this is skipped. */
   public void setMaxTokenLength(int length) {
+    if (length < 1) {
+      throw new IllegalArgumentException("maxTokenLength must be greater than zero");
+    }
     this.maxTokenLength = length;
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/StandardTokenizer.java Tue Apr  1 04:52:27 2014
@@ -111,6 +111,9 @@ public final class StandardTokenizer ext
   /** Set the max allowed token length.  Any token longer
    *  than this is skipped. */
   public void setMaxTokenLength(int length) {
+    if (length < 1) {
+      throw new IllegalArgumentException("maxTokenLength must be greater than zero");
+    }
     this.maxTokenLength = length;
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/UAX29URLEmailTokenizer.java Tue Apr  1 04:52:27 2014
@@ -92,6 +92,9 @@ public final class UAX29URLEmailTokenize
   /** Set the max allowed token length.  Any token longer
    *  than this is skipped. */
   public void setMaxTokenLength(int length) {
+    if (length < 1) {
+      throw new IllegalArgumentException("maxTokenLength must be greater than zero");
+    }
     this.maxTokenLength = length;
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilter.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilter.java Tue Apr  1 04:52:27 2014
@@ -25,6 +25,14 @@ import java.io.IOException;
 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.core.KeywordTokenizer;
+import org.junit.Test;
+
 public class TestLengthFilter extends BaseTokenStreamTestCase {
   
   public void testFilterNoPosIncr() throws Exception {
@@ -58,4 +66,11 @@ public class TestLengthFilter extends Ba
     checkOneTerm(a, "", "");
   }
 
+  /**
+   * checking the validity of constructor arguments
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalArguments() throws Exception {
+    new LengthFilter(TEST_VERSION_CURRENT, new MockTokenizer(new StringReader("accept only valid arguments")), -4, -1);
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilterFactory.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilterFactory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLengthFilterFactory.java Tue Apr  1 04:52:27 2014
@@ -1,11 +1,12 @@
 package org.apache.lucene.analysis.miscellaneous;
 
-/**
- * Copyright 2004 The Apache Software Foundation
- *
- * Licensed 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
+/*
+ * 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
  *
@@ -42,22 +43,35 @@ public class TestLengthFilterFactory ext
     Reader reader = new StringReader("foo foobar super-duper-trooper");
     TokenStream stream = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
     stream = tokenFilterFactory("Length",
-        "min", "4",
-        "max", "10",
-        "enablePositionIncrements", "true").create(stream);
+        LengthFilterFactory.MIN_KEY, "4",
+        LengthFilterFactory.MAX_KEY, "10").create(stream);
     assertTokenStreamContents(stream, new String[] { "foobar" }, new int[] { 2 });
   }
-  
+
   /** Test that bogus arguments result in exception */
   public void testBogusArguments() throws Exception {
     try {
-      tokenFilterFactory("Length", 
-          "min", "4", 
-          "max", "5", 
+      tokenFilterFactory("Length",
+          LengthFilterFactory.MIN_KEY, "4",
+          LengthFilterFactory.MAX_KEY, "5",
           "bogusArg", "bogusValue");
       fail();
     } catch (IllegalArgumentException expected) {
       assertTrue(expected.getMessage().contains("Unknown parameters"));
     }
   }
+
+  /** Test that invalid arguments result in exception */
+  public void testInvalidArguments() throws Exception {
+    try {
+      Reader reader = new StringReader("foo foobar super-duper-trooper");
+      TokenStream stream = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+      tokenFilterFactory("Length",
+          LengthFilterFactory.MIN_KEY, "5",
+          LengthFilterFactory.MAX_KEY, "4").create(stream);
+      fail();
+    } catch (IllegalArgumentException expected) {
+      assertTrue(expected.getMessage().contains("maximum length must not be greater than minimum length"));
+    }
+  }
 }
\ No newline at end of file

Copied: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java (from r1583530, lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java?p2=lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java&p1=lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java&r1=1583530&r2=1583531&rev=1583531&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilter.java Tue Apr  1 04:52:27 2014
@@ -17,6 +17,8 @@ package org.apache.lucene.analysis.misce
  * limitations under the License.
  */
 
+import java.io.StringReader;
+
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
@@ -26,7 +28,7 @@ public class TestLimitTokenCountFilter e
 
   public void test() throws Exception {
     for (final boolean consumeAll : new boolean[]{true, false}) {
-      MockTokenizer tokenizer = whitespaceMockTokenizer("A1 B2 C3 D4 E5 F6");
+      MockTokenizer tokenizer = new MockTokenizer(new StringReader("A1 B2 C3 D4 E5 F6"), MockTokenizer.WHITESPACE, false);
       tokenizer.setEnableChecks(consumeAll);
       TokenStream stream = new LimitTokenCountFilter(tokenizer, 3, consumeAll);
       assertTokenStreamContents(stream, new String[]{"A1", "B2", "C3"});
@@ -35,6 +37,6 @@ public class TestLimitTokenCountFilter e
 
   @Test(expected = IllegalArgumentException.class)
   public void testIllegalArguments() throws Exception {
-    new LimitTokenCountFilter(whitespaceMockTokenizer("A1 B2 C3 D4 E5 F6"), -1);
+    new LimitTokenCountFilter(new MockTokenizer(new StringReader("A1 B2 C3 D4 E5 F6"), MockTokenizer.WHITESPACE, false), -1);
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java Tue Apr  1 04:52:27 2014
@@ -1,11 +1,12 @@
 package org.apache.lucene.analysis.miscellaneous;
 
-/**
- * Copyright 2004 The Apache Software Foundation
- *
- * Licensed 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
+/*
+ * 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
  *
@@ -16,24 +17,27 @@ package org.apache.lucene.analysis.misce
  * limitations under the License.
  */
 
-import java.io.Reader;
-import java.io.StringReader;
-
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
 
+import java.io.Reader;
+import java.io.StringReader;
+
 public class TestLimitTokenCountFilterFactory extends BaseTokenStreamFactoryTestCase {
 
   public void test() throws Exception {
-    Reader reader = new StringReader("A1 B2 C3 D4 E5 F6");
-    MockTokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
-    // LimitTokenCountFilter doesn't consume the entire stream that it wraps
-    tokenizer.setEnableChecks(false);
-    TokenStream stream = tokenizer;
-    stream = tokenFilterFactory("LimitTokenCount",
-        "maxTokenCount", "3").create(stream);
-    assertTokenStreamContents(stream, new String[] { "A1", "B2", "C3" });
+    for (final boolean consumeAll : new boolean[]{true, false}) {
+      Reader reader = new StringReader("A1 B2 C3 D4 E5 F6");
+      MockTokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+      tokenizer.setEnableChecks(consumeAll);
+      TokenStream stream = tokenizer;
+      stream = tokenFilterFactory("LimitTokenCount",
+          LimitTokenCountFilterFactory.MAX_TOKEN_COUNT_KEY, "3",
+          LimitTokenCountFilterFactory.CONSUME_ALL_TOKENS_KEY, Boolean.toString(consumeAll)
+      ).create(stream);
+      assertTokenStreamContents(stream, new String[]{"A1", "B2", "C3"});
+    }
   }
 
   public void testRequired() throws Exception {
@@ -43,15 +47,17 @@ public class TestLimitTokenCountFilterFa
       fail();
     } catch (IllegalArgumentException e) {
       assertTrue("exception doesn't mention param: " + e.getMessage(),
-                 0 < e.getMessage().indexOf(LimitTokenCountFilterFactory.MAX_TOKEN_COUNT_KEY));
+          0 < e.getMessage().indexOf(LimitTokenCountFilterFactory.MAX_TOKEN_COUNT_KEY));
     }
   }
-  
-  /** Test that bogus arguments result in exception */
+
+  /**
+   * Test that bogus arguments result in exception
+   */
   public void testBogusArguments() throws Exception {
     try {
-      tokenFilterFactory("LimitTokenCount", 
-          "maxTokenCount", "3", 
+      tokenFilterFactory("LimitTokenCount",
+          LimitTokenCountFilterFactory.MAX_TOKEN_COUNT_KEY, "3",
           "bogusArg", "bogusValue");
       fail();
     } catch (IllegalArgumentException expected) {

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilter.java Tue Apr  1 04:52:27 2014
@@ -16,10 +16,6 @@ package org.apache.lucene.analysis.misce
  * limitations under the License.
  */
 
-import java.io.IOException;
-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;
@@ -27,11 +23,16 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.synonym.SynonymFilter;
 import org.apache.lucene.analysis.synonym.SynonymMap;
 import org.apache.lucene.util.CharsRef;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.io.StringReader;
 
 public class TestLimitTokenPositionFilter extends BaseTokenStreamTestCase {
 
   public void testMaxPosition2() throws IOException {
-    for (final boolean consumeAll : new boolean[] { true, false }) {
+    for (final boolean consumeAll : new boolean[]{true, false}) {
       Analyzer a = new Analyzer() {
         @Override
         protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
@@ -42,43 +43,50 @@ public class TestLimitTokenPositionFilte
         }
       };
 
-      // dont use assertAnalyzesTo here, as the end offset is not the end of the string (unless consumeAll is true, in which case its correct)!
-      assertTokenStreamContents(a.tokenStream("dummy", "1  2     3  4  5"), 
-                                new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 16 : null);
-      assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")), 
-                                new String[] { "1", "2" }, new int[] { 0, 2 }, new int[] { 1, 3 }, consumeAll ? 9 : null);
+      // don't use assertAnalyzesTo here, as the end offset is not the end of the string (unless consumeAll is true, in which case its correct)!
+      assertTokenStreamContents(a.tokenStream("dummy", "1  2     3  4  5"),
+          new String[]{"1", "2"}, new int[]{0, 3}, new int[]{1, 4}, consumeAll ? 16 : null);
+      assertTokenStreamContents(a.tokenStream("dummy", new StringReader("1 2 3 4 5")),
+          new String[]{"1", "2"}, new int[]{0, 2}, new int[]{1, 3}, consumeAll ? 9 : null);
 
       // less than the limit, ensure we behave correctly
       assertTokenStreamContents(a.tokenStream("dummy", "1  "),
-                                new String[] { "1" }, new int[] { 0 }, new int[] { 1 }, consumeAll ? 3 : null);
-                                                                                   
+          new String[]{"1"}, new int[]{0}, new int[]{1}, consumeAll ? 3 : null);
+
       // equal to limit
-      assertTokenStreamContents(a.tokenStream("dummy", "1  2  "), 
-                                new String[] { "1", "2" }, new int[] { 0, 3 }, new int[] { 1, 4 }, consumeAll ? 6 : null);
+      assertTokenStreamContents(a.tokenStream("dummy", "1  2  "),
+          new String[]{"1", "2"}, new int[]{0, 3}, new int[]{1, 4}, consumeAll ? 6 : null);
     }
   }
-  
+
   public void testMaxPosition3WithSynomyms() throws IOException {
-    MockTokenizer tokenizer = new MockTokenizer(new StringReader("one two three four five"), MockTokenizer.WHITESPACE, false);
-    tokenizer.setEnableChecks(false); // LimitTokenPositionFilter doesn't consume the entire stream that it wraps
-    
-    SynonymMap.Builder builder = new SynonymMap.Builder(true);
-    builder.add(new CharsRef("one"), new CharsRef("first"), true);
-    builder.add(new CharsRef("one"), new CharsRef("alpha"), true);
-    builder.add(new CharsRef("one"), new CharsRef("beguine"), true);
-    CharsRef multiWordCharsRef = new CharsRef();
-    SynonymMap.Builder.join(new String[] { "and", "indubitably", "single", "only" }, multiWordCharsRef);
-    builder.add(new CharsRef("one"), multiWordCharsRef, true);
-    SynonymMap.Builder.join(new String[]{"dopple", "ganger"}, multiWordCharsRef);
-    builder.add(new CharsRef("two"), multiWordCharsRef, true);
-    SynonymMap synonymMap = builder.build();
-    TokenStream stream = new SynonymFilter(tokenizer, synonymMap, true);
-    stream = new LimitTokenPositionFilter(stream, 3); // consumeAllTokens defaults to false
-    
-    // "only", the 4th word of multi-word synonym "and indubitably single only" is not emitted, since its position is greater than 3.
-    assertTokenStreamContents(stream, 
-        new String[] { "one", "first", "alpha", "beguine", "and", "two", "indubitably", "dopple", "three", "single", "ganger" },
-        new int[]    {     1,       0,       0,         0,    0,     1,              0,        0,       1,       0,         0 });
-    
+    for (final boolean consumeAll : new boolean[]{true, false}) {
+      MockTokenizer tokenizer = new MockTokenizer(new StringReader("one two three four five"), MockTokenizer.WHITESPACE, false);
+      // if we are consuming all tokens, we can use the checks, otherwise we can't
+      tokenizer.setEnableChecks(consumeAll);
+
+      SynonymMap.Builder builder = new SynonymMap.Builder(true);
+      builder.add(new CharsRef("one"), new CharsRef("first"), true);
+      builder.add(new CharsRef("one"), new CharsRef("alpha"), true);
+      builder.add(new CharsRef("one"), new CharsRef("beguine"), true);
+      CharsRef multiWordCharsRef = new CharsRef();
+      SynonymMap.Builder.join(new String[]{"and", "indubitably", "single", "only"}, multiWordCharsRef);
+      builder.add(new CharsRef("one"), multiWordCharsRef, true);
+      SynonymMap.Builder.join(new String[]{"dopple", "ganger"}, multiWordCharsRef);
+      builder.add(new CharsRef("two"), multiWordCharsRef, true);
+      SynonymMap synonymMap = builder.build();
+      TokenStream stream = new SynonymFilter(tokenizer, synonymMap, true);
+      stream = new LimitTokenPositionFilter(stream, 3, consumeAll);
+
+      // "only", the 4th word of multi-word synonym "and indubitably single only" is not emitted, since its position is greater than 3.
+      assertTokenStreamContents(stream,
+          new String[]{"one", "first", "alpha", "beguine", "and", "two", "indubitably", "dopple", "three", "single", "ganger"},
+          new int[]{1, 0, 0, 0, 0, 1, 0, 0, 1, 0, 0});
+    }
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalArguments() throws Exception {
+    new LimitTokenPositionFilter(new MockTokenizer(new StringReader("one two three four five")), 0);
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilterFactory.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilterFactory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenPositionFilterFactory.java Tue Apr  1 04:52:27 2014
@@ -16,26 +16,30 @@ package org.apache.lucene.analysis.misce
  * limitations under the License.
  */
 
-import java.io.Reader;
-import java.io.StringReader;
-
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
 
+import java.io.Reader;
+import java.io.StringReader;
+
 public class TestLimitTokenPositionFilterFactory extends BaseTokenStreamFactoryTestCase {
 
   public void testMaxPosition1() throws Exception {
-    Reader reader = new StringReader("A1 B2 C3 D4 E5 F6");
-    MockTokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
-    // LimitTokenPositionFilter doesn't consume the entire stream that it wraps
-    tokenizer.setEnableChecks(false);
-    TokenStream stream = tokenizer;
-    stream = tokenFilterFactory("LimitTokenPosition",
-        "maxTokenPosition", "1").create(stream);
-    assertTokenStreamContents(stream, new String[] { "A1" });
+    for (final boolean consumeAll : new boolean[]{true, false}) {
+      Reader reader = new StringReader("A1 B2 C3 D4 E5 F6");
+      MockTokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+      // if we are consuming all tokens, we can use the checks, otherwise we can't
+      tokenizer.setEnableChecks(consumeAll);
+      TokenStream stream = tokenizer;
+      stream = tokenFilterFactory("LimitTokenPosition",
+          LimitTokenPositionFilterFactory.MAX_TOKEN_POSITION_KEY, "1",
+          LimitTokenPositionFilterFactory.CONSUME_ALL_TOKENS_KEY, Boolean.toString(consumeAll)
+      ).create(stream);
+      assertTokenStreamContents(stream, new String[]{"A1"});
+    }
   }
-  
+
   public void testMissingParam() throws Exception {
     try {
       tokenFilterFactory("LimitTokenPosition");
@@ -47,18 +51,22 @@ public class TestLimitTokenPositionFilte
   }
 
   public void testMaxPosition1WithShingles() throws Exception {
-    Reader reader = new StringReader("one two three four five");
-    MockTokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
-    // LimitTokenPositionFilter doesn't consume the entire stream that it wraps
-    tokenizer.setEnableChecks(false);
-    TokenStream stream = tokenizer;
-    stream = tokenFilterFactory("Shingle",
-        "minShingleSize", "2",
-        "maxShingleSize", "3",
-        "outputUnigrams", "true").create(stream);
-    stream = tokenFilterFactory("LimitTokenPosition",
-        "maxTokenPosition", "1").create(stream);
-    assertTokenStreamContents(stream, new String[] { "one", "one two", "one two three" });
+    for (final boolean consumeAll : new boolean[]{true, false}) {
+      Reader reader = new StringReader("one two three four five");
+      MockTokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
+      // if we are consuming all tokens, we can use the checks, otherwise we can't
+      tokenizer.setEnableChecks(consumeAll);
+      TokenStream stream = tokenizer;
+      stream = tokenFilterFactory("Shingle",
+          "minShingleSize", "2",
+          "maxShingleSize", "3",
+          "outputUnigrams", "true").create(stream);
+      stream = tokenFilterFactory("LimitTokenPosition",
+          LimitTokenPositionFilterFactory.MAX_TOKEN_POSITION_KEY, "1",
+          LimitTokenPositionFilterFactory.CONSUME_ALL_TOKENS_KEY, Boolean.toString(consumeAll)
+      ).create(stream);
+      assertTokenStreamContents(stream, new String[]{"one", "one two", "one two three"});
+    }
   }
   
   public void testConsumeAllTokens() throws Exception {
@@ -69,12 +77,14 @@ public class TestLimitTokenPositionFilte
         "consumeAllTokens", "true").create(stream);
     assertTokenStreamContents(stream, new String[] { "A1", "B2", "C3" });
   }
-  
-  /** Test that bogus arguments result in exception */
+ 
+  /**
+   * Test that bogus arguments result in exception
+   */
   public void testBogusArguments() throws Exception {
     try {
-      tokenFilterFactory("LimitTokenPosition", 
-          "maxTokenPosition", "3", 
+      tokenFilterFactory("LimitTokenPosition",
+          "maxTokenPosition", "3",
           "bogusArg", "bogusValue");
       fail();
     } catch (IllegalArgumentException expected) {

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TokenRangeSinkTokenizerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TokenRangeSinkTokenizerTest.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TokenRangeSinkTokenizerTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TokenRangeSinkTokenizerTest.java Tue Apr  1 04:52:27 2014
@@ -1,11 +1,12 @@
 package org.apache.lucene.analysis.sinks;
 
-/**
- * Copyright 2004 The Apache Software Foundation
- *
- * Licensed 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
+/*
+ * 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
  *
@@ -21,6 +22,7 @@ import java.io.StringReader;
 
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.MockTokenizer;
+import org.junit.Test;
 
 public class TokenRangeSinkTokenizerTest extends BaseTokenStreamTestCase {
 
@@ -29,20 +31,25 @@ public class TokenRangeSinkTokenizerTest
     String test = "The quick red fox jumped over the lazy brown dogs";
     TeeSinkTokenFilter tee = new TeeSinkTokenFilter(new MockTokenizer(new StringReader(test), MockTokenizer.WHITESPACE, false));
     TeeSinkTokenFilter.SinkTokenStream rangeToks = tee.newSinkTokenStream(sinkFilter);
-    
+
     int count = 0;
     tee.reset();
     while(tee.incrementToken()) {
       count++;
     }
-    
+
     int sinkCount = 0;
     rangeToks.reset();
     while (rangeToks.incrementToken()) {
       sinkCount++;
     }
-    
+
     assertTrue(count + " does not equal: " + 10, count == 10);
     assertTrue("rangeToks Size: " + sinkCount + " is not: " + 2, sinkCount == 2);
   }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testIllegalArguments() throws Exception {
+    new TokenRangeSinkFilter(4, 2);
+  }
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizerFactory.java?rev=1583531&r1=1583530&r2=1583531&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizerFactory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizerFactory.java Tue Apr  1 04:52:27 2014
@@ -180,4 +180,13 @@ public class TestUAX29URLEmailTokenizerF
       assertTrue(expected.getMessage().contains("Unknown parameters"));
     }
   }
+
+ public void testIllegalArguments() throws Exception {
+    try {
+      tokenizerFactory("UAX29URLEmail", "maxTokenLength", "-1").create(new StringReader("hello"));
+      fail();
+    } catch (IllegalArgumentException expected) {
+      assertTrue(expected.getMessage().contains("maxTokenLength must be greater than zero"));
+    }
+  }
 }