lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From uschind...@apache.org
Subject svn commit: r826290 - /lucene/java/trunk/src/java/org/apache/lucene/index/
Date Sat, 17 Oct 2009 19:15:08 GMT
Author: uschindler
Date: Sat Oct 17 19:15:07 2009
New Revision: 826290

URL: http://svn.apache.org/viewvc?rev=826290&view=rev
Log:
LUCENE-1257: More generified APIs and implementations. Thanks Kay Kay!

Modified:
    lucene/java/trunk/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FilterIndexReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentWriteState.java

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/CheckIndex.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/CheckIndex.java Sat Oct 17 19:15:07
2009
@@ -28,7 +28,7 @@
 import java.io.IOException;
 import java.io.File;
 import java.util.Collection;
-import java.util.Iterator;
+
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Map;
@@ -113,7 +113,7 @@
     public boolean partial;
 
     /** Holds the userData of the last commit in the index */
-    public Map userData;
+    public Map<String, String> userData;
 
     /** Holds the status of each segment in the index.
      *  See {@link #segmentInfos}.
@@ -172,10 +172,10 @@
        *  @see AbstractField#setOmitTermFreqAndPositions */
       public boolean hasProx;
 
-      /** Map<String, String> that includes certain
+      /** Map that includes certain
        *  debugging details that IndexWriter records into
        *  each segment it creates */
-      public Map diagnostics;
+      public Map<String,String> diagnostics;
 
       /** Status for testing of field norms (null if field norms could not be tested). */
       public FieldNormStatus fieldNormStatus;
@@ -309,7 +309,7 @@
    *  <p><b>WARNING</b>: make sure
    *  you only call this when the index is not opened by any
    *  writer. */
