lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From uschind...@apache.org
Subject svn commit: r834847 - in /lucene/java/trunk: contrib/analyzers/common/src/java/org/apache/lucene/analysis/compound/hyphenation/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/query/ contrib/analyzers/common/src/java/org/apache/lucene/anal...
Date Wed, 11 Nov 2009 12:18:35 GMT
Author: uschindler
Date: Wed Nov 11 12:18:34 2009
New Revision: 834847

URL: http://svn.apache.org/viewvc?rev=834847&view=rev
Log:
LUCENE-1257: Generics: *heavy* Robert Muir & mine patch

Modified:
    lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java
    lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java
    lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java
    lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java
    lucene/java/trunk/contrib/analyzers/smartcn/src/java/org/apache/lucene/analysis/cn/smart/hhmm/BiSegGraph.java
    lucene/java/trunk/contrib/ant/src/java/org/apache/lucene/ant/IndexTask.java
    lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Format.java
    lucene/java/trunk/contrib/db/bdb-je/src/java/org/apache/lucene/store/je/JEDirectory.java
    lucene/java/trunk/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/DuplicateFilter.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java
    lucene/java/trunk/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java
    lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java
    lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java
    lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java
    lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java
    lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java
    lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java
    lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java
    lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java
    lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
    lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
    lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java

Modified: lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java (original)
+++ lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java Wed Nov 11 12:18:34 2009
@@ -44,7 +44,7 @@
   /**
    * This map stores hyphenation exceptions
    */
