lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From uschind...@apache.org
Subject svn commit: r826527 [1/2] - in /lucene/java/trunk/src: java/org/apache/lucene/document/ java/org/apache/lucene/index/ java/org/apache/lucene/search/ java/org/apache/lucene/search/payloads/ java/org/apache/lucene/search/spans/ java/org/apache/lucene/sto...
Date Sun, 18 Oct 2009 21:11:37 GMT
Author: uschindler
Date: Sun Oct 18 21:11:33 2009
New Revision: 826527

URL: http://svn.apache.org/viewvc?rev=826527&view=rev
Log:
LUCENE-1257: More generified APIs and implementations (when will we be finished?). Thanks Kay Kay!

Modified:
    lucene/java/trunk/src/java/org/apache/lucene/document/MapFieldSelector.java
    lucene/java/trunk/src/java/org/apache/lucene/document/SetBasedFieldSelector.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/CachingSpanFilter.java
    lucene/java/trunk/src/java/org/apache/lucene/search/ConjunctionScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Explanation.java
    lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java
    lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java
    lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Query.java
    lucene/java/trunk/src/java/org/apache/lucene/search/QueryTermVector.java
    lucene/java/trunk/src/java/org/apache/lucene/search/Similarity.java
    lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
    lucene/java/trunk/src/java/org/apache/lucene/search/SpanFilterResult.java
    lucene/java/trunk/src/java/org/apache/lucene/search/SpanQueryFilter.java
    lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNotQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/Spans.java
    lucene/java/trunk/src/java/org/apache/lucene/search/spans/TermSpans.java
    lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/FileSwitchDirectory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/IndexOutput.java
    lucene/java/trunk/src/java/org/apache/lucene/store/MMapDirectory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java
    lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java
    lucene/java/trunk/src/java/org/apache/lucene/util/AverageGuessMemoryModel.java
    lucene/java/trunk/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java
    lucene/java/trunk/src/java/org/apache/lucene/util/MapOfSets.java
    lucene/java/trunk/src/java/org/apache/lucene/util/Parameter.java
    lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java
    lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java
    lucene/java/trunk/src/java/org/apache/lucene/util/cache/Cache.java
    lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java
    lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java
    lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java

