lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From markrmil...@apache.org
Subject svn commit: r827772 [3/6] - in /lucene/java/branches/flex_1458: ./ contrib/ contrib/instantiated/src/java/org/apache/lucene/store/instantiated/ contrib/misc/src/java/org/apache/lucene/queryParser/precedence/ contrib/queries/src/java/org/apache/lucene/s...
Date Tue, 20 Oct 2009 19:58:22 GMT
Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessor.java Tue Oct 20 19:58:18 2009
@@ -21,7 +21,7 @@
 import java.util.Collection;
 import java.util.Map;
 import java.util.HashMap;
-import java.util.Iterator;
+
 
 /**
  * This is a DocConsumer that gathers all fields under the
@@ -50,12 +50,11 @@
     fieldsWriter.closeDocStore(state);
   }
 
-  public void flush(Collection threads, SegmentWriteState state) throws IOException {
+  public void flush(Collection<DocConsumerPerThread> threads, SegmentWriteState state) throws IOException {
 
-    Map childThreadsAndFields = new HashMap();
-    Iterator it = threads.iterator();
-    while(it.hasNext()) {
-      DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) it.next();
+    Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> childThreadsAndFields = new HashMap<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>>();
+    for ( DocConsumerPerThread thread : threads) {
+      DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread;
       childThreadsAndFields.put(perThread.consumer, perThread.fields());
       perThread.trimFields(state);
     }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java Tue Oct 20 19:58:18 2009
@@ -76,8 +76,8 @@
     consumer.abort();
   }
 
-  public Collection fields() {
-    Collection fields = new HashSet();
+  public Collection<DocFieldConsumerPerField> fields() {
+    Collection<DocFieldConsumerPerField> fields = new HashSet<DocFieldConsumerPerField>();
     for(int i=0;i<fieldHash.length;i++) {
       DocFieldProcessorPerField field = fieldHash[i];
       while(field != null) {
@@ -163,7 +163,7 @@
     
     final int thisFieldGen = fieldGen++;
 
-    final List docFields = doc.getFields();
+    final List<Fieldable> docFields = doc.getFields();
     final int numDocFields = docFields.size();
 
     // Absorb any new fields first seen in this document.

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverter.java Tue Oct 20 19:58:18 2009
@@ -21,10 +21,9 @@
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
+
 import java.util.Map;
 
-import org.apache.lucene.util.AttributeSource;
 
 /** This is a DocFieldConsumer that inverts each field,
  *  separately, from a Document, and accepts a
@@ -46,25 +45,20 @@
     endConsumer.setFieldInfos(fieldInfos);
   }
 
-  void flush(Map threadsAndFields, SegmentWriteState state) throws IOException {
+  void flush(Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
 
-    Map childThreadsAndFields = new HashMap();
-    Map endChildThreadsAndFields = new HashMap();
+    Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> childThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
+    Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> endChildThreadsAndFields = new HashMap<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>>();
 
-    Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
+    for (Map.Entry<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> entry : threadsAndFields.entrySet() ) {
 
-      Map.Entry entry = (Map.Entry) it.next();
 
       DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey();
 
-      Collection fields = (Collection) entry.getValue();
-
-      Iterator fieldsIt = fields.iterator();
-      Collection childFields = new HashSet();
-      Collection endChildFields = new HashSet();
-      while(fieldsIt.hasNext()) {
-        DocInverterPerField perField = (DocInverterPerField) fieldsIt.next();
+      Collection<InvertedDocConsumerPerField> childFields = new HashSet<InvertedDocConsumerPerField>();
+      Collection<InvertedDocEndConsumerPerField> endChildFields = new HashSet<InvertedDocEndConsumerPerField>();
+      for (final DocFieldConsumerPerField field: entry.getValue() ) {  
+        DocInverterPerField perField = (DocInverterPerField) field;
         childFields.add(perField.consumer);
         endChildFields.add(perField.endConsumer);
       }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java Tue Oct 20 19:58:18 2009
@@ -24,7 +24,6 @@
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 
@@ -126,7 +125,7 @@
   // than this they share ThreadStates
   private final static int MAX_THREAD_STATE = 5;
   private DocumentsWriterThreadState[] threadStates = new DocumentsWriterThreadState[0];
-  private final HashMap threadBindings = new HashMap();
+  private final HashMap<Thread,DocumentsWriterThreadState> threadBindings = new HashMap<Thread,DocumentsWriterThreadState>();
 
   private int pauseThreads;               // Non-zero when we need all threads to
                                           // pause (eg to flush)
@@ -140,7 +139,7 @@
   int maxFieldLength = IndexWriter.DEFAULT_MAX_FIELD_LENGTH;
   Similarity similarity;
 
-  List newFiles;
+  List<String> newFiles;
 
   static class DocState {
     DocumentsWriter docWriter;
@@ -385,11 +384,11 @@
     }
   }
 
-  private Collection abortedFiles;               // List of files that were written before last abort()
+  private Collection<String> abortedFiles;               // List of files that were written before last abort()
 
   private SegmentWriteState flushState;
 
-  Collection abortedFiles() {
+  Collection<String> abortedFiles() {
     return abortedFiles;
   }
 
@@ -398,17 +397,17 @@
       writer.message("DW: " + message);
   }
 
-  final List openFiles = new ArrayList();
-  final List closedFiles = new ArrayList();
+  final List<String> openFiles = new ArrayList<String>();
+  final List<String> closedFiles = new ArrayList<String>();
 
   /* Returns Collection of files in use by this instance,
    * including any flushed segments. */