-  protected HashMap stoplist;
+  protected HashMap<String,ArrayList> stoplist;
 
   /**
    * This map stores the character classes
@@ -57,7 +57,7 @@
   private transient TernaryTree ivalues;
 
   public HyphenationTree() {
-    stoplist = new HashMap(23); // usually a small table
+    stoplist = new HashMap<String,ArrayList>(23); // usually a small table
     classmap = new TernaryTree();
     vspace = new ByteVector();
     vspace.alloc(1); // this reserves index 0, which we don't use
@@ -363,7 +363,7 @@
     if (stoplist.containsKey(sw)) {
       // assume only simple hyphens (Hyphen.pre="-", Hyphen.post = Hyphen.no =
       // null)
-      ArrayList hw = (ArrayList) stoplist.get(sw);
+      ArrayList hw = stoplist.get(sw);
       int j = 0;
       for (i = 0; i < hw.size(); i++) {
         Object o = hw.get(i);

Modified: lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java (original)
+++ lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/query/QueryAutoStopWordAnalyzer.java Wed Nov 11 12:18:34 2009
@@ -45,7 +45,7 @@
  */
 public class QueryAutoStopWordAnalyzer extends Analyzer {
   Analyzer delegate;
-  HashMap stopWordsPerField = new HashMap();
+  HashMap<String,HashSet<String>> stopWordsPerField = new HashMap<String,HashSet<String>>();
   //The default maximum percentage (40%) of index documents which
   //can contain a term, after which the term is considered to be a stop word.
   public static final float defaultMaxDocFreqPercent = 0.4f;
@@ -86,9 +86,9 @@
    */
   public int addStopWords(IndexReader reader, int maxDocFreq) throws IOException {
     int numStopWords = 0;
-    Collection fieldNames = reader.getFieldNames(IndexReader.FieldOption.INDEXED);
-    for (Iterator iter = fieldNames.iterator(); iter.hasNext();) {
-      String fieldName = (String) iter.next();
+    Collection<String> fieldNames = reader.getFieldNames(IndexReader.FieldOption.INDEXED);
+    for (Iterator<String> iter = fieldNames.iterator(); iter.hasNext();) {
+      String fieldName = iter.next();
       numStopWords += addStopWords(reader, fieldName, maxDocFreq);
     }
     return numStopWords;
@@ -106,9 +106,9 @@
    */
   public int addStopWords(IndexReader reader, float maxPercentDocs) throws IOException {
     int numStopWords = 0;
-    Collection fieldNames = reader.getFieldNames(IndexReader.FieldOption.INDEXED);
-    for (Iterator iter = fieldNames.iterator(); iter.hasNext();) {
-      String fieldName = (String) iter.next();
+    Collection<String> fieldNames = reader.getFieldNames(IndexReader.FieldOption.INDEXED);
+    for (Iterator<String> iter = fieldNames.iterator(); iter.hasNext();) {
+      String fieldName = iter.next();
       numStopWords += addStopWords(reader, fieldName, maxPercentDocs);
     }
     return numStopWords;
@@ -141,7 +141,7 @@
    * @throws IOException
    */
   public int addStopWords(IndexReader reader, String fieldName, int maxDocFreq) throws IOException {
-    HashSet stopWords = new HashSet();
+    HashSet<String> stopWords = new HashSet<String>();
     String internedFieldName = StringHelper.intern(fieldName);
     TermEnum te = reader.terms(new Term(fieldName));
     Term term = te.term();
@@ -162,7 +162,7 @@
     /* if the stopwords for a field are changed,
      * then saved streams for that field are erased.
      */
-    Map streamMap = (Map) getPreviousTokenStream();
+    Map<String,SavedStreams> streamMap = (Map<String,SavedStreams>) getPreviousTokenStream();
     if (streamMap != null)
       streamMap.remove(fieldName);
     
@@ -177,7 +177,7 @@
     } catch (IOException e) {
       result = delegate.tokenStream(fieldName, reader);
     }
-    HashSet stopWords = (HashSet) stopWordsPerField.get(fieldName);
+    HashSet<String> stopWords = stopWordsPerField.get(fieldName);
     if (stopWords != null) {
       result = new StopFilter(StopFilter.getEnablePositionIncrementsVersionDefault(matchVersion),
                               result, stopWords);
@@ -207,13 +207,13 @@
     }
 
     /* map of SavedStreams for each field */
-    Map streamMap = (Map) getPreviousTokenStream();
+    Map<String,SavedStreams> streamMap = (Map<String,SavedStreams>) getPreviousTokenStream();
     if (streamMap == null) {
-      streamMap = new HashMap();
+      streamMap = new HashMap<String, SavedStreams>();
       setPreviousTokenStream(streamMap);
     }
 
-    SavedStreams streams = (SavedStreams) streamMap.get(fieldName);
+    SavedStreams streams = streamMap.get(fieldName);
     if (streams == null) {
       /* an entry for this field does not exist, create one */
       streams = new SavedStreams();
@@ -221,7 +221,7 @@
       streams.wrapped = delegate.reusableTokenStream(fieldName, reader);
 
       /* if there are any stopwords for the field, save the stopfilter */
-      HashSet stopWords = (HashSet) stopWordsPerField.get(fieldName);
+      HashSet<String> stopWords = stopWordsPerField.get(fieldName);
       if (stopWords != null)
         streams.withStopFilter = new StopFilter(StopFilter.getEnablePositionIncrementsVersionDefault(matchVersion),
                                                 streams.wrapped, stopWords);
@@ -243,7 +243,7 @@
          * field, create a new StopFilter around the new stream
          */
         streams.wrapped = result;
-        HashSet stopWords = (HashSet) stopWordsPerField.get(fieldName);
+        HashSet<String> stopWords = stopWordsPerField.get(fieldName);
         if (stopWords != null)
           streams.withStopFilter = new StopFilter(StopFilter.getEnablePositionIncrementsVersionDefault(matchVersion),
                                                   streams.wrapped, stopWords);
@@ -264,9 +264,9 @@
    */
   public String[] getStopWords(String fieldName) {
     String[] result;
-    HashSet stopWords = (HashSet) stopWordsPerField.get(fieldName);
+    HashSet<String> stopWords = stopWordsPerField.get(fieldName);
     if (stopWords != null) {
-      result = (String[]) stopWords.toArray(new String[stopWords.size()]);
+      result = stopWords.toArray(new String[stopWords.size()]);
     } else {
       result = new String[0];
     }
@@ -279,16 +279,16 @@
    * @return the stop words (as terms)
    */
   public Term[] getStopWords() {
-    ArrayList allStopWords = new ArrayList();
-    for (Iterator iter = stopWordsPerField.keySet().iterator(); iter.hasNext();) {
-      String fieldName = (String) iter.next();
-      HashSet stopWords = (HashSet) stopWordsPerField.get(fieldName);
-      for (Iterator iterator = stopWords.iterator(); iterator.hasNext();) {
-        String text = (String) iterator.next();
+    ArrayList<Term> allStopWords = new ArrayList<Term>();
+    for (Iterator<String> iter = stopWordsPerField.keySet().iterator(); iter.hasNext();) {
+      String fieldName = iter.next();
+      HashSet<String> stopWords = stopWordsPerField.get(fieldName);
+      for (Iterator<String> iterator = stopWords.iterator(); iterator.hasNext();) {
+        String text = iterator.next();
         allStopWords.add(new Term(fieldName, text));
       }
     }
-    return (Term[]) allStopWords.toArray(new Term[allStopWords.size()]);
+    return allStopWords.toArray(new Term[allStopWords.size()]);
 	}
 
 }

Modified: lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java (original)
+++ lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java Wed Nov 11 12:18:34 2009
@@ -29,6 +29,7 @@
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.AttributeSource.State;
 
 /**
  * <p>A ShingleFilter constructs shingles (token n-grams) from a token stream.
@@ -43,7 +44,7 @@
  */
 public final class ShingleFilter extends TokenFilter {
 
-  private LinkedList shingleBuf = new LinkedList();
+  private LinkedList<State> shingleBuf = new LinkedList<State>();
   private StringBuilder[] shingles;
   private String tokenType = "shingle";
 
@@ -172,7 +173,7 @@
         }
       }
       
-      nextToken = (AttributeSource.State) shingleBuf.getFirst();
+      nextToken = shingleBuf.getFirst();
       
       if (outputUnigrams) {
         if (shingleBufferPosition == 0) {
@@ -313,8 +314,8 @@
     }
 
     int i = 0;
-    for (Iterator it = shingleBuf.iterator(); it.hasNext(); ) {
-      restoreState((AttributeSource.State) it.next());
+    for (Iterator<State> it = shingleBuf.iterator(); it.hasNext(); ) {
+      restoreState(it.next());
       for (int j = i; j < shingles.length; j++) {
         if (shingles[j].length() != 0) {
           shingles[j].append(TOKEN_SEPARATOR);

Modified: lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java (original)
+++ lucene/java/trunk/contrib/analyzers/common/src/java/org/apache/lucene/analysis/shingle/ShingleMatrixFilter.java Wed Nov 11 12:18:34 2009
@@ -30,6 +30,8 @@
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.miscellaneous.EmptyTokenStream;
 import org.apache.lucene.analysis.payloads.PayloadHelper;
+import org.apache.lucene.analysis.shingle.ShingleMatrixFilter.Matrix.Column;
+import org.apache.lucene.analysis.shingle.ShingleMatrixFilter.Matrix.Column.Row;
 import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
@@ -577,12 +579,12 @@
    * @param currentPermutationRows index to Matrix.Column.Row from the position of tokens in parameter currentPermutationTokens
    * @param currentPermuationTokens tokens of the current permutation of rows in the matrix.
    */
-  public void updateToken(Token token, List shingle, int currentPermutationStartOffset, List currentPermutationRows, List currentPermuationTokens) {
+  public void updateToken(Token token, List<Token> shingle, int currentPermutationStartOffset, List<Row> currentPermutationRows, List<Token> currentPermuationTokens) {
     token.setType(ShingleMatrixFilter.class.getName());
     token.setFlags(0);
     token.setPositionIncrement(1);
-    token.setStartOffset(((Token) shingle.get(0)).startOffset());
-    token.setEndOffset(((Token) shingle.get(shingle.size() - 1)).endOffset());
+    token.setStartOffset(shingle.get(0).startOffset());
+    token.setEndOffset(shingle.get(shingle.size() - 1).endOffset());
     settingsCodec.setWeight(token, calculateShingleWeight(token, shingle, currentPermutationStartOffset, currentPermutationRows, currentPermuationTokens));
   }
 
@@ -602,7 +604,7 @@
    * @param currentPermuationTokens all tokens in the current row permutation of the matrix. A sub list (parameter offset, parameter shingle.size) equals parameter shingle.
    * @return weight to be set for parameter shingleToken
    */
-  public float calculateShingleWeight(Token shingleToken, List shingle, int currentPermutationStartOffset, List currentPermutationRows, List currentPermuationTokens) {
+  public float calculateShingleWeight(Token shingleToken, List<Token> shingle, int currentPermutationStartOffset, List<Row> currentPermutationRows, List<Token> currentPermuationTokens) {
     double[] weights = new double[shingle.size()];
 
     double total = 0f;
@@ -610,7 +612,7 @@
 
 
     for (int i=0; i<weights.length; i++) {
-      weights[i] = settingsCodec.getWeight((Token) shingle.get(i));
+      weights[i] = settingsCodec.getWeight(shingle.get(i));
 
       double tmp = weights[i];
       if (tmp > top) {
@@ -705,7 +707,7 @@
 
     private List<Column> columns = new ArrayList<Column>();
 
-    public List getColumns() {
+    public List<Column> getColumns() {
       return columns;
     }
 

Modified: lucene/java/trunk/contrib/analyzers/smartcn/src/java/org/apache/lucene/analysis/cn/smart/hhmm/BiSegGraph.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/analyzers/smartcn/src/java/org/apache/lucene/analysis/cn/smart/hhmm/BiSegGraph.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/analyzers/smartcn/src/java/org/apache/lucene/analysis/cn/smart/hhmm/BiSegGraph.java (original)
+++ lucene/java/trunk/contrib/analyzers/smartcn/src/java/org/apache/lucene/analysis/cn/smart/hhmm/BiSegGraph.java Wed Nov 11 12:18:34 2009
@@ -206,7 +206,7 @@
 
     rpath.add(current);
     while (current != 0) {
-      PathNode currentPathNode = (PathNode) path.get(current);
+      PathNode currentPathNode = path.get(current);
       preNode = currentPathNode.preNode;
       rpath.add(Integer.valueOf(preNode));
       current = preNode;

Modified: lucene/java/trunk/contrib/ant/src/java/org/apache/lucene/ant/IndexTask.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/ant/src/java/org/apache/lucene/ant/IndexTask.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/ant/src/java/org/apache/lucene/ant/IndexTask.java (original)
+++ lucene/java/trunk/contrib/ant/src/java/org/apache/lucene/ant/IndexTask.java Wed Nov 11 12:18:34 2009
@@ -63,7 +63,7 @@
   /**
    *  resources
    */
-  protected Vector rcs = new Vector();
+  protected Vector<ResourceCollection> rcs = new Vector<ResourceCollection>();
 
   /**
    *  overwrite index?
@@ -225,10 +225,9 @@
 
     // construct handler and analyzer dynamically
     try {
-      Class clazz = Class.forName(handlerClassName);
-      handler = (DocumentHandler) clazz.newInstance();
+      handler = Class.forName(handlerClassName).asSubclass(DocumentHandler.class).newInstance();
 
-      analyzer = this.createAnalyzer(analyzerClassName);
+      analyzer = IndexTask.createAnalyzer(analyzerClassName);
     } catch (Exception e) {
       throw new BuildException(e);
     }
@@ -292,7 +291,7 @@
         writer.setMergeFactor(mergeFactor);
 
         for (int i = 0; i < rcs.size(); i++) {
-          ResourceCollection rc = (ResourceCollection) rcs.elementAt(i);
+          ResourceCollection rc = rcs.elementAt(i);
           if (rc.isFilesystemOnly()) {
             Iterator resources = rc.iterator();
             while (resources.hasNext()) {
@@ -415,7 +414,7 @@
   }
 
  public static class AnalyzerType extends EnumeratedAttribute {
-    private static Map analyzerLookup = new HashMap();
+    private static Map<String,String> analyzerLookup = new HashMap<String,String>();
 
     static {
       analyzerLookup.put("simple", SimpleAnalyzer.class.getName());
@@ -429,12 +428,12 @@
      */
     @Override
     public String[] getValues() {
-      Set keys = analyzerLookup.keySet();
-      return (String[]) keys.toArray(new String[0]);
+      Set<String> keys = analyzerLookup.keySet();
+      return keys.toArray(new String[0]);
     }
 
     public String getClassname() {
-      return (String) analyzerLookup.get(getValue());
+      return analyzerLookup.get(getValue());
     }
   }
 }

Modified: lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Format.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Format.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Format.java (original)
+++ lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Format.java Wed Nov 11 12:18:34 2009
@@ -112,7 +112,7 @@
    * @param cls class whose simple name is required 
    * @return simple class name
    */
-  public static String simpleName (Class cls) {
+  public static String simpleName (Class<?> cls) {
     String c = cls.getName();
     String p = cls.getPackage().getName();
     int k = c.lastIndexOf(p+".");

Modified: lucene/java/trunk/contrib/db/bdb-je/src/java/org/apache/lucene/store/je/JEDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/db/bdb-je/src/java/org/apache/lucene/store/je/JEDirectory.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/db/bdb-je/src/java/org/apache/lucene/store/je/JEDirectory.java (original)
+++ lucene/java/trunk/contrib/db/bdb-je/src/java/org/apache/lucene/store/je/JEDirectory.java Wed Nov 11 12:18:34 2009
@@ -51,7 +51,7 @@
 
 public class JEDirectory extends Directory {
 
-    protected Set openFiles = Collections.synchronizedSet(new HashSet());
+    protected Set<JEIndexOutput> openFiles = Collections.synchronizedSet(new HashSet<JEIndexOutput>());
 
     protected Database files, blocks;
 
@@ -102,11 +102,11 @@
      * @see #setTransaction
      */
     public void flush() throws IOException {
-        Iterator iterator = openFiles.iterator();
+        Iterator<JEIndexOutput> iterator = openFiles.iterator();
 
         while (iterator.hasNext()) {
             System.out
-                    .println(((JEIndexOutput) iterator.next()).file.getName());
+                    .println(iterator.next().file.getName());
             // ((IndexOutput) iterator.next()).flush();
         }
     }
@@ -149,7 +149,7 @@
     @Override
     public String[] listAll() throws IOException {
         Cursor cursor = null;
-        List list = new ArrayList();
+        List<String> list = new ArrayList<String>();
 
         try {
             try {
@@ -186,7 +186,7 @@
             throw new IOException(e.getMessage());
         }
 
-        return (String[]) list.toArray(new String[list.size()]);
+        return list.toArray(new String[list.size()]);
     }
 
     @Override

Modified: lucene/java/trunk/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java (original)
+++ lucene/java/trunk/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java Wed Nov 11 12:18:34 2009
@@ -54,7 +54,7 @@
 
 public class DbDirectory extends Directory {
 
-    protected Set openFiles = Collections.synchronizedSet(new HashSet());
+    protected Set<DbIndexOutput> openFiles = Collections.synchronizedSet(new HashSet<DbIndexOutput>());
     protected Db files, blocks;
     protected DbTxn txn;
     protected int flags;
@@ -113,10 +113,10 @@
     public void flush()
         throws IOException
     {
-        Iterator iterator = openFiles.iterator();
+        Iterator<DbIndexOutput> iterator = openFiles.iterator();
         
         while (iterator.hasNext())
-            ((IndexOutput) iterator.next()).flush();
+            iterator.next().flush();
     }
 
     @Override
@@ -169,7 +169,7 @@
         throws IOException
     {
         Dbc cursor = null;
-        List list = new ArrayList();
+        List<String> list = new ArrayList<String>();
 
         try {
             try {
@@ -209,7 +209,7 @@
             throw new IOException(e.getMessage());
         }
 
-        return (String[]) list.toArray(new String[list.size()]);
+        return list.toArray(new String[list.size()]);
     }
 
     @Override

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java Wed Nov 11 12:18:34 2009
@@ -24,7 +24,6 @@
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.util.OpenBitSet;
 import org.apache.lucene.util.OpenBitSetDISI;
-import org.apache.lucene.util.SortedVIntList;
 
 /**
  * A container Filter that allows Boolean composition of Filters.
@@ -62,7 +61,7 @@
         if (res == null) {
           res = new OpenBitSetDISI(getDISI(shouldFilters, i, reader), reader.maxDoc());
         } else { 
-          DocIdSet dis = ((Filter)shouldFilters.get(i)).getDocIdSet(reader);
+          DocIdSet dis = shouldFilters.get(i).getDocIdSet(reader);
           if(dis instanceof OpenBitSet) {
             // optimized case for OpenBitSets
             res.or((OpenBitSet) dis);

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/DuplicateFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/DuplicateFilter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/DuplicateFilter.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/DuplicateFilter.java Wed Nov 11 12:18:34 2009
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 import java.io.IOException;
-import java.util.BitSet;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java Wed Nov 11 12:18:34 2009
@@ -25,7 +25,6 @@
 import java.util.Iterator;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 import org.apache.lucene.index.IndexReader;
@@ -52,7 +51,7 @@
 {
     static Similarity sim=new DefaultSimilarity();
     Query rewrittenQuery=null;
-    ArrayList fieldVals=new ArrayList();
+    ArrayList<FieldVals> fieldVals=new ArrayList<FieldVals>();
     Analyzer analyzer;
     
     ScoreTermQueue q;
@@ -190,7 +189,7 @@
         
         int corpusNumDocs=reader.numDocs();
         Term internSavingTemplateTerm =new Term(f.fieldName); //optimization to avoid constructing new Term() objects
-        HashSet processedTerms=new HashSet();
+        HashSet<String> processedTerms=new HashSet<String>();
         while (ts.incrementToken()) 
         {
                 String term = termAtt.term();
@@ -220,7 +219,7 @@
     	                if(variantsQ.size() < MAX_VARIANTS_PER_TERM || score > minScore){
     	                    ScoreTerm st=new ScoreTerm(possibleMatch,score,startTerm);                    
     	                    variantsQ.insertWithOverflow(st);
-    	                    minScore = ((ScoreTerm)variantsQ.top()).score; // maintain minScore
+    	                    minScore = variantsQ.top().score; // maintain minScore
     	                }
                     }
                 }
@@ -239,7 +238,7 @@
 	                int size = variantsQ.size();
 	                for(int i = 0; i < size; i++)
 	                {
-	                  ScoreTerm st = (ScoreTerm) variantsQ.pop();
+	                  ScoreTerm st = variantsQ.pop();
 	                  st.score=(st.score*st.score)*sim.idf(df,corpusNumDocs);
 	                  q.insertWithOverflow(st);
 	                }                            
@@ -256,9 +255,9 @@
             return rewrittenQuery;
         }
         //load up the list of possible terms
-        for (Iterator iter = fieldVals.iterator(); iter.hasNext();)
+        for (Iterator<FieldVals> iter = fieldVals.iterator(); iter.hasNext();)
 		{
-			FieldVals f = (FieldVals) iter.next();
+			FieldVals f = iter.next();
 			addTerms(reader,f);			
 		}
         //clear the list of fields
@@ -270,27 +269,27 @@
         //create BooleanQueries to hold the variants for each token/field pair and ensure it
         // has no coord factor
         //Step 1: sort the termqueries by term/field
-        HashMap variantQueries=new HashMap();
+        HashMap<Term,ArrayList<ScoreTerm>> variantQueries=new HashMap<Term,ArrayList<ScoreTerm>>();
         int size = q.size();
         for(int i = 0; i < size; i++)
         {
-          ScoreTerm st = (ScoreTerm) q.pop();
-          ArrayList l=(ArrayList) variantQueries.get(st.fuzziedSourceTerm);
+          ScoreTerm st = q.pop();
+          ArrayList<ScoreTerm> l= variantQueries.get(st.fuzziedSourceTerm);
           if(l==null)
           {
-              l=new ArrayList();
+              l=new ArrayList<ScoreTerm>();
               variantQueries.put(st.fuzziedSourceTerm,l);
           }
           l.add(st);
         }
         //Step 2: Organize the sorted termqueries into zero-coord scoring boolean queries
-        for (Iterator iter = variantQueries.values().iterator(); iter.hasNext();)
+        for (Iterator<ArrayList<ScoreTerm>> iter = variantQueries.values().iterator(); iter.hasNext();)
         {
-            ArrayList variants = (ArrayList) iter.next();
+            ArrayList<ScoreTerm> variants = iter.next();
             if(variants.size()==1)
             {
                 //optimize where only one selected variant
-                ScoreTerm st=(ScoreTerm) variants.get(0);
+                ScoreTerm st= variants.get(0);
                 TermQuery tq = new FuzzyTermQuery(st.term,ignoreTF);
                 tq.setBoost(st.score); // set the boost to a mix of IDF and score
                 bq.add(tq, BooleanClause.Occur.SHOULD); 
@@ -298,10 +297,10 @@
             else
             {
                 BooleanQuery termVariants=new BooleanQuery(true); //disable coord and IDF for these term variants
-                for (Iterator iterator2 = variants.iterator(); iterator2
+                for (Iterator<ScoreTerm> iterator2 = variants.iterator(); iterator2
                         .hasNext();)
                 {
-                    ScoreTerm st = (ScoreTerm) iterator2.next();
+                    ScoreTerm st = iterator2.next();
                     TermQuery tq = new FuzzyTermQuery(st.term,ignoreTF);      // found a match
                     tq.setBoost(st.score); // set the boost using the ScoreTerm's score
                     termVariants.add(tq, BooleanClause.Occur.SHOULD);          // add to query                    

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java Wed Nov 11 12:18:34 2009
@@ -18,7 +18,6 @@
  */
 
 import java.io.IOException;
-import java.util.BitSet;
 import java.util.Iterator;
 import java.util.Set;
 import java.util.TreeSet;
@@ -38,7 +37,7 @@
  */
 public class TermsFilter extends Filter
 {
-	Set terms=new TreeSet();
+	Set<Term> terms=new TreeSet<Term>();
 	
 	/**
 	 * Adds a term to the list of acceptable terms   
@@ -59,9 +58,9 @@
         TermDocs td = reader.termDocs();
         try
         {
-            for (Iterator iter = terms.iterator(); iter.hasNext();)
+            for (Iterator<Term> iter = terms.iterator(); iter.hasNext();)
             {
-                Term term = (Term) iter.next();
+                Term term = iter.next();
                 td.seek(term);
                 while (td.next())
                 {
@@ -92,9 +91,9 @@
 	public int hashCode()
 	{
 		int hash=9;
-		for (Iterator iter = terms.iterator(); iter.hasNext();)
+		for (Iterator<Term> iter = terms.iterator(); iter.hasNext();)
 		{
-			Term term = (Term) iter.next();
+			Term term = iter.next();
 			hash = 31 * hash + term.hashCode();			
 		}
 		return hash;

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java Wed Nov 11 12:18:34 2009
@@ -221,12 +221,12 @@
 	 * @see #setStopWords
 	 * @see #getStopWords
 	 */
-	public static final Set DEFAULT_STOP_WORDS = null;
+	public static final Set<?> DEFAULT_STOP_WORDS = null;
 
 	/**
 	 * Current set of stop words.
 	 */
-	private Set stopWords = DEFAULT_STOP_WORDS;
+	private Set<?> stopWords = DEFAULT_STOP_WORDS;
 
     /**
      * Return a Query with no more than this many terms.
@@ -532,7 +532,7 @@
 	 * @see org.apache.lucene.analysis.StopFilter#makeStopSet StopFilter.makeStopSet()
 	 * @see #getStopWords	 
 	 */
-	public void setStopWords(Set stopWords) {
+	public void setStopWords(Set<?> stopWords) {
 		this.stopWords = stopWords;
 	}
 
@@ -540,7 +540,7 @@
 	 * Get the current stop words being used.
 	 * @see #setStopWords
 	 */
-	public Set getStopWords() {
+	public Set<?> getStopWords() {
 		return stopWords;
 	}
 		
@@ -594,8 +594,8 @@
     public Query like(int docNum) throws IOException {
         if (fieldNames == null) {
             // gather list of valid fields from lucene
-            Collection fields = ir.getFieldNames( IndexReader.FieldOption.INDEXED);
-            fieldNames = (String[]) fields.toArray(new String[fields.size()]);
+            Collection<String> fields = ir.getFieldNames( IndexReader.FieldOption.INDEXED);
+            fieldNames = fields.toArray(new String[fields.size()]);
         }
 
         return createQuery(retrieveTerms(docNum));
@@ -609,8 +609,8 @@
     public Query like(File f) throws IOException {
         if (fieldNames == null) {
             // gather list of valid fields from lucene
-            Collection fields = ir.getFieldNames( IndexReader.FieldOption.INDEXED);
-            fieldNames = (String[]) fields.toArray(new String[fields.size()]);
+            Collection<String> fields = ir.getFieldNames( IndexReader.FieldOption.INDEXED);
+            fieldNames = fields.toArray(new String[fields.size()]);
         }
 
         return like(new FileReader(f));
@@ -646,7 +646,7 @@
     /**
      * Create the More like query from a PriorityQueue
      */
-    private Query createQuery(PriorityQueue q) {
+    private Query createQuery(PriorityQueue<Object[]> q) {
         BooleanQuery query = new BooleanQuery();
         Object cur;
         int qterms = 0;
@@ -686,16 +686,16 @@
      *
      * @param words a map of words keyed on the word(String) with Int objects as the values.
      */
-    private PriorityQueue createQueue(Map words) throws IOException {
+    private PriorityQueue<Object[]> createQueue(Map<String,Int> words) throws IOException {
         // have collected all words in doc and their freqs
         int numDocs = ir.numDocs();
         FreqQ res = new FreqQ(words.size()); // will order words by score
 
-        Iterator it = words.keySet().iterator();
+        Iterator<String> it = words.keySet().iterator();
         while (it.hasNext()) { // for every word
-            String word = (String) it.next();
+            String word = it.next();
 
-            int tf = ((Int) words.get(word)).x; // term freq in the source doc
+            int tf = words.get(word).x; // term freq in the source doc
             if (minTermFreq > 0 && tf < minTermFreq) {
                 continue; // filter out words that don't occur enough times in the source
             }
@@ -825,8 +825,8 @@
      *
      * @param docNum the id of the lucene document from which to find terms
      */
-    public PriorityQueue retrieveTerms(int docNum) throws IOException {
-        Map termFreqMap = new HashMap();
+    public PriorityQueue<Object[]> retrieveTerms(int docNum) throws IOException {
+        Map<String,Int> termFreqMap = new HashMap<String,Int>();
         for (int i = 0; i < fieldNames.length; i++) {
             String fieldName = fieldNames[i];
             TermFreqVector vector = ir.getTermFreqVector(docNum, fieldName);
@@ -856,7 +856,7 @@
 	 * @param termFreqMap a Map of terms and their frequencies
 	 * @param vector List of terms and their frequencies for a doc/field
 	 */
-	private void addTermFrequencies(Map termFreqMap, TermFreqVector vector)
+	private void addTermFrequencies(Map<String,Int> termFreqMap, TermFreqVector vector)
 	{
 		String[] terms = vector.getTerms();
 		int freqs[]=vector.getTermFrequencies();
@@ -867,7 +867,7 @@
 				continue;
 			}
 		    // increment frequency
-		    Int cnt = (Int) termFreqMap.get(term);
+		    Int cnt = termFreqMap.get(term);
 		    if (cnt == null) {
 		    	cnt=new Int();
 				termFreqMap.put(term, cnt);
@@ -884,7 +884,7 @@
 	 * @param termFreqMap a Map of terms and their frequencies
 	 * @param fieldName Used by analyzer for any special per-field analysis
 	 */
-	private void addTermFrequencies(Reader r, Map termFreqMap, String fieldName)
+	private void addTermFrequencies(Reader r, Map<String,Int> termFreqMap, String fieldName)
 		throws IOException
 	{
 		   TokenStream ts = analyzer.tokenStream(fieldName, r);
@@ -904,7 +904,7 @@
 				}
 				
 				// increment frequency
-				Int cnt = (Int) termFreqMap.get(word);
+				Int cnt = termFreqMap.get(word);
 				if (cnt == null) {
 					termFreqMap.put(word, new Int());
 				}
@@ -958,8 +958,8 @@
 	 *
 	 * @see #retrieveInterestingTerms
      */
-    public PriorityQueue retrieveTerms(Reader r) throws IOException {
-        Map words = new HashMap();
+    public PriorityQueue<Object[]> retrieveTerms(Reader r) throws IOException {
+        Map<String,Int> words = new HashMap<String,Int>();
         for (int i = 0; i < fieldNames.length; i++) {
             String fieldName = fieldNames[i];
 			addTermFrequencies(r, words, fieldName);
@@ -971,8 +971,8 @@
    * @see #retrieveInterestingTerms(java.io.Reader) 
    */
   public String [] retrieveInterestingTerms(int docNum) throws IOException{
-    ArrayList al = new ArrayList( maxQueryTerms);
-		PriorityQueue pq = retrieveTerms(docNum);
+    ArrayList<Object> al = new ArrayList<Object>( maxQueryTerms);
+		PriorityQueue<Object[]> pq = retrieveTerms(docNum);
 		Object cur;
 		int lim = maxQueryTerms; // have to be careful, retrieveTerms returns all words but that's probably not useful to our caller...
 		// we just want to return the top words
@@ -981,7 +981,7 @@
 			al.add( ar[ 0]); // the 1st entry is the interesting word
 		}
 		String[] res = new String[ al.size()];
-		return (String[]) al.toArray( res);
+		return al.toArray( res);
   }
 
   /**
@@ -994,8 +994,8 @@
 	 * @see #setMaxQueryTerms
 	 */
 	public String[] retrieveInterestingTerms( Reader r) throws IOException {
-		ArrayList al = new ArrayList( maxQueryTerms);
-		PriorityQueue pq = retrieveTerms( r);
+		ArrayList<Object> al = new ArrayList<Object>( maxQueryTerms);
+		PriorityQueue<Object[]> pq = retrieveTerms( r);
 		Object cur;
 		int lim = maxQueryTerms; // have to be careful, retrieveTerms returns all words but that's probably not useful to our caller...
 		// we just want to return the top words
@@ -1004,7 +1004,7 @@
 			al.add( ar[ 0]); // the 1st entry is the interesting word
 		}
 		String[] res = new String[ al.size()];
-		return (String[]) al.toArray( res);
+		return al.toArray( res);
 	}
 
     /**

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java Wed Nov 11 12:18:34 2009
@@ -46,7 +46,7 @@
     float percentTermsToMatch=0.3f;
     int minTermFrequency=1;
     int maxQueryTerms=5;
-    Set stopWords=null;
+    Set<?> stopWords=null;
 	int minDocFreq=-1;
     
     
@@ -145,11 +145,11 @@
 	{
 		this.moreLikeFields = moreLikeFields;
 	}
-    public Set getStopWords()
+    public Set<?> getStopWords()
     {
         return stopWords;
     }
-    public void setStopWords(Set stopWords)
+    public void setStopWords(Set<?> stopWords)
     {
         this.stopWords = stopWords;
     }

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java Wed Nov 11 12:18:34 2009
@@ -82,14 +82,14 @@
     public static Query formSimilarQuery( String body,
 										  Analyzer a,
 										  String field,
-										  Set stop)
+										  Set<?> stop)
 										  throws IOException
 	{	
 		TokenStream ts = a.tokenStream( field, new StringReader( body));
 		TermAttribute termAtt = ts.addAttribute(TermAttribute.class);
 		
 		BooleanQuery tmp = new BooleanQuery();
-		Set already = new HashSet(); // ignore dups
+		Set<String> already = new HashSet<String>(); // ignore dups
 		while (ts.incrementToken()) {
 		  String word = termAtt.term();
 			// ignore opt stop words

Modified: lucene/java/trunk/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java (original)
+++ lucene/java/trunk/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java Wed Nov 11 12:18:34 2009
@@ -82,8 +82,8 @@
     return term.field();
   }
 
-  public Collection getTerms() {
-    Collection terms = new ArrayList();
+  public Collection<Term> getTerms() {
+    Collection<Term> terms = new ArrayList<Term>();
     terms.add(term);
     return terms;
   }

Modified: lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java (original)
+++ lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java Wed Nov 11 12:18:34 2009
@@ -37,7 +37,7 @@
  */
 public class SnowballAnalyzer extends Analyzer {
   private String name;
-  private Set stopSet;
+  private Set<?> stopSet;
   private final Version matchVersion;
 
   /** Builds the named analyzer with no stop words. */

Modified: lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java (original)
+++ lucene/java/trunk/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java Wed Nov 11 12:18:34 2009
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
@@ -55,7 +54,7 @@
   public SnowballFilter(TokenStream in, String name) {
     super(in);
     try {      
-      Class stemClass = Class.forName("org.tartarus.snowball.ext." + name + "Stemmer");
+      Class<?> stemClass = Class.forName("org.tartarus.snowball.ext." + name + "Stemmer");
       stemmer = (SnowballProgram) stemClass.newInstance();
     } catch (Exception e) {
       throw new RuntimeException(e.toString());

Modified: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java (original)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java Wed Nov 11 12:18:34 2009
@@ -18,8 +18,6 @@
 
 import java.io.IOException;
 import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;

Modified: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java (original)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java Wed Nov 11 12:18:34 2009
@@ -98,7 +98,7 @@
   
   
   public static void main(String args[]){ 
-    DistanceHandler db = new DistanceHandler(new HashMap(), new HashMap(), Precision.TWOHUNDREDFEET);
+    DistanceHandler db = new DistanceHandler(new HashMap<Integer,Double>(), new HashMap<String,Double>(), Precision.TWOHUNDREDFEET);
     System.out.println(DistanceHandler.getPrecision(-1234.123456789, db.getPrecision()));
   }
 }

Modified: lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java (original)
+++ lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java Wed Nov 11 12:18:34 2009
@@ -31,5 +31,5 @@
    * Return all words present in the dictionary
    * @return Iterator
    */
-  Iterator getWordsIterator();
+  Iterator<String> getWordsIterator();
 }

Modified: lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java (original)
+++ lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java Wed Nov 11 12:18:34 2009
@@ -46,12 +46,12 @@
     this.field = StringHelper.intern(field);
   }
 
-  public final Iterator getWordsIterator() {
+  public final Iterator<String> getWordsIterator() {
     return new LuceneIterator();
   }
 
 
-  final class LuceneIterator implements Iterator {
+  final class LuceneIterator implements Iterator<String> {
     private TermEnum termEnum;
     private Term actualTerm;
     private boolean hasNextCalled;
@@ -64,7 +64,7 @@
       }
     }
 
-    public Object next() {
+    public String next() {
       if (!hasNextCalled) {
         hasNext();
       }

Modified: lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java (original)
+++ lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java Wed Nov 11 12:18:34 2009
@@ -51,12 +51,12 @@
     in = new BufferedReader(reader);
   }
 
-  public Iterator getWordsIterator() {
+  public Iterator<String> getWordsIterator() {
     return new fileIterator();
   }
 
-  final class fileIterator implements Iterator {
-    public Object next() {
+  final class fileIterator implements Iterator<String> {
+    public String next() {
       if (!hasNextCalled) {
         hasNext();
       }

Modified: lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java (original)
+++ lucene/java/trunk/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java Wed Nov 11 12:18:34 2009
@@ -249,7 +249,7 @@
       sugQueue.insertWithOverflow(sugWord);
       if (sugQueue.size() == numSug) {
         // if queue full, maintain the minScore score
-        min = ((SuggestWord) sugQueue.top()).score;
+        min = sugQueue.top().score;
       }
       sugWord = new SuggestWord();
     }
@@ -257,7 +257,7 @@
     // convert to array string
     String[] list = new String[sugQueue.size()];
     for (int i = sugQueue.size() - 1; i >= 0; i--) {
-      list[i] = ((SuggestWord) sugQueue.pop()).string;
+      list[i] = sugQueue.pop().string;
     }
 
     return list;
@@ -329,9 +329,9 @@
     writer.setMergeFactor(mergeFactor);
     writer.setRAMBufferSizeMB(ramMB);
 
-    Iterator iter = dict.getWordsIterator();
+    Iterator<String> iter = dict.getWordsIterator();
     while (iter.hasNext()) {
-      String word = (String) iter.next();
+      String word = iter.next();
 
       int len = word.length();
       if (len < 3) {

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java Wed Nov 11 12:18:34 2009
@@ -22,7 +22,7 @@
 import org.apache.lucene.search.BooleanClause;
 
 public class AndQuery extends ComposedQuery { 
-  public AndQuery(List queries, boolean inf, String opName) { 
+  public AndQuery(List<SrndQuery> queries, boolean inf, String opName) { 
     super(queries, inf, opName);
   }
   

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java Wed Nov 11 12:18:34 2009
@@ -20,6 +20,8 @@
 import java.util.ArrayList;
 import java.util.Iterator;
 
+import org.apache.lucene.search.Query;
+
 public abstract class ComposedQuery extends SrndQuery { 
   
   public ComposedQuery(List qs, boolean operatorInfix, String opName) {
@@ -47,8 +49,8 @@
   private boolean operatorInfix; 
   public boolean isOperatorInfix() { return operatorInfix; } /* else prefix operator */
   
-  public List makeLuceneSubQueriesField(String fn, BasicQueryFactory qf) {
-    List luceneSubQueries = new ArrayList();
+  public List<Query> makeLuceneSubQueriesField(String fn, BasicQueryFactory qf) {
+    List<Query> luceneSubQueries = new ArrayList<Query>();
     Iterator sqi = getSubQueriesIterator();
     while (sqi.hasNext()) {
       luceneSubQueries.add( ((SrndQuery) sqi.next()).makeLuceneQueryField(fn, qf));

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java Wed Nov 11 12:18:34 2009
@@ -29,7 +29,7 @@
 
 public class DistanceQuery extends ComposedQuery implements DistanceSubQuery {
   public DistanceQuery(
-      List queries,
+      List<SrndQuery> queries,
       boolean infix,
       int opDistance,
       String opName,

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java Wed Nov 11 12:18:34 2009
@@ -24,11 +24,11 @@
 
 public class FieldsQuery extends SrndQuery { /* mostly untested */
   private SrndQuery q;
-  private List fieldNames;
+  private List<String> fieldNames;
   private final char fieldOp;
   private final String OrOperatorName = "OR"; /* for expanded queries, not normally visible */
   
-  public FieldsQuery(SrndQuery q, List fieldNames, char fieldOp) {
+  public FieldsQuery(SrndQuery q, List<String> fieldNames, char fieldOp) {
     this.q = q;
     this.fieldNames = fieldNames;
     this.fieldOp = fieldOp;
@@ -36,7 +36,7 @@
   
   public FieldsQuery(SrndQuery q, String fieldName, char fieldOp) {
     this.q = q;
-    fieldNames = new ArrayList();
+    fieldNames = new ArrayList<String>();
     fieldNames.add(fieldName);
     this.fieldOp = fieldOp;
   }
@@ -48,14 +48,14 @@
   
   public Query makeLuceneQueryNoBoost(BasicQueryFactory qf) {
     if (fieldNames.size() == 1) { /* single field name: no new queries needed */
-      return q.makeLuceneQueryFieldNoBoost((String) fieldNames.get(0), qf);
+      return q.makeLuceneQueryFieldNoBoost(fieldNames.get(0), qf);
     } else { /* OR query over the fields */
-      List queries = new ArrayList();
-      Iterator fni = getFieldNames().listIterator();
+      List<SrndQuery> queries = new ArrayList<SrndQuery>();
+      Iterator<String> fni = getFieldNames().listIterator();
       SrndQuery qc;
       while (fni.hasNext()) {
         qc = (SrndQuery) q.clone();
-        queries.add( new FieldsQuery( qc, (String) fni.next(), fieldOp));
+        queries.add( new FieldsQuery( qc, fni.next(), fieldOp));
       }
       boolean infix = true;
       OrQuery oq = new OrQuery(queries,
@@ -72,7 +72,7 @@
   }
 
   
-  public List getFieldNames() {return fieldNames;}
+  public List<String> getFieldNames() {return fieldNames;}
 
   public char getFieldOperator() { return fieldOp;}
   
@@ -87,9 +87,9 @@
   }
   
   protected void fieldNamesToString(StringBuilder r) {
-    Iterator fni = getFieldNames().listIterator();
+    Iterator<String> fni = getFieldNames().listIterator();
     while (fni.hasNext()) {
-      r.append((String) fni.next());
+      r.append(fni.next());
       r.append(getFieldOperator());
     }
   }

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java Wed Nov 11 12:18:34 2009
@@ -22,13 +22,13 @@
 import org.apache.lucene.search.BooleanClause;
 
 public class NotQuery extends ComposedQuery { 
-  public NotQuery(List queries, String opName) { super(queries, true /* infix */, opName); }
+  public NotQuery(List<SrndQuery> queries, String opName) { super(queries, true /* infix */, opName); }
   
   @Override
   public Query makeLuceneQueryFieldNoBoost(String fieldName, BasicQueryFactory qf) {
-    List luceneSubQueries = makeLuceneSubQueriesField(fieldName, qf);
+    List<Query> luceneSubQueries = makeLuceneSubQueriesField(fieldName, qf);
     BooleanQuery bq = new BooleanQuery();
-    bq.add( (Query) luceneSubQueries.get(0), BooleanClause.Occur.MUST);
+    bq.add( luceneSubQueries.get(0), BooleanClause.Occur.MUST);
     SrndBooleanQuery.addQueriesToBoolean(bq,
             // FIXME: do not allow weights on prohibited subqueries.
             luceneSubQueries.subList(1, luceneSubQueries.size()),

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java Wed Nov 11 12:18:34 2009
@@ -24,7 +24,7 @@
 import java.io.IOException;
 
 public class OrQuery extends ComposedQuery implements DistanceSubQuery { 
-  public OrQuery(List queries, boolean infix, String opName) {
+  public OrQuery(List<SrndQuery> queries, boolean infix, String opName) {
     super(queries, infix, opName);
   }
   

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java Wed Nov 11 12:18:34 2009
@@ -83,7 +83,7 @@
       
       @Override
       public Query rewrite(IndexReader reader) throws IOException {
-        final List luceneSubQueries = new ArrayList();
+        final List<Query> luceneSubQueries = new ArrayList<Query>();
         visitMatchingTerms( reader, fieldName,
             new MatchingTermVisitor() {
               public void visitMatchingTerm(Term term) throws IOException {
@@ -91,7 +91,7 @@
               }
             });
         return  (luceneSubQueries.size() == 0) ? SrndQuery.theEmptyLcnQuery
-              : (luceneSubQueries.size() == 1) ? (Query) luceneSubQueries.get(0)
+              : (luceneSubQueries.size() == 1) ? luceneSubQueries.get(0)
               : SrndBooleanQuery.makeBooleanQuery(
                   /* luceneSubQueries all have default weight */
                   luceneSubQueries, BooleanClause.Occur.SHOULD); /* OR the subquery terms */ 

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java Wed Nov 11 12:18:34 2009
@@ -76,12 +76,12 @@
   public SpanNearClauseFactory(IndexReader reader, String fieldName, BasicQueryFactory qf) {
     this.reader = reader;
     this.fieldName = fieldName;
-    this.weightBySpanQuery = new HashMap(); 
+    this.weightBySpanQuery = new HashMap<SpanQuery, Float>(); 
     this.qf = qf;
   }
   private IndexReader reader;
   private String fieldName;
-  private HashMap weightBySpanQuery;
+  private HashMap<SpanQuery, Float> weightBySpanQuery;
   private BasicQueryFactory qf;
   
   public IndexReader getIndexReader() {return reader;}
@@ -99,7 +99,7 @@
   public void clear() {weightBySpanQuery.clear();}
 
   protected void addSpanQueryWeighted(SpanQuery sq, float weight) {
-    Float w = (Float) weightBySpanQuery.get(sq);
+    Float w = weightBySpanQuery.get(sq);
     if (w != null)
       w = Float.valueOf(w.floatValue() + weight);
     else
@@ -124,11 +124,11 @@
   
   public SpanQuery makeSpanNearClause() {
     SpanQuery [] spanQueries = new SpanQuery[size()];
-    Iterator sqi = weightBySpanQuery.keySet().iterator();
+    Iterator<SpanQuery> sqi = weightBySpanQuery.keySet().iterator();
     int i = 0;
     while (sqi.hasNext()) {
-      SpanQuery sq = (SpanQuery) sqi.next();
-      sq.setBoost(((Float)weightBySpanQuery.get(sq)).floatValue());
+      SpanQuery sq = sqi.next();
+      sq.setBoost(weightBySpanQuery.get(sq).floatValue());
       spanQueries[i++] = sq;
     }
     

Modified: lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java (original)
+++ lucene/java/trunk/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java Wed Nov 11 12:18:34 2009
@@ -25,15 +25,15 @@
 class SrndBooleanQuery {
   public static void addQueriesToBoolean(
           BooleanQuery bq,
-          List queries,
+          List<Query> queries,
           BooleanClause.Occur occur) {
     for (int i = 0; i < queries.size(); i++) {
-      bq.add( (Query) queries.get(i), occur);
+      bq.add( queries.get(i), occur);
     }
   }
   
   public static Query makeBooleanQuery(
-          List queries,
+          List<Query> queries,
           BooleanClause.Occur occur) {
     if (queries.size() <= 1) {
       throw new AssertionError("Too few subqueries: " + queries.size());

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java Wed Nov 11 12:18:34 2009
@@ -29,8 +29,6 @@
 /** This is just a "splitter" class: it lets you wrap two
  *  DocFieldConsumer instances as a single consumer. */
 
-// TODO: Fix the unchecked collections, I do not understand the whole code here -- Uwe
-@SuppressWarnings("unchecked")
 final class DocFieldConsumers extends DocFieldConsumer {
   final DocFieldConsumer one;
   final DocFieldConsumer two;
@@ -50,21 +48,18 @@
   @Override
   public void flush(Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
 
-    Map oneThreadsAndFields = new HashMap();
-    Map twoThreadsAndFields = new HashMap();
+    Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> oneThreadsAndFields = new HashMap<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>>();
+    Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> twoThreadsAndFields = new HashMap<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>>();
 
-    Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
+    for (Map.Entry<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> entry : threadsAndFields.entrySet()) {
 
-      Map.Entry entry = (Map.Entry) it.next();
+      final DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
 
-      DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
+      final Collection<DocFieldConsumerPerField> fields = entry.getValue();
 
-      Collection fields = (Collection) entry.getValue();
-
-      Iterator fieldsIt = fields.iterator();
-      Collection oneFields = new HashSet();
-      Collection twoFields = new HashSet();
+      Iterator<DocFieldConsumerPerField> fieldsIt = fields.iterator();
+      Collection<DocFieldConsumerPerField> oneFields = new HashSet<DocFieldConsumerPerField>();
+      Collection<DocFieldConsumerPerField> twoFields = new HashSet<DocFieldConsumerPerField>();
       while(fieldsIt.hasNext()) {
         DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldsIt.next();
         oneFields.add(perField.one);

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java Wed Nov 11 12:18:34 2009
@@ -1196,7 +1196,7 @@
           numBytesAlloc += BYTE_BLOCK_SIZE;
           b = new byte[BYTE_BLOCK_SIZE];
         } else
-          b = (byte[]) freeByteBlocks.remove(size-1);
+          b = freeByteBlocks.remove(size-1);
         if (trackAllocations)
           numBytesUsed += BYTE_BLOCK_SIZE;
         assert numBytesUsed <= numBytesAlloc;
@@ -1236,7 +1236,7 @@
       numBytesAlloc += INT_BLOCK_SIZE*INT_NUM_BYTE;
       b = new int[INT_BLOCK_SIZE];
     } else
-      b = (int[]) freeIntBlocks.remove(size-1);
+      b = freeIntBlocks.remove(size-1);
     if (trackAllocations)
       numBytesUsed += INT_BLOCK_SIZE*INT_NUM_BYTE;
     assert numBytesUsed <= numBytesAlloc;
@@ -1279,7 +1279,7 @@
       numBytesAlloc += CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
       c = new char[CHAR_BLOCK_SIZE];
     } else
-      c = (char[]) freeCharBlocks.remove(size-1);
+      c = freeCharBlocks.remove(size-1);
     // We always track allocations of char blocks, for now,
     // because nothing that skips allocation tracking
     // (currently only term vectors) uses its own char

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java Wed Nov 11 12:18:34 2009
@@ -282,7 +282,7 @@
    * doesn't exist.
    */  
   public FieldInfo fieldInfo(int fieldNumber) {
-	return (fieldNumber >= 0) ? (FieldInfo) byNumber.get(fieldNumber) : null;
+	return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null;
   }
 
   public int size() {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Wed Nov 11 12:18:34 2009
@@ -25,10 +25,7 @@
 import java.util.Map;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Iterator;
 
-// TODO: Fix the unchecked collections, I do not understand the whole code here -- Uwe
-@SuppressWarnings("unchecked")
 final class FreqProxTermsWriter extends TermsHashConsumer {
 
   @Override
@@ -75,19 +72,15 @@
 
     // Gather all FieldData's that have postings, across all
     // ThreadStates
-    List allFields = new ArrayList();
+    List<FreqProxTermsWriterPerField> allFields = new ArrayList<FreqProxTermsWriterPerField>();
 
-    Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
+    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
 
-      Map.Entry entry = (Map.Entry) it.next();
+      Collection<TermsHashConsumerPerField> fields = entry.getValue();
 
-      Collection fields = (Collection) entry.getValue();
 
-      Iterator fieldsIt = fields.iterator();
-
-      while(fieldsIt.hasNext()) {
-        FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) fieldsIt.next();
+      for (final TermsHashConsumerPerField i : fields) {
+        final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) i;
         if (perField.termsHashPerField.numPostings > 0)
           allFields.add(perField);
       }
@@ -113,16 +106,16 @@
 
     int start = 0;
     while(start < numAllFields) {
-      final FieldInfo fieldInfo = ((FreqProxTermsWriterPerField) allFields.get(start)).fieldInfo;
+      final FieldInfo fieldInfo = allFields.get(start).fieldInfo;
       final String fieldName = fieldInfo.name;
 
       int end = start+1;
-      while(end < numAllFields && ((FreqProxTermsWriterPerField) allFields.get(end)).fieldInfo.name.equals(fieldName))
+      while(end < numAllFields && allFields.get(end).fieldInfo.name.equals(fieldName))
         end++;
       
       FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start];
       for(int i=start;i<end;i++) {
-        fields[i-start] = (FreqProxTermsWriterPerField) allFields.get(i);
+        fields[i-start] = allFields.get(i);
 
         // Aggregate the storePayload as seen by the same
         // field across multiple threads
@@ -144,9 +137,7 @@
       start = end;
     }
 
-    it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
-      Map.Entry entry = (Map.Entry) it.next();
+    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
       FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
       perThread.termsHashPerThread.reset(true);
     }
@@ -280,12 +271,8 @@
     termsConsumer.finish();
   }
 
-  private final TermInfo termInfo = new TermInfo(); // minimize consing
-
   final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
 
-  void files(Collection<String> files) {}
-
   static final class PostingList extends RawPostingList {
     int docFreq;                                    // # times this term occurs in the current doc
     int lastDocID;                                  // Last docID where this term occurred

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java Wed Nov 11 12:18:34 2009
@@ -44,7 +44,7 @@
     // called by onInit above):
     int size = commits.size();
     for(int i=0;i<size-1;i++) {
-      ((IndexCommit) commits.get(i)).delete();
+      commits.get(i).delete();
     }
   }
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriter.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriter.java Wed Nov 11 12:18:34 2009
@@ -35,8 +35,6 @@
  *  merges all of these together into a single _X.nrm file.
  */
 
-// TODO: Fix the unchecked collections, I do not understand the whole code here -- Uwe
-@SuppressWarnings("unchecked")
 final class NormsWriter extends InvertedDocEndConsumer {
 
   private static final byte defaultNorm = Similarity.encodeNorm(1.0f);
@@ -62,27 +60,24 @@
   @Override
   public void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
 
-    final Map byField = new HashMap();
+    final Map<FieldInfo,List<NormsWriterPerField>> byField = new HashMap<FieldInfo,List<NormsWriterPerField>>();
 
     // Typically, each thread will have encountered the same
     // field.  So first we collate by field, ie, all
     // per-thread field instances that correspond to the
     // same FieldInfo
-    final Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
-      Map.Entry entry = (Map.Entry) it.next();
+    for (final Map.Entry<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> entry : threadsAndFields.entrySet()) {
+      final Collection<InvertedDocEndConsumerPerField> fields = entry.getValue();
+      final Iterator<InvertedDocEndConsumerPerField> fieldsIt = fields.iterator();
 
-      Collection fields = (Collection) entry.getValue();
-      Iterator fieldsIt = fields.iterator();
-
-      while(fieldsIt.hasNext()) {
-        NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next();
+      while (fieldsIt.hasNext()) {
+        final NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next();
 
         if (perField.upto > 0) {
           // It has some norms
-          List l = (List) byField.get(perField.fieldInfo);
+          List<NormsWriterPerField> l = byField.get(perField.fieldInfo);
           if (l == null) {
-            l = new ArrayList();
+            l = new ArrayList<NormsWriterPerField>();
             byField.put(perField.fieldInfo, l);
           }
           l.add(perField);
@@ -108,7 +103,7 @@
 
         final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
 
-        List toMerge = (List) byField.get(fieldInfo);
+        List<NormsWriterPerField> toMerge = byField.get(fieldInfo);
         int upto = 0;
         if (toMerge != null) {
 
@@ -120,7 +115,7 @@
           int[] uptos = new int[numFields];
 
           for(int j=0;j<numFields;j++)
-            fields[j] = (NormsWriterPerField) toMerge.get(j);
+            fields[j] = toMerge.get(j);
 
           int numLeft = numFields;
               

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java Wed Nov 11 12:18:34 2009
@@ -35,9 +35,6 @@
  *  TermVectorsTermsWriter}, write their own byte streams
  *  under each term.
  */
-
-// TODO: Fix the unchecked collections, I do not understand the whole code here -- Uwe
-@SuppressWarnings("unchecked")
 final class TermsHash extends InvertedDocConsumer {
 
   final TermsHashConsumer consumer;
@@ -45,8 +42,6 @@
   final int bytesPerPosting;
   final int postingsFreeChunk;
   final DocumentsWriter docWriter;
-  
-  private TermsHash primaryTermsHash;
 
   private RawPostingList[] postingsFreeList = new RawPostingList[1];
   private int postingsFreeCount;
@@ -90,7 +85,7 @@
       nextTermsHash.abort();
   }
 
-  void shrinkFreePostings(Map threadsAndFields, SegmentWriteState state) {
+  void shrinkFreePostings(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, SegmentWriteState state) {
 
     assert postingsFreeCount == postingsAllocCount: Thread.currentThread().getName() + ": postingsFreeCount=" + postingsFreeCount + " postingsAllocCount=" + postingsAllocCount + " consumer=" + consumer;
 
@@ -111,29 +106,26 @@
 
   @Override
   synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
-    Map childThreadsAndFields = new HashMap();
-    Map nextThreadsAndFields;
+    Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
+    Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;
 
     if (nextTermsHash != null)
-      nextThreadsAndFields = new HashMap();
+      nextThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
     else
       nextThreadsAndFields = null;
 
-    Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
-
-      Map.Entry entry = (Map.Entry) it.next();
+    for (final Map.Entry<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> entry : threadsAndFields.entrySet()) {
 
       TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
 
-      Collection fields = (Collection) entry.getValue();
+      Collection<InvertedDocConsumerPerField> fields = entry.getValue();
 
-      Iterator fieldsIt = fields.iterator();
-      Collection childFields = new HashSet();
-      Collection nextChildFields;
+      Iterator<InvertedDocConsumerPerField> fieldsIt = fields.iterator();
+      Collection<TermsHashConsumerPerField> childFields = new HashSet<TermsHashConsumerPerField>();
+      Collection<InvertedDocConsumerPerField> nextChildFields;
 
       if (nextTermsHash != null)
-        nextChildFields = new HashSet();
+        nextChildFields = new HashSet<InvertedDocConsumerPerField>();
       else
         nextChildFields = null;
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=834847&r1=834846&r2=834847&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/BooleanScorer2.java Wed Nov 11 12:18:34 2009
@@ -261,7 +261,7 @@
           ? requiredCountingSumScorer // no prohibited
           : new ReqExclScorer(requiredCountingSumScorer,
                               ((prohibitedScorers.size() == 1)
-                                ? (Scorer) prohibitedScorers.get(0)
+                                ? prohibitedScorers.get(0)
                                 : new DisjunctionSumScorer(prohibitedScorers)));
   }
 



Mime
View raw message