Modified: lucene/java/trunk/src/java/org/apache/lucene/document/MapFieldSelector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/document/MapFieldSelector.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/document/MapFieldSelector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/document/MapFieldSelector.java Sun Oct 18 21:11:33 2009
@@ -17,6 +17,7 @@
  * limitations under the License.
  */
 
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -27,39 +28,39 @@
  */
 public class MapFieldSelector implements FieldSelector {
     
-    Map fieldSelections;
+    Map<String,FieldSelectorResult> fieldSelections;
     
     /** Create a a MapFieldSelector
      * @param fieldSelections maps from field names (String) to {@link FieldSelectorResult}s
      */
-    public MapFieldSelector(Map fieldSelections) {
+    public MapFieldSelector(Map<String,FieldSelectorResult> fieldSelections) {
         this.fieldSelections = fieldSelections;
     }
     
     /** Create a a MapFieldSelector
      * @param fields fields to LOAD.  List of Strings.  All other fields are NO_LOAD.
      */
-    public MapFieldSelector(List fields) {
-        fieldSelections = new HashMap(fields.size()*5/3);
-        for (int i=0; i<fields.size(); i++)
-            fieldSelections.put(fields.get(i), FieldSelectorResult.LOAD);
+    public MapFieldSelector(List<String> fields) {
+        fieldSelections = new HashMap<String,FieldSelectorResult>(fields.size()*5/3);
+        for (final String field : fields)
+            fieldSelections.put(field, FieldSelectorResult.LOAD);
     }
     
     /** Create a a MapFieldSelector
      * @param fields fields to LOAD.  All other fields are NO_LOAD.
      */
-    public MapFieldSelector(String[] fields) {
-        fieldSelections = new HashMap(fields.length*5/3);
-        for (int i=0; i<fields.length; i++)
-            fieldSelections.put(fields[i], FieldSelectorResult.LOAD);
+    public MapFieldSelector(String... fields) {
+      this(Arrays.asList(fields));
     }
+
+
     
     /** Load field according to its associated value in fieldSelections
      * @param field a field name
      * @return the fieldSelections value that field maps to or NO_LOAD if none.
      */
     public FieldSelectorResult accept(String field) {
-        FieldSelectorResult selection = (FieldSelectorResult) fieldSelections.get(field);
+        FieldSelectorResult selection = fieldSelections.get(field);
         return selection!=null ? selection : FieldSelectorResult.NO_LOAD;
     }
     

Modified: lucene/java/trunk/src/java/org/apache/lucene/document/SetBasedFieldSelector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/document/SetBasedFieldSelector.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/document/SetBasedFieldSelector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/document/SetBasedFieldSelector.java Sun Oct 18 21:11:33 2009
@@ -23,18 +23,16 @@
  **/
 public class SetBasedFieldSelector implements FieldSelector {
   
-  private Set fieldsToLoad;
-  private Set lazyFieldsToLoad;
+  private Set<String> fieldsToLoad;
+  private Set<String> lazyFieldsToLoad;
   
-  
-
   /**
    * Pass in the Set of {@link Field} names to load and the Set of {@link Field} names to load lazily.  If both are null, the
    * Document will not have any {@link Field} on it.  
    * @param fieldsToLoad A Set of {@link String} field names to load.  May be empty, but not null
    * @param lazyFieldsToLoad A Set of {@link String} field names to load lazily.  May be empty, but not null  
    */
-  public SetBasedFieldSelector(Set fieldsToLoad, Set lazyFieldsToLoad) {
+  public SetBasedFieldSelector(Set<String> fieldsToLoad, Set<String> lazyFieldsToLoad) {
     this.fieldsToLoad = fieldsToLoad;
     this.lazyFieldsToLoad = lazyFieldsToLoad;
   }

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=826527&r1=826526&r2=826527&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 Sun Oct 18 21:11:33 2009
@@ -17,8 +17,6 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.UnicodeUtil;
 
 import java.io.IOException;
@@ -66,7 +64,7 @@
   // under the same FieldInfo together, up into TermsHash*.
   // Other writers would presumably share alot of this...
 
-  public void flush(Map threadsAndFields, final SegmentWriteState state) throws IOException {
+  public void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
 
     // Gather all FieldData's that have postings, across all
     // ThreadStates
@@ -279,7 +277,7 @@
 
   final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
 
-  void files(Collection files) {}
+  void files(Collection<String> files) {}
 
   static final class PostingList extends RawPostingList {
     int docFreq;                                    // # times this term occurs in the current doc

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java Sun Oct 18 21:11:33 2009
@@ -24,7 +24,7 @@
 import java.io.PrintStream;
 import java.util.Map;
 import java.util.HashMap;
-import java.util.Iterator;
+
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -72,12 +72,12 @@
   /* Files that we tried to delete but failed (likely
    * because they are open and we are running on Windows),
    * so we will retry them again later: */
-  private List deletable;
+  private List<String> deletable;
 
   /* Reference count for all files in the index.  
    * Counts how many existing commits reference a file.
-   * Maps String to RefCount (class below) instances: */
-  private Map refCounts = new HashMap();
+   **/
+  private Map<String, RefCount> refCounts = new HashMap<String, RefCount>();
 
   /* Holds all commits (segments_N) currently in the index.
    * This will have just 1 commit if you are using the
@@ -88,7 +88,7 @@
 
   /* Holds files we had incref'd from the previous
    * non-commit checkpoint: */
-  private List lastFiles = new ArrayList();
+  private List<Collection<String>> lastFiles = new ArrayList<Collection<String>>();
 
   /* Commits that the IndexDeletionPolicy have decided to delete: */ 
   private List commitsToDelete = new ArrayList();
@@ -217,10 +217,9 @@
     // Now delete anything with ref count at 0.  These are
     // presumably abandoned files eg due to crash of
     // IndexWriter.
-    Iterator it = refCounts.keySet().iterator();
-    while(it.hasNext()) {
-      String fileName = (String) it.next();
-      RefCount rc = (RefCount) refCounts.get(fileName);
+    for(Map.Entry<String, RefCount> entry : refCounts.entrySet() ) {  
+      RefCount rc = entry.getValue();
+      final String fileName = entry.getKey();
       if (0 == rc.count) {
         if (infoStream != null) {
           message("init: removing unreferenced file \"" + fileName + "\"");
@@ -259,9 +258,8 @@
         if (infoStream != null) {
           message("deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
         }
-        Iterator it = commit.files.iterator();
-        while(it.hasNext()) {
-          decRef((String) it.next());
+        for (final String file : commit.files) {
+          decRef(file);
         }
       }
       commitsToDelete.clear();
@@ -333,7 +331,7 @@
     int size = lastFiles.size();
     if (size > 0) {
       for(int i=0;i<size;i++)
-        decRef((Collection) lastFiles.get(i));
+        decRef(lastFiles.get(i));
       lastFiles.clear();
     }
 
@@ -342,13 +340,13 @@
 
   private void deletePendingFiles() throws IOException {
     if (deletable != null) {
-      List oldDeletable = deletable;
+      List<String> oldDeletable = deletable;
       deletable = null;
       int size = oldDeletable.size();
       for(int i=0;i<size;i++) {
         if (infoStream != null)
           message("delete pending file " + oldDeletable.get(i));
-        deleteFile((String) oldDeletable.get(i));
+        deleteFile(oldDeletable.get(i));
       }
     }
   }
@@ -397,7 +395,7 @@
       deleteCommits();
     } else {
 
-      final List docWriterFiles;
+      final List<String> docWriterFiles;
       if (docWriter != null) {
         docWriterFiles = docWriter.openFiles();
         if (docWriterFiles != null)
@@ -412,7 +410,7 @@
       int size = lastFiles.size();
       if (size > 0) {
         for(int i=0;i<size;i++)
-          decRef((Collection) lastFiles.get(i));
+          decRef(lastFiles.get(i));
         lastFiles.clear();
       }
 
@@ -427,16 +425,14 @@
   void incRef(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
      // If this is a commit point, also incRef the
      // segments_N file:
-    Iterator it = segmentInfos.files(directory, isCommit).iterator();
-    while(it.hasNext()) {
-      incRef((String) it.next());
+    for( final String fileName: segmentInfos.files(directory, isCommit) ) {
+      incRef(fileName);
     }
   }
 
-  void incRef(List files) throws IOException {
-    int size = files.size();
-    for(int i=0;i<size;i++) {
-      incRef((String) files.get(i));
+  void incRef(List<String> files) throws IOException {
+    for(final String file : files) {
+      incRef(file);
     }
   }
 
@@ -448,10 +444,9 @@
     rc.IncRef();
   }
 
-  void decRef(Collection files) throws IOException {
-    Iterator it = files.iterator();
-    while(it.hasNext()) {
-      decRef((String) it.next());
+  void decRef(Collection<String> files) throws IOException {
+    for(final String file : files) {
+      decRef(file);
     }
   }
 
@@ -469,9 +464,8 @@
   }
 
   void decRef(SegmentInfos segmentInfos) throws IOException {
-    Iterator it = segmentInfos.files(directory, false).iterator();
-    while(it.hasNext()) {
-      decRef((String) it.next());
+    for (final String file : segmentInfos.files(directory, false)) {
+      decRef(file);
     }
   }
 
@@ -481,23 +475,20 @@
       rc = new RefCount(fileName);
       refCounts.put(fileName, rc);
     } else {
-      rc = (RefCount) refCounts.get(fileName);
+      rc = refCounts.get(fileName);
     }
     return rc;
   }
 
-  void deleteFiles(List files) throws IOException {
-    final int size = files.size();
-    for(int i=0;i<size;i++)
-      deleteFile((String) files.get(i));
+  void deleteFiles(List<String> files) throws IOException {
+    for(final String file: files)
+      deleteFile(file);
   }
 
   /** Deletes the specified files, but only if they are new
    *  (have not yet been incref'd). */
-  void deleteNewFiles(Collection files) throws IOException {
-    final Iterator it = files.iterator();
-    while(it.hasNext()) {
-      final String fileName = (String) it.next();
+  void deleteNewFiles(Collection<String> files) throws IOException {
+    for (final String fileName: files) {
       if (!refCounts.containsKey(fileName))
         deleteFile(fileName);
     }
@@ -524,7 +515,7 @@
           message("IndexFileDeleter: unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
         }
         if (deletable == null) {
-          deletable = new ArrayList();
+          deletable = new ArrayList<String>();
         }
         deletable.add(fileName);                  // add to deletable
       }
@@ -574,13 +565,13 @@
     String segmentsFileName;
     boolean deleted;
     Directory directory;
-    Collection commitsToDelete;
+    Collection<IndexCommit> commitsToDelete;
     long version;
     long generation;
     final boolean isOptimized;
-    final Map userData;
+    final Map<String,String> userData;
 
-    public CommitPoint(Collection commitsToDelete, Directory directory, SegmentInfos segmentInfos) throws IOException {
+    public CommitPoint(Collection<IndexCommit> commitsToDelete, Directory directory, SegmentInfos segmentInfos) throws IOException {
       this.directory = directory;
       this.commitsToDelete = commitsToDelete;
       userData = segmentInfos.getUserData();
@@ -618,7 +609,7 @@
       return generation;
     }
 
-    public Map getUserData() {
+    public Map<String,String> getUserData() {
       return userData;
     }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileNameFilter.java Sun Oct 18 21:11:33 2009
@@ -27,16 +27,16 @@
 public class IndexFileNameFilter implements FilenameFilter {
 
   private static IndexFileNameFilter singleton = new IndexFileNameFilter();
-  private HashSet extensions;
-  private HashSet extensionsInCFS;
+  private HashSet<String> extensions;
+  private HashSet<String> extensionsInCFS;
 
   // Prevent instantiation.
   private IndexFileNameFilter() {
-    extensions = new HashSet();
+    extensions = new HashSet<String>();
     for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS.length; i++) {
       extensions.add(IndexFileNames.INDEX_EXTENSIONS[i]);
     }
-    extensionsInCFS = new HashSet();
+    extensionsInCFS = new HashSet<String>();
     for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE.length; i++) {
       extensionsInCFS.add(IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE[i]);
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java Sun Oct 18 21:11:33 2009
@@ -244,7 +244,7 @@
   private long lastCommitChangeCount; // last changeCount that was committed
 
   private SegmentInfos rollbackSegmentInfos;      // segmentInfos we will fallback to if the commit fails
-  private HashMap rollbackSegments;
+  private HashMap<SegmentInfo,Integer> rollbackSegments;
 
   volatile SegmentInfos pendingCommit;            // set when a commit is pending (after prepareCommit() & before commit())
   volatile long pendingCommitChangeCount;
@@ -257,7 +257,7 @@
   private DocumentsWriter docWriter;
   private IndexFileDeleter deleter;
 
-  private Set segmentsToOptimize = new HashSet();           // used by optimize to note those needing optimization
+  private Set<SegmentInfo> segmentsToOptimize = new HashSet<SegmentInfo>();           // used by optimize to note those needing optimization
 
   private Lock writeLock;
 
@@ -268,13 +268,13 @@
 
   // Holds all SegmentInfo instances currently involved in
   // merges
-  private HashSet mergingSegments = new HashSet();
+  private HashSet<SegmentInfo> mergingSegments = new HashSet<SegmentInfo>();
 
   private MergePolicy mergePolicy = new LogByteSizeMergePolicy(this);
   private MergeScheduler mergeScheduler = new ConcurrentMergeScheduler();
-  private LinkedList pendingMerges = new LinkedList();
-  private Set runningMerges = new HashSet();
-  private List mergeExceptions = new ArrayList();
+  private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<MergePolicy.OneMerge>();
+  private Set<MergePolicy.OneMerge> runningMerges = new HashSet<MergePolicy.OneMerge>();
+  private List<MergePolicy.OneMerge> mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
   private long mergeGen;
   private boolean stopMerges;
 
@@ -402,23 +402,19 @@
 
   class ReaderPool {
 
-    private final Map readerMap = new HashMap();
+    private final Map<SegmentInfo,SegmentReader> readerMap = new HashMap<SegmentInfo,SegmentReader>();
 
     /** Forcefully clear changes for the specified segments,
      *  and remove from the pool.   This is called on successful merge. */
     synchronized void clear(SegmentInfos infos) throws IOException {
       if (infos == null) {
-        Iterator iter = readerMap.entrySet().iterator();
-        while (iter.hasNext()) {
-          Map.Entry ent = (Map.Entry) iter.next();
-          ((SegmentReader) ent.getValue()).hasChanges = false;
+        for (Map.Entry<SegmentInfo,SegmentReader> ent: readerMap.entrySet()) {
+          ent.getValue().hasChanges = false;
         }
       } else {
-        final int numSegments = infos.size();
-        for(int i=0;i<numSegments;i++) {
-          final SegmentInfo info = infos.info(i);
+        for (final SegmentInfo info: infos) {
           if (readerMap.containsKey(info)) {
-            ((SegmentReader) readerMap.get(info)).hasChanges = false;
+            readerMap.get(info).hasChanges = false;
           }
         }     
       }
@@ -435,7 +431,7 @@
     public synchronized SegmentInfo mapToLive(SegmentInfo info) {
       int idx = segmentInfos.indexOf(info);
       if (idx != -1) {
-        info = (SegmentInfo) segmentInfos.get(idx);
+        info = segmentInfos.get(idx);
       }
       return info;
     }
@@ -497,11 +493,12 @@
     /** Remove all our references to readers, and commits
      *  any pending changes. */
     synchronized void close() throws IOException {
-      Iterator iter = readerMap.entrySet().iterator();
+      Iterator<Map.Entry<SegmentInfo,SegmentReader>> iter = readerMap.entrySet().iterator();
       while (iter.hasNext()) {
-        Map.Entry ent = (Map.Entry) iter.next();
+        
+        Map.Entry<SegmentInfo,SegmentReader> ent = iter.next();
 
-        SegmentReader sr = (SegmentReader) ent.getValue();
+        SegmentReader sr = ent.getValue();
         if (sr.hasChanges) {
           assert infoIsLive(sr.getSegmentInfo());
           sr.startCommit();
@@ -531,11 +528,9 @@
      * @throws IOException
      */
     synchronized void commit() throws IOException {
-      Iterator iter = readerMap.entrySet().iterator();
-      while (iter.hasNext()) {
-        Map.Entry ent = (Map.Entry) iter.next();
+      for (Map.Entry<SegmentInfo,SegmentReader> ent : readerMap.entrySet()) {
 
-        SegmentReader sr = (SegmentReader) ent.getValue();
+        SegmentReader sr = ent.getValue();
         if (sr.hasChanges) {
           assert infoIsLive(sr.getSegmentInfo());
           sr.startCommit();
@@ -1125,7 +1120,7 @@
   private synchronized void setRollbackSegmentInfos(SegmentInfos infos) {
     rollbackSegmentInfos = (SegmentInfos) infos.clone();
     assert !rollbackSegmentInfos.hasExternalSegments(directory);
-    rollbackSegments = new HashMap();
+    rollbackSegments = new HashMap<SegmentInfo,Integer>();
     final int size = rollbackSegmentInfos.size();
     for(int i=0;i<size;i++)
       rollbackSegments.put(rollbackSegmentInfos.info(i), Integer.valueOf(i));
@@ -1731,9 +1726,9 @@
 
       try {
         CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
-        final Iterator it = docWriter.closedFiles().iterator();
-        while(it.hasNext())
-          cfsWriter.addFile((String) it.next());
+        for (final String file :  docWriter.closedFiles() ) {
+          cfsWriter.addFile(file);
+        }
       
         // Perform the merge
         cfsWriter.close();
@@ -1923,7 +1918,7 @@
             // If docWriter has some aborted files that were
             // never incref'd, then we clean them up here
             if (docWriter != null) {
-              final Collection files = docWriter.abortedFiles();
+              final Collection<String> files = docWriter.abortedFiles();
               if (files != null)
                 deleter.deleteNewFiles(files);
             }
@@ -2079,7 +2074,7 @@
           synchronized (this) {
             // If docWriter has some aborted files that were
             // never incref'd, then we clean them up here
-            final Collection files = docWriter.abortedFiles();
+            final Collection<String> files = docWriter.abortedFiles();
             if (files != null)
               deleter.deleteNewFiles(files);
           }
@@ -2254,23 +2249,19 @@
 
     synchronized(this) {
       resetMergeExceptions();
-      segmentsToOptimize = new HashSet();
+      segmentsToOptimize = new HashSet<SegmentInfo>();
       final int numSegments = segmentInfos.size();
       for(int i=0;i<numSegments;i++)
         segmentsToOptimize.add(segmentInfos.info(i));
       
       // Now mark all pending & running merges as optimize
       // merge:
-      Iterator it = pendingMerges.iterator();
-      while(it.hasNext()) {
-        final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
+      for(final MergePolicy.OneMerge merge  : pendingMerges) {
         merge.optimize = true;
         merge.maxNumSegmentsOptimize = maxNumSegments;
       }
 
-      it = runningMerges.iterator();
-      while(it.hasNext()) {
-        final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
+      for ( final MergePolicy.OneMerge merge: runningMerges ) {
         merge.optimize = true;
         merge.maxNumSegmentsOptimize = maxNumSegments;
       }
@@ -2291,7 +2282,7 @@
             // threads to the current thread:
             final int size = mergeExceptions.size();
             for(int i=0;i<size;i++) {
-              final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) mergeExceptions.get(0);
+              final MergePolicy.OneMerge merge = mergeExceptions.get(i);
               if (merge.optimize) {
                 IOException err = new IOException("background merge hit exception: " + merge.segString(directory));
                 final Throwable t = merge.getException();
@@ -2324,16 +2315,16 @@
   /** Returns true if any merges in pendingMerges or
    *  runningMerges are optimization merges. */
   private synchronized boolean optimizeMergesPending() {
-    Iterator it = pendingMerges.iterator();
-    while(it.hasNext())
-      if (((MergePolicy.OneMerge) it.next()).optimize)
+    for (final MergePolicy.OneMerge merge : pendingMerges) {
+      if (merge.optimize)
         return true;
-
-    it = runningMerges.iterator();
-    while(it.hasNext())
-      if (((MergePolicy.OneMerge) it.next()).optimize)
+    }
+    
+    for (final MergePolicy.OneMerge merge : runningMerges) {
+      if (merge.optimize)
         return true;
-
+    }
+    
     return false;
   }
 
@@ -2513,7 +2504,7 @@
     if (pendingMerges.size() == 0)
       return null;
     else {
-      Iterator it = pendingMerges.iterator();
+      Iterator<MergePolicy.OneMerge> it = pendingMerges.iterator();
       while(it.hasNext()) {
         MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
         if (merge.isExternal) {
@@ -2810,9 +2801,7 @@
       stopMerges = true;
 
       // Abort all pending & running merges:
-      Iterator it = pendingMerges.iterator();
-      while(it.hasNext()) {
-        final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
+      for (final MergePolicy.OneMerge merge : pendingMerges) {
         if (infoStream != null)
           message("now abort pending merge " + merge.segString(directory));
         merge.abort();
@@ -2820,9 +2809,7 @@
       }
       pendingMerges.clear();
       
-      it = runningMerges.iterator();
-      while(it.hasNext()) {
-        final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
+      for (final MergePolicy.OneMerge merge : runningMerges) {
         if (infoStream != null)
           message("now abort running merge " + merge.segString(directory));
         merge.abort();
@@ -2918,12 +2905,12 @@
   }
 
   private synchronized void resetMergeExceptions() {
-    mergeExceptions = new ArrayList();
+    mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
     mergeGen++;
   }
 
   private void noDupDirs(Directory... dirs) {
-    HashSet dups = new HashSet();
+    HashSet<Directory> dups = new HashSet<Directory>();
     for(int i=0;i<dirs.length;i++) {
       if (dups.contains(dirs[i]))
         throw new IllegalArgumentException("Directory " + dirs[i] + " appears more than once");
@@ -3251,7 +3238,7 @@
     
       if (mergePolicy instanceof LogMergePolicy && getUseCompoundFile()) {
 
-        List files = null;
+        List<String> files = null;
 
         synchronized(this) {
           // Must incRef our files so that if another thread
@@ -3351,7 +3338,7 @@
    *  only "stick" if there are actually changes in the
    *  index to commit.
    */
-  public final void prepareCommit(Map commitUserData) throws CorruptIndexException, IOException {
+  public final void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
 
     if (hitOOM) {
       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
@@ -3416,7 +3403,7 @@
    * you should immediately close the writer.  See <a
    * href="#OOME">above</a> for details.</p>
    */
-  public final void commit(Map commitUserData) throws CorruptIndexException, IOException {
+  public final void commit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
 
     ensureOpen();
 
@@ -4115,7 +4102,7 @@
                                  false);
 
 
-    Map details = new HashMap();
+    Map<String,String> details = new HashMap<String,String>();
     details.put("optimize", merge.optimize+"");
     details.put("mergeFactor", end+"");
     details.put("mergeDocStores", mergeDocStores+"");
@@ -4132,8 +4119,8 @@
     setDiagnostics(info, source, null);
   }
 
-  private void setDiagnostics(SegmentInfo info, String source, Map details) {
-    Map diagnostics = new HashMap();
+  private void setDiagnostics(SegmentInfo info, String source, Map<String,String> details) {
+    Map<String,String> diagnostics = new HashMap<String,String>();
     diagnostics.put("source", source);
     diagnostics.put("lucene.version", Constants.LUCENE_VERSION);
     diagnostics.put("os", Constants.OS_NAME+"");
@@ -4199,7 +4186,7 @@
 
     boolean mergeDocStores = false;
 
-    final Set dss = new HashSet();
+    final Set<String> dss = new HashSet<String>();
     
     // This is try/finally to make sure merger's readers are
     // closed:
@@ -4463,12 +4450,12 @@
   }
 
   // Files that have been sync'd already
-  private HashSet synced = new HashSet();
+  private HashSet<String> synced = new HashSet<String>();
 
   // Files that are now being sync'd
-  private HashSet syncing = new HashSet();
+  private HashSet<String> syncing = new HashSet<String>();
 
-  private boolean startSync(String fileName, Collection pending) {
+  private boolean startSync(String fileName, Collection<String> pending) {
     synchronized(synced) {
       if (!synced.contains(fileName)) {
         if (!syncing.contains(fileName)) {
@@ -4494,11 +4481,11 @@
   }
 
   /** Blocks until all files in syncing are sync'd */
-  private boolean waitForAllSynced(Collection syncing) throws IOException {
+  private boolean waitForAllSynced(Collection<String> syncing) throws IOException {
     synchronized(synced) {
-      Iterator it = syncing.iterator();
+      Iterator<String> it = syncing.iterator();
       while(it.hasNext()) {
-        final String fileName = (String) it.next();
+        final String fileName = it.next();
         while(!synced.contains(fileName)) {
           if (!syncing.contains(fileName))
             // There was an error because a file that was
@@ -4541,7 +4528,7 @@
    *  if it wasn't already.  If that succeeds, then we
    *  prepare a new segments_N file but do not fully commit
    *  it. */
-  private void startCommit(long sizeInBytes, Map commitUserData) throws IOException {
+  private void startCommit(long sizeInBytes, Map<String,String> commitUserData) throws IOException {
 
     assert testPoint("startStartCommit");
 
@@ -4597,9 +4584,8 @@
           deleter.incRef(toSync, false);
           myChangeCount = changeCount;
 
-          Iterator it = toSync.files(directory, false).iterator();
-          while(it.hasNext()) {
-            String fileName = (String) it.next();
+          Collection<String> files = toSync.files(directory, false);
+          for(final String fileName: files) {
             assert directory.fileExists(fileName): "file " + fileName + " does not exist";
           }
 
@@ -4617,11 +4603,11 @@
         // Loop until all files toSync references are sync'd:
         while(true) {
 
-          final Collection pending = new ArrayList();
+          final Collection<String> pending = new ArrayList<String>();
 
-          Iterator it = toSync.files(directory, false).iterator();
+          Iterator<String> it = toSync.files(directory, false).iterator();
           while(it.hasNext()) {
-            final String fileName = (String) it.next();
+            final String fileName = it.next();
             if (startSync(fileName, pending)) {
               boolean success = false;
               try {

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=826527&r1=826526&r2=826527&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 Sun Oct 18 21:11:33 2009
@@ -46,7 +46,7 @@
   public void abort() {}
 
   // We only write the _X.nrm file at flush
-  void files(Collection files) {}
+  void files(Collection<String> files) {}
 
   void setFieldInfos(FieldInfos fieldInfos) {
     this.fieldInfos = fieldInfos;

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java Sun Oct 18 21:11:33 2009
@@ -112,7 +112,7 @@
   private static PrintStream infoStream;
 
   public final SegmentInfo info(int i) {
-    return (SegmentInfo) get(i);
+    return get(i);
   }
 
   /**

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java Sun Oct 18 21:11:33 2009
@@ -1031,7 +1031,7 @@
 
   // can return null if norms aren't stored
   protected synchronized byte[] getNorms(String field) throws IOException {
-    Norm norm = (Norm) norms.get(field);
+    Norm norm = norms.get(field);
     if (norm == null) return null;  // not indexed, or norms not stored
     return norm.bytes();
   }
@@ -1045,7 +1045,7 @@
 
   protected void doSetNorm(int doc, String field, byte value)
           throws IOException {
-    Norm norm = (Norm) norms.get(field);
+    Norm norm = norms.get(field);
     if (norm == null)                             // not an indexed field
       return;
 
@@ -1058,7 +1058,7 @@
     throws IOException {
 
     ensureOpen();
-    Norm norm = (Norm) norms.get(field);
+    Norm norm = norms.get(field);
     if (norm == null) {
       Arrays.fill(bytes, offset, bytes.length, DefaultSimilarity.encodeNorm(1.0f));
       return;

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java Sun Oct 18 21:11:33 2009
@@ -52,7 +52,7 @@
     SegmentTermEnum termEnum;
     
     // Used for caching the least recently looked-up Terms
-    Cache termInfoCache;
+    Cache<Term,TermInfo> termInfoCache;
   }
   
   TermInfosReader(Directory dir, String seg, FieldInfos fis, int readBufferSize, int indexDivisor)
@@ -143,7 +143,7 @@
       resources = new ThreadResources();
       resources.termEnum = terms();
       // Cache does not have to be thread-safe, it is only used by one thread at the same time
-      resources.termInfoCache = new SimpleLRUCache(DEFAULT_CACHE_SIZE);
+      resources.termInfoCache = new SimpleLRUCache<Term,TermInfo>(DEFAULT_CACHE_SIZE);
       threadResources.set(resources);
     }
     return resources;
@@ -187,12 +187,12 @@
 
     TermInfo ti;
     ThreadResources resources = getThreadResources();
-    Cache cache = null;
+    Cache<Term,TermInfo> cache = null;
     
     if (useCache) {
       cache = resources.termInfoCache;
       // check the cache first if the term was recently looked up
-      ti = (TermInfo) cache.get(term);
+      ti = cache.get(term);
       if (ti != null) {
         return ti;
       }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Sun Oct 18 21:11:33 2009
@@ -23,7 +23,7 @@
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Iterator;
+
 import java.util.Map;
 
 final class TermVectorsTermsWriter extends TermsHashConsumer {
@@ -51,7 +51,7 @@
       postings[i] = new PostingList();
   }
 
-  synchronized void flush(Map threadsAndFields, final SegmentWriteState state) throws IOException {
+  synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
 
     if (tvx != null) {
 
@@ -65,12 +65,9 @@
       tvf.flush();
     }
 
-    Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
-      Map.Entry entry = (Map.Entry) it.next();
-      Iterator it2 = ((Collection) entry.getValue()).iterator();
-      while(it2.hasNext()) {
-        TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) it2.next();
+    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
+      for (final TermsHashConsumerPerField field : entry.getValue() ) {
+        TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
         perField.termsHashPerField.reset();
         perField.shrinkHash();
       }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumer.java Sun Oct 18 21:11:33 2009
@@ -18,13 +18,14 @@
  */
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Map;
 
 abstract class TermsHashConsumer {
   abstract int bytesPerPosting();
   abstract void createPostings(RawPostingList[] postings, int start, int count);
   abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
-  abstract void flush(Map threadsAndFields, final SegmentWriteState state) throws IOException;
+  abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract void closeDocStore(SegmentWriteState state) throws IOException;
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/CachingSpanFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/CachingSpanFilter.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/CachingSpanFilter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/CachingSpanFilter.java Sun Oct 18 21:11:33 2009
@@ -19,7 +19,7 @@
 import org.apache.lucene.index.IndexReader;
 
 import java.io.IOException;
-import java.util.BitSet;
+
 import java.util.Map;
 import java.util.WeakHashMap;
 
@@ -33,7 +33,7 @@
   /**
    * A transient Filter cache.
    */
-  protected transient Map cache;
+  protected transient Map<IndexReader,SpanFilterResult> cache;
 
   /**
    * @param filter Filter to cache results of
@@ -50,11 +50,11 @@
   private SpanFilterResult getCachedResult(IndexReader reader) throws IOException {
     SpanFilterResult result = null;
     if (cache == null) {
-      cache = new WeakHashMap();
+      cache = new WeakHashMap<IndexReader,SpanFilterResult>();
     }
 
     synchronized (cache) {  // check cache
-      result = (SpanFilterResult) cache.get(reader);
+      result = cache.get(reader);
       if (result == null) {
         result = filter.bitSpans(reader);
         cache.put(reader, result);

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ConjunctionScorer.java Sun Oct 18 21:11:33 2009
@@ -29,8 +29,8 @@
   private final float coord;
   private int lastDoc = -1;
 
-  public ConjunctionScorer(Similarity similarity, Collection scorers) throws IOException {
-    this(similarity, (Scorer[]) scorers.toArray(new Scorer[scorers.size()]));
+  public ConjunctionScorer(Similarity similarity, Collection<Scorer> scorers) throws IOException {
+    this(similarity, scorers.toArray(new Scorer[scorers.size()]));
   }
 
   public ConjunctionScorer(Similarity similarity, Scorer[] scorers) throws IOException {
@@ -52,9 +52,9 @@
     // it will already start off sorted (all scorers on same doc).
     
     // note that this comparator is not consistent with equals!
-    Arrays.sort(scorers, new Comparator() {         // sort the array
-      public int compare(Object o1, Object o2) {
-        return ((Scorer) o1).docID() - ((Scorer) o2).docID();
+    Arrays.sort(scorers, new Comparator<Scorer>() {         // sort the array
+      public int compare(Scorer o1, Scorer o2) {
+        return o1.docID() - o2.docID();
       }
     });
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Explanation.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Explanation.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Explanation.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Explanation.java Sun Oct 18 21:11:33 2009
@@ -24,7 +24,7 @@
 public class Explanation implements java.io.Serializable {
   private float value;                            // the value of this node
   private String description;                     // what it represents
-  private ArrayList details;                      // sub-explanations
+  private ArrayList<Explanation> details;                      // sub-explanations
 
   public Explanation() {}
 
@@ -71,13 +71,13 @@
   public Explanation[] getDetails() {
     if (details == null)
       return null;
-    return (Explanation[])details.toArray(new Explanation[0]);
+    return details.toArray(new Explanation[0]);
   }
 
   /** Adds a sub-node to this explanation node. */
   public void addDetail(Explanation detail) {
     if (details == null)
-      details = new ArrayList();
+      details = new ArrayList<Explanation>();
     details.add(detail);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/FilterManager.java Sun Oct 18 21:11:33 2009
@@ -46,7 +46,7 @@
   protected static final long DEFAULT_CACHE_SLEEP_TIME = 1000 * 60 * 10;
 
   /** The cache itself */
-  protected Map           cache;
+  protected Map<Integer,FilterItem>           cache;
   /** Maximum allowed cache size */
   protected int           cacheCleanSize;
   /** Cache cleaning frequency */
@@ -65,7 +65,7 @@
    * Sets up the FilterManager singleton.
    */
   protected FilterManager() {
-    cache            = new HashMap();
+    cache            = new HashMap<Integer,FilterItem>();
     cacheCleanSize   = DEFAULT_CACHE_CLEAN_SIZE; // Let the cache get to 100 items
     cleanSleepTime   = DEFAULT_CACHE_SLEEP_TIME; // 10 minutes between cleanings
 
@@ -103,7 +103,7 @@
   public Filter getFilter(Filter filter) {
     synchronized(cache) {
       FilterItem fi = null;
-      fi = (FilterItem)cache.get(Integer.valueOf(filter.hashCode()));
+      fi = cache.get(Integer.valueOf(filter.hashCode()));
       if (fi != null) {
         fi.timestamp = new Date().getTime();
         return fi.filter;
@@ -146,14 +146,13 @@
   protected class FilterCleaner implements Runnable  {
 
     private boolean running = true;
-    private TreeSet sortedFilterItems;
+    private TreeSet<Map.Entry<Integer,FilterItem>> sortedFilterItems;
 
     public FilterCleaner() {
-      sortedFilterItems = new TreeSet(new Comparator() {
-        public int compare(Object a, Object b) {
-          if( a instanceof Map.Entry && b instanceof Map.Entry) {
-            FilterItem fia = (FilterItem) ((Map.Entry)a).getValue();
-            FilterItem fib = (FilterItem) ((Map.Entry)b).getValue();
+      sortedFilterItems = new TreeSet<Map.Entry<Integer,FilterItem>>(new Comparator<Map.Entry<Integer,FilterItem>>() {
+        public int compare(Map.Entry<Integer,FilterItem> a, Map.Entry<Integer,FilterItem> b) {
+            FilterItem fia = a.getValue();
+            FilterItem fib = b.getValue();
             if ( fia.timestamp == fib.timestamp ) {
               return 0;
             }
@@ -163,9 +162,7 @@
             }
             // larger timestamp last
             return 1;
-          } else {
-            throw new ClassCastException("Objects are not Map.Entry");
-          }
+          
         }
       });
     }
@@ -180,12 +177,12 @@
           sortedFilterItems.clear();
           synchronized (cache) {
             sortedFilterItems.addAll(cache.entrySet());
-            Iterator it = sortedFilterItems.iterator();
+            Iterator<Map.Entry<Integer,FilterItem>> it = sortedFilterItems.iterator();
             int numToDelete = (int) ((cache.size() - cacheCleanSize) * 1.5);
             int counter = 0;
             // loop over the set and delete all of the cache entries not used in a while
             while (it.hasNext() && counter++ < numToDelete) {
-              Map.Entry entry = (Map.Entry)it.next();
+              Map.Entry<Integer,FilterItem> entry = it.next();
               cache.remove(entry.getKey());
             }
           }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/IndexSearcher.java Sun Oct 18 21:11:33 2009
@@ -90,9 +90,9 @@
     reader = r;
     this.closeReader = closeReader;
 
-    List subReadersList = new ArrayList();
+    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
     gatherSubReaders(subReadersList, reader);
-    subReaders = (IndexReader[]) subReadersList.toArray(new IndexReader[subReadersList.size()]);
+    subReaders = subReadersList.toArray(new IndexReader[subReadersList.size()]);
     docStarts = new int[subReaders.length];
     int maxDoc = 0;
     for (int i = 0; i < subReaders.length; i++) {
@@ -101,7 +101,7 @@
     }
   }
 
-  protected void gatherSubReaders(List allSubReaders, IndexReader r) {
+  protected void gatherSubReaders(List<IndexReader> allSubReaders, IndexReader r) {
     ReaderUtil.gatherSubReaders(allSubReaders, r);
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/MultiPhraseQuery.java Sun Oct 18 21:11:33 2009
@@ -38,8 +38,8 @@
  */
 public class MultiPhraseQuery extends Query {
   private String field;
-  private ArrayList termArrays = new ArrayList();
-  private ArrayList positions = new ArrayList();
+  private ArrayList<Term[]> termArrays = new ArrayList<Term[]>();
+  private ArrayList<Integer> positions = new ArrayList<Integer>();
 
   private int slop = 0;
 
@@ -95,10 +95,10 @@
   }
 
   /**
-   * Returns a List<Term[]> of the terms in the multiphrase.
+   * Returns a List of the terms in the multiphrase.
    * Do not modify the List or its contents.
    */
-  public List getTermArrays() {
+  public List<Term[]> getTermArrays() {
 	  return Collections.unmodifiableList(termArrays);
   }
 
@@ -114,10 +114,9 @@
 
   // inherit javadoc
   public void extractTerms(Set<Term> terms) {
-    for (Iterator iter = termArrays.iterator(); iter.hasNext();) {
-      Term[] arr = (Term[])iter.next();
-      for (int i=0; i<arr.length; i++) {
-        terms.add(arr[i]);
+    for (final Term[] arr : termArrays) {
+      for (final Term term: arr) {
+        terms.add(term);
       }
     }
   }
@@ -135,11 +134,9 @@
       this.similarity = getSimilarity(searcher);
 
       // compute idf
-      Iterator i = termArrays.iterator();
-      while (i.hasNext()) {
-        Term[] terms = (Term[])i.next();
-        for (int j=0; j<terms.length; j++) {
-          idf += getSimilarity(searcher).idf(terms[j], searcher);
+      for(final Term[] terms: termArrays) {
+        for (Term term: terms) {
+          idf += getSimilarity(searcher).idf(term, searcher);
         }
       }
     }
@@ -278,9 +275,9 @@
     }
 
     buffer.append("\"");
-    Iterator i = termArrays.iterator();
+    Iterator<Term[]> i = termArrays.iterator();
     while (i.hasNext()) {
-      Term[] terms = (Term[])i.next();
+      Term[] terms = i.next();
       if (terms.length > 1) {
         buffer.append("(");
         for (int j = 0; j < terms.length; j++) {
@@ -330,9 +327,7 @@
   // Breakout calculation of the termArrays hashcode
   private int termArraysHashCode() {
     int hashCode = 1;
-    Iterator iterator = termArrays.iterator();
-    while (iterator.hasNext()) {
-      Term[] termArray = (Term[]) iterator.next();
+    for (final Term[] termArray: termArrays) {
       hashCode = 31 * hashCode
           + (termArray == null ? 0 : arraysHashCode(termArray));
     }
@@ -354,15 +349,15 @@
   }
 
   // Breakout calculation of the termArrays equals
-  private boolean termArraysEquals(List termArrays1, List termArrays2) {
+  private boolean termArraysEquals(List<Term[]> termArrays1, List<Term[]> termArrays2) {
     if (termArrays1.size() != termArrays2.size()) {
       return false;
     }
-    ListIterator iterator1 = termArrays1.listIterator();
-    ListIterator iterator2 = termArrays2.listIterator();
+    ListIterator<Term[]> iterator1 = termArrays1.listIterator();
+    ListIterator<Term[]> iterator2 = termArrays2.listIterator();
     while (iterator1.hasNext()) {
-      Term[] termArray1 = (Term[]) iterator1.next();
-      Term[] termArray2 = (Term[]) iterator2.next();
+      Term[] termArray1 = iterator1.next();
+      Term[] termArray2 = iterator2.next();
       if (!(termArray1 == null ? termArray2 == null : Arrays.equals(termArray1,
           termArray2))) {
         return false;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/MultiSearcher.java Sun Oct 18 21:11:33 2009
@@ -43,10 +43,10 @@
    * initialize Weights.
    */
   private static class CachedDfSource extends Searcher {
-    private Map dfMap; // Map from Terms to corresponding doc freqs
+    private Map<Term,Integer> dfMap; // Map from Terms to corresponding doc freqs
     private int maxDoc; // document count
 
-    public CachedDfSource(Map dfMap, int maxDoc, Similarity similarity) {
+    public CachedDfSource(Map<Term,Integer> dfMap, int maxDoc, Similarity similarity) {
       this.dfMap = dfMap;
       this.maxDoc = maxDoc;
       setSimilarity(similarity);
@@ -55,7 +55,7 @@
     public int docFreq(Term term) {
       int df;
       try {
-        df = ((Integer) dfMap.get(term)).intValue();
+        df = dfMap.get(term).intValue();
       } catch (NullPointerException e) {
         throw new IllegalArgumentException("df for term " + term.text()
             + " not available");
@@ -305,7 +305,7 @@
     Query rewrittenQuery = rewrite(original);
 
     // step 2
-    Set terms = new HashSet();
+    Set<Term> terms = new HashSet<Term>();
     rewrittenQuery.extractTerms(terms);
 
     // step3
@@ -319,7 +319,7 @@
       }
     }
 
-    HashMap dfMap = new HashMap();
+    HashMap<Term,Integer> dfMap = new HashMap<Term,Integer>();
     for(int i=0; i<allTermsArray.length; i++) {
       dfMap.put(allTermsArray[i], Integer.valueOf(aggregatedDfs[i]));
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQuery.java Sun Oct 18 21:11:33 2009
@@ -21,11 +21,11 @@
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Iterator;
+
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.util.ToStringUtils;
+
 import org.apache.lucene.queryParser.QueryParser; // for javadoc
 
 /**
@@ -217,7 +217,7 @@
       // exhaust the enum before hitting either of the
       // cutoffs, we use ConstantBooleanQueryRewrite; else,
       // ConstantFilterRewrite:
-      final Collection pendingTerms = new ArrayList();
+      final Collection<Term> pendingTerms = new ArrayList<Term>();
       final int docCountCutoff = (int) ((docCountPercent / 100.) * reader.maxDoc());
       final int termCountLimit = Math.min(BooleanQuery.getMaxClauseCount(), termCountCutoff);
       int docVisitCount = 0;
@@ -244,10 +244,9 @@
             // Enumeration is done, and we hit a small
             // enough number of terms & docs -- just make a
             // BooleanQuery, now
-            Iterator it = pendingTerms.iterator();
             BooleanQuery bq = new BooleanQuery(true);
-            while(it.hasNext()) {
-              TermQuery tq = new TermQuery((Term) it.next());
+            for (final Term term: pendingTerms) {
+              TermQuery tq = new TermQuery(term);
               bq.add(tq, BooleanClause.Occur.SHOULD);
             }
             // Strip scores

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/PhraseQuery.java Sun Oct 18 21:11:33 2009
@@ -34,8 +34,8 @@
  */
 public class PhraseQuery extends Query {
   private String field;
-  private ArrayList terms = new ArrayList(4);
-  private ArrayList positions = new ArrayList(4);
+  private ArrayList<Term> terms = new ArrayList<Term>(4);
+  private ArrayList<Integer> positions = new ArrayList<Integer>(4);
   private int maxPosition = 0;
   private int slop = 0;
 
@@ -67,7 +67,7 @@
   public void add(Term term) {
     int position = 0;
     if(positions.size() > 0)
-        position = ((Integer) positions.get(positions.size()-1)).intValue() + 1;
+        position = positions.get(positions.size()-1).intValue() + 1;
 
     add(term, position);
   }
@@ -94,7 +94,7 @@
 
   /** Returns the set of terms in this phrase. */
   public Term[] getTerms() {
-    return (Term[])terms.toArray(new Term[0]);
+    return terms.toArray(new Term[0]);
   }
 
   /**
@@ -103,7 +103,7 @@
   public int[] getPositions() {
       int[] result = new int[positions.size()];
       for(int i = 0; i < positions.size(); i++)
-          result[i] = ((Integer) positions.get(i)).intValue();
+          result[i] = positions.get(i).intValue();
       return result;
   }
 
@@ -145,7 +145,7 @@
 
       TermPositions[] tps = new TermPositions[terms.size()];
       for (int i = 0; i < terms.size(); i++) {
-        TermPositions p = reader.termPositions((Term)terms.get(i));
+        TermPositions p = reader.termPositions(terms.get(i));
         if (p == null)
           return null;
         tps[i] = p;
@@ -176,7 +176,7 @@
           query.append(" ");
         }
 
-        Term term = (Term)terms.get(i);
+        Term term = terms.get(i);
 
         query.append(term.text());
       }
@@ -242,7 +242,7 @@
 
   public Weight createWeight(Searcher searcher) throws IOException {
     if (terms.size() == 1) {			  // optimize one-term case
-      Term term = (Term)terms.get(0);
+      Term term = terms.get(0);
       Query termQuery = new TermQuery(term);
       termQuery.setBoost(getBoost());
       return termQuery.createWeight(searcher);
@@ -268,12 +268,12 @@
     buffer.append("\"");
     String[] pieces = new String[maxPosition + 1];
     for (int i = 0; i < terms.size(); i++) {
-      int pos = ((Integer)positions.get(i)).intValue();
+      int pos = positions.get(i).intValue();
       String s = pieces[pos];
       if (s == null) {
-        s = ((Term)terms.get(i)).text();
+        s = (terms.get(i)).text();
       } else {
-        s = s + "|" + ((Term)terms.get(i)).text();
+        s = s + "|" + (terms.get(i)).text();
       }
       pieces[pos] = s;
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Query.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Query.java Sun Oct 18 21:11:33 2009
@@ -20,7 +20,7 @@
 import java.io.IOException;
 
 import java.util.HashSet;
-import java.util.Iterator;
+
 import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
@@ -126,7 +126,7 @@
    * the other queries.
   */
   public Query combine(Query[] queries) {
-    HashSet uniques = new HashSet();
+    HashSet<Query> uniques = new HashSet<Query>();
     for (int i = 0; i < queries.length; i++) {
       Query query = queries[i];
       BooleanClause[] clauses = null;
@@ -152,10 +152,9 @@
     if(uniques.size() == 1){
         return (Query)uniques.iterator().next();
     }
-    Iterator it = uniques.iterator();
     BooleanQuery result = new BooleanQuery(true);
-    while (it.hasNext())
-      result.add((Query) it.next(), BooleanClause.Occur.SHOULD);
+    for (final Query query : uniques)
+      result.add(query, BooleanClause.Occur.SHOULD);
     return result;
   }
   
@@ -179,20 +178,18 @@
    *<p>A utility for use by {@link #combine(Query[])} implementations.
    */
   public static Query mergeBooleanQueries(BooleanQuery[] queries) {
-    HashSet allClauses = new HashSet();
-    for (int i = 0; i < queries.length; i++) {
-      BooleanClause[] clauses = queries[i].getClauses();
-      for (int j = 0; j < clauses.length; j++) {
-        allClauses.add(clauses[j]);
+    HashSet<BooleanClause> allClauses = new HashSet<BooleanClause>();
+    for (BooleanQuery booleanQuery : queries) {
+      for (BooleanClause clause : booleanQuery) {
+        allClauses.add(clause);
       }
     }
 
     boolean coordDisabled =
       queries.length==0? false : queries[0].isCoordDisabled();
     BooleanQuery result = new BooleanQuery(coordDisabled);
-    Iterator i = allClauses.iterator();
-    while (i.hasNext()) {
-      result.add((BooleanClause)i.next());
+    for(BooleanClause clause2 : allClauses) {
+      result.add(clause2);
     }
     return result;
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/QueryTermVector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/QueryTermVector.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/QueryTermVector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/QueryTermVector.java Sun Oct 18 21:11:33 2009
@@ -22,7 +22,7 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.Iterator;
+
 import java.util.List;
 import java.util.Map;
 
@@ -56,7 +56,7 @@
       TokenStream stream = analyzer.tokenStream("", new StringReader(queryString));
       if (stream != null)
       {
-        List terms = new ArrayList();
+        List<String> terms = new ArrayList<String>();
         try {
           boolean hasMoreTokens = false;
           
@@ -78,30 +78,29 @@
   private void processTerms(String[] queryTerms) {
     if (queryTerms != null) {
       Arrays.sort(queryTerms);
-      Map tmpSet = new HashMap(queryTerms.length);
+      Map<String,Integer> tmpSet = new HashMap<String,Integer>(queryTerms.length);
       //filter out duplicates
-      List tmpList = new ArrayList(queryTerms.length);
-      List tmpFreqs = new ArrayList(queryTerms.length);
+      List<String> tmpList = new ArrayList<String>(queryTerms.length);
+      List<Integer> tmpFreqs = new ArrayList<Integer>(queryTerms.length);
       int j = 0;
       for (int i = 0; i < queryTerms.length; i++) {
         String term = queryTerms[i];
-        Integer position = (Integer)tmpSet.get(term);
+        Integer position = tmpSet.get(term);
         if (position == null) {
           tmpSet.put(term, Integer.valueOf(j++));
           tmpList.add(term);
           tmpFreqs.add(Integer.valueOf(1));
         }       
         else {
-          Integer integer = (Integer)tmpFreqs.get(position.intValue());
+          Integer integer = tmpFreqs.get(position.intValue());
           tmpFreqs.set(position.intValue(), Integer.valueOf(integer.intValue() + 1));          
         }
       }
-      terms = (String[])tmpList.toArray(terms);
+      terms = tmpList.toArray(terms);
       //termFreqs = (int[])tmpFreqs.toArray(termFreqs);
       termFreqs = new int[tmpFreqs.size()];
       int i = 0;
-      for (Iterator iter = tmpFreqs.iterator(); iter.hasNext();) {
-        Integer integer = (Integer) iter.next();
+      for (final Integer integer : tmpFreqs) {
         termFreqs[i++] = integer.intValue();
       }
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Similarity.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Similarity.java Sun Oct 18 21:11:33 2009
@@ -27,7 +27,7 @@
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.IdentityHashMap;
-import java.util.Iterator;
+
 
 /** 
  * Expert: Scoring API.
@@ -787,11 +787,11 @@
    * @return idf score factor
    * @deprecated see {@link #idfExplain(Collection, Searcher)}
    */
-  public float idf(Collection terms, Searcher searcher) throws IOException {
+  public float idf(Collection<Term> terms, Searcher searcher) throws IOException {
     float idf = 0.0f;
-    Iterator i = terms.iterator();
-    while (i.hasNext()) {
-      idf += idf((Term)i.next(), searcher);
+
+    for(final Term term: terms) {
+      idf += idf(term, searcher);
     }
     return idf;
   }
@@ -810,7 +810,7 @@
    *         for each term.
    * @throws IOException
    */
-  public IDFExplanation idfExplain(Collection terms, Searcher searcher) throws IOException {
+  public IDFExplanation idfExplain(Collection<Term> terms, Searcher searcher) throws IOException {
     if(supportedMethods.overridesCollectionIDF) {
       final float idf = idf(terms, searcher);
       return new IDFExplanation() {
@@ -827,9 +827,7 @@
     final int max = searcher.maxDoc();
     float idf = 0.0f;
     final StringBuilder exp = new StringBuilder();
-    Iterator i = terms.iterator();
-    while (i.hasNext()) {
-      Term term = (Term)i.next();
+    for (final Term term : terms ) {
       final int df = searcher.docFreq(term);
       idf += idf(df, max);
       exp.append(" ");
@@ -955,7 +953,7 @@
   }
   
   /** @deprecated Remove this when old API is removed! */
-  private static final IdentityHashMap/*<Class<? extends Similarity>,MethodSupport>*/ knownMethodSupport = new IdentityHashMap();
+  private static final IdentityHashMap<Class<? extends Similarity>,MethodSupport> knownMethodSupport = new IdentityHashMap();
   
   /** @deprecated Remove this when old API is removed! */
   private static MethodSupport getSupportedMethods(Class clazz) {

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Sun Oct 18 21:11:33 2009
@@ -146,14 +146,14 @@
         if (!checkedRepeats) {
             checkedRepeats = true;
             // check for repeats
-            HashMap m = null;
+            HashMap<PhrasePositions, Object> m = null;
             for (PhrasePositions pp = first; pp != null; pp = pp.next) {
                 int tpPos = pp.position + pp.offset;
                 for (PhrasePositions pp2 = pp.next; pp2 != null; pp2 = pp2.next) {
                     int tpPos2 = pp2.position + pp2.offset;
                     if (tpPos2 == tpPos) { 
                         if (m == null)
-                            m = new HashMap();
+                            m = new HashMap<PhrasePositions, Object>();
                         pp.repeats = true;
                         pp2.repeats = true;
                         m.put(pp,null);
@@ -162,7 +162,7 @@
                 }
             }
             if (m!=null)
-                repeats = (PhrasePositions[]) m.keySet().toArray(new PhrasePositions[0]);
+                repeats = m.keySet().toArray(new PhrasePositions[0]);
         }
         
         // with repeats must advance some repeating pp's so they all start with differing tp's       

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/SpanFilterResult.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/SpanFilterResult.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/SpanFilterResult.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/SpanFilterResult.java Sun Oct 18 21:11:33 2009
@@ -16,7 +16,7 @@
  */
 
 import java.util.ArrayList;
-import java.util.BitSet;
+
 import java.util.List;
 
 
@@ -29,14 +29,14 @@
  **/
 public class SpanFilterResult {
   private DocIdSet docIdSet;
-  private List positions;//Spans spans;
+  private List<PositionInfo> positions;//Spans spans;
   
   /**
   *
   * @param docIdSet The DocIdSet for the Filter
   * @param positions A List of {@link org.apache.lucene.search.SpanFilterResult.PositionInfo} objects
   */
-  public SpanFilterResult(DocIdSet docIdSet, List positions) {
+  public SpanFilterResult(DocIdSet docIdSet, List<PositionInfo> positions) {
     this.docIdSet = docIdSet;
     this.positions = positions;
   }
@@ -46,7 +46,7 @@
    * Entries are increasing by document order
    * @return A List of PositionInfo objects
    */
-  public List getPositions() {
+  public List<PositionInfo> getPositions() {
     return positions;
   }
 
@@ -57,12 +57,12 @@
 
   public static class PositionInfo {
     private int doc;
-    private List positions;
+    private List<StartEnd> positions;
 
 
     public PositionInfo(int doc) {
       this.doc = doc;
-      positions = new ArrayList();
+      positions = new ArrayList<StartEnd>();
     }
 
     public void addPosition(int start, int end)
@@ -76,9 +76,9 @@
 
     /**
      *
-     * @return A List of {@link org.apache.lucene.search.SpanFilterResult.StartEnd} objects
+     * @return Positions
      */
-    public List getPositions() {
+    public List<StartEnd> getPositions() {
       return positions;
     }
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/SpanQueryFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/SpanQueryFilter.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/SpanQueryFilter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/SpanQueryFilter.java Sun Oct 18 21:11:33 2009
@@ -63,7 +63,7 @@
 
     final OpenBitSet bits = new OpenBitSet(reader.maxDoc());
     Spans spans = query.getSpans(reader);
-    List tmp = new ArrayList(20);
+    List<SpanFilterResult.PositionInfo> tmp = new ArrayList<SpanFilterResult.PositionInfo>(20);
     int currentDoc = -1;
     SpanFilterResult.PositionInfo currentInfo = null;
     while (spans.next())

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Sun Oct 18 21:11:33 2009
@@ -77,8 +77,7 @@
     SpanQuery[] newClauses = new SpanQuery[sz];
 
     for (int i = 0; i < sz; i++) {
-      SpanQuery clause = (SpanQuery) clauses.get(i);
-      newClauses[i] = (SpanQuery) clause.clone();
+      newClauses[i] = (SpanQuery) clauses.get(i).clone();
     }
     PayloadNearQuery boostingNearQuery = new PayloadNearQuery(newClauses, slop,
         inOrder);
@@ -90,9 +89,9 @@
   public String toString(String field) {
     StringBuilder buffer = new StringBuilder();
     buffer.append("payloadNear([");
-    Iterator i = clauses.iterator();
+    Iterator<SpanQuery> i = clauses.iterator();
     while (i.hasNext()) {
-      SpanQuery clause = (SpanQuery) i.next();
+      SpanQuery clause = i.next();
       buffer.append(clause.toString(field));
       if (i.hasNext()) {
         buffer.append(", ");
@@ -194,9 +193,8 @@
      * 
      * @see Spans
      */
-    protected void processPayloads(Collection payLoads, int start, int end) {
-      for (Iterator iterator = payLoads.iterator(); iterator.hasNext();) {
-        byte[] thePayload = (byte[]) iterator.next();
+    protected void processPayloads(Collection<byte[]> payLoads, int start, int end) {
+      for (final byte[] thePayload : payLoads) {
         payloadScore = function.currentScore(doc, fieldName, start, end,
             payloadsSeen, payloadScore, similarity.scorePayload(doc, fieldName,
                 spans.start(), spans.end(), thePayload, 0, thePayload.length));

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java Sun Oct 18 21:11:33 2009
@@ -69,13 +69,13 @@
    * @return payloads Collection
    * @throws IOException
    */
-  public Collection getPayloadsForQuery(Query query) throws IOException {
-    Collection payloads = new ArrayList();
+  public Collection<byte[]> getPayloadsForQuery(Query query) throws IOException {
+    Collection<byte[]> payloads = new ArrayList<byte[]>();
     queryToSpanQuery(query, payloads);
     return payloads;
   }
 
-  private void queryToSpanQuery(Query query, Collection payloads)
+  private void queryToSpanQuery(Query query, Collection<byte[]> payloads)
       throws IOException {
     if (query instanceof BooleanQuery) {
       BooleanClause[] queryClauses = ((BooleanQuery) query).getClauses();
@@ -113,14 +113,14 @@
       queryToSpanQuery(((FilteredQuery) query).getQuery(), payloads);
     } else if (query instanceof DisjunctionMaxQuery) {
 
-      for (Iterator iterator = ((DisjunctionMaxQuery) query).iterator(); iterator
+      for (Iterator<Query> iterator = ((DisjunctionMaxQuery) query).iterator(); iterator
           .hasNext();) {
-        queryToSpanQuery((Query) iterator.next(), payloads);
+        queryToSpanQuery(iterator.next(), payloads);
       }
 
     } else if (query instanceof MultiPhraseQuery) {
       final MultiPhraseQuery mpq = (MultiPhraseQuery) query;
-      final List termArrays = mpq.getTermArrays();
+      final List<Term[]> termArrays = mpq.getTermArrays();
       final int[] positions = mpq.getPositions();
       if (positions.length > 0) {
 
@@ -131,19 +131,19 @@
           }
         }
 
-        final List[] disjunctLists = new List[maxPosition + 1];
+        final List<Query>[] disjunctLists = new List[maxPosition + 1];
         int distinctPositions = 0;
 
         for (int i = 0; i < termArrays.size(); ++i) {
-          final Term[] termArray = (Term[]) termArrays.get(i);
-          List disjuncts = disjunctLists[positions[i]];
+          final Term[] termArray = termArrays.get(i);
+          List<Query> disjuncts = disjunctLists[positions[i]];
           if (disjuncts == null) {
-            disjuncts = (disjunctLists[positions[i]] = new ArrayList(
+            disjuncts = (disjunctLists[positions[i]] = new ArrayList<Query>(
                 termArray.length));
             ++distinctPositions;
           }
-          for (int j = 0; j < termArray.length; ++j) {
-            disjuncts.add(new SpanTermQuery(termArray[j]));
+          for (final Term term : termArray) {
+            disjuncts.add(new SpanTermQuery(term));
           }
         }
 
@@ -151,9 +151,9 @@
         int position = 0;
         final SpanQuery[] clauses = new SpanQuery[distinctPositions];
         for (int i = 0; i < disjunctLists.length; ++i) {
-          List disjuncts = disjunctLists[i];
+          List<Query> disjuncts = disjunctLists[i];
           if (disjuncts != null) {
-            clauses[position++] = new SpanOrQuery((SpanQuery[]) disjuncts
+            clauses[position++] = new SpanOrQuery(disjuncts
                 .toArray(new SpanQuery[disjuncts.size()]));
           } else {
             ++positionGaps;
@@ -171,16 +171,14 @@
     }
   }
 
-  private void getPayloads(Collection payloads, SpanQuery query)
+  private void getPayloads(Collection<byte []> payloads, SpanQuery query)
       throws IOException {
     Spans spans = query.getSpans(reader);
 
     while (spans.next() == true) {
       if (spans.isPayloadAvailable()) {
-        Collection payload = spans.getPayload();
-        Iterator it = payload.iterator();
-        while (it.hasNext()) {
-          byte[] bytes = (byte[]) it.next();
+        Collection<byte[]> payload = spans.getPayload();
+        for (byte [] bytes : payload) {
           payloads.add(bytes);
         }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java Sun Oct 18 21:11:33 2009
@@ -18,7 +18,6 @@
  */
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Sun Oct 18 21:11:33 2009
@@ -65,12 +65,12 @@
   private int matchDoc = -1;
   private int matchStart = -1;
   private int matchEnd = -1;
-  private List/*<byte[]>*/ matchPayload;
+  private List<byte[]> matchPayload;
 
   private final Spans[] subSpansByDoc;
-  private final Comparator spanDocComparator = new Comparator() {
-    public int compare(Object o1, Object o2) {
-      return ((Spans)o1).doc() - ((Spans)o2).doc();
+  private final Comparator<Spans> spanDocComparator = new Comparator<Spans>() {
+    public int compare(Spans o1, Spans o2) {
+      return o1.doc() - o2.doc();
     }
   };
   
@@ -91,7 +91,7 @@
     allowedSlop = spanNearQuery.getSlop();
     SpanQuery[] clauses = spanNearQuery.getClauses();
     subSpans = new Spans[clauses.length];
-    matchPayload = new LinkedList();
+    matchPayload = new LinkedList<byte[]>();
     subSpansByDoc = new Spans[clauses.length];
     for (int i = 0; i < clauses.length; i++) {
       subSpans[i] = clauses[i].getSpans(reader);
@@ -115,7 +115,7 @@
 
   // TODO: Remove warning after API has been finalized
   // TODO: Would be nice to be able to lazy load payloads
-  public Collection/*<byte[]>*/ getPayload() throws IOException {
+  public Collection<byte[]> getPayload() throws IOException {
     return matchPayload;
   }
 
@@ -256,12 +256,12 @@
   private boolean shrinkToAfterShortestMatch() throws IOException {
     matchStart = subSpans[subSpans.length - 1].start();
     matchEnd = subSpans[subSpans.length - 1].end();
-    Set possibleMatchPayloads = new HashSet();
+    Set<byte[]> possibleMatchPayloads = new HashSet<byte[]>();
     if (subSpans[subSpans.length - 1].isPayloadAvailable()) {
       possibleMatchPayloads.addAll(subSpans[subSpans.length - 1].getPayload());
     }
 
-    Collection possiblePayload = null;
+    Collection<byte[]> possiblePayload = null;
     
     int matchSlop = 0;
     int lastStart = matchStart;
@@ -269,8 +269,8 @@
     for (int i = subSpans.length - 2; i >= 0; i--) {
       Spans prevSpans = subSpans[i];
       if (collectPayloads && prevSpans.isPayloadAvailable()) {
-        Collection payload = prevSpans.getPayload();
-        possiblePayload = new ArrayList(payload.size());
+        Collection<byte[]> payload = prevSpans.getPayload();
+        possiblePayload = new ArrayList<byte[]>(payload.size());
         possiblePayload.addAll(payload);
       }
       
@@ -293,8 +293,8 @@
             prevStart = ppStart;
             prevEnd = ppEnd;
             if (collectPayloads && prevSpans.isPayloadAvailable()) {
-              Collection payload = prevSpans.getPayload();
-              possiblePayload = new ArrayList(payload.size());
+              Collection<byte[]> payload = prevSpans.getPayload();
+              possiblePayload = new ArrayList<byte[]>(payload.size());
               possiblePayload.addAll(payload);
             }
           }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Sun Oct 18 21:11:33 2009
@@ -36,7 +36,7 @@
 public class NearSpansUnordered extends Spans {
   private SpanNearQuery query;
 
-  private List ordered = new ArrayList();         // spans in query order
+  private List<SpansCell> ordered = new ArrayList<SpansCell>();         // spans in query order
   private Spans[] subSpans;  
   private int slop;                               // from query
 
@@ -107,8 +107,8 @@
     public int start() { return spans.start(); }
     public int end() { return spans.end(); }
                     // TODO: Remove warning after API has been finalized
-    public Collection/*<byte[]>*/ getPayload() throws IOException {
-      return new ArrayList(spans.getPayload());
+    public Collection<byte[]> getPayload() throws IOException {
+      return new ArrayList<byte[]>(spans.getPayload());
     }
 
     // TODO: Remove warning after API has been finalized
@@ -223,8 +223,8 @@
    * @return Collection of <code>byte[]</code> payloads
    * @throws IOException
    */
-  public Collection/*<byte[]>*/ getPayload() throws IOException {
-    Set/*<byte[]*/ matchPayload = new HashSet();
+  public Collection<byte[]> getPayload() throws IOException {
+    Set<byte[]> matchPayload = new HashSet<byte[]>();
     for (SpansCell cell = first; cell != null; cell = cell.next) {
       if (cell.isPayloadAvailable()) {
         matchPayload.addAll(cell.getPayload());
@@ -253,7 +253,7 @@
 
   private void initList(boolean next) throws IOException {
     for (int i = 0; more && i < ordered.size(); i++) {
-      SpansCell cell = (SpansCell)ordered.get(i);
+      SpansCell cell = ordered.get(i);
       if (next)
         more = cell.next();                       // move to first entry
       if (more) {

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java Sun Oct 18 21:11:33 2009
@@ -94,10 +94,10 @@
         public int end() { return spans.end(); }
 
       // TODO: Remove warning after API has been finalized
-      public Collection/*<byte[]>*/ getPayload() throws IOException {
-        ArrayList result = null;
+      public Collection<byte[]> getPayload() throws IOException {
+        ArrayList<byte[]> result = null;
         if (spans.isPayloadAvailable()) {
-          result = new ArrayList(spans.getPayload());
+          result = new ArrayList<byte[]>(spans.getPayload());
         }
         return result;//TODO: any way to avoid the new construction?
       }



Mime
View raw message