You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by ma...@apache.org on 2009/11/16 22:24:47 UTC

svn commit: r880963 [3/7] - in /lucene/java/branches/flex_1458: ./ contrib/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/ar/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/br/ contrib/analyzers/common/src/java/org/apache/l...

Modified: lucene/java/branches/flex_1458/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java (original)
+++ lucene/java/branches/flex_1458/contrib/db/bdb/src/java/org/apache/lucene/store/db/DbDirectory.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java (original)
+++ lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java Mon Nov 16 21:24:41 2009
@@ -36,10 +36,7 @@
 public class Highlighter
 {
   public static final int DEFAULT_MAX_CHARS_TO_ANALYZE = 50*1024;
-  /**
-   * @deprecated See {@link #DEFAULT_MAX_CHARS_TO_ANALYZE}
-   */
-	public static final  int DEFAULT_MAX_DOC_BYTES_TO_ANALYZE=DEFAULT_MAX_CHARS_TO_ANALYZE;
+
   private int maxDocCharsToAnalyze = DEFAULT_MAX_CHARS_TO_ANALYZE;
 	private Formatter formatter;
 	private Encoder encoder;
@@ -119,29 +116,6 @@
 	 *
 	 * @param analyzer   the analyzer that will be used to split <code>text</code>
 	 * into chunks
-	 * @param text        	text to highlight terms in
-	 * @param maxNumFragments  the maximum number of fragments.
-	 * @deprecated This method incorrectly hardcodes the choice of fieldname. Use the
-	 * method of the same name that takes a fieldname.
-	 * @return highlighted text fragments (between 0 and maxNumFragments number of fragments)
-	 * @throws InvalidTokenOffsetsException  thrown if any token's endOffset exceeds the provided text's length
-	 */
-	public final String[] getBestFragments(
-		Analyzer analyzer,
-		String text,
-		int maxNumFragments)
-		throws IOException, InvalidTokenOffsetsException
-	{
-		TokenStream tokenStream = analyzer.tokenStream("field", new StringReader(text));
-		return getBestFragments(tokenStream, text, maxNumFragments);
-	}
-	/**
-	 * Highlights chosen terms in a text, extracting the most relevant sections.
-	 * This is a convenience method that calls
-	 * {@link #getBestFragments(TokenStream, String, int)}
-	 *
-	 * @param analyzer   the analyzer that will be used to split <code>text</code>
-	 * into chunks
 	 * @param fieldName     the name of the field being highlighted (used by analyzer)
 	 * @param text        	text to highlight terms in
 	 * @param maxNumFragments  the maximum number of fragments.
@@ -498,27 +472,6 @@
 		return result.toString();
 	}
 
-	/**
-	 * @return the maximum number of bytes to be tokenized per doc
-   *
-   * @deprecated See {@link #getMaxDocCharsToAnalyze()}, since this value has always counted on chars.  They both set the same internal value, however
-	 */
-	public int getMaxDocBytesToAnalyze()
-	{
-		return maxDocCharsToAnalyze;
-	}
-
-	/**
-	 * @param byteCount the maximum number of bytes to be tokenized per doc
-	 * (This can improve performance with large documents)
-   *
-   * @deprecated See {@link #setMaxDocCharsToAnalyze(int)}, since this value has always counted chars
-	 */
-	public void setMaxDocBytesToAnalyze(int byteCount)
-	{
-		maxDocCharsToAnalyze = byteCount;
-	}
-
   public int getMaxDocCharsToAnalyze() {
     return maxDocCharsToAnalyze;
   }

Modified: lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TextFragment.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TextFragment.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TextFragment.java (original)
+++ lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/TextFragment.java Mon Nov 16 21:24:41 2009
@@ -37,7 +37,10 @@
 		this.textStartPos = textStartPos;
 		this.fragNum = fragNum;
 	}