-  synchronized List openFiles() {
-    return (List) ((ArrayList) openFiles).clone();
+  synchronized List<String> openFiles() {
+    return ( List<String>) ((ArrayList<String>) openFiles).clone();
   }
 
-  synchronized List closedFiles() {
-    return (List) ((ArrayList) closedFiles).clone();
+  synchronized List<String> closedFiles() {
+    return (List<String>) ((ArrayList<String>) closedFiles).clone();
   }
 
   synchronized void addOpenFile(String name) {
@@ -578,7 +577,7 @@
         flushState.numDocsInStore = 0;
       }
 
-      Collection threads = new HashSet();
+      Collection<DocConsumerPerThread> threads = new HashSet<DocConsumerPerThread>();
       for(int i=0;i<threadStates.length;i++)
         threads.add(threadStates[i].consumer);
       consumer.flush(threads, flushState);
@@ -614,9 +613,7 @@
   void createCompoundFile(String segment) throws IOException {
     
     CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segment + "." + IndexFileNames.COMPOUND_FILE_EXTENSION);
-    Iterator it = flushState.flushedFiles.iterator();
-    while(it.hasNext()) {
-      final String fileName = (String) it.next();
+    for(String fileName : flushState.flushedFiles) {
       if (Codec.DEBUG)
         System.out.println("make cfs " + fileName);
       cfsWriter.addFile(fileName);
@@ -835,7 +832,7 @@
   }
 
   // for testing
-  synchronized HashMap getBufferedDeleteTerms() {
+  synchronized HashMap<Term,BufferedDeletes.Num> getBufferedDeleteTerms() {
     return deletesInRAM.terms;
   }
 
@@ -981,7 +978,7 @@
 
         DocsEnum docs = reader.termDocsEnum(reader.getDeletedDocs(), term.field, new TermRef(term.text));
         if (docs != null) {
-          int limit = ((BufferedDeletes.Num) entry.getValue()).getNum();
+          int limit = entry.getValue().getNum();
           while (true) {
             final int docID = docs.next();
             if (docID == DocsEnum.NO_MORE_DOCS || docIDStart+docID >= limit) {
@@ -1182,7 +1179,7 @@
 
   private class ByteBlockAllocator extends ByteBlockPool.Allocator {
 
-    ArrayList freeByteBlocks = new ArrayList();
+    ArrayList<byte[]> freeByteBlocks = new ArrayList<byte[]>();
     
     /* Allocate another byte[] from the shared pool */
     byte[] getByteBlock(boolean trackAllocations) {
@@ -1222,7 +1219,7 @@
   final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT;
   final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
 
-  private ArrayList freeIntBlocks = new ArrayList();
+  private ArrayList<int[]> freeIntBlocks = new ArrayList<int[]>();
 
   /* Allocate another int[] from the shared pool */
   synchronized int[] getIntBlock(boolean trackAllocations) {
@@ -1271,7 +1268,7 @@
 
   final static int MAX_TERM_LENGTH = CHAR_BLOCK_SIZE-1;
 
-  private ArrayList freeCharBlocks = new ArrayList();
+  private ArrayList<char[]> freeCharBlocks = new ArrayList<char[]>();
 
   /* Allocate another char[] from the shared pool */
   synchronized char[] getCharBlock() {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java Tue Oct 20 19:58:18 2009
@@ -52,8 +52,8 @@
   static final byte STORE_PAYLOADS = 0x20;
   static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
   
-  private final ArrayList byNumber = new ArrayList();
-  private final HashMap byName = new HashMap();
+  private final ArrayList<FieldInfo> byNumber = new ArrayList<FieldInfo>();
+  private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
   private int format;
 
   FieldInfos() { }
@@ -112,10 +112,8 @@
 
   /** Adds field info for a Document. */
   synchronized public void add(Document doc) {
-    List fields = doc.getFields();
-    Iterator fieldIterator = fields.iterator();
-    while (fieldIterator.hasNext()) {
-      Fieldable field = (Fieldable) fieldIterator.next();
+    List<Fieldable> fields = doc.getFields();
+    for (Fieldable field : fields) {
       add(field.name(), field.isIndexed(), field.isTermVectorStored(), field.isStorePositionWithTermVector(),
               field.isStoreOffsetWithTermVector(), field.getOmitNorms(), false, field.getOmitTermFreqAndPositions());
     }
@@ -146,11 +144,10 @@
    * @param storePositionWithTermVector true if positions should be stored.
    * @param storeOffsetWithTermVector true if offsets should be stored
    */
-  synchronized public void addIndexed(Collection names, boolean storeTermVectors, boolean storePositionWithTermVector, 
+  synchronized public void addIndexed(Collection<String> names, boolean storeTermVectors, boolean storePositionWithTermVector, 
                          boolean storeOffsetWithTermVector) {
-    Iterator i = names.iterator();
-    while (i.hasNext()) {
-      add((String)i.next(), true, storeTermVectors, storePositionWithTermVector, storeOffsetWithTermVector);
+    for (String name : names) {
+      add(name, true, storeTermVectors, storePositionWithTermVector, storeOffsetWithTermVector);
     }
   }
 
@@ -162,10 +159,9 @@
    * 
    * @see #add(String, boolean)
    */
-  synchronized public void add(Collection names, boolean isIndexed) {
-    Iterator i = names.iterator();
-    while (i.hasNext()) {
-      add((String)i.next(), isIndexed);
+  synchronized public void add(Collection<String> names, boolean isIndexed) {
+    for (String name : names) {
+      add(name, isIndexed);
     }
   }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldSortedTermVectorMapper.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldSortedTermVectorMapper.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldSortedTermVectorMapper.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldSortedTermVectorMapper.java Tue Oct 20 19:58:18 2009
@@ -24,21 +24,21 @@
  * This is not thread-safe.
  */
 public class FieldSortedTermVectorMapper extends TermVectorMapper{
-  private Map fieldToTerms = new HashMap();
-  private SortedSet currentSet;
+  private Map<String,SortedSet<TermVectorEntry>> fieldToTerms = new HashMap<String,SortedSet<TermVectorEntry>>();
+  private SortedSet<TermVectorEntry> currentSet;
   private String currentField;
-  private Comparator comparator;
+  private Comparator<TermVectorEntry> comparator;
 
   /**
    *
    * @param comparator A Comparator for sorting {@link TermVectorEntry}s
    */
-  public FieldSortedTermVectorMapper(Comparator comparator) {
+  public FieldSortedTermVectorMapper(Comparator<TermVectorEntry> comparator) {
     this(false, false, comparator);
   }
 
 
-  public FieldSortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator comparator) {
+  public FieldSortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator<TermVectorEntry> comparator) {
     super(ignoringPositions, ignoringOffsets);
     this.comparator = comparator;
   }
@@ -49,7 +49,7 @@
   }
 
   public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
-    currentSet = new TreeSet(comparator);
+    currentSet = new TreeSet<TermVectorEntry>(comparator);
     currentField = field;
     fieldToTerms.put(field, currentSet);
   }
@@ -59,12 +59,12 @@
    *
    * @return A map between field names and {@link java.util.SortedSet}s per field.  SortedSet entries are {@link TermVectorEntry}
    */
-  public Map getFieldToTerms() {
+  public Map<String,SortedSet<TermVectorEntry>> getFieldToTerms() {
     return fieldToTerms;
   }
 
 
-  public Comparator getComparator() {
+  public Comparator<TermVectorEntry> getComparator() {
     return comparator;
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsReader.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsReader.java Tue Oct 20 19:58:18 2009
@@ -55,7 +55,7 @@
   // file.  This will be 0 if we have our own private file.
   private int docStoreOffset;
 
-  private CloseableThreadLocal fieldsStreamTL = new CloseableThreadLocal();
+  private CloseableThreadLocal<IndexInput> fieldsStreamTL = new CloseableThreadLocal<IndexInput>();
   private boolean isOriginal = false;
 
   /** Returns a cloned FieldsReader that shares open
@@ -417,7 +417,7 @@
     }
 
     private IndexInput getFieldStream() {
-      IndexInput localFieldsStream = (IndexInput) fieldsStreamTL.get();
+      IndexInput localFieldsStream = fieldsStreamTL.get();
       if (localFieldsStream == null) {
         localFieldsStream = (IndexInput) cloneableFieldsStream.clone();
         fieldsStreamTL.set(localFieldsStream);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsWriter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsWriter.java Tue Oct 20 19:58:18 2009
@@ -17,7 +17,7 @@
  */
 
 import java.io.IOException;
-import java.util.Iterator;
+import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
@@ -215,17 +215,16 @@
         indexStream.writeLong(fieldsStream.getFilePointer());
 
         int storedCount = 0;
-        Iterator fieldIterator = doc.getFields().iterator();
-        while (fieldIterator.hasNext()) {
-            Fieldable field = (Fieldable) fieldIterator.next();
+        List<Fieldable> fields = doc.getFields();
+        for (Fieldable field : fields) {
             if (field.isStored())
                 storedCount++;
         }
         fieldsStream.writeVInt(storedCount);
 
-        fieldIterator = doc.getFields().iterator();
-        while (fieldIterator.hasNext()) {
-            Fieldable field = (Fieldable) fieldIterator.next();
+        
+
+        for (Fieldable field : fields) {
             if (field.isStored())
               writeField(fieldInfos.fieldInfo(field.name()), field);
         }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FilterIndexReader.java Tue Oct 20 19:58:18 2009
@@ -221,12 +221,12 @@
 
   protected void doDelete(int n) throws  CorruptIndexException, IOException { in.deleteDocument(n); }
   
-  protected void doCommit(Map commitUserData) throws IOException { in.commit(commitUserData); }
+  protected void doCommit(Map<String,String> commitUserData) throws IOException { in.commit(commitUserData); }
   
   protected void doClose() throws IOException { in.close(); }
 
 
-  public Collection getFieldNames(IndexReader.FieldOption fieldNames) {
+  public Collection<String> getFieldNames(IndexReader.FieldOption fieldNames) {
     ensureOpen();
     return in.getFieldNames(fieldNames);
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Tue Oct 20 19:58:18 2009
@@ -69,7 +69,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
@@ -295,7 +295,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/branches/flex_1458/src/java/org/apache/lucene/index/IndexCommit.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexCommit.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexCommit.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexCommit.java Tue Oct 20 19:58:18 2009
@@ -52,7 +52,7 @@
   /**
    * Returns all index files referenced by this commit point.
    */
-  public abstract Collection getFileNames() throws IOException;
+  public abstract Collection<String> getFileNames() throws IOException;
 
   /**
    * Returns the {@link Directory} for the index.
@@ -125,7 +125,7 @@
   /** Returns userData, previously passed to {@link
    *  IndexWriter#commit(Map)} for this commit.  Map is
    *  String -> String. */
-  public Map getUserData() throws IOException {
+  public Map<String,String> getUserData() throws IOException {
     throw new UnsupportedOperationException("This IndexCommit does not support this method.");
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java Tue Oct 20 19:58:18 2009
@@ -70,7 +70,7 @@
    * {@link IndexCommit point-in-time commits},
    *  sorted by age (the 0th one is the oldest commit).
    */
-  public void onInit(List commits) throws IOException;
+  public void onInit(List<? extends IndexCommit> commits) throws IOException;
 
   /**
    * <p>This is called each time the writer completed a commit.
@@ -94,5 +94,5 @@
    * @param commits List of {@link IndexCommit},
    *  sorted by age (the 0th one is the oldest commit).
    */
-  public void onCommit(List commits) throws IOException;
+  public void onCommit(List<? extends IndexCommit> commits) throws IOException;
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java Tue Oct 20 19:58:18 2009
@@ -26,7 +26,7 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
+
 import java.util.List;
 import java.util.Map;
 
@@ -75,26 +75,26 @@
   /* 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
    * default delete policy (KeepOnlyLastCommitDeletionPolicy).
    * Other policies may leave commit points live for longer
    * in which case this list would be longer than 1: */
-  private List commits = new ArrayList();
+  private List<CommitPoint> commits = new ArrayList<CommitPoint>();
 
   /* 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();
+  private List<CommitPoint> commitsToDelete = new ArrayList<CommitPoint>();
 
   private PrintStream infoStream;
   private Directory directory;
@@ -243,10 +243,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 + "\"");
@@ -285,9 +284,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();
@@ -358,7 +356,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();
     }
 
@@ -367,13 +365,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));
       }
     }
   }
@@ -422,7 +420,7 @@
       deleteCommits();
     } else {
 
-      final List docWriterFiles;
+      final List<String> docWriterFiles;
       if (docWriter != null) {
         docWriterFiles = docWriter.openFiles();
         if (docWriterFiles != null)
@@ -437,7 +435,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();
       }
 
@@ -452,16 +450,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);
     }
   }
 
@@ -473,10 +469,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);
     }
   }
 
@@ -494,9 +489,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);
     }
   }
 
@@ -506,23 +500,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);
     }
@@ -549,7 +540,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
       }
@@ -595,17 +586,17 @@
   final private static class CommitPoint extends IndexCommit implements Comparable {
 
     long gen;
-    Collection files;
+    Collection<String> files;
     String segmentsFileName;
     boolean deleted;
     Directory directory;
-    Collection commitsToDelete;
+    Collection<CommitPoint> 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<CommitPoint> commitsToDelete, Directory directory, SegmentInfos segmentInfos) throws IOException {
       this.directory = directory;
       this.commitsToDelete = commitsToDelete;
       userData = segmentInfos.getUserData();
@@ -627,7 +618,7 @@
       return segmentsFileName;
     }
 
-    public Collection getFileNames() throws IOException {
+    public Collection<String> getFileNames() throws IOException {
       return files;
     }
 
@@ -643,7 +634,7 @@
       return generation;
     }
 
-    public Map getUserData() {
+    public Map<String,String> getUserData() {
       return userData;
     }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileNameFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileNameFilter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileNameFilter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileNameFilter.java Tue Oct 20 19:58:18 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/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java Tue Oct 20 19:58:18 2009
@@ -28,6 +28,7 @@
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.Closeable;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Map;
@@ -78,7 +79,7 @@
  <code>IndexReader</code> instance; use your own
  (non-Lucene) objects instead.
 */
-public abstract class IndexReader implements Cloneable {
+public abstract class IndexReader implements Cloneable,Closeable {
 
   /**
    * Constants describing field properties, for example used for
@@ -537,7 +538,7 @@
    *
    * @see #getCommitUserData()
    */
-  public static Map getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
+  public static Map<String,String> getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
     return SegmentInfos.readCurrentUserData(directory, Codecs.getDefault());
   }
 
@@ -557,7 +558,7 @@
    *
    * @see #getCommitUserData(Directory)
    */
-  public Map getCommitUserData() {
+  public Map<String,String> getCommitUserData() {
     throw new UnsupportedOperationException("This reader does not support this method.");
   }
 
@@ -1059,7 +1060,7 @@
    *  IndexReader#getCommitUserData}.
    * @throws IOException
    */
-  public final synchronized void flush(Map commitUserData) throws IOException {
+  public final synchronized void flush(Map<String, String> commitUserData) throws IOException {
     ensureOpen();
     commit(commitUserData);
   }
@@ -1086,7 +1087,7 @@
    * (transactional semantics).
    * @throws IOException if there is a low-level IO error
    */
-  protected final synchronized void commit(Map commitUserData) throws IOException {
+  protected final synchronized void commit(Map<String, String> commitUserData) throws IOException {
     if (hasChanges) {
       doCommit(commitUserData);
     }
@@ -1095,7 +1096,7 @@
 
   /** Implements commit.  NOTE: subclasses should override
    *  this.  In 3.0 this will become an abstract method. */
-  protected abstract void doCommit(Map commitUserData) throws IOException;
+  protected abstract void doCommit(Map<String, String> commitUserData) throws IOException;
 
   /**
    * Closes files associated with this index.
@@ -1121,7 +1122,7 @@
    * @return Collection of Strings indicating the names of the fields.
    * @see IndexReader.FieldOption
    */
-  public abstract Collection getFieldNames(FieldOption fldOption);
+  public abstract Collection<String> getFieldNames(FieldOption fldOption);
 
   private final class DeletedDocsBits implements Bits {
     public boolean get(int docID) {
@@ -1251,7 +1252,7 @@
    *  java.io.IOException}.  Note that if a commit is in
    *  progress while this method is running, that commit
    *  may or may not be returned array.  */
-  public static Collection listCommits(Directory dir) throws IOException {
+  public static Collection<IndexCommit> listCommits(Directory dir) throws IOException {
     return DirectoryReader.listCommits(dir);
   }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java Tue Oct 20 19:58:18 2009
@@ -31,6 +31,7 @@
 import org.apache.lucene.index.codecs.Codecs;
 
 import java.io.IOException;
+import java.io.Closeable;
 import java.io.PrintStream;
 import java.util.List;
 import java.util.Collection;
@@ -166,7 +167,7 @@
  * referenced by the "front" of the index). For this, IndexFileDeleter 
  * keeps track of the last non commit checkpoint.
  */
-public class IndexWriter {
+public class IndexWriter implements Closeable {
 
   /**
    * Default value for the write lock timeout (1,000).
@@ -245,7 +246,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;
@@ -258,7 +259,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;
 
@@ -269,13 +270,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;
 
@@ -403,23 +404,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;
           }
         }     
       }
@@ -436,7 +433,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;
     }
@@ -498,11 +495,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();
@@ -532,11 +530,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();
@@ -1136,7 +1132,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));
@@ -1353,6 +1349,14 @@
    * instead of RAM usage (each buffered delete Query counts
    * as one).
    *
+   * <p> <b>NOTE</b>: because IndexWriter uses
+   * <code>int</code>s when managing its internal storage,
+   * the absolute maximum value for this setting is somewhat
+   * less than 2048 MB.  The precise limit depends on
+   * various factors, such as how large your documents are,
+   * how many fields have norms, etc., so it's best to set
+   * this value comfortably under 2048.</p>
+   *
    * <p> The default value is {@link #DEFAULT_RAM_BUFFER_SIZE_MB}.</p>
    * 
    * @throws IllegalArgumentException if ramBufferSize is
@@ -1360,6 +1364,9 @@
    * when maxBufferedDocs is already disabled
    */
   public void setRAMBufferSizeMB(double mb) {
+    if (mb > 2048.0) {
+      throw new IllegalArgumentException("ramBufferSize " + mb + " is too large; should be comfortably less than 2048");
+    }
     if (mb != DISABLE_AUTO_FLUSH && mb <= 0.0)
       throw new IllegalArgumentException(
           "ramBufferSize should be > 0.0 MB when enabled");
@@ -1742,9 +1749,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();
@@ -1934,7 +1941,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);
             }
@@ -2090,7 +2097,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);
           }
@@ -2265,23 +2272,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;
       }
@@ -2302,7 +2305,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();
@@ -2335,16 +2338,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;
   }
 
@@ -2524,7 +2527,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) {
@@ -2821,9 +2824,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();
@@ -2831,9 +2832,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();
@@ -2929,12 +2928,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");
@@ -3262,7 +3261,7 @@
     
       if (mergePolicy instanceof LogMergePolicy && getUseCompoundFile()) {
 
-        List files = null;
+        List<String> files = null;
 
         synchronized(this) {
           // Must incRef our files so that if another thread
@@ -3362,7 +3361,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");
@@ -3427,7 +3426,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();
 
@@ -3840,7 +3839,7 @@
 
     // Must note the change to segmentInfos so any commits
     // in-flight don't lose it:
-    changeCount++;
+    checkpoint();
 
     // If the merged segments had pending changes, clear
     // them so that they don't bother writing them to
@@ -4132,7 +4131,7 @@
                                  null);
 
 
-    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+"");
@@ -4149,8 +4148,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+"");
@@ -4216,7 +4215,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:
@@ -4489,12 +4488,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)) {
@@ -4520,11 +4519,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
@@ -4567,7 +4566,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");
 
@@ -4623,9 +4622,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";
           }
 
@@ -4643,11 +4641,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/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocConsumer.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocConsumer.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocConsumer.java Tue Oct 20 19:58:18 2009
@@ -17,6 +17,7 @@
  * limitations under the License.
  */
 
+import java.util.Collection;
 import java.util.Map;
 import java.io.IOException;
 
@@ -29,7 +30,7 @@
   abstract void abort();
 
   /** Flush a new segment */
-  abstract void flush(Map threadsAndFields, SegmentWriteState state) throws IOException;
+  abstract void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
 
   /** Close doc stores */
   abstract void closeDocStore(SegmentWriteState state) throws IOException;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java Tue Oct 20 19:58:18 2009
@@ -17,12 +17,13 @@
  * limitations under the License.
  */
 
+import java.util.Collection;
 import java.util.Map;
 import java.io.IOException;
 
 abstract class InvertedDocEndConsumer {
   abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
-  abstract void flush(Map threadsAndFields, SegmentWriteState state) throws IOException;
+  abstract void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
   abstract void closeDocStore(SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract void setFieldInfos(FieldInfos fieldInfos);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java Tue Oct 20 19:58:18 2009
@@ -31,7 +31,7 @@
   /**
    * Deletes all commits except the most recent one.
    */
-  public void onInit(List commits) {
+  public void onInit(List<? extends IndexCommit> commits) {
     // Note that commits.size() should normally be 1:
     onCommit(commits);
   }
@@ -39,7 +39,7 @@
   /**
    * Deletes all commits except the most recent one.
    */
-  public void onCommit(List commits) {
+  public void onCommit(List<? extends IndexCommit> commits) {
     // Note that commits.size() should normally be 2 (if not
     // called by onInit above):
     int size = commits.size();

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/LogMergePolicy.java Tue Oct 20 19:58:18 2009
@@ -175,7 +175,7 @@
     }
   }
   
-  private boolean isOptimized(SegmentInfos infos, int maxNumSegments, Set segmentsToOptimize) throws IOException {
+  private boolean isOptimized(SegmentInfos infos, int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
     final int numSegments = infos.size();
     int numToOptimize = 0;
     SegmentInfo optimizeInfo = null;
@@ -212,7 +212,7 @@
    *  (mergeFactor at a time) so the {@link MergeScheduler}
    *  in use may make use of concurrency. */
   public MergeSpecification findMergesForOptimize(SegmentInfos infos,
-      int maxNumSegments, Set segmentsToOptimize) throws IOException {
+      int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
     MergeSpecification spec;
 
     assert maxNumSegments > 0;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MergePolicy.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MergePolicy.java Tue Oct 20 19:58:18 2009
@@ -61,7 +61,7 @@
  * these APIs.
  */
 
-public abstract class MergePolicy {
+public abstract class MergePolicy implements java.io.Closeable {
 
   /** OneMerge provides the information necessary to perform
    *  an individual primitive merge operation, resulting in
@@ -152,7 +152,7 @@
      * The subset of segments to be included in the primitive merge.
      */
 
-    public List merges = new ArrayList();
+    public List<OneMerge> merges = new ArrayList<OneMerge>();
 
     public void add(OneMerge merge) {
       merges.add(merge);
@@ -233,7 +233,7 @@
    *          away. This may be a subset of all SegmentInfos.
    */
   public abstract MergeSpecification findMergesForOptimize(
-      SegmentInfos segmentInfos, int maxSegmentCount, Set segmentsToOptimize)
+      SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
       throws CorruptIndexException, IOException;
 
   /**

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java Tue Oct 20 19:58:18 2009
@@ -39,7 +39,7 @@
   protected IndexReader[] subReaders;
   private int[] starts;                           // 1st docno for each segment
   private boolean[] decrefOnClose;                // remember which subreaders to decRef on close
-  private Map normsCache = new HashMap();
+  private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
   private int maxDoc = 0;
   private int numDocs = -1;
   private boolean hasDeletions = false;
@@ -52,9 +52,10 @@
   * left to the subreaders. </p>
   * <p>Note that all subreaders are closed if this Multireader is closed.</p>
   * @param subReaders set of (sub)readers
+ * @throws IOException 
   * @throws IOException
   */
-  public MultiReader(IndexReader[] subReaders) throws IOException {
+  public MultiReader(IndexReader... subReaders) throws IOException {
     initialize(subReaders, true);
   }
 
@@ -306,7 +307,7 @@
   
   public synchronized byte[] norms(String field) throws IOException {
     ensureOpen();
-    byte[] bytes = (byte[])normsCache.get(field);
+    byte[] bytes = normsCache.get(field);
     if (bytes != null)
       return bytes;          // cache hit
     if (!hasNorms(field))
@@ -322,7 +323,7 @@
   public synchronized void norms(String field, byte[] result, int offset)
     throws IOException {
     ensureOpen();
-    byte[] bytes = (byte[])normsCache.get(field);
+    byte[] bytes = normsCache.get(field);
     for (int i = 0; i < subReaders.length; i++)      // read from segments
       subReaders[i].norms(field, result, offset + starts[i]);
 
@@ -388,7 +389,7 @@
     doCommit(null);
   }
   
-  protected void doCommit(Map commitUserData) throws IOException {
+  protected void doCommit(Map<String,String> commitUserData) throws IOException {
     for (int i = 0; i < subReaders.length; i++)
       subReaders[i].commit(commitUserData);
   }
@@ -403,7 +404,7 @@
     }
   }
   
-  public Collection getFieldNames (IndexReader.FieldOption fieldNames) {
+  public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
     ensureOpen();
     return DirectoryReader.getFieldNames(fieldNames, this.subReaders);
   }  

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultipleTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultipleTermPositions.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultipleTermPositions.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultipleTermPositions.java Tue Oct 20 19:58:18 2009
@@ -21,7 +21,7 @@
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Iterator;
+
 import java.util.LinkedList;
 import java.util.List;
 
@@ -34,19 +34,17 @@
 public class MultipleTermPositions implements TermPositions {
 
   private static final class TermPositionsQueue extends PriorityQueue<TermPositions> {
-    TermPositionsQueue(List termPositions) throws IOException {
+    TermPositionsQueue(List<TermPositions> termPositions) throws IOException {
       initialize(termPositions.size());
 
-      Iterator i = termPositions.iterator();
-      while (i.hasNext()) {
-        TermPositions tp = (TermPositions) i.next();
+      for (TermPositions tp : termPositions) {
         if (tp.next())
           add(tp);
       }
     }
 
     final TermPositions peek() {
-      return (TermPositions) top();
+      return top();
     }
 
     public final boolean lessThan(TermPositions a, TermPositions b) {
@@ -103,7 +101,7 @@
    * @exception IOException
    */
   public MultipleTermPositions(IndexReader indexReader, Term[] terms) throws IOException {
-    List termPositions = new LinkedList();
+    List<TermPositions> termPositions = new LinkedList<TermPositions>();
 
     for (int i = 0; i < terms.length; i++)
       termPositions.add(indexReader.termPositions(terms[i]));

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java Tue Oct 20 19:58:18 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;
@@ -54,7 +54,7 @@
 
   /** Produce _X.nrm if any document had a field with norms
    *  not disabled */
-  public void flush(Map threadsAndFields, SegmentWriteState state) throws IOException {
+  public void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
 
     final Map byField = new HashMap();
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java Tue Oct 20 19:58:18 2009
@@ -45,12 +45,12 @@
  * undefined behavior</em>.
  */
 public class ParallelReader extends IndexReader {
-  private List readers = new ArrayList();
-  private List decrefOnClose = new ArrayList(); // remember which subreaders to decRef on close
+  private List<IndexReader> readers = new ArrayList<IndexReader>();
+  private List<Boolean> decrefOnClose = new ArrayList<Boolean>(); // remember which subreaders to decRef on close
   boolean incRefReaders = false;
-  private SortedMap< String, IndexReader> fieldToReader = new TreeMap<String, IndexReader>();
-  private Map readerToFields = new HashMap();
-  private List storedFieldReaders = new ArrayList();
+  private SortedMap<String,IndexReader> fieldToReader = new TreeMap<String,IndexReader>();
+  private Map<IndexReader,Collection<String>> readerToFields = new HashMap<IndexReader,Collection<String>>();
+  private List<IndexReader> storedFieldReaders = new ArrayList<IndexReader>();
 
   private int maxDoc;
   private int numDocs;
@@ -107,11 +107,9 @@
       throw new IllegalArgumentException
         ("All readers must have same numDocs: "+numDocs+"!="+reader.numDocs());
 
-    Collection fields = reader.getFieldNames(IndexReader.FieldOption.ALL);
+    Collection<String> fields = reader.getFieldNames(IndexReader.FieldOption.ALL);
     readerToFields.put(reader, fields);
-    Iterator i = fields.iterator();
-    while (i.hasNext()) {                         // update fieldToReader map
-      String field = (String)i.next();
+    for (final String field : fields) {               // update fieldToReader map
       if (fieldToReader.get(field) == null) {
         fieldToReader.put(field, reader);
       }
@@ -214,13 +212,12 @@
     ensureOpen();
     
     boolean reopened = false;
-    List newReaders = new ArrayList();
+    List<IndexReader> newReaders = new ArrayList<IndexReader>();
     
     boolean success = false;
     
     try {
-      for (int i = 0; i < readers.size(); i++) {
-        IndexReader oldReader = (IndexReader) readers.get(i);
+      for (final IndexReader oldReader : readers) {
         IndexReader newReader = null;
         if (doClone) {
           newReader = (IndexReader) oldReader.clone();
@@ -238,7 +235,7 @@
     } finally {
       if (!success && reopened) {
         for (int i = 0; i < newReaders.size(); i++) {
-          IndexReader r = (IndexReader) newReaders.get(i);
+          IndexReader r = newReaders.get(i);
           if (r != readers.get(i)) {
             try {
               r.close();
@@ -251,7 +248,7 @@
     }
 
     if (reopened) {
-      List newDecrefOnClose = new ArrayList();
+      List<Boolean> newDecrefOnClose = new ArrayList<Boolean>();
       ParallelReader pr = new ParallelReader();
       for (int i = 0; i < readers.size(); i++) {
         IndexReader oldReader = (IndexReader) readers.get(i);
@@ -295,22 +292,22 @@
   public boolean isDeleted(int n) {
     // Don't call ensureOpen() here (it could affect performance)
     if (readers.size() > 0)
-      return ((IndexReader)readers.get(0)).isDeleted(n);
+      return readers.get(0).isDeleted(n);
     return false;
   }
 
   // delete in all readers
   protected void doDelete(int n) throws CorruptIndexException, IOException {
-    for (int i = 0; i < readers.size(); i++) {
-      ((IndexReader)readers.get(i)).deleteDocument(n);
+    for (final IndexReader reader : readers) {
+      reader.deleteDocument(n);
     }
     hasDeletions = true;
   }
 
   // undeleteAll in all readers
   protected void doUndeleteAll() throws CorruptIndexException, IOException {
-    for (int i = 0; i < readers.size(); i++) {
-      ((IndexReader)readers.get(i)).undeleteAll();
+    for (final IndexReader reader : readers) {
+      reader.undeleteAll();
     }
     hasDeletions = false;
   }
@@ -319,22 +316,21 @@
   public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
     ensureOpen();
     Document result = new Document();
-    for (int i = 0; i < storedFieldReaders.size(); i++) {
-      IndexReader reader = (IndexReader)storedFieldReaders.get(i);
+    for (final IndexReader reader: storedFieldReaders) {
 
       boolean include = (fieldSelector==null);
       if (!include) {
-        Iterator it = ((Collection) readerToFields.get(reader)).iterator();
-        while (it.hasNext())
-          if (fieldSelector.accept((String)it.next())!=FieldSelectorResult.NO_LOAD) {
+        Collection<String> fields = readerToFields.get(reader);
+        for (final String field : fields)
+          if (fieldSelector.accept(field) != FieldSelectorResult.NO_LOAD) {
             include = true;
             break;
           }
       }
       if (include) {
-        Iterator fieldIterator = reader.document(n, fieldSelector).getFields().iterator();
-        while (fieldIterator.hasNext()) {
-          result.add((Fieldable)fieldIterator.next());
+        List<Fieldable> fields = reader.document(n, fieldSelector).getFields();
+        for (Fieldable field : fields) {
+          result.add(field);
         }
       }
     }
@@ -344,12 +340,11 @@
   // get all vectors
   public TermFreqVector[] getTermFreqVectors(int n) throws IOException {
     ensureOpen();
-    ArrayList results = new ArrayList();
-    Iterator i = fieldToReader.entrySet().iterator();
-    while (i.hasNext()) {
-      Map.Entry e = (Map.Entry)i.next();
-      String field = (String)e.getKey();
-      IndexReader reader = (IndexReader)e.getValue();
+    ArrayList<TermFreqVector> results = new ArrayList<TermFreqVector>();
+    for (final Map.Entry<String,IndexReader> e: fieldToReader.entrySet()) {
+
+      String field = e.getKey();
+      IndexReader reader = e.getValue();
       TermFreqVector vector = reader.getTermFreqVector(n, field);
       if (vector != null)
         results.add(vector);
@@ -361,14 +356,14 @@
   public TermFreqVector getTermFreqVector(int n, String field)
     throws IOException {
     ensureOpen();
-    IndexReader reader = ((IndexReader)fieldToReader.get(field));
+    IndexReader reader = fieldToReader.get(field);
     return reader==null ? null : reader.getTermFreqVector(n, field);
   }
 
 
   public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
     ensureOpen();
-    IndexReader reader = ((IndexReader)fieldToReader.get(field));
+    IndexReader reader = fieldToReader.get(field);
     if (reader != null) {
       reader.getTermFreqVector(docNumber, field, mapper); 
     }
@@ -377,11 +372,10 @@
   public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
     ensureOpen();
 
-    Iterator i = fieldToReader.entrySet().iterator();
-    while (i.hasNext()) {
-      Map.Entry e = (Map.Entry)i.next();
-      String field = (String)e.getKey();
-      IndexReader reader = (IndexReader)e.getValue();
+    for (final Map.Entry<String,IndexReader> e : fieldToReader.entrySet()) {
+
+      String field = e.getKey();
+      IndexReader reader = e.getValue();
       reader.getTermFreqVector(docNumber, field, mapper);
     }
 
@@ -389,27 +383,27 @@
 
   public boolean hasNorms(String field) throws IOException {
     ensureOpen();
-    IndexReader reader = ((IndexReader)fieldToReader.get(field));
+    IndexReader reader = fieldToReader.get(field);
     return reader==null ? false : reader.hasNorms(field);
   }
 
   public byte[] norms(String field) throws IOException {
     ensureOpen();
-    IndexReader reader = ((IndexReader)fieldToReader.get(field));
+    IndexReader reader = fieldToReader.get(field);
     return reader==null ? null : reader.norms(field);
   }
 
   public void norms(String field, byte[] result, int offset)
     throws IOException {
     ensureOpen();
-    IndexReader reader = ((IndexReader)fieldToReader.get(field));
+    IndexReader reader = fieldToReader.get(field);
     if (reader!=null)
       reader.norms(field, result, offset);
   }
 
   protected void doSetNorm(int n, String field, byte value)
     throws CorruptIndexException, IOException {
-    IndexReader reader = ((IndexReader)fieldToReader.get(field));
+    IndexReader reader = fieldToReader.get(field);
     if (reader!=null)
       reader.doSetNorm(n, field, value);
   }
@@ -426,7 +420,7 @@
 
   public int docFreq(Term term) throws IOException {
     ensureOpen();
-    IndexReader reader = ((IndexReader)fieldToReader.get(term.field()));
+    IndexReader reader = fieldToReader.get(term.field());
     return reader==null ? 0 : reader.docFreq(term);
   }
 
@@ -460,8 +454,8 @@
    * Checks recursively if all subreaders are up to date. 
    */
   public boolean isCurrent() throws CorruptIndexException, IOException {
-    for (int i = 0; i < readers.size(); i++) {
-      if (!((IndexReader)readers.get(i)).isCurrent()) {
+    for (final IndexReader reader : readers) {
+      if (!reader.isCurrent()) {
         return false;
       }
     }
@@ -474,8 +468,8 @@
    * Checks recursively if all subindexes are optimized 
    */
   public boolean isOptimized() {
-    for (int i = 0; i < readers.size(); i++) {
-      if (!((IndexReader)readers.get(i)).isOptimized()) {
+    for (final IndexReader reader : readers) {
+      if (!reader.isOptimized()) {
         return false;
       }
     }
@@ -494,30 +488,29 @@
 
   // for testing
   IndexReader[] getSubReaders() {
-    return (IndexReader[]) readers.toArray(new IndexReader[readers.size()]);
+    return readers.toArray(new IndexReader[readers.size()]);
   }
 
-  protected void doCommit(Map commitUserData) throws IOException {
-    for (int i = 0; i < readers.size(); i++)
-      ((IndexReader)readers.get(i)).commit(commitUserData);
+  protected void doCommit(Map<String,String> commitUserData) throws IOException {
+    for (final IndexReader reader : readers)
+      reader.commit(commitUserData);
   }
 
   protected synchronized void doClose() throws IOException {
     for (int i = 0; i < readers.size(); i++) {
-      if (((Boolean) decrefOnClose.get(i)).booleanValue()) {
-        ((IndexReader)readers.get(i)).decRef();
+      if (decrefOnClose.get(i).booleanValue()) {
+        readers.get(i).decRef();
       } else {
-        ((IndexReader)readers.get(i)).close();
+        readers.get(i).close();
       }
     }
   }
 
-  public Collection getFieldNames (IndexReader.FieldOption fieldNames) {
+  public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
     ensureOpen();
-    Set fieldSet = new HashSet();
-    for (int i = 0; i < readers.size(); i++) {
-      IndexReader reader = ((IndexReader)readers.get(i));
-      Collection names = reader.getFieldNames(fieldNames);
+    Set<String> fieldSet = new HashSet<String>();
+    for (final IndexReader reader : readers) {
+      Collection<String> names = reader.getFieldNames(fieldNames);
       fieldSet.addAll(names);
     }
     return fieldSet;
@@ -530,7 +523,7 @@
 
     public ParallelTermEnum() throws IOException {
       try {
-        field = (String)fieldToReader.firstKey();
+        field = fieldToReader.firstKey();
       } catch(NoSuchElementException e) {
         // No fields, so keep field == null, termEnum == null
         return;
@@ -562,7 +555,7 @@
         fieldIterator.next();                     // Skip field to get next one
       }
       while (fieldIterator.hasNext()) {
-        field = (String) fieldIterator.next();
+        field = fieldIterator.next();
         termEnum = fieldToReader.get(field).terms(new Term(field));
         Term term = termEnum.term();
         if (term!=null && term.field()==field)
@@ -602,7 +595,7 @@
     public ParallelTermDocs() {}
     public ParallelTermDocs(Term term) throws IOException {
       if (term == null)
-        termDocs = readers.isEmpty() ? null : ((IndexReader)readers.get(0)).termDocs(null);
+        termDocs = readers.isEmpty() ? null : readers.get(0).termDocs(null);
       else
         seek(term);
     }
@@ -611,7 +604,7 @@
     public int freq() { return termDocs.freq(); }
 
     public void seek(Term term) throws IOException {
-      IndexReader reader = ((IndexReader)fieldToReader.get(term.field()));
+      IndexReader reader = fieldToReader.get(term.field());
       termDocs = reader!=null ? reader.termDocs(term) : null;
     }
 
@@ -654,7 +647,7 @@
     public ParallelTermPositions(Term term) throws IOException { seek(term); }
 
     public void seek(Term term) throws IOException {
-      IndexReader reader = ((IndexReader)fieldToReader.get(term.field()));
+      IndexReader reader = fieldToReader.get(term.field());
       termDocs = reader!=null ? reader.termPositions(term) : null;
     }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionBasedTermVectorMapper.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionBasedTermVectorMapper.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionBasedTermVectorMapper.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/PositionBasedTermVectorMapper.java Tue Oct 20 19:58:18 2009
@@ -27,13 +27,13 @@
  * This is not thread-safe.
  */
 public class PositionBasedTermVectorMapper extends TermVectorMapper{
-  private Map/*<String, Map<Integer, TVPositionInfo>>*/ fieldToTerms;
+  private Map<String, Map<Integer,TVPositionInfo>> fieldToTerms;
 
   private String currentField;
   /**
    * A Map of Integer and TVPositionInfo
    */
-  private Map/*<Integer, TVPositionInfo>*/ currentPositions;
+  private Map<Integer,TVPositionInfo> currentPositions;
   private boolean storeOffsets;
 
   
@@ -95,10 +95,10 @@
     {
       //ignoring offsets
     }
-    fieldToTerms = new HashMap(numTerms);
+    fieldToTerms = new HashMap<String,Map<Integer,TVPositionInfo>>(numTerms);
     this.storeOffsets = storeOffsets;
     currentField = field;
-    currentPositions = new HashMap();
+    currentPositions = new HashMap<Integer,TVPositionInfo>();
     fieldToTerms.put(currentField, currentPositions);
   }
 
@@ -107,7 +107,7 @@
    *
    * @return A map between field names and a Map.  The sub-Map key is the position as the integer, the value is {@link org.apache.lucene.index.PositionBasedTermVectorMapper.TVPositionInfo}.
    */
-  public Map getFieldToTerms() {
+  public Map<String, Map<Integer, TVPositionInfo>>  getFieldToTerms() {
     return fieldToTerms;
   }
 
@@ -116,17 +116,17 @@
    */
   public static class TVPositionInfo{
     private int position;
-    //a list of Strings
-    private List terms;
-    //A list of TermVectorOffsetInfo
-    private List offsets;
+
+    private List<String> terms;
+
+    private List<TermVectorOffsetInfo> offsets;
 
 
     public TVPositionInfo(int position, boolean storeOffsets) {
       this.position = position;
-      terms = new ArrayList();
+      terms = new ArrayList<String>();
       if (storeOffsets) {
-        offsets = new ArrayList();
+        offsets = new ArrayList<TermVectorOffsetInfo>();
       }
     }
 
@@ -150,15 +150,15 @@
      * Note, there may be multiple terms at the same position
      * @return A List of Strings
      */
-    public List getTerms() {
+    public List<String> getTerms() {
       return terms;
     }
 
     /**
      * Parallel list (to {@link #getTerms()}) of TermVectorOffsetInfo objects.  There may be multiple entries since there may be multiple terms at a position
-     * @return A List of TermVectorOffsetInfo objects, if offsets are store.
+     * @return A List of TermVectorOffsetInfo objects, if offsets are stored.
      */
-    public List getOffsets() {
+    public List<TermVectorOffsetInfo> getOffsets() {
       return offsets;
     }
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java Tue Oct 20 19:58:18 2009
@@ -28,7 +28,7 @@
     super(directory, sis, deletionPolicy, true, termInfosIndexDivisor, codecs);
   }
 
-  ReadOnlyDirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[] oldStarts, Map oldNormsCache, boolean doClone,
+  ReadOnlyDirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[] oldStarts,  Map<String,byte[]> oldNormsCache, boolean doClone,
                           int termInfosIndexDivisor, Codecs codecs) throws IOException {
     super(directory, infos, oldReaders, oldStarts, oldNormsCache, true, doClone, termInfosIndexDivisor, codecs);
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentFieldMergeQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentFieldMergeQueue.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentFieldMergeQueue.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentFieldMergeQueue.java Tue Oct 20 19:58:18 2009
@@ -20,14 +20,14 @@
 import org.apache.lucene.util.PriorityQueue;
 
 // Used to merge-sort by SegmentMergeInfo.field
-final class SegmentFieldMergeQueue extends PriorityQueue {
+final class SegmentFieldMergeQueue extends PriorityQueue<SegmentMergeInfo> {
   SegmentFieldMergeQueue(int size) {
     initialize(size);
   }
 
-  protected final boolean lessThan(Object a, Object b) {
-    SegmentMergeInfo stiA = (SegmentMergeInfo)a;
-    SegmentMergeInfo stiB = (SegmentMergeInfo)b;
+  protected final boolean lessThan(SegmentMergeInfo a, SegmentMergeInfo b) {
+    SegmentMergeInfo stiA = a;
+    SegmentMergeInfo stiB = b;
     // nocommit ok not to break ties?
     return stiA.field.compareTo(stiB.field) < 0;
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java Tue Oct 20 19:58:18 2009
@@ -96,7 +96,7 @@
   private Codec codec;
 
 
-  private Map diagnostics;
+  private Map<String,String> diagnostics;
 
   public String toString() {
     return "si: "+dir.toString()+" "+name+" docCount: "+docCount+" delCount: "+delCount+" delFileName: "+getDelFileName();
@@ -167,12 +167,12 @@
   }
 
   // must be Map<String, String>
-  void setDiagnostics(Map diagnostics) {
+  void setDiagnostics(Map<String, String> diagnostics) {
     this.diagnostics = diagnostics;
   }
 
   // returns Map<String, String>
-  public Map getDiagnostics() {
+  public Map<String, String> getDiagnostics() {
     return diagnostics;
   }
 
@@ -241,7 +241,7 @@
       if (format <= SegmentInfos.FORMAT_DIAGNOSTICS) {
         diagnostics = input.readStringStringMap();
       } else {
-        diagnostics = Collections.EMPTY_MAP;
+        diagnostics = Collections.<String,String>emptyMap();
       }
     } else {
       delGen = CHECK_DIR;
@@ -285,11 +285,11 @@
    *  this segment. */
   public long sizeInBytes() throws IOException {
     if (sizeInBytes == -1) {
-      List files = files();
+      List<String> files = files();
       final int size = files.size();
       sizeInBytes = 0;
       for(int i=0;i<size;i++) {
-        final String fileName = (String) files.get(i);
+        final String fileName = files.get(i);
         // We don't count bytes used by a shared doc store
         // against this segment:
         if (docStoreOffset == -1 || !IndexFileNames.isDocStoreFile(fileName))
@@ -347,7 +347,7 @@
     si.hasProx = hasProx;
     si.preLockless = preLockless;
     si.hasSingleNormFile = hasSingleNormFile;
-    si.diagnostics = new HashMap(diagnostics);
+    si.diagnostics = new HashMap<String, String>(diagnostics);
     if (normGen != null) {
       si.normGen = (long[]) normGen.clone();
     }
@@ -614,7 +614,7 @@
     return codec;
   }
 
-  private void addIfExists(List files, String fileName) throws IOException {
+  private void addIfExists(List<String> files, String fileName) throws IOException {
     if (dir.fileExists(fileName))
       files.add(fileName);
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java?rev=827772&r1=827771&r2=827772&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java Tue Oct 20 19:58:18 2009
@@ -117,7 +117,7 @@
   private static PrintStream infoStream;
 
   public final SegmentInfo info(int i) {
-    return (SegmentInfo) get(i);
+    return get(i);
   }
 
   /**



Mime
View raw message