lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From uschind...@apache.org
Subject svn commit: r826527 [2/2] - in /lucene/java/trunk/src: java/org/apache/lucene/document/ java/org/apache/lucene/index/ java/org/apache/lucene/search/ java/org/apache/lucene/search/payloads/ java/org/apache/lucene/search/spans/ java/org/apache/lucene/sto...
Date Sun, 18 Oct 2009 21:11:37 GMT
Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanNearQuery.java Sun Oct 18 21:11:33 2009
@@ -19,7 +19,7 @@
 
 import java.io.IOException;
 
-import java.util.Collection;
+
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -35,7 +35,7 @@
  * maximum number of intervening unmatched positions, as well as whether
  * matches are required to be in-order. */
 public class SpanNearQuery extends SpanQuery implements Cloneable {
-  protected List clauses;
+  protected List<SpanQuery> clauses;
   protected int slop;
   protected boolean inOrder;
 
@@ -53,7 +53,7 @@
   public SpanNearQuery(SpanQuery[] clauses, int slop, boolean inOrder, boolean collectPayloads) {
 
     // copy clauses array into an ArrayList
-    this.clauses = new ArrayList(clauses.length);
+    this.clauses = new ArrayList<SpanQuery>(clauses.length);
     for (int i = 0; i < clauses.length; i++) {
       SpanQuery clause = clauses[i];
       if (i == 0) {                               // check field
@@ -70,7 +70,7 @@
 
   /** Return the clauses whose spans are matched. */
   public SpanQuery[] getClauses() {
-    return (SpanQuery[])clauses.toArray(new SpanQuery[clauses.size()]);
+    return clauses.toArray(new SpanQuery[clauses.size()]);
   }
 
   /** Return the maximum number of intervening unmatched positions permitted.*/
@@ -82,9 +82,7 @@
   public String getField() { return field; }
   
   public void extractTerms(Set<Term> terms) {
-	    Iterator i = clauses.iterator();
-	    while (i.hasNext()) {
-	      SpanQuery clause = (SpanQuery)i.next();
+	    for (final SpanQuery clause : clauses) {
 	      clause.extractTerms(terms);
 	    }
   }  
@@ -93,9 +91,9 @@
   public String toString(String field) {
     StringBuilder buffer = new StringBuilder();
     buffer.append("spanNear([");
-    Iterator i = clauses.iterator();
+    Iterator<SpanQuery> i = clauses.iterator();
     while (i.hasNext()) {
-      SpanQuery clause = (SpanQuery)i.next();
+      SpanQuery clause = i.next();
       buffer.append(clause.toString(field));
       if (i.hasNext()) {
         buffer.append(", ");
@@ -115,7 +113,7 @@
       return new SpanOrQuery(getClauses()).getSpans(reader);
 
     if (clauses.size() == 1)                      // optimize 1-clause case
-      return ((SpanQuery)clauses.get(0)).getSpans(reader);
+      return clauses.get(0).getSpans(reader);
 
     return inOrder
             ? (Spans) new NearSpansOrdered(this, reader, collectPayloads)
@@ -125,7 +123,7 @@
   public Query rewrite(IndexReader reader) throws IOException {
     SpanNearQuery clone = null;
     for (int i = 0 ; i < clauses.size(); i++) {
-      SpanQuery c = (SpanQuery)clauses.get(i);
+      SpanQuery c = clauses.get(i);
       SpanQuery query = (SpanQuery) c.rewrite(reader);
       if (query != c) {                     // clause rewrote: must clone
         if (clone == null)
@@ -145,8 +143,7 @@
     SpanQuery[] newClauses = new SpanQuery[sz];
 
     for (int i = 0; i < sz; i++) {
-      SpanQuery clause = (SpanQuery) clauses.get(i);
-      newClauses[i] = (SpanQuery) clause.clone();
+      newClauses[i] = (SpanQuery) clauses.get(i).clone();
     }
     SpanNearQuery spanNearQuery = new SpanNearQuery(newClauses, slop, inOrder);
     spanNearQuery.setBoost(getBoost());

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

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanOrQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanOrQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanOrQuery.java Sun Oct 18 21:11:33 2009
@@ -33,14 +33,14 @@
 
 /** Matches the union of its clauses.*/
 public class SpanOrQuery extends SpanQuery implements Cloneable {
-  private List clauses;
+  private List<SpanQuery> clauses;
   private String field;
 
   /** Construct a SpanOrQuery merging the provided clauses. */
   public SpanOrQuery(SpanQuery[] clauses) {
 
     // copy clauses array into an ArrayList
-    this.clauses = new ArrayList(clauses.length);
+    this.clauses = new ArrayList<SpanQuery>(clauses.length);
     for (int i = 0; i < clauses.length; i++) {
       SpanQuery clause = clauses[i];
       if (i == 0) {                               // check field
@@ -60,9 +60,7 @@
   public String getField() { return field; }
 
   public void extractTerms(Set<Term> terms) {
-    Iterator i = clauses.iterator();
-    while (i.hasNext()) {
-      SpanQuery clause = (SpanQuery)i.next();
+    for(final SpanQuery clause: clauses) {
       clause.extractTerms(terms);
     }
   }
@@ -72,8 +70,7 @@
     SpanQuery[] newClauses = new SpanQuery[sz];
 
     for (int i = 0; i < sz; i++) {
-      SpanQuery clause = (SpanQuery) clauses.get(i);
-      newClauses[i] = (SpanQuery) clause.clone();
+      newClauses[i] = (SpanQuery) clauses.get(i).clone();
     }
     SpanOrQuery soq = new SpanOrQuery(newClauses);
     soq.setBoost(getBoost());
@@ -83,7 +80,7 @@
   public Query rewrite(IndexReader reader) throws IOException {
     SpanOrQuery clone = null;
     for (int i = 0 ; i < clauses.size(); i++) {
-      SpanQuery c = (SpanQuery)clauses.get(i);
+      SpanQuery c = clauses.get(i);
       SpanQuery query = (SpanQuery) c.rewrite(reader);
       if (query != c) {                     // clause rewrote: must clone
         if (clone == null)
@@ -101,9 +98,9 @@
   public String toString(String field) {
     StringBuilder buffer = new StringBuilder();
     buffer.append("spanOr([");
-    Iterator i = clauses.iterator();
+    Iterator<SpanQuery> i = clauses.iterator();
     while (i.hasNext()) {
-      SpanQuery clause = (SpanQuery)i.next();
+      SpanQuery clause = i.next();
       buffer.append(clause.toString(field));
       if (i.hasNext()) {
         buffer.append(", ");
@@ -134,14 +131,12 @@
   }
 
 
-  private class SpanQueue extends PriorityQueue {
+  private class SpanQueue extends PriorityQueue<Spans> {
     public SpanQueue(int size) {
       initialize(size);
     }
 
-    protected final boolean lessThan(Object o1, Object o2) {
-      Spans spans1 = (Spans)o1;
-      Spans spans2 = (Spans)o2;
+    protected final boolean lessThan(Spans spans1, Spans spans2) {
       if (spans1.doc() == spans2.doc()) {
         if (spans1.start() == spans2.start()) {
           return spans1.end() < spans2.end();
@@ -163,9 +158,9 @@
 
         private boolean initSpanQueue(int target) throws IOException {
           queue = new SpanQueue(clauses.size());
-          Iterator i = clauses.iterator();
+          Iterator<SpanQuery> i = clauses.iterator();
           while (i.hasNext()) {
-            Spans spans = ((SpanQuery)i.next()).getSpans(reader);
+            Spans spans = i.next().getSpans(reader);
             if (   ((target == -1) && spans.next())
                 || ((target != -1) && spans.skipTo(target))) {
               queue.add(spans);
@@ -219,11 +214,11 @@
         public int start() { return top().start(); }
         public int end() { return top().end(); }
 
-      public Collection/*<byte[]>*/ getPayload() throws IOException {
-        ArrayList result = null;
+      public Collection<byte[]> getPayload() throws IOException {
+        ArrayList<byte[]> result = null;
         Spans theTop = top();
         if (theTop != null && theTop.isPayloadAvailable()) {
-          result = new ArrayList(theTop.getPayload());
+          result = new ArrayList<byte[]>(theTop.getPayload());
         }
         return result;
       }

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

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Sun Oct 18 21:11:33 2009
@@ -22,8 +22,6 @@
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Set;
 
 /** Matches spans containing a term. */

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanWeight.java Sun Oct 18 21:11:33 2009
@@ -18,6 +18,7 @@
  */
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.Explanation.IDFExplanation;
 
@@ -35,7 +36,7 @@
   protected float queryNorm;
   protected float queryWeight;
 
-  protected Set terms;
+  protected Set<Term> terms;
   protected SpanQuery query;
   private IDFExplanation idfExp;
 
@@ -43,8 +44,10 @@
     throws IOException {
     this.similarity = query.getSimilarity(searcher);
     this.query = query;
-    terms=new HashSet();
+    
+    terms=new HashSet<Term>();
     query.extractTerms(terms);
+    
     idfExp = similarity.idfExplain(terms, searcher);
     idf = idfExp.getIdf();
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/Spans.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/Spans.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/Spans.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/Spans.java Sun Oct 18 21:11:33 2009
@@ -75,7 +75,7 @@
    * @throws java.io.IOException
     */
   // TODO: Remove warning after API has been finalized
-  public abstract Collection/*<byte[]>*/ getPayload() throws IOException;
+  public abstract Collection<byte[]> getPayload() throws IOException;
 
   /**
    * Checks if a payload can be loaded at this position.

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/TermSpans.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/TermSpans.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/TermSpans.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/TermSpans.java Sun Oct 18 21:11:33 2009
@@ -87,7 +87,7 @@
   }
 
   // TODO: Remove warning after API has been finalized
-  public Collection/*<byte[]>*/ getPayload() throws IOException {
+  public Collection<byte[]> getPayload() throws IOException {
     byte [] bytes = new byte[positions.getPayloadLength()]; 
     bytes = positions.getPayload(bytes, 0);
     return Collections.singletonList(bytes);

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/FSDirectory.java Sun Oct 18 21:11:33 2009
@@ -19,14 +19,10 @@
 
 import java.io.File;
 import java.io.FilenameFilter;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.lucene.util.Constants;
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/FileSwitchDirectory.java Sun Oct 18 21:11:33 2009
@@ -18,8 +18,7 @@
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
+
 import java.util.Set;
 
 /**
@@ -39,10 +38,10 @@
 public class FileSwitchDirectory extends Directory {
   private final Directory secondaryDir;
   private final Directory primaryDir;
-  private final Set primaryExtensions;
+  private final Set<String> primaryExtensions;
   private boolean doClose;
 
-  public FileSwitchDirectory(Set primaryExtensions, Directory primaryDir, Directory secondaryDir, boolean doClose) {
+  public FileSwitchDirectory(Set<String> primaryExtensions, Directory primaryDir, Directory secondaryDir, boolean doClose) {
     this.primaryExtensions = primaryExtensions;
     this.primaryDir = primaryDir;
     this.secondaryDir = secondaryDir;

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/IndexOutput.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/IndexOutput.java Sun Oct 18 21:11:33 2009
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.Iterator;
 import org.apache.lucene.util.UnicodeUtil;
 
 /** Abstract base class for output to a file in a Directory.  A random-access
@@ -210,17 +209,14 @@
    */
   public void setLength(long length) throws IOException {};
 
-  // map must be Map<String, String>
-  public void writeStringStringMap(Map map) throws IOException {
+  public void writeStringStringMap(Map<String,String> map) throws IOException {
     if (map == null) {
       writeInt(0);
     } else {
       writeInt(map.size());
-      final Iterator it = map.entrySet().iterator();
-      while(it.hasNext()) {
-        Map.Entry entry = (Map.Entry) it.next();
-        writeString((String) entry.getKey());
-        writeString((String) entry.getValue());
+      for(final Map.Entry<String, String> entry: map.entrySet()) {
+        writeString(entry.getKey());
+        writeString(entry.getValue());
       }
     }
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/MMapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/MMapDirectory.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/MMapDirectory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/MMapDirectory.java Sun Oct 18 21:11:33 2009
@@ -148,7 +148,7 @@
   final void cleanMapping(final ByteBuffer buffer) throws IOException {
     if (useUnmapHack) {
       try {
-        AccessController.doPrivileged(new PrivilegedExceptionAction() {
+        AccessController.doPrivileged(new PrivilegedExceptionAction<Object>() {
           public Object run() throws Exception {
             final Method getCleanerMethod = buffer.getClass()
               .getMethod("cleaner", NO_PARAM_TYPES);

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/NativeFSLockFactory.java Sun Oct 18 21:11:33 2009
@@ -164,7 +164,7 @@
    * one JVM (each with their own NativeFSLockFactory
    * instance) have set the same lock dir and lock prefix.
    */
-  private static HashSet LOCK_HELD = new HashSet();
+  private static HashSet<String> LOCK_HELD = new HashSet<String>();
 
   public NativeFSLock(File lockDir, String lockFileName) {
     this.lockDir = lockDir;

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMDirectory.java Sun Oct 18 21:11:33 2009
@@ -19,10 +19,8 @@
 
 import java.io.IOException;
 import java.io.FileNotFoundException;
-import java.io.File;
 import java.io.Serializable;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Set;
 
 /**
@@ -34,7 +32,7 @@
 
   private static final long serialVersionUID = 1l;
 
-  HashMap fileMap = new HashMap();
+  HashMap<String,RAMFile> fileMap = new HashMap<String,RAMFile>();
   long sizeInBytes = 0;
   
   // *****
@@ -73,12 +71,11 @@
 
   public synchronized final String[] listAll() {
     ensureOpen();
-    Set fileNames = fileMap.keySet();
+    Set<String> fileNames = fileMap.keySet();
     String[] result = new String[fileNames.size()];
     int i = 0;
-    Iterator it = fileNames.iterator();
-    while (it.hasNext())
-      result[i++] = (String)it.next();
+    for(final String fileName: fileNames) 
+      result[i++] = fileName;
     return result;
   }
 
@@ -87,7 +84,7 @@
     ensureOpen();
     RAMFile file;
     synchronized (this) {
-      file = (RAMFile)fileMap.get(name);
+      file = fileMap.get(name);
     }
     return file != null;
   }
@@ -99,7 +96,7 @@
     ensureOpen();
     RAMFile file;
     synchronized (this) {
-      file = (RAMFile)fileMap.get(name);
+      file = fileMap.get(name);
     }
     if (file==null)
       throw new FileNotFoundException(name);
@@ -113,7 +110,7 @@
     ensureOpen();
     RAMFile file;
     synchronized (this) {
-      file = (RAMFile)fileMap.get(name);
+      file = fileMap.get(name);
     }
     if (file==null)
       throw new FileNotFoundException(name);
@@ -141,7 +138,7 @@
     ensureOpen();
     RAMFile file;
     synchronized (this) {
-      file = (RAMFile)fileMap.get(name);
+      file = fileMap.get(name);
     }
     if (file==null)
       throw new FileNotFoundException(name);
@@ -161,7 +158,7 @@
    */
   public synchronized void deleteFile(String name) throws IOException {
     ensureOpen();
-    RAMFile file = (RAMFile)fileMap.get(name);
+    RAMFile file = fileMap.get(name);
     if (file!=null) {
         fileMap.remove(name);
         file.directory = null;
@@ -175,7 +172,7 @@
     ensureOpen();
     RAMFile file = new RAMFile(this);
     synchronized (this) {
-      RAMFile existing = (RAMFile)fileMap.get(name);
+      RAMFile existing = fileMap.get(name);
       if (existing!=null) {
         sizeInBytes -= existing.sizeInBytes;
         existing.directory = null;
@@ -190,7 +187,7 @@
     ensureOpen();
     RAMFile file;
     synchronized (this) {
-      file = (RAMFile)fileMap.get(name);
+      file = fileMap.get(name);
     }
     if (file == null)
       throw new FileNotFoundException(name);

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/RAMFile.java Sun Oct 18 21:11:33 2009
@@ -24,7 +24,7 @@
 
   private static final long serialVersionUID = 1l;
 
-  private ArrayList buffers = new ArrayList();
+  private ArrayList<byte[]> buffers = new ArrayList<byte[]>();
   long length;
   RAMDirectory directory;
   long sizeInBytes;                  // Only maintained if in a directory; updates synchronized on directory

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java Sun Oct 18 21:11:33 2009
@@ -33,7 +33,7 @@
 
 public class SingleInstanceLockFactory extends LockFactory {
 
-  private HashSet locks = new HashSet();
+  private HashSet<String> locks = new HashSet<String>();
 
   public Lock makeLock(String lockName) {
     // We do not use the LockPrefix at all, because the private
@@ -54,9 +54,9 @@
 class SingleInstanceLock extends Lock {
 
   String lockName;
-  private HashSet locks;
+  private HashSet<String> locks;
 
-  public SingleInstanceLock(HashSet locks, String lockName) {
+  public SingleInstanceLock(HashSet<String> locks, String lockName) {
     this.locks = locks;
     this.lockName = lockName;
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/AverageGuessMemoryModel.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/AverageGuessMemoryModel.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/AverageGuessMemoryModel.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/AverageGuessMemoryModel.java Sun Oct 18 21:11:33 2009
@@ -26,7 +26,7 @@
  */
 public class AverageGuessMemoryModel extends MemoryModel {
   // best guess primitive sizes
-  private final Map sizes = new IdentityHashMap() {
+  private final Map<Class,Integer> sizes = new IdentityHashMap<Class,Integer>() {
     {
       put(boolean.class, Integer.valueOf(1));
       put(byte.class, Integer.valueOf(1));
@@ -61,7 +61,7 @@
    * @see org.apache.lucene.util.MemoryModel#getPrimitiveSize(java.lang.Class)
    */
   public int getPrimitiveSize(Class clazz) {
-    return ((Integer) sizes.get(clazz)).intValue();
+    return sizes.get(clazz).intValue();
   }
 
   /* (non-Javadoc)

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java Sun Oct 18 21:11:33 2009
@@ -19,7 +19,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;
 import java.util.Set;
@@ -111,13 +110,13 @@
     //
     // maps the (valId) identityhashCode of cache values to 
     // sets of CacheEntry instances
-    final MapOfSets valIdToItems = new MapOfSets(new HashMap(17));
+    final MapOfSets<Integer, CacheEntry> valIdToItems = new MapOfSets<Integer, CacheEntry>(new HashMap<Integer, Set<CacheEntry>>(17));
     // maps ReaderField keys to Sets of ValueIds
-    final MapOfSets readerFieldToValIds = new MapOfSets(new HashMap(17));
+    final MapOfSets<ReaderField, Integer> readerFieldToValIds = new MapOfSets<ReaderField, Integer>(new HashMap<ReaderField, Set<Integer>>(17));
     //
 
     // any keys that we know result in more then one valId
-    final Set valMismatchKeys = new HashSet();
+    final Set<ReaderField> valMismatchKeys = new HashSet<ReaderField>();
 
     // iterate over all the cacheEntries to get the mappings we'll need
     for (int i = 0; i < cacheEntries.length; i++) {
@@ -139,7 +138,7 @@
       }
     }
 
-    final List insanity = new ArrayList(valMismatchKeys.size() * 3);
+    final List<Insanity> insanity = new ArrayList<Insanity>(valMismatchKeys.size() * 3);
 
     insanity.addAll(checkValueMismatch(valIdToItems, 
                                        readerFieldToValIds, 
@@ -147,7 +146,7 @@
     insanity.addAll(checkSubreaders(valIdToItems, 
                                     readerFieldToValIds));
                     
-    return (Insanity[]) insanity.toArray(new Insanity[insanity.size()]);
+    return insanity.toArray(new Insanity[insanity.size()]);
   }
 
   /** 
@@ -157,31 +156,27 @@
    * the Insanity objects. 
    * @see InsanityType#VALUEMISMATCH
    */
-  private Collection checkValueMismatch(MapOfSets valIdToItems,
-                                        MapOfSets readerFieldToValIds,
-                                        Set valMismatchKeys) {
+  private Collection<Insanity> checkValueMismatch(MapOfSets<Integer, CacheEntry> valIdToItems,
+                                        MapOfSets<ReaderField, Integer> readerFieldToValIds,
+                                        Set<ReaderField> valMismatchKeys) {
 
-    final List insanity = new ArrayList(valMismatchKeys.size() * 3);
+    final List<Insanity> insanity = new ArrayList<Insanity>(valMismatchKeys.size() * 3);
 
     if (! valMismatchKeys.isEmpty() ) { 
       // we have multiple values for some ReaderFields
 
-      final Map rfMap = readerFieldToValIds.getMap();
-      final Map valMap = valIdToItems.getMap();
-      final Iterator mismatchIter = valMismatchKeys.iterator();
-      while (mismatchIter.hasNext()) {
-        final ReaderField rf = (ReaderField)mismatchIter.next();
-        final List badEntries = new ArrayList(valMismatchKeys.size() * 2);
-        final Iterator valIter = ((Set)rfMap.get(rf)).iterator();
-        while (valIter.hasNext()) {
-          Iterator entriesIter = ((Set)valMap.get(valIter.next())).iterator();
-          while (entriesIter.hasNext()) {
-            badEntries.add(entriesIter.next());
+      final Map<ReaderField, Set<Integer>> rfMap = readerFieldToValIds.getMap();
+      final Map<Integer, Set<CacheEntry>> valMap = valIdToItems.getMap();
+      for (final ReaderField rf : valMismatchKeys) {
+        final List<CacheEntry> badEntries = new ArrayList<CacheEntry>(valMismatchKeys.size() * 2);
+        for(final Integer value: rfMap.get(rf)) {
+          for (final CacheEntry cacheEntry : valMap.get(value)) {
+            badEntries.add(cacheEntry);
           }
         }
 
         CacheEntry[] badness = new CacheEntry[badEntries.size()];
-        badness = (CacheEntry[]) badEntries.toArray(badness);
+        badness = badEntries.toArray(badness);
 
         insanity.add(new Insanity(InsanityType.VALUEMISMATCH,
                                   "Multiple distinct value objects for " + 
@@ -199,35 +194,33 @@
    *
    * @see InsanityType#SUBREADER
    */
-  private Collection checkSubreaders(MapOfSets valIdToItems,
-                                     MapOfSets readerFieldToValIds) {
+  private Collection<Insanity> checkSubreaders( MapOfSets<Integer, CacheEntry>  valIdToItems,
+                                      MapOfSets<ReaderField, Integer> readerFieldToValIds) {
 
-    final List insanity = new ArrayList(23);
+    final List<Insanity> insanity = new ArrayList<Insanity>(23);
 
-    Map badChildren = new HashMap(17);
-    MapOfSets badKids = new MapOfSets(badChildren); // wrapper
+    Map<ReaderField, Set<ReaderField>> badChildren = new HashMap<ReaderField, Set<ReaderField>>(17);
+    MapOfSets<ReaderField, ReaderField> badKids = new MapOfSets<ReaderField, ReaderField>(badChildren); // wrapper
 
-    Map viToItemSets = valIdToItems.getMap();
-    Map rfToValIdSets = readerFieldToValIds.getMap();
+    Map<Integer, Set<CacheEntry>> viToItemSets = valIdToItems.getMap();
+    Map<ReaderField, Set<Integer>> rfToValIdSets = readerFieldToValIds.getMap();
 
-    Set seen = new HashSet(17);
+    Set<ReaderField> seen = new HashSet<ReaderField>(17);
 
-    Set readerFields = rfToValIdSets.keySet();
-    Iterator rfIter = readerFields.iterator();
-    while (rfIter.hasNext()) {
-      ReaderField rf = (ReaderField) rfIter.next();
+    Set<ReaderField> readerFields = rfToValIdSets.keySet();
+    for (final ReaderField rf : readerFields) {
       
       if (seen.contains(rf)) continue;
 
       List kids = getAllDecendentReaderKeys(rf.readerKey);
-      for (int i = 0; i < kids.size(); i++) {
-        ReaderField kid = new ReaderField(kids.get(i), rf.fieldName);
+      for (Object kidKey : kids) {
+        ReaderField kid = new ReaderField(kidKey, rf.fieldName);
         
         if (badChildren.containsKey(kid)) {
           // we've already process this kid as RF and found other problems
           // track those problems as our own
           badKids.put(rf, kid);
-          badKids.putAll(rf, (Collection)badChildren.get(kid));
+          badKids.putAll(rf, badChildren.get(kid));
           badChildren.remove(kid);
           
         } else if (rfToValIdSets.containsKey(kid)) {
@@ -240,33 +233,27 @@
     }
 
     // every mapping in badKids represents an Insanity
-    Iterator parentsIter = badChildren.keySet().iterator();
-    while (parentsIter.hasNext()) {
-      ReaderField parent = (ReaderField) parentsIter.next();
-      Set kids = (Set) badChildren.get(parent);
+    for (final ReaderField parent : badChildren.keySet()) {
+      Set<ReaderField> kids = badChildren.get(parent);
 
-      List badEntries = new ArrayList(kids.size() * 2);
+      List<CacheEntry> badEntries = new ArrayList<CacheEntry>(kids.size() * 2);
 
       // put parent entr(ies) in first
       {
-        Iterator valIter =((Set)rfToValIdSets.get(parent)).iterator();
-        while (valIter.hasNext()) {
-          badEntries.addAll((Set)viToItemSets.get(valIter.next()));
+        for (final Integer value  : rfToValIdSets.get(parent)) {
+          badEntries.addAll(viToItemSets.get(value));
         }
       }
 
       // now the entries for the descendants
-      Iterator kidsIter = kids.iterator();
-      while (kidsIter.hasNext()) {
-        ReaderField kid = (ReaderField) kidsIter.next();
-        Iterator valIter =((Set)rfToValIdSets.get(kid)).iterator();
-        while (valIter.hasNext()) {
-          badEntries.addAll((Set)viToItemSets.get(valIter.next()));
+      for (final ReaderField kid : kids) {
+        for (final Integer value : rfToValIdSets.get(kid)) {
+          badEntries.addAll(viToItemSets.get(value));
         }
       }
 
       CacheEntry[] badness = new CacheEntry[badEntries.size()];
-      badness = (CacheEntry[]) badEntries.toArray(badness);
+      badness = badEntries.toArray(badness);
 
       insanity.add(new Insanity(InsanityType.SUBREADER,
                                 "Found caches for decendents of " + 

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/MapOfSets.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/MapOfSets.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/MapOfSets.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/MapOfSets.java Sun Oct 18 21:11:33 2009
@@ -26,21 +26,21 @@
 /**
  * Helper class for keeping Lists of Objects associated with keys. <b>WARNING: THIS CLASS IS NOT THREAD SAFE</b>
  */
-public class MapOfSets {
+public class MapOfSets<K, V> {
 
-  private final Map theMap;
+  private final Map<K, Set<V>> theMap;
 
   /**
    * @param m the backing store for this object
    */
-  public MapOfSets(Map m) {
+  public MapOfSets(Map<K, Set<V>> m) {
     theMap = m;
   }
 
   /**
    * @return direct access to the map backing this object.
    */
-  public Map getMap() {
+  public Map<K, Set<V>> getMap() {
     return theMap;
   }
 
@@ -49,12 +49,12 @@
    * already in the map, a new Set will first be created.
    * @return the size of the Set associated with key once val is added to it.
    */
-  public int put(Object key, Object val) {
-    final Set theSet;
+  public int put(K key, V val) {
+    final Set<V> theSet;
     if (theMap.containsKey(key)) {
-      theSet = (Set)theMap.get(key);
+      theSet = theMap.get(key);
     } else {
-      theSet = new HashSet(23);
+      theSet = new HashSet<V>(23);
       theMap.put(key, theSet);
     }
     theSet.add(val);
@@ -66,12 +66,12 @@
    * already in the map, a new Set will first be created.
    * @return the size of the Set associated with key once val is added to it.
    */
-  public int putAll(Object key, Collection vals) {
-    final Set theSet;
+  public int putAll(K key, Collection<? extends V> vals) {
+    final Set<V> theSet;
     if (theMap.containsKey(key)) {
-      theSet = (Set)theMap.get(key);
+      theSet = theMap.get(key);
     } else {
-      theSet = new HashSet(23);
+      theSet = new HashSet<V>(23);
       theMap.put(key, theSet);
     }
     theSet.addAll(vals);

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/Parameter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/Parameter.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/Parameter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/Parameter.java Sun Oct 18 21:11:33 2009
@@ -28,7 +28,7 @@
  */
 public abstract class Parameter implements Serializable
 {
-  static Map allParameters = new HashMap();
+  static Map<String,Parameter> allParameters = new HashMap<String,Parameter>();
   
   private String name;
   

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/RamUsageEstimator.java Sun Oct 18 21:11:33 2009
@@ -37,7 +37,7 @@
 public final class RamUsageEstimator {
   private MemoryModel memoryModel;
 
-  private final Map seen;
+  private final Map<Object,Object> seen;
 
   private int refSize;
   private int arraySize;
@@ -82,7 +82,7 @@
     this.checkInterned = checkInterned;
     // Use Map rather than Set so that we can use an IdentityHashMap - not
     // seeing an IdentityHashSet
-    seen = new IdentityHashMap(64);
+    seen = new IdentityHashMap<Object,Object>(64);
     this.refSize = memoryModel.getReferenceSize();
     this.arraySize = memoryModel.getArraySize();
     this.classSize = memoryModel.getClassSize();

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/ReaderUtil.java Sun Oct 18 21:11:33 2009
@@ -34,7 +34,7 @@
    * @param allSubReaders
    * @param reader
    */
-  public static void gatherSubReaders(List allSubReaders, IndexReader reader) {
+  public static void gatherSubReaders(List<IndexReader> allSubReaders, IndexReader reader) {
     IndexReader[] subReaders = reader.getSequentialSubReaders();
     if (subReaders == null) {
       // Add the reader itself, and do not recurse
@@ -54,7 +54,7 @@
    * @return sub reader of parent which contains the specified doc id
    */
   public static IndexReader subReader(int doc, IndexReader reader) {
-    List subReadersList = new ArrayList();
+    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
     ReaderUtil.gatherSubReaders(subReadersList, reader);
     IndexReader[] subReaders = (IndexReader[]) subReadersList
         .toArray(new IndexReader[subReadersList.size()]);
@@ -75,7 +75,7 @@
    * @return the subreader at subIndex
    */
   public static IndexReader subReader(IndexReader reader, int subIndex) {
-    List subReadersList = new ArrayList();
+    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
     ReaderUtil.gatherSubReaders(subReadersList, reader);
     IndexReader[] subReaders = (IndexReader[]) subReadersList
         .toArray(new IndexReader[subReadersList.size()]);

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/cache/Cache.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/cache/Cache.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/cache/Cache.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/cache/Cache.java Sun Oct 18 21:11:33 2009
@@ -21,43 +21,48 @@
 /**
  * Base class for cache implementations.
  */
-public abstract class Cache {
+public abstract class Cache<K,V> {
   
   /**
    * Simple Cache wrapper that synchronizes all
    * calls that access the cache. 
    */
-  static class SynchronizedCache extends Cache {
-    Object mutex;
-    Cache  cache;
+  static class SynchronizedCache<K,V> extends Cache<K,V> {
+    private Object mutex;
+    private Cache<K,V> cache;
     
-    SynchronizedCache(Cache cache) {
+    SynchronizedCache(Cache<K,V> cache) {
       this.cache = cache;
       this.mutex = this;
     }
     
-    SynchronizedCache(Cache cache, Object mutex) {
+    SynchronizedCache(Cache<K,V> cache, Object mutex) {
       this.cache = cache;
       this.mutex = mutex;
     }
     
-    public void put(Object key, Object value) {
+    @Override
+    public void put(K key, V value) {
       synchronized(mutex) {cache.put(key, value);}
     }
     
-    public Object get(Object key) {
+    @Override
+    public V get(Object key) {
       synchronized(mutex) {return cache.get(key);}
     }
     
+    @Override
     public boolean containsKey(Object key) {
       synchronized(mutex) {return cache.containsKey(key);}
     }
     
+    @Override
     public void close() {
       synchronized(mutex) {cache.close();}
     }
     
-    Cache getSynchronizedCache() {
+    @Override
+    Cache<K,V> getSynchronizedCache() {
       return this;
     }
   }
@@ -67,7 +72,7 @@
    * In order to guarantee thread-safety, all access to the backed cache must
    * be accomplished through the returned cache.
    */
-  public static Cache synchronizedCache(Cache cache) {
+  public static <K,V> Cache<K,V> synchronizedCache(Cache<K,V> cache) {
     return cache.getSynchronizedCache();
   }
 
@@ -78,19 +83,19 @@
    * e. g. subclasses of {@link SynchronizedCache} or this
    * in case this cache is already synchronized.
    */
-  Cache getSynchronizedCache() {
-    return new SynchronizedCache(this);
+  Cache<K,V> getSynchronizedCache() {
+    return new SynchronizedCache<K,V>(this);
   }
   
   /**
    * Puts a (key, value)-pair into the cache. 
    */
-  public abstract void put(Object key, Object value);
+  public abstract void put(K key, V value);
   
   /**
    * Returns the value for the given key. 
    */
-  public abstract Object get(Object key);
+  public abstract V get(Object key);
   
   /**
    * Returns whether the given key is in this cache. 

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java Sun Oct 18 21:11:33 2009
@@ -26,24 +26,19 @@
  * if needed.
  * 
  */
-public class SimpleLRUCache extends SimpleMapCache {
+public class SimpleLRUCache<K,V> extends SimpleMapCache<K,V> {
   private final static float LOADFACTOR = 0.75f;
 
-  private int cacheSize;
-
   /**
    * Creates a last-recently-used cache with the specified size. 
    */
-  public SimpleLRUCache(int cacheSize) {
-    super(null);
-    this.cacheSize = cacheSize;
-    int capacity = (int) Math.ceil(cacheSize / LOADFACTOR) + 1;
-
-    super.map = new LinkedHashMap(capacity, LOADFACTOR, true) {
-      protected boolean removeEldestEntry(Map.Entry eldest) {
-        return size() > SimpleLRUCache.this.cacheSize;
+  public SimpleLRUCache(final int cacheSize) {
+    super(new LinkedHashMap<K,V>((int) Math.ceil(cacheSize / LOADFACTOR) + 1, LOADFACTOR, true) {
+      @Override
+      protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
+        return size() > cacheSize;
       }
-    };
+    });
   }
 
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.java Sun Oct 18 21:11:33 2009
@@ -26,29 +26,33 @@
  * This cache is not synchronized, use {@link Cache#synchronizedCache(Cache)}
  * if needed.
  */
-public class SimpleMapCache extends Cache {
-  Map map;
+public class SimpleMapCache<K,V> extends Cache<K,V> {
+  protected Map<K,V> map;
   
   public SimpleMapCache() {
-    this(new HashMap());
+    this(new HashMap<K,V>());
   }
 
-  public SimpleMapCache(Map map) {
+  public SimpleMapCache(Map<K,V> map) {
     this.map = map;
   }
   
-  public Object get(Object key) {
+  @Override
+  public V get(Object key) {
     return map.get(key);
   }
 
-  public void put(Object key, Object value) {
+  @Override
+  public void put(K key, V value) {
     map.put(key, value);
   }
 
+  @Override
   public void close() {
     // NOOP
   }
 
+  @Override
   public boolean containsKey(Object key) {
     return map.containsKey(key);
   }
@@ -56,44 +60,51 @@
   /**
    * Returns a Set containing all keys in this cache.
    */
-  public Set keySet() {
+  public Set<K> keySet() {
     return map.keySet();
   }
   
-  Cache getSynchronizedCache() {
-    return new SynchronizedSimpleMapCache(this);
+  @Override
+  Cache<K,V> getSynchronizedCache() {
+    return new SynchronizedSimpleMapCache<K,V>(this);
   }
   
-  private static class SynchronizedSimpleMapCache extends SimpleMapCache {
-    Object mutex;
-    SimpleMapCache cache;
+  private static class SynchronizedSimpleMapCache<K,V> extends SimpleMapCache<K,V> {
+    private Object mutex;
+    private SimpleMapCache<K,V> cache;
     
-    SynchronizedSimpleMapCache(SimpleMapCache cache) {
+    SynchronizedSimpleMapCache(SimpleMapCache<K,V> cache) {
         this.cache = cache;
         this.mutex = this;
     }
     
-    public void put(Object key, Object value) {
+    @Override
+    public void put(K key, V value) {
         synchronized(mutex) {cache.put(key, value);}
     }
     
-    public Object get(Object key) {
+    @Override
+    public V get(Object key) {
         synchronized(mutex) {return cache.get(key);}
     }
     
+    @Override
     public boolean containsKey(Object key) {
         synchronized(mutex) {return cache.containsKey(key);}
     }
     
+    @Override
     public void close() {
         synchronized(mutex) {cache.close();}
     }
     
-    public Set keySet() {
+    @Override
+    public Set<K> keySet() {
       synchronized(mutex) {return cache.keySet();}
     }
     
-    Cache getSynchronizedCache() {
+    @Override
+    Cache<K,V> getSynchronizedCache() {
       return this;
     }
   }

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java Sun Oct 18 21:11:33 2009
@@ -335,7 +335,7 @@
     origSegmentReader.close();
     assertDelDocsRefCountEquals(1, origSegmentReader);
     // check the norm refs
-    Norm norm = (Norm) clonedSegmentReader.norms.get("field1");
+    Norm norm = clonedSegmentReader.norms.get("field1");
     assertEquals(1, norm.bytesRef().refCount());
     clonedSegmentReader.close();
     dir1.close();

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java Sun Oct 18 21:11:33 2009
@@ -170,7 +170,7 @@
     TestIndexReaderReopen.createIndex(dir1, false);
     SegmentReader reader1 = SegmentReader.getOnlySegmentReader(dir1);
     reader1.norms("field1");
-    Norm r1norm = (Norm)reader1.norms.get("field1");
+    Norm r1norm = reader1.norms.get("field1");
     SegmentReader.Ref r1BytesRef = r1norm.bytesRef();
     SegmentReader reader2 = (SegmentReader)reader1.clone();
     assertEquals(2, r1norm.bytesRef().refCount());
@@ -189,14 +189,14 @@
     IndexReader reader2C = (IndexReader) reader1.clone();
     SegmentReader segmentReader2C = SegmentReader.getOnlySegmentReader(reader2C);
     segmentReader2C.norms("field1"); // load the norms for the field
-    Norm reader2CNorm = (Norm)segmentReader2C.norms.get("field1");
+    Norm reader2CNorm = segmentReader2C.norms.get("field1");
     assertTrue("reader2CNorm.bytesRef()=" + reader2CNorm.bytesRef(), reader2CNorm.bytesRef().refCount() == 2);
     
     
     
     IndexReader reader3C = (IndexReader) reader2C.clone();
     SegmentReader segmentReader3C = SegmentReader.getOnlySegmentReader(reader3C);
-    Norm reader3CCNorm = (Norm)segmentReader3C.norms.get("field1");
+    Norm reader3CCNorm = segmentReader3C.norms.get("field1");
     assertEquals(3, reader3CCNorm.bytesRef().refCount());
     
     // edit a norm and the refcount should be 1
@@ -215,13 +215,13 @@
     
     // norm values should be different 
     assertTrue(Similarity.decodeNorm(segmentReader3C.norms("field1")[5]) != Similarity.decodeNorm(segmentReader4C.norms("field1")[5]));
-    Norm reader4CCNorm = (Norm)segmentReader4C.norms.get("field1");
+    Norm reader4CCNorm = segmentReader4C.norms.get("field1");
     assertEquals(3, reader3CCNorm.bytesRef().refCount());
     assertEquals(1, reader4CCNorm.bytesRef().refCount());
         
     IndexReader reader5C = (IndexReader) reader4C.clone();
     SegmentReader segmentReader5C = SegmentReader.getOnlySegmentReader(reader5C);
-    Norm reader5CCNorm = (Norm)segmentReader5C.norms.get("field1");
+    Norm reader5CCNorm = segmentReader5C.norms.get("field1");
     reader5C.setNorm(5, "field1", 0.7f);
     assertEquals(1, reader5CCNorm.bytesRef().refCount());    
 

Modified: lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java?rev=826527&r1=826526&r2=826527&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/store/MockRAMDirectory.java Sun Oct 18 21:11:33 2009
@@ -43,19 +43,19 @@
   Random randomState;
   boolean noDeleteOpenFile = true;
   boolean preventDoubleWrite = true;
-  private Set unSyncedFiles;
-  private Set createdFiles;
+  private Set<String> unSyncedFiles;
+  private Set<String> createdFiles;
   volatile boolean crashed;
 
   // NOTE: we cannot initialize the Map here due to the
   // order in which our constructor actually does this
   // member initialization vs when it calls super.  It seems
   // like super is called, then our members are initialized:
-  Map openFiles;
+  Map<String,Integer> openFiles;
 
   private synchronized void init() {
     if (openFiles == null)
-      openFiles = new HashMap();
+      openFiles = new HashMap<String,Integer>();
     if (createdFiles == null)
       createdFiles = new HashSet();
     if (unSyncedFiles == null)
@@ -90,12 +90,12 @@
   public synchronized void crash() throws IOException {
     crashed = true;
     openFiles = new HashMap();
-    Iterator it = unSyncedFiles.iterator();
+    Iterator<String> it = unSyncedFiles.iterator();
     unSyncedFiles = new HashSet();
     int count = 0;
     while(it.hasNext()) {
-      String name = (String) it.next();
-      RAMFile file = (RAMFile) fileMap.get(name);
+      String name = it.next();
+      RAMFile file = fileMap.get(name);
       if (count % 3 == 0) {
         deleteFile(name, true);
       } else if (count % 3 == 1) {
@@ -204,7 +204,7 @@
       throw new IOException("cannot createOutput after crash");
     unSyncedFiles.add(name);
     createdFiles.add(name);
-    RAMFile existing = (RAMFile)fileMap.get(name);
+    RAMFile existing = fileMap.get(name);
     // Enforce write once:
     if (existing!=null && !name.equals("segments.gen") && preventDoubleWrite)
       throw new IOException("file " + name + " already exists");
@@ -221,7 +221,7 @@
   }
 
   public synchronized IndexInput openInput(String name) throws IOException {
-    RAMFile file = (RAMFile)fileMap.get(name);
+    RAMFile file = fileMap.get(name);
     if (file == null)
       throw new FileNotFoundException(name);
     else {
@@ -239,9 +239,9 @@
   /** Provided for testing purposes.  Use sizeInBytes() instead. */
   public synchronized final long getRecomputedSizeInBytes() {
     long size = 0;
-    Iterator it = fileMap.values().iterator();
-    while (it.hasNext())
-      size += ((RAMFile) it.next()).getSizeInBytes();
+    for(final RAMFile file: fileMap.values()) {
+      size += file.getSizeInBytes();
+    }
     return size;
   }
 
@@ -253,9 +253,8 @@
 
   public final synchronized long getRecomputedActualSizeInBytes() {
     long size = 0;
-    Iterator it = fileMap.values().iterator();
-    while (it.hasNext())
-      size += ((RAMFile) it.next()).length;
+    for (final RAMFile file : fileMap.values())
+      size += file.length;
     return size;
   }
 



Mime
View raw message