-  public Status checkIndex(List onlySegments) throws IOException {
+  public Status checkIndex(List<String> onlySegments) throws IOException {
     NumberFormat nf = NumberFormat.getInstance();
     SegmentInfos sis = new SegmentInfos();
     Status result = new Status();
@@ -397,10 +397,9 @@
       result.partial = true;
       if (infoStream != null)
         infoStream.print("\nChecking only these segments:");
-      Iterator it = onlySegments.iterator();
-      while (it.hasNext()) {
+      for (String s : onlySegments) {
         if (infoStream != null)
-          infoStream.print(" " + it.next());
+          infoStream.print(" " + s);
       }
       result.segmentsChecked.addAll(onlySegments);
       msg(":");
@@ -439,7 +438,7 @@
         segInfoStat.numFiles = info.files().size();
         msg("    size (MB)=" + nf.format(info.sizeInBytes()/(1024.*1024.)));
         segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
-        Map diagnostics = info.getDiagnostics();
+        Map<String,String> diagnostics = info.getDiagnostics();
         segInfoStat.diagnostics = diagnostics;
         if (diagnostics.size() > 0) {
           msg("    diagnostics = " + diagnostics);
@@ -497,7 +496,7 @@
         if (infoStream != null) {
           infoStream.print("    test: fields..............");
         }         
-        Collection fieldNames = reader.getFieldNames(IndexReader.FieldOption.ALL);
+        Collection<String> fieldNames = reader.getFieldNames(IndexReader.FieldOption.ALL);
         msg("OK [" + fieldNames.size() + " fields]");
         segInfoStat.numFields = fieldNames.size();
         
@@ -559,7 +558,7 @@
   /**
    * Test field norms.
    */
-  private Status.FieldNormStatus testFieldNorms(Collection fieldNames, SegmentReader reader)
{
+  private Status.FieldNormStatus testFieldNorms(Collection<String> fieldNames, SegmentReader
reader) {
     final Status.FieldNormStatus status = new Status.FieldNormStatus();
 
     try {
@@ -567,10 +566,8 @@
       if (infoStream != null) {
         infoStream.print("    test: field norms.........");
       }
-      Iterator it = fieldNames.iterator();
       final byte[] b = new byte[reader.maxDoc()];
-      while (it.hasNext()) {
-        final String fieldName = (String) it.next();
+      for (final String fieldName : fieldNames) {
         reader.norms(fieldName, b, 0);
         ++status.totFields;
       }
@@ -807,7 +804,7 @@
   public static void main(String[] args) throws IOException, InterruptedException {
 
     boolean doFix = false;
-    List onlySegments = new ArrayList();
+    List<String> onlySegments = new ArrayList<String>();
     String indexPath = null;
     int i = 0;
     while(i < args.length) {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileReader.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileReader.java Sat Oct 17
19:15:07 2009
@@ -47,7 +47,7 @@
     private String fileName;
 
     private IndexInput stream;
-    private HashMap entries = new HashMap();
+    private HashMap<String,FileEntry> entries = new HashMap<String,FileEntry>();
 
 
   public CompoundFileReader(Directory dir, String name) throws IOException {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileWriter.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/CompoundFileWriter.java Sat Oct 17
19:15:07 2009
@@ -22,7 +22,7 @@
 import org.apache.lucene.store.IndexInput;
 import java.util.LinkedList;
 import java.util.HashSet;
-import java.util.Iterator;
+
 import java.io.IOException;
 
 
@@ -62,8 +62,8 @@
 
     private Directory directory;
     private String fileName;
-    private HashSet ids;
-    private LinkedList entries;
+    private HashSet<String> ids;
+    private LinkedList<FileEntry> entries;
     private boolean merged = false;
     private SegmentMerger.CheckAbort checkAbort;
 
@@ -83,8 +83,8 @@
         this.checkAbort = checkAbort;
         directory = dir;
         fileName = name;
-        ids = new HashSet();
-        entries = new LinkedList();
+        ids = new HashSet<String>();
+        entries = new LinkedList<FileEntry>();
     }
 
     /** Returns the directory of the compound file. */
@@ -152,10 +152,8 @@
             // Write the directory with all offsets at 0.
             // Remember the positions of directory entries so that we can
             // adjust the offsets later
-            Iterator it = entries.iterator();
             long totalSize = 0;
-            while(it.hasNext()) {
-                FileEntry fe = (FileEntry) it.next();
+            for (FileEntry fe : entries) {
                 fe.directoryOffset = os.getFilePointer();
                 os.writeLong(0);    // for now
                 os.writeString(fe.file);
@@ -174,17 +172,13 @@
             // Open the files and copy their data into the stream.
             // Remember the locations of each file's data section.
             byte buffer[] = new byte[16384];
-            it = entries.iterator();
-            while(it.hasNext()) {
-                FileEntry fe = (FileEntry) it.next();
+            for (FileEntry fe : entries) {
                 fe.dataOffset = os.getFilePointer();
                 copyFile(fe, os, buffer);
             }
 
             // Write the data offsets into the directory of the compound stream
-            it = entries.iterator();
-            while(it.hasNext()) {
-                FileEntry fe = (FileEntry) it.next();
+            for (FileEntry fe : entries) {
                 os.seek(fe.directoryOffset);
                 os.writeLong(fe.dataOffset);
             }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Sat Oct
17 19:15:07 2009
@@ -36,7 +36,7 @@
 
   private int mergeThreadPriority = -1;
 
-  protected List mergeThreads = new ArrayList();
+  protected List<MergeThread> mergeThreads = new ArrayList<MergeThread>();
 
   // Max number of threads allowed to be merging at once
   private int maxThreadCount = 3;
@@ -363,7 +363,7 @@
       // Make sure all outstanding threads are done so we see
       // any exceptions they may produce:
       for(int i=0;i<count;i++)
-        ((ConcurrentMergeScheduler) allInstances.get(i)).sync();
+        allInstances.get(i).sync();
       boolean v = anyExceptions;
       anyExceptions = false;
       return v;
@@ -382,7 +382,7 @@
       final int size = allInstances.size();
       int upto = 0;
       for(int i=0;i<size;i++) {
-        final ConcurrentMergeScheduler other = (ConcurrentMergeScheduler) allInstances.get(i);
+        final ConcurrentMergeScheduler other = allInstances.get(i);
         if (!(other.closed && 0 == other.mergeThreadCount()))
           // Keep this one for now: it still has threads or
           // may spawn new threads
@@ -406,8 +406,8 @@
   }
 
   /** Used for testing */
-  private static List allInstances;
+  private static List<ConcurrentMergeScheduler> allInstances;
   public static void setTestMode() {
-    allInstances = new ArrayList();
+    allInstances = new ArrayList<ConcurrentMergeScheduler>();
   }
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryReader.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryReader.java Sat Oct 17 19:15:07
2009
@@ -25,7 +25,7 @@
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
+
 import java.util.Map;
 import java.util.Set;
 
@@ -47,7 +47,7 @@
   IndexWriter writer;
 
   private IndexDeletionPolicy deletionPolicy;
-  private final HashSet synced = new HashSet();
+  private final HashSet<String> synced = new HashSet<String>();
   private Lock writeLock;
   private SegmentInfos segmentInfos;
   private SegmentInfos segmentInfosStart;
@@ -59,7 +59,7 @@
 
   private SegmentReader[] subReaders;
   private int[] starts;                           // 1st docno for each segment
-  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;
@@ -177,7 +177,7 @@
 
   /** This constructor is only used for {@link #reopen()} */
   DirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[]
oldStarts,
-                  Map oldNormsCache, boolean readOnly, boolean doClone, int termInfosIndexDivisor)
throws IOException {
+                  Map<String,byte[]> oldNormsCache, boolean readOnly, boolean doClone,
int termInfosIndexDivisor) throws IOException {
     this.directory = directory;
     this.readOnly = readOnly;
     this.segmentInfos = infos;
@@ -190,7 +190,7 @@
 
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
-    Map segmentReaders = new HashMap();
+    Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
 
     if (oldReaders != null) {
       // create a Map SegmentName->SegmentReader
@@ -267,9 +267,7 @@
     
     // try to copy unchanged norms from the old normsCache to the new one
     if (oldNormsCache != null) {
-      Iterator it = oldNormsCache.entrySet().iterator();
-      while (it.hasNext()) {
-        Map.Entry entry = (Map.Entry) it.next();
+      for (Map.Entry<String,byte[]> entry: oldNormsCache.entrySet()) {
         String field = (String) entry.getKey();
         if (!hasNorms(field)) {
           continue;
@@ -681,7 +679,7 @@
    *
    * @throws IOException if there is a low-level IO error
    */
-  protected void doCommit(Map commitUserData) throws IOException {
+  protected void doCommit(Map<String,String> commitUserData) throws IOException {
     if (hasChanges) {
       segmentInfos.setUserData(commitUserData);
       // Default deleter (for backwards compatibility) is
@@ -700,9 +698,8 @@
           subReaders[i].commit();
 
         // Sync all files we just wrote
-        Iterator it = segmentInfos.files(directory, false).iterator();
-        while (it.hasNext()) {
-          final String fileName = (String) it.next();
+        final Collection<String> files = segmentInfos.files(directory, false);
+        for (final String fileName : files) { 
           if (!synced.contains(fileName)) {
             assert directory.fileExists(fileName);
             directory.sync(fileName);
@@ -766,7 +763,7 @@
     }
   }
 
-  public Map getCommitUserData() {
+  public Map<String,String> getCommitUserData() {
     ensureOpen();
     return segmentInfos.getUserData();
   }
@@ -796,17 +793,16 @@
     if (ioe != null) throw ioe;
   }
 
-  public Collection getFieldNames (IndexReader.FieldOption fieldNames) {
+  public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
     ensureOpen();
     return getFieldNames(fieldNames, this.subReaders);
   }
   
-  static Collection getFieldNames (IndexReader.FieldOption fieldNames, IndexReader[] subReaders)
{
+  static Collection<String> getFieldNames (IndexReader.FieldOption fieldNames, IndexReader[]
subReaders) {
     // maintain a unique set of field names
-    Set fieldSet = new HashSet();
-    for (int i = 0; i < subReaders.length; i++) {
-      IndexReader reader = subReaders[i];
-      Collection names = reader.getFieldNames(fieldNames);
+    Set<String> fieldSet = new HashSet<String>();
+    for (IndexReader reader : subReaders) {
+      Collection<String> names = reader.getFieldNames(fieldNames);
       fieldSet.addAll(names);
     }
     return fieldSet;
@@ -838,10 +834,10 @@
   }
 
   /** @see org.apache.lucene.index.IndexReader#listCommits */
-  public static Collection listCommits(Directory dir) throws IOException {
+  public static Collection<IndexCommit> listCommits(Directory dir) throws IOException
{
     final String[] files = dir.listAll();
 
-    Collection commits = new ArrayList();
+    Collection<IndexCommit> commits = new ArrayList<IndexCommit>();
 
     SegmentInfos latest = new SegmentInfos();
     latest.read(dir);
@@ -883,12 +879,12 @@
 
   private static final class ReaderCommit extends IndexCommit {
     private String segmentsFileName;
-    Collection files;
+    Collection<String> files;
     Directory dir;
     long generation;
     long version;
     final boolean isOptimized;
-    final Map userData;
+    final Map<String,String> userData;
 
     ReaderCommit(SegmentInfos infos, Directory dir) throws IOException {
       segmentsFileName = infos.getCurrentSegmentFileName();
@@ -908,7 +904,7 @@
       return segmentsFileName;
     }
 
-    public Collection getFileNames() {
+    public Collection<String> getFileNames() {
       return files;
     }
 
@@ -928,7 +924,7 @@
       return false;
     }
 
-    public Map getUserData() {
+    public Map<String,String> getUserData() {
       return userData;
     }
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java Sat Oct 17 19:15:07
2009
@@ -22,7 +22,7 @@
 
 abstract class DocConsumer {
   abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException;
-  abstract void flush(final Collection threads, final SegmentWriteState state) throws IOException;
+  abstract void flush(final Collection<DocFieldProcessorPerThread> threads, final SegmentWriteState
state) throws IOException;
   abstract void closeDocStore(final SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract boolean freeRAM();

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java Sat Oct 17 19:15:07
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,10 @@
     fieldsWriter.closeDocStore(state);
   }
 
-  public void flush(Collection threads, SegmentWriteState state) throws IOException {
+  public void flush(Collection<DocFieldProcessorPerThread> 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 ( DocFieldProcessorPerThread perThread : threads) {
       childThreadsAndFields.put(perThread.consumer, perThread.fields());
       perThread.trimFields(state);
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java Sat
Oct 17 19:15:07 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/trunk/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FilterIndexReader.java Sat Oct 17 19:15:07
2009
@@ -211,12 +211,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/trunk/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java Sat Oct 17 19:15:07
2009
@@ -463,7 +463,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);
   }
 
@@ -503,7 +503,7 @@
    *
    * @see #getCommitUserData(Directory)
    */
-  public Map getCommitUserData() {
+  public Map<String,String> getCommitUserData() {
     throw new UnsupportedOperationException("This reader does not support this method.");
   }
 
@@ -944,7 +944,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);
   }
@@ -971,7 +971,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);
     }
@@ -980,7 +980,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.
@@ -1006,7 +1006,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);
 
   /**
    * Expert: return the IndexCommit that this reader has
@@ -1111,7 +1111,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/trunk/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java Sat Oct 17 19:15:07
2009
@@ -352,7 +352,7 @@
     return new MultiTermPositions(this, subReaders, starts);
   }
 
-  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);
   }
@@ -367,7 +367,7 @@
     }
   }
   
-  public Collection getFieldNames (IndexReader.FieldOption fieldNames) {
+  public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
     ensureOpen();
     return DirectoryReader.getFieldNames(fieldNames, this.subReaders);
   }  

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java Sat Oct 17 19:15:07
2009
@@ -104,7 +104,7 @@
       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
@@ -435,7 +435,7 @@
     return (IndexReader[]) readers.toArray(new IndexReader[readers.size()]);
   }
 
-  protected void doCommit(Map commitUserData) throws IOException {
+  protected void doCommit(Map<String,String> commitUserData) throws IOException {
     for (int i = 0; i < readers.size(); i++)
       ((IndexReader)readers.get(i)).commit(commitUserData);
   }
@@ -450,12 +450,12 @@
     }
   }
 
-  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);
+      Collection<String> names = reader.getFieldNames(fieldNames);
       fieldSet.addAll(names);
     }
     return fieldSet;

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfo.java Sat Oct 17 19:15:07
2009
@@ -89,7 +89,7 @@
 
   private boolean hasProx;                        // True if this segment has any fields
with omitTermFreqAndPositions==false
 
-  private Map diagnostics;
+  private Map<String,String> diagnostics;
 
   public String toString() {
     return "si: "+dir.toString()+" "+name+" docCount: "+docCount+" delCount: "+delCount+"
delFileName: "+getDelFileName();
@@ -152,12 +152,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;
   }
 

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=826290&r1=826289&r2=826290&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 Sat Oct 17 19:15:07
2009
@@ -792,7 +792,7 @@
     return clone;
   }
 
-  protected void doCommit(Map commitUserData) throws IOException {
+  protected void doCommit(Map<String,String> commitUserData) throws IOException {
     if (hasChanges) {
       if (deletedDocsDirty) {               // re-write deleted
         si.advanceDelGen();
@@ -971,10 +971,10 @@
   /**
    * @see IndexReader#getFieldNames(IndexReader.FieldOption fldOption)
    */
-  public Collection getFieldNames(IndexReader.FieldOption fieldOption) {
+  public Collection<String> getFieldNames(IndexReader.FieldOption fieldOption) {
     ensureOpen();
 
-    Set fieldSet = new HashSet();
+    Set<String> fieldSet = new HashSet<String>();
     for (int i = 0; i < core.fieldInfos.size(); i++) {
       FieldInfo fi = core.fieldInfos.fieldInfo(i);
       if (fieldOption == IndexReader.FieldOption.ALL) {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=826290&r1=826289&r2=826290&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentWriteState.java Sat Oct 17 19:15:07
2009
@@ -30,7 +30,7 @@
   int numDocs;
   int termIndexInterval;
   int numDocsInStore;
-  Collection flushedFiles;
+  Collection<String> flushedFiles;
 
   public SegmentWriteState(DocumentsWriter docWriter, Directory directory, String segmentName,
String docStoreSegmentName, int numDocs,
                            int numDocsInStore, int termIndexInterval) {
@@ -41,7 +41,7 @@
     this.numDocs = numDocs;
     this.numDocsInStore = numDocsInStore;
     this.termIndexInterval = termIndexInterval;
-    flushedFiles = new HashSet();
+    flushedFiles = new HashSet<String>();
   }
 
   public String segmentFileName(String ext) {



Mime
View raw message