-  /** @deprecated */
+  /** 
+   * @deprecated Use {@link #TextFragment(CharSequence, int, int)} instead.
+   * This constructor will be removed in Lucene 4.0
+   */
 	public TextFragment(StringBuffer markedUpText,int textStartPos, int fragNum)
 	{
 		this.markedUpText=markedUpText;

Modified: lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java (original)
+++ lucene/java/branches/flex_1458/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java Mon Nov 16 21:24:41 2009
@@ -28,8 +28,10 @@
 
 import org.apache.lucene.analysis.CachingTokenFilter;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.index.FilterIndexReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermEnum;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.spans.FieldMaskingSpanQuery;
@@ -144,22 +146,14 @@
     } else if (query instanceof MultiTermQuery && expandMultiTermQuery) {
       MultiTermQuery mtq = ((MultiTermQuery)query);
       if(mtq.getRewriteMethod() != MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE) {
-        mtq = copyMultiTermQuery(mtq);
+        mtq = (MultiTermQuery) mtq.clone();
         mtq.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
         query = mtq;
       }
-      String field = null;
-      if(mtq instanceof TermRangeQuery) {
-        field = ((TermRangeQuery)mtq).getField();
-      } else if (mtq instanceof PrefixQuery) {
-        field = ((PrefixQuery) mtq).getPrefix().field();
-      } else if (mtq instanceof WildcardQuery) {
-        field = ((WildcardQuery) mtq).getTerm().field();
-      } else if (mtq instanceof FuzzyQuery) {
-        field = ((FuzzyQuery) mtq).getTerm().field();
-      }
-      if (field != null) {
-        IndexReader ir = getReaderForField(field);
+      FakeReader fReader = new FakeReader();
+      MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE.rewrite(fReader, mtq);
+      if (fReader.field != null) {
+        IndexReader ir = getReaderForField(fReader.field);
         extract(query.rewrite(ir), terms);
       }
     } else if (query instanceof MultiPhraseQuery) {
@@ -527,29 +521,6 @@
     
   }
   
-  private MultiTermQuery copyMultiTermQuery(MultiTermQuery query) {
-    if(query instanceof TermRangeQuery) {
-      TermRangeQuery q = (TermRangeQuery)query;
-      q.setBoost(query.getBoost());
-      return new TermRangeQuery(q.getField(), q.getLowerTerm(), q.getUpperTerm(), q.includesLower(), q.includesUpper());
-    } else if(query instanceof WildcardQuery) {
-      MultiTermQuery q = new WildcardQuery(((WildcardQuery) query).getTerm());
-      q.setBoost(query.getBoost());
-      return q;
-    } else if(query instanceof PrefixQuery) {
-      MultiTermQuery q = new PrefixQuery(((PrefixQuery) query).getPrefix());
-      q.setBoost(q.getBoost());
-      return q;
-    } else if(query instanceof FuzzyQuery) {
-      FuzzyQuery q = (FuzzyQuery)query;
-      q.setBoost(q.getBoost());
-      return new FuzzyQuery(q.getTerm(), q.getMinSimilarity(), q.getPrefixLength());
-    }
-    
-    return query;
-  }
-  
-  
   public boolean getExpandMultiTermQuery() {
     return expandMultiTermQuery;
   }
@@ -578,4 +549,32 @@
   public void setWrapIfNotCachingTokenFilter(boolean wrap) {
     this.wrapToCaching = wrap;
   }
+  
+  /**
+   * 
+   * A fake IndexReader class to extract the field from a MultiTermQuery
+   * 
+   */
+  static final class FakeReader extends FilterIndexReader {
+
+    private static final IndexReader EMPTY_MEMORY_INDEX_READER =
+      new MemoryIndex().createSearcher().getIndexReader();
+    
+    String field;
+
+    FakeReader() {
+      super(EMPTY_MEMORY_INDEX_READER);
+    }
+
+    @Override
+    public TermEnum terms(final Term t) throws IOException {
+      // only set first fieldname, maybe use a Set?
+      if (t != null && field == null)
+        field = t.field();
+      return super.terms(t);
+    }
+
+
+  }
+
 }

Propchange: lucene/java/branches/flex_1458/contrib/highlighter/src/test/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Nov 16 21:24:41 2009
@@ -1,4 +1,5 @@
 /lucene/java/branches/lucene_2_4/contrib/highlighter/src/test:748824
 /lucene/java/branches/lucene_2_9/contrib/highlighter/src/test:817269-818600,825998,826775,829134,829816,829881,831036
 /lucene/java/branches/lucene_2_9_back_compat_tests/contrib/highlighter/src/test:818601-821336
-/lucene/java/trunk/contrib/highlighter/src/test:829439-833960
+/lucene/java/branches/lucene_3_0/contrib/highlighter/src/test:880793
+/lucene/java/trunk/contrib/highlighter/src/test:829439-833960,880727-880846

Modified: lucene/java/branches/flex_1458/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java (original)
+++ lucene/java/branches/flex_1458/contrib/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java Mon Nov 16 21:24:41 2009
@@ -70,6 +70,7 @@
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.highlight.SynonymTokenizer.TestHighlightRunner;
+import org.apache.lucene.search.regex.RegexQuery;
 import org.apache.lucene.search.regex.SpanRegexQuery;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanNotQuery;
@@ -333,6 +334,30 @@
         numHighlights == 5);
   }
   
+  public void testRegexQuery() throws Exception {
+    query = new RegexQuery(new Term(FIELD_NAME, "ken.*"));
+    searcher = new IndexSearcher(ramDir, true);
+    hits = searcher.search(query, 100);
+    int maxNumFragmentsRequired = 2;
+
+    QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
+    Highlighter highlighter = new Highlighter(this, scorer);
+    
+    for (int i = 0; i < hits.totalHits; i++) {
+      String text = searcher.doc(hits.scoreDocs[i].doc).get(FIELD_NAME);
+      TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(text));
+
+      highlighter.setTextFragmenter(new SimpleFragmenter(40));
+
+      String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
+          "...");
+      System.out.println("\t" + result);
+    }
+    
+    assertTrue("Failed to find correct number of highlights " + numHighlights + " found",
+        numHighlights == 5);
+  }
+  
   public void testNumericRangeQuery() throws Exception {
     // doesn't currently highlight, but make sure it doesn't cause exception either
     query = NumericRangeQuery.newIntRange(NUMERIC_FIELD_NAME, 2, 6, true, true);
@@ -1096,7 +1121,7 @@
         Highlighter highlighter = getHighlighter(query, FIELD_NAME, tokenStream,
             HighlighterTest.this);// new Highlighter(this, new
         // QueryTermScorer(query));
-        highlighter.setMaxDocBytesToAnalyze(30);
+        highlighter.setMaxDocCharsToAnalyze(30);
 
         highlighter.getBestFragment(tokenStream, texts[0]);
         assertTrue("Setting MaxDocBytesToAnalyze should have prevented "
@@ -1133,10 +1158,10 @@
         // new
         // QueryTermScorer(query));
         hg.setTextFragmenter(new NullFragmenter());
-        hg.setMaxDocBytesToAnalyze(100);
+        hg.setMaxDocCharsToAnalyze(100);
         match = hg.getBestFragment(new StandardAnalyzer(TEST_VERSION, stopWords), "data", sb.toString());
         assertTrue("Matched text should be no more than 100 chars in length ", match.length() < hg
-            .getMaxDocBytesToAnalyze());
+            .getMaxDocCharsToAnalyze());
 
         // add another tokenized word to the overrall length - but set way
         // beyond
@@ -1147,7 +1172,7 @@
         sb.append(goodWord);
         match = hg.getBestFragment(new StandardAnalyzer(TEST_VERSION, stopWords), "data", sb.toString());
         assertTrue("Matched text should be no more than 100 chars in length ", match.length() < hg
-            .getMaxDocBytesToAnalyze());
+            .getMaxDocCharsToAnalyze());
       }
     };
 

Modified: lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexReader.java (original)
+++ lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexReader.java Mon Nov 16 21:24:41 2009
@@ -133,7 +133,7 @@
   public boolean isDeleted(int n) {
     return getIndex().getDeletedDocuments().contains(n) || deletedDocumentNumbers.contains(n);
   }
-  
+
   @Override
   public boolean hasDeletions() {
     return getIndex().getDeletedDocuments().size() > 0 || deletedDocumentNumbers.size() > 0;

Modified: lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermDocumentInformation.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermDocumentInformation.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermDocumentInformation.java (original)
+++ lucene/java/branches/flex_1458/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermDocumentInformation.java Mon Nov 16 21:24:41 2009
@@ -45,7 +45,7 @@
 
   public static final Comparator<InstantiatedTermDocumentInformation> termComparator = new Comparator<InstantiatedTermDocumentInformation>() {
     public int compare(InstantiatedTermDocumentInformation instantiatedTermDocumentInformation, InstantiatedTermDocumentInformation instantiatedTermDocumentInformation1) {
-      return instantiatedTermDocumentInformation.getTerm().getTerm().compareTo(instantiatedTermDocumentInformation1.getTerm());
+      return instantiatedTermDocumentInformation.getTerm().getTerm().compareTo(instantiatedTermDocumentInformation1.getTerm().getTerm());
     }
   };
 

Modified: lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/LuceneMethods.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/LuceneMethods.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/LuceneMethods.java (original)
+++ lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/LuceneMethods.java Mon Nov 16 21:24:41 2009
@@ -30,26 +30,20 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
-import java.util.Map.Entry;
 
 import jline.ConsoleReader;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FieldsEnum;
-import org.apache.lucene.index.TermRef;
+import org.apache.lucene.index.TermEnum;
 import org.apache.lucene.index.IndexReader.FieldOption;
 import org.apache.lucene.queryParser.MultiFieldQueryParser;
 import org.apache.lucene.queryParser.ParseException;
@@ -70,10 +64,9 @@
 class LuceneMethods {
 
   private int numDocs;
-  private FSDirectory indexName; //directory of this index
-  private java.util.Iterator fieldIterator;
-  private List fields; //Fields as a vector
-  private List indexedFields; //Fields as a vector
+  private final FSDirectory indexName; //directory of this index
+  private List<String> fields; //Fields as a vector
+  private List<String> indexedFields; //Fields as a vector
   private String fieldsArray[]; //Fields as an array
   private Searcher searcher;
   private Query query; //current query string
@@ -87,13 +80,10 @@
     private Analyzer createAnalyzer() {
         if (analyzerClassFQN == null) return new StandardAnalyzer(Version.LUCENE_CURRENT);
         try {
-            Class aClass = Class.forName(analyzerClassFQN);
-            Object obj = aClass.newInstance();
-            if (!(obj instanceof Analyzer)) {
-                message("Given class is not an Analyzer: " + analyzerClassFQN);
-                return new StandardAnalyzer(Version.LUCENE_CURRENT);
-            }
-            return (Analyzer)obj;
+            return Class.forName(analyzerClassFQN).asSubclass(Analyzer.class).newInstance();
+        } catch (ClassCastException cce) {
+            message("Given class is not an Analyzer: " + analyzerClassFQN);
+            return new StandardAnalyzer(Version.LUCENE_CURRENT);
         } catch (Exception e) {
             message("Unable to use Analyzer " + analyzerClassFQN);
             return new StandardAnalyzer(Version.LUCENE_CURRENT);
@@ -125,7 +115,7 @@
   		throws java.io.IOException, org.apache.lucene.queryParser.ParseException {
     initSearch(queryString);
     int numHits = computeCount(query);
-    System.out.println(numHits + " total matching documents");
+    message(numHits + " total matching documents");
     if (explain) {
       query = explainQuery(queryString);
     }
@@ -197,11 +187,11 @@
     int arraySize = indexedFields.size();
     String indexedArray[] = new String[arraySize];
     for (int ii = 0; ii < arraySize; ii++) {
-      indexedArray[ii] = (String) indexedFields.get(ii);
+      indexedArray[ii] = indexedFields.get(ii);
     }
     MultiFieldQueryParser parser = new MultiFieldQueryParser(Version.LUCENE_CURRENT, indexedArray, analyzer);
     query = parser.parse(queryString);
-    System.out.println("Searching for: " + query.toString());
+    message("Searching for: " + query.toString());
     return (query);
 
   }
@@ -218,7 +208,7 @@
     int arraySize = fields.size();
     fieldsArray = new String[arraySize];
     for (int ii = 0; ii < arraySize; ii++) {
-      fieldsArray[ii] = (String) fields.get(ii);
+      fieldsArray[ii] = fields.get(ii);
     }
     MultiFieldQueryParser parser = new MultiFieldQueryParser(Version.LUCENE_CURRENT, fieldsArray, analyzer);
     query = parser.parse(queryString);
@@ -252,7 +242,7 @@
 
   public void count(String queryString) throws java.io.IOException, ParseException {
     initSearch(queryString);
-    System.out.println(computeCount(query) + " total documents");
+    message(computeCount(query) + " total documents");
     searcher.close();
   }
   
@@ -266,21 +256,17 @@
 
   private void getFieldInfo() throws IOException {
     IndexReader indexReader = IndexReader.open(indexName, true);
-    fields = new ArrayList();
-    indexedFields = new ArrayList();
+    fields = new ArrayList<String>();
+    indexedFields = new ArrayList<String>();
 
     //get the list of all field names
-    fieldIterator = indexReader.getFieldNames(FieldOption.ALL).iterator();
-    while (fieldIterator.hasNext()) {
-      Object field = fieldIterator.next();
+    for(String field : indexReader.getFieldNames(FieldOption.ALL)) {
       if (field != null && !field.equals(""))
         fields.add(field.toString());
     }
     //
     //get the list of indexed field names
-    fieldIterator = indexReader.getFieldNames(FieldOption.INDEXED).iterator();
-    while (fieldIterator.hasNext()) {
-      Object field = fieldIterator.next();
+    for(String field : indexReader.getFieldNames(FieldOption.INDEXED)) {
       if (field != null && !field.equals(""))
         indexedFields.add(field.toString());
     }
@@ -293,17 +279,12 @@
   private void invertDocument(Document doc)
     throws IOException {
 
-    Map tokenMap = new HashMap();
+    Map<String,Integer> tokenMap = new HashMap<String,Integer>();
     final int maxFieldLength = 10000;
 
     Analyzer analyzer = createAnalyzer();
-    Iterator fields = doc.getFields().iterator();
-    final Token reusableToken = new Token();
-    while (fields.hasNext()) {
-      Field field = (Field) fields.next();
+    for (Fieldable field : doc.getFields()) {
       String fieldName = field.name();
-
-
       if (field.isIndexed()) {
         if (field.isTokenized()) {     // un-tokenized field
           Reader reader;        // find or make Reader
@@ -326,7 +307,7 @@
               position += (posIncrAtt.getPositionIncrement() - 1);
               position++;
               String name = termAtt.term();
-              Integer Count = (Integer) tokenMap.get(name);
+              Integer Count = tokenMap.get(name);
               if (Count == null) { // not in there yet
                 tokenMap.put(name, Integer.valueOf(1)); //first one
               } else {
@@ -342,9 +323,9 @@
 
       }
     }
-    Entry[] sortedHash = getSortedMapEntries(tokenMap);
+    Map.Entry<String,Integer>[] sortedHash = getSortedMapEntries(tokenMap);
     for (int ii = 0; ii < sortedHash.length && ii < 10; ii++) {
-      Entry currentEntry = sortedHash[ii];
+      Map.Entry<String,Integer> currentEntry = sortedHash[ii];
       message((ii + 1) + ":" + currentEntry.getKey() + " " + currentEntry.getValue());
     }
   }
@@ -355,45 +336,21 @@
    * @param field  - the name of the command or null for all of them.
    */
   public void terms(String field) throws IOException {
-    TreeMap termMap = new TreeMap();
+    TreeMap<String,Integer> termMap = new TreeMap<String,Integer>();
     IndexReader indexReader = IndexReader.open(indexName, true);
-    if (field == null) {
-      FieldsEnum fields = indexReader.fields().iterator();
-      while(true) {
-        final String field2 = fields.next();
-        if (field2 != null) {
-          TermsEnum terms = fields.terms();
-          while(true) {
-            TermRef text = terms.next();
-            if (text != null) {
-              termMap.put(field2 + ":" + text, new Integer(terms.docFreq()));
-            } else {
-              break;
-            }
-          }
-        } else {
-          break;
-        }
-      }
-    } else {
-      Terms terms = indexReader.fields().terms(field);
-      if (terms != null) {
-        TermsEnum termsEnum = terms.iterator();
-        while(true) {
-          TermRef text = termsEnum.next();
-          if (text != null) {
-            termMap.put(field + ":" + text, new Integer(termsEnum.docFreq()));
-          } else {
-            break;
-          }
-        }
-      }
+    TermEnum terms = indexReader.terms();
+    while (terms.next()) {
+      Term term = terms.term();
+      //message(term.field() + ":" + term.text() + " freq:" + terms.docFreq());
+      //if we're either not looking by field or we're matching the specific field
+      if ((field == null) || field.equals(term.field()))
+        termMap.put(term.field() + ":" + term.text(), Integer.valueOf((terms.docFreq())));
     }
-    
-    Iterator termIterator = termMap.keySet().iterator();
+
+    Iterator<String> termIterator = termMap.keySet().iterator();
     for (int ii = 0; termIterator.hasNext() && ii < 100; ii++) {
-      String termDetails = (String) termIterator.next();
-      Integer termFreq = (Integer) termMap.get(termDetails);
+      String termDetails = termIterator.next();
+      Integer termFreq = termMap.get(termDetails);
       message(termDetails + ": " + termFreq);
     }
     indexReader.close();
@@ -403,17 +360,17 @@
    * @param m the map we're sorting
    * from http://developer.java.sun.com/developer/qow/archive/170/index.jsp
    */
-  public static Entry[]
-    getSortedMapEntries(Map m) {
-    Set set = m.entrySet();
-    Entry[] entries =
-      (Entry[]) set.toArray(
-          new Entry[set.size()]);
-    Arrays.sort(entries, new Comparator() {
-      public int compare(Object o1, Object o2) {
-        Object v1 = ((Entry) o1).getValue();
-        Object v2 = ((Entry) o2).getValue();
-        return ((Comparable) v2).compareTo(v1); //descending order
+  @SuppressWarnings("unchecked")
+  public static <K,V extends Comparable<V>> Map.Entry<K,V>[]
+    getSortedMapEntries(Map<K,V> m) {
+    Set<Map.Entry<K, V>> set = m.entrySet();
+    Map.Entry<K,V>[] entries =
+       set.toArray(new Map.Entry[set.size()]);
+    Arrays.sort(entries, new Comparator<Map.Entry<K,V>>() {
+      public int compare(Map.Entry<K,V> o1, Map.Entry<K,V> o2) {
+        V v1 = o1.getValue();
+        V v2 = o2.getValue();
+        return v2.compareTo(v1); //descending order
       }
     });
     return entries;

Modified: lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/Lucli.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/Lucli.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/Lucli.java (original)
+++ lucene/java/branches/flex_1458/contrib/lucli/src/java/lucli/Lucli.java Mon Nov 16 21:24:41 2009
@@ -61,7 +61,7 @@
 	final static int ANALYZER = 10;
 
 	String historyFile;
-	TreeMap commandMap = new TreeMap();
+	TreeMap<String,Command> commandMap = new TreeMap<String,Command>();
 	LuceneMethods luceneMethods; //current cli class we're using
 	boolean enableReadline; //false: use plain java. True: shared library readline
 
@@ -126,11 +126,11 @@
 	}
 
 	private String[] getCommandsAsArray() {
-		Set commandSet = commandMap.keySet();
+		Set<String> commandSet = commandMap.keySet();
 		String[] commands = new String[commandMap.size()];
 		int i = 0;
-		for (Iterator iter = commandSet.iterator(); iter.hasNext();) {
-			String	cmd = (String) iter.next();
+		for (Iterator<String> iter = commandSet.iterator(); iter.hasNext();) {
+			String	cmd = iter.next();
 			commands[i++] = cmd;
 		}
 		return commands;
@@ -252,7 +252,7 @@
 
 	private int getCommandId(String name, int params) {
 		name = name.toLowerCase(); //treat uppercase and lower case commands the same
-		Command command = (Command) commandMap.get(name);
+		Command command = commandMap.get(name);
 		if (command == null) {
 			return(UNKOWN);
 		}
@@ -266,9 +266,9 @@
 	}
 
 	private void help() {
-		Iterator commands = commandMap.keySet().iterator();
+		Iterator<String> commands = commandMap.keySet().iterator();
 		while (commands.hasNext()) {
-			Command command = (Command) commandMap.get(commands.next());
+			Command command = commandMap.get(commands.next());
 			System.out.println("\t" + command.name + ": " + command.help);
 
 		}
@@ -300,7 +300,6 @@
 	private class Command {
 		String name;
 		int id;
-		int numberArgs;
 		String help;
 		int params;
 

Modified: lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Mon Nov 16 21:24:41 2009
@@ -35,29 +35,22 @@
 import org.apache.lucene.analysis.tokenattributes.TermAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.PositionsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermDocs;
 import org.apache.lucene.index.TermEnum;
 import org.apache.lucene.index.TermFreqVector;
 import org.apache.lucene.index.TermPositionVector;
 import org.apache.lucene.index.TermPositions;
-import org.apache.lucene.index.TermRef;
 import org.apache.lucene.index.TermVectorMapper;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Similarity;
-import org.apache.lucene.util.Bits;
+import org.apache.lucene.store.RAMDirectory;
 
 /**
  * High-performance single-document main memory Apache Lucene fulltext search index. 
@@ -735,8 +728,7 @@
   private final class MemoryIndexReader extends IndexReader {
     
     private Searcher searcher; // needed to find searcher.getSimilarity() 
-    private final MemoryFields memoryFields = new MemoryFields();
-
+    
     private MemoryIndexReader() {
       super(); // avoid as much superclass baggage as possible
     }
@@ -748,10 +740,6 @@
     private Info getInfo(int pos) {
       return sortedFields[pos].getValue();
     }
-
-    public Bits getDeletedDocs() {
-      return null;
-    }
     
     @Override
     public int docFreq(Term term) {
@@ -761,208 +749,6 @@
       if (DEBUG) System.err.println("MemoryIndexReader.docFreq: " + term + ", freq:" + freq);
       return freq;
     }
-
-    private final class MemoryFields extends Fields {
-
-      public FieldsEnum iterator() {
-        return new MemoryFieldsEnum();
-      }
-
-      public Terms terms(String field) {
-        return new MemoryTerms(field);
-      }
-    }
-
-    private final class MemoryTerms extends Terms {
-      private final String field;
-      private final Info info;
-
-      public MemoryTerms(String field) {
-        this.field = field;
-        info = getInfo(field);
-      }
-
-      public int docFreq(TermRef text) {
-        if (info != null) {
-          return info.getPositions(text.toString()) != null ? 1 : 0;
-        } else {
-          return 0;
-        }
-      }
-
-      public DocsEnum docs(Bits skipDocs, TermRef text) {
-        return new MemoryDocsEnum(skipDocs, info == null ? null : info.getPositions(text.toString()));
-      }
-
-      public TermsEnum iterator() {
-        return new MemoryTermsEnum(info);
-      }
-    }
-
-    private final class MemoryFieldsEnum extends FieldsEnum {
-      Map.Entry[] fields;
-      int pos;
-
-      public MemoryFieldsEnum() {
-        sortFields();
-        fields = MemoryIndex.this.sortedFields;
-      }
-
-      public String next() {
-        if (pos >= fields.length) {
-          return null;
-        } else {
-          return (String) fields[pos++].getKey();
-        }
-      }
-
-      public TermsEnum terms() {
-        return new MemoryTermsEnum(getInfo((String) fields[pos-1].getKey()));
-      }
-    }
-
-    private final class MemoryTermsEnum extends TermsEnum {
-      private final Info info;
-      private final TermRef term = new TermRef();
-      private final Map.Entry[] sortedTerms;
-      private int pos;
-
-      public MemoryTermsEnum(Info info) {
-        this.info = info;
-        // nocommit -- understand why this null check became necessary?
-        if (info != null) {
-          info.sortTerms();
-          this.sortedTerms = info.sortedTerms;
-        } else {
-          sortedTerms = new Map.Entry[0];
-        }
-      }
-
-      public TermRef next() {
-        if (pos < sortedTerms.length) {
-          // TODO: would be more efficient to store TermRefs
-          // in MemoryIndex
-          term.copy((String) sortedTerms[pos++].getKey());
-          return term;
-        } else {
-          return null;
-        }
-      }
-
-      public long ord() {
-        return pos;
-      }
-
-      public TermRef term() {
-        return term;
-      }
-
-      public SeekStatus seek(TermRef seekTerm) {
-        int i = Arrays.binarySearch(sortedTerms, seekTerm.toString(), termComparator);
-        if (i < 0) {
-          // not found; choose successor
-          pos = -i-1;
-          if (pos < sortedTerms.length) {
-            term.copy((String) sortedTerms[pos].getKey());
-            return SeekStatus.NOT_FOUND;
-          } else {
-            // no successor
-            return SeekStatus.END;
-          }
-        } else {
-          // found
-          term.copy(seekTerm);
-          pos = i;
-          return SeekStatus.FOUND;
-        }
-      }
-
-      public SeekStatus seek(long ord) {
-        if (ord < sortedTerms.length) {
-          pos = (int) ord;
-          term.copy((String) sortedTerms[pos].getKey());
-          // always found
-          return SeekStatus.FOUND;
-        } else {
-          return SeekStatus.END;
-        }
-      }
-
-      public int docFreq() {
-        return 1;
-      }
-
-      public DocsEnum docs(Bits skipDocs) {
-        return new MemoryDocsEnum(skipDocs, (ArrayIntList) sortedTerms[pos].getValue());
-      }
-    }
-
-    private final class MemoryDocsEnum extends DocsEnum {
-      private final ArrayIntList positions;
-      private boolean hasNext = true;
-      private final MemoryPositionsEnum positionsEnum;
-
-      public MemoryDocsEnum(Bits skipDocs, ArrayIntList positions) {
-        this.positions = positions;
-        if (positions == null || (skipDocs != null && skipDocs.get(0))) {
-          hasNext = false;
-        }
-        positionsEnum = new MemoryPositionsEnum(positions);
-      }
-
-      public int next() {
-        if (hasNext) {
-          hasNext = false;
-          return 0;
-        } else {
-          return NO_MORE_DOCS;
-        }
-      }
-
-      public int advance(int target) {
-        return next();
-      }
-
-      public int freq() {
-        return positions == null ? 0 : numPositions(positions);
-      }
-
-      public PositionsEnum positions() {
-        return positionsEnum;
-      }
-    }
-
-    private final class MemoryPositionsEnum extends PositionsEnum {
-      private int cursor;
-      private final ArrayIntList positions;
-
-      public MemoryPositionsEnum(ArrayIntList positions) {
-        this.positions = positions;
-      }
-
-      public int next() {
-        final int pos = positions.get(cursor);
-        cursor += stride;
-        return pos;
-      }
-
-      public boolean hasPayload() {
-        return false;
-      }
-
-      public int getPayloadLength() {
-        throw new UnsupportedOperationException();
-      }
-
-      public byte[] getPayload(byte[] data, int offset) {
-        throw new UnsupportedOperationException();
-      }
-    }
-
-    // Flex API
-    public Fields fields() {
-      return memoryFields;
-    }
   
     @Override
     public TermEnum terms() {

Modified: lucene/java/branches/flex_1458/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java (original)
+++ lucene/java/branches/flex_1458/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java Mon Nov 16 21:24:41 2009
@@ -37,11 +37,13 @@
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.SimpleAnalyzer;
 import org.apache.lucene.analysis.StopAnalyzer;
+import org.apache.lucene.analysis.StopFilter;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.queryParser.QueryParser;
 import org.apache.lucene.search.Collector;
@@ -203,7 +205,7 @@
   
   private Analyzer analyzer;
   private boolean fastMode = false;
-
+  
   private final boolean verbose = false;
   
   private static final String FIELD_NAME = "content";
@@ -335,7 +337,7 @@
                 if (useMemIndex && useRAMIndex) {
                   if (verbose) System.out.println("diff="+ (score1-score2) + ", query=" + queries[q] + ", s1=" + score1 + ", s2=" + score2);
                   if (score1 != score2 || score1 < 0.0f || score2 < 0.0f || score1 > 1.0f || score2 > 1.0f) {
-                    throw new IllegalStateException("BUG DETECTED:" + (i*(q+1)) + " at query=" + queries[q] + ", file=" + file + ", anal=" + analyzer + " score1=" + score1 + " score2=" + score2);
+                    throw new IllegalStateException("BUG DETECTED:" + (i*(q+1)) + " at query=" + queries[q] + ", file=" + file + ", anal=" + analyzer);
                   }
                 }
               }
@@ -390,9 +392,9 @@
   
   private MemoryIndex createMemoryIndex(Document doc) {
     MemoryIndex index = new MemoryIndex();
-    Iterator iter = doc.getFields().iterator();
+    Iterator<Fieldable> iter = doc.getFields().iterator();
     while (iter.hasNext()) {
-      Field field = (Field) iter.next();
+      Fieldable field = iter.next();
       index.addField(field.name(), field.stringValue(), analyzer);
     }
     return index;

Added: lucene/java/branches/flex_1458/contrib/misc/junit8183035598261200813.properties
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/misc/junit8183035598261200813.properties?rev=880963&view=auto
==============================================================================
    (empty)

Added: lucene/java/branches/flex_1458/contrib/misc/junitvmwatcher4205547712589018705.properties
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/misc/junitvmwatcher4205547712589018705.properties?rev=880963&view=auto
==============================================================================
    (empty)

Modified: lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/misc/ChainedFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/misc/ChainedFilter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/misc/ChainedFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/misc/ChainedFilter.java Mon Nov 16 21:24:41 2009
@@ -150,16 +150,14 @@
         return result;
     }
 
-    // TODO: in 3.0, instead of removing this deprecated
-    // method, make it a no-op and mark it final
     /** Provide a SortedVIntList when it is definitely
      *  smaller than an OpenBitSet
      *  @deprecated Either use CachingWrapperFilter, or
-     *  switch to a different DocIdSet implementation yourself. */
-    protected DocIdSet finalResult(OpenBitSetDISI result, int maxDocs) {
-        return (result.cardinality() < (maxDocs / 9))
-              ? (DocIdSet) new SortedVIntList(result)
-              : (DocIdSet) result;
+     *  switch to a different DocIdSet implementation yourself.
+     *  This method will be removed in Lucene 4.0 
+     **/
+    protected final DocIdSet finalResult(OpenBitSetDISI result, int maxDocs) {
+        return result;
     }
         
 

Modified: lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/BooleanFilter.java Mon Nov 16 21:24:41 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);
@@ -112,16 +111,14 @@
     return DocIdSet.EMPTY_DOCIDSET;
   }
 
-  // TODO: in 3.0, instead of removing this deprecated
-  // method, make it a no-op and mark it final
   /** Provide a SortedVIntList when it is definitely smaller
    * than an OpenBitSet.
    * @deprecated Either use CachingWrapperFilter, or
-   * switch to a different DocIdSet implementation yourself. */
-  protected DocIdSet finalResult(OpenBitSetDISI result, int maxDocs) {
-    return (result.cardinality() < (maxDocs / 9))
-      ? (DocIdSet) new SortedVIntList(result)
-      : (DocIdSet) result;
+   * switch to a different DocIdSet implementation yourself.
+   * This method will be removed in Lucene 4.0 
+   */
+  protected final DocIdSet finalResult(OpenBitSetDISI result, int maxDocs) {
+    return result;
   }
 
   /**

Modified: lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java Mon Nov 16 21:24:41 2009
@@ -51,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;
@@ -189,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();
@@ -214,7 +214,7 @@
                         if (variantsQ.size() < MAX_VARIANTS_PER_TERM || score > minScore){
                           ScoreTerm st=new ScoreTerm(new Term(startTerm.field(), possibleMatch.toString()),score,startTerm);                    
                           variantsQ.insertWithOverflow(st);
-                          minScore = ((ScoreTerm)variantsQ.top()).score; // maintain minScore
+                          minScore = variantsQ.top().score; // maintain minScore
                         }
                       }
                     } while(fe.next() != null);
@@ -234,7 +234,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);
 	                }                            
@@ -251,9 +251,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
@@ -265,27 +265,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); 
@@ -293,10 +293,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/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/TermsFilter.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java (original)
+++ lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThis.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/MoreLikeThisQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java (original)
+++ lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/similar/SimilarityQueries.java Mon Nov 16 21:24:41 2009
@@ -82,7 +82,7 @@
     public static Query formSimilarQuery( String body,
 										  Analyzer a,
 										  String field,
-										  Set stop)
+										  Set<?> stop)
 										  throws IOException
 	{	
 		TokenStream ts = a.tokenStream( field, new StringReader( body));

Modified: lucene/java/branches/flex_1458/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java (original)
+++ lucene/java/branches/flex_1458/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java Mon Nov 16 21:24:41 2009
@@ -30,7 +30,7 @@
  * 
  * @see ParametricRangeQueryNodeProcessor
  * @see RangeCollatorAttribute
- * @see org.apache.lucene.search.RangeQuery
+ * @see org.apache.lucene.search.TermRangeQuery
  */
 public class RangeQueryNode extends ParametricRangeQueryNode {
 

Modified: lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java (original)
+++ lucene/java/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballAnalyzer.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/snowball/src/java/org/apache/lucene/analysis/snowball/SnowballFilter.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/geometry/shape/DistanceApproximation.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/geometry/shape/DistanceApproximation.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/geometry/shape/DistanceApproximation.java (original)
+++ lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/geometry/shape/DistanceApproximation.java Mon Nov 16 21:24:41 2009
@@ -25,7 +25,7 @@
  * release.</font>
  *
  * @deprecated This has been replaced with more accurate
- * math in {@link LLRect}.
+ * math in {@link LLRect}. This class will be removed in a future release.
  */
 public class DistanceApproximation
 {

Modified: lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java (original)
+++ lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java (original)
+++ lucene/java/branches/flex_1458/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java (original)
+++ lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/Dictionary.java Mon Nov 16 21:24:41 2009
@@ -31,5 +31,5 @@
    * Return all words present in the dictionary
    * @return Iterator
    */
-  Iterator getWordsIterator();
+  Iterator<String> getWordsIterator();
 }

Modified: lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java (original)
+++ lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java (original)
+++ lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/AndQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/ComposedQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/DistanceQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/FieldsQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/NotQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/OrQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SimpleTerm.java Mon Nov 16 21:24:41 2009
@@ -27,7 +27,7 @@
 
 public abstract class SimpleTerm
   extends SrndQuery
-  implements DistanceSubQuery, Comparable
+  implements DistanceSubQuery, Comparable<SimpleTerm>
 {
   public SimpleTerm(boolean q) {quoted = q;}
   
@@ -39,9 +39,8 @@
   
   public abstract String toStringUnquoted();
   
-  public int compareTo(Object o) {
+  public int compareTo(SimpleTerm ost) {
     /* for ordering terms and prefixes before using an index, not used */
-    SimpleTerm ost = (SimpleTerm) o;
     return this.toStringUnquoted().compareTo( ost.toStringUnquoted());
   }
   
@@ -84,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 {
@@ -92,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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SpanNearClauseFactory.java Mon Nov 16 21:24:41 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;}
@@ -89,8 +89,7 @@
   public String getFieldName() {return fieldName;}
 
   public BasicQueryFactory getBasicQueryFactory() {return qf;}
-
-  /* @deprecated */
+  
   public TermEnum getTermEnum(String termText) throws IOException {
     return getIndexReader().terms(new Term(getFieldName(), termText));
   }
@@ -100,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
@@ -125,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/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndBooleanQuery.java Mon Nov 16 21:24:41 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/branches/flex_1458/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java (original)
+++ lucene/java/branches/flex_1458/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java Mon Nov 16 21:24:41 2009
@@ -51,7 +51,7 @@
      * The reference links between the decorated ListModel
      * and this list model based on search criteria
      */
-    private ArrayList rowToModelIndex = new ArrayList();
+    private ArrayList<Integer> rowToModelIndex = new ArrayList<Integer>();
 
     /**
      * In memory lucene index
@@ -256,12 +256,12 @@
         searchString = null;
         rowToModelIndex.clear();
         for (int t=0; t<listModel.getSize(); t++){
-            rowToModelIndex.add(Integer.valueOf(t));
+            rowToModelIndex.add(t);
         }
     }
 
     private int getModelRow(int row){
-        return ((Integer) rowToModelIndex.get(row)).intValue();
+        return rowToModelIndex.get(row);
     }
 
     public int getSize() {