You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/03/16 20:39:37 UTC

svn commit: r1578144 [9/37] - in /lucene/dev/branches/lucene5376_2: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/contrib/map-reduce/ dev-tools/idea/solr/core/src/test/ dev-tools/script...

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/package.html Sun Mar 16 19:39:10 2014
@@ -53,8 +53,8 @@ org.apache.lucene.search.IndexSearcher#s
 <p>
 Once a Query has been created and submitted to the {@link org.apache.lucene.search.IndexSearcher IndexSearcher}, the scoring
 process begins. After some infrastructure setup, control finally passes to the {@link org.apache.lucene.search.Weight Weight}
-implementation and its {@link org.apache.lucene.search.Scorer Scorer} instances. See the <a href="#algorithm">Algorithm</a> 
-section for more notes on the process.
+implementation and its {@link org.apache.lucene.search.Scorer Scorer} or {@link org.apache.lucene.search.BulkScorer BulkScore}
+instances. See the <a href="#algorithm">Algorithm</a> section for more notes on the process.
 </p>
     <!-- FILL IN MORE HERE -->   
     <!-- TODO: this page over-links the same things too many times -->
@@ -370,6 +370,12 @@ on the built-in available scoring models
             {@link org.apache.lucene.search.Scorer Scorer} &mdash; An abstract class containing common
             functionality for scoring. Provides both scoring and
             explanation capabilities.  This is created per-segment.</li>
+        <li>
+            {@link org.apache.lucene.search.BulkScorer BulkScorer} &mdash; An abstract class that scores
+	    a range of documents.  A default implementation simply iterates through the hits from
+	    {@link org.apache.lucene.search.Scorer Scorer}, but some queries such as
+	    {@link org.apache.lucene.search.BooleanQuery BooleanQuery} have more efficient
+	    implementations.</li>
     </ol>
     Details on each of these classes, and their children, can be found in the subsections below.
 </p>
@@ -430,13 +436,19 @@ on the built-in available scoring models
                 that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will just defer to the Similarity's implementation:
                 {@link org.apache.lucene.search.similarities.Similarity.SimWeight#normalize SimWeight#normalize(float,float)}.</li>
             <li>
-                {@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, boolean, boolean, org.apache.lucene.util.Bits)
-                  scorer(AtomicReaderContext context, boolean scoresDocsInOrder, boolean topScorer, Bits acceptDocs)} &mdash;
+                {@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, org.apache.lucene.util.Bits)
+                  scorer(AtomicReaderContext context, Bits acceptDocs)} &mdash;
                 Construct a new {@link org.apache.lucene.search.Scorer Scorer} for this Weight. See <a href="#scorerClass">The Scorer Class</a>
                 below for help defining a Scorer. As the name implies, the Scorer is responsible for doing the actual scoring of documents 
                 given the Query.
             </li>
             <li>
+                {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.AtomicReaderContext, boolean, org.apache.lucene.util.Bits)
+                  scorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} &mdash;
+                Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a>
+                below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
+            </li>
+            <li>
                 {@link org.apache.lucene.search.Weight#explain(org.apache.lucene.index.AtomicReaderContext, int) 
                   explain(AtomicReaderContext context, int doc)} &mdash; Provide a means for explaining why a given document was
                 scored the way it was.
@@ -489,6 +501,18 @@ on the built-in available scoring models
             </li>
         </ol>
     </p>
+<a name="bulkScorerClass"></a>
+<h4>The BulkScorer Class</h4>
+    <p>The
+        {@link org.apache.lucene.search.BulkScorer BulkScorer} scores a range of documents.  There is only one 
+        abstract method:
+        <ol>
+            <li>
+                {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector,int) score(Collector,int)} &mdash;
+		Score all documents up to but not including the specified max document.
+	    </li>
+        </ol>
+    </p>
 <h4>Why would I want to add my own Query?</h4>
 
     <p>In a nutshell, you want to add your own custom Query implementation when you think that Lucene's
@@ -539,7 +563,7 @@ on the built-in available scoring models
 <p>If a Filter is being used, some initial setup is done to determine which docs to include. 
    Otherwise, we ask the Weight for a {@link org.apache.lucene.search.Scorer Scorer} for each
    {@link org.apache.lucene.index.IndexReader IndexReader} segment and proceed by calling
-   {@link org.apache.lucene.search.Scorer#score(org.apache.lucene.search.Collector) Scorer.score()}.
+   {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector) BulkScorer.score(Collector)}.
 </p>
 <p>At last, we are actually going to score some documents. The score method takes in the Collector
    (most likely the TopScoreDocCollector or TopFieldCollector) and does its business.Of course, here 

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Sun Mar 16 19:39:10 2014
@@ -148,15 +148,14 @@ public class PayloadNearQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs, termContexts), this,
           similarity, similarity.simScorer(stats, context));
     }
     
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
+      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java Sun Mar 16 19:39:10 2014
@@ -74,7 +74,7 @@ public class PayloadSpanUtil {
    * @throws IOException if there is a low-level I/O error
    */
   public Collection<byte[]> getPayloadsForQuery(Query query) throws IOException {
-    Collection<byte[]> payloads = new ArrayList<byte[]>();
+    Collection<byte[]> payloads = new ArrayList<>();
     queryToSpanQuery(query, payloads);
     return payloads;
   }
@@ -143,7 +143,7 @@ public class PayloadSpanUtil {
           final Term[] termArray = termArrays.get(i);
           List<Query> disjuncts = disjunctLists[positions[i]];
           if (disjuncts == null) {
-            disjuncts = (disjunctLists[positions[i]] = new ArrayList<Query>(
+            disjuncts = (disjunctLists[positions[i]] = new ArrayList<>(
                 termArray.length));
             ++distinctPositions;
           }
@@ -178,8 +178,8 @@ public class PayloadSpanUtil {
 
   private void getPayloads(Collection<byte []> payloads, SpanQuery query)
       throws IOException {
-    Map<Term,TermContext> termContexts = new HashMap<Term,TermContext>();
-    TreeSet<Term> terms = new TreeSet<Term>();
+    Map<Term,TermContext> termContexts = new HashMap<>();
+    TreeSet<Term> terms = new TreeSet<>();
     query.extractTerms(terms);
     for (Term term : terms) {
       termContexts.put(term, TermContext.build(context, term));

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Sun Mar 16 19:39:10 2014
@@ -79,8 +79,7 @@ public class PayloadTermQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs, termContexts),
           this, similarity.simScorer(stats, context));
     }
@@ -177,7 +176,7 @@ public class PayloadTermQuery extends Sp
     
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
+      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Sun Mar 16 19:39:10 2014
@@ -103,7 +103,7 @@ public class NearSpansOrdered extends Sp
     allowedSlop = spanNearQuery.getSlop();
     SpanQuery[] clauses = spanNearQuery.getClauses();
     subSpans = new Spans[clauses.length];
-    matchPayload = new LinkedList<byte[]>();
+    matchPayload = new LinkedList<>();
     subSpansByDoc = new Spans[clauses.length];
     for (int i = 0; i < clauses.length; i++) {
       subSpans[i] = clauses[i].getSpans(context, acceptDocs, termContexts);
@@ -282,7 +282,7 @@ public class NearSpansOrdered extends Sp
   private boolean shrinkToAfterShortestMatch() throws IOException {
     matchStart = subSpans[subSpans.length - 1].start();
     matchEnd = subSpans[subSpans.length - 1].end();
-    Set<byte[]> possibleMatchPayloads = new HashSet<byte[]>();
+    Set<byte[]> possibleMatchPayloads = new HashSet<>();
     if (subSpans[subSpans.length - 1].isPayloadAvailable()) {
       possibleMatchPayloads.addAll(subSpans[subSpans.length - 1].getPayload());
     }
@@ -296,7 +296,7 @@ public class NearSpansOrdered extends Sp
       Spans prevSpans = subSpans[i];
       if (collectPayloads && prevSpans.isPayloadAvailable()) {
         Collection<byte[]> payload = prevSpans.getPayload();
-        possiblePayload = new ArrayList<byte[]>(payload.size());
+        possiblePayload = new ArrayList<>(payload.size());
         possiblePayload.addAll(payload);
       }
       
@@ -320,7 +320,7 @@ public class NearSpansOrdered extends Sp
             prevEnd = ppEnd;
             if (collectPayloads && prevSpans.isPayloadAvailable()) {
               Collection<byte[]> payload = prevSpans.getPayload();
-              possiblePayload = new ArrayList<byte[]>(payload.size());
+              possiblePayload = new ArrayList<>(payload.size());
               possiblePayload.addAll(payload);
             }
           }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Sun Mar 16 19:39:10 2014
@@ -40,7 +40,7 @@ import java.util.HashSet;
 public class NearSpansUnordered extends Spans {
   private SpanNearQuery query;
 
-  private List<SpansCell> ordered = new ArrayList<SpansCell>();         // spans in query order
+  private List<SpansCell> ordered = new ArrayList<>();         // spans in query order
   private Spans[] subSpans;  
   private int slop;                               // from query
 
@@ -121,7 +121,7 @@ public class NearSpansUnordered extends 
                     // TODO: Remove warning after API has been finalized
     @Override
     public Collection<byte[]> getPayload() throws IOException {
-      return new ArrayList<byte[]>(spans.getPayload());
+      return new ArrayList<>(spans.getPayload());
     }
 
     // TODO: Remove warning after API has been finalized
@@ -250,7 +250,7 @@ public class NearSpansUnordered extends 
    */
   @Override
   public Collection<byte[]> getPayload() throws IOException {
-    Set<byte[]> matchPayload = new HashSet<byte[]>();
+    Set<byte[]> matchPayload = new HashSet<>();
     for (SpansCell cell = first; cell != null; cell = cell.next) {
       if (cell.isPayloadAvailable()) {
         matchPayload.addAll(cell.getPayload());

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java Sun Mar 16 19:39:10 2014
@@ -61,7 +61,7 @@ public class SpanNearQuery extends SpanQ
   public SpanNearQuery(SpanQuery[] clauses, int slop, boolean inOrder, boolean collectPayloads) {
 
     // copy clauses array into an ArrayList
-    this.clauses = new ArrayList<SpanQuery>(clauses.length);
+    this.clauses = new ArrayList<>(clauses.length);
     for (int i = 0; i < clauses.length; i++) {
       SpanQuery clause = clauses[i];
       if (field == null) {                               // check field

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java Sun Mar 16 19:39:10 2014
@@ -175,7 +175,7 @@ public class SpanNotQuery extends SpanQu
       public Collection<byte[]> getPayload() throws IOException {
         ArrayList<byte[]> result = null;
         if (includeSpans.isPayloadAvailable()) {
-          result = new ArrayList<byte[]>(includeSpans.getPayload());
+          result = new ArrayList<>(includeSpans.getPayload());
         }
         return result;
       }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java Sun Mar 16 19:39:10 2014
@@ -44,7 +44,7 @@ public class SpanOrQuery extends SpanQue
   public SpanOrQuery(SpanQuery... clauses) {
 
     // copy clauses array into an ArrayList
-    this.clauses = new ArrayList<SpanQuery>(clauses.length);
+    this.clauses = new ArrayList<>(clauses.length);
     for (int i = 0; i < clauses.length; i++) {
       addClause(clauses[i]);
     }
@@ -242,7 +242,7 @@ public class SpanOrQuery extends SpanQue
         ArrayList<byte[]> result = null;
         Spans theTop = top();
         if (theTop != null && theTop.isPayloadAvailable()) {
-          result = new ArrayList<byte[]>(theTop.getPayload());
+          result = new ArrayList<>(theTop.getPayload());
         }
         return result;
       }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java Sun Mar 16 19:39:10 2014
@@ -169,7 +169,7 @@ public abstract class SpanPositionCheckQ
     public Collection<byte[]> getPayload() throws IOException {
       ArrayList<byte[]> result = null;
       if (spans.isPayloadAvailable()) {
-        result = new ArrayList<byte[]>(spans.getPayload());
+        result = new ArrayList<>(spans.getPayload());
       }
       return result;//TODO: any way to avoid the new construction?
     }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java Sun Mar 16 19:39:10 2014
@@ -45,8 +45,8 @@ public class SpanWeight extends Weight {
     this.similarity = searcher.getSimilarity();
     this.query = query;
     
-    termContexts = new HashMap<Term,TermContext>();
-    TreeSet<Term> terms = new TreeSet<Term>();
+    termContexts = new HashMap<>();
+    TreeSet<Term> terms = new TreeSet<>();
     query.extractTerms(terms);
     final IndexReaderContext context = searcher.getTopReaderContext();
     final TermStatistics termStats[] = new TermStatistics[terms.size()];
@@ -81,8 +81,7 @@ public class SpanWeight extends Weight {
   }
 
   @Override
-  public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-      boolean topScorer, Bits acceptDocs) throws IOException {
+  public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
     if (stats == null) {
       return null;
     } else {
@@ -92,7 +91,7 @@ public class SpanWeight extends Weight {
 
   @Override
   public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-    SpanScorer scorer = (SpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
+    SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs());
     if (scorer != null) {
       int newDoc = scorer.advance(doc);
       if (newDoc == doc) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java Sun Mar 16 19:39:10 2014
@@ -130,7 +130,7 @@ public final class CompoundFileDirectory
       entriesStream = dir.openInput(entriesFileName, IOContext.READONCE);
       CodecUtil.checkHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_START);
       final int numEntries = entriesStream.readVInt();
-      final Map<String, FileEntry> mapping = new HashMap<String,FileEntry>(numEntries);
+      final Map<String, FileEntry> mapping = new HashMap<>(numEntries);
       for (int i = 0; i < numEntries; i++) {
         final FileEntry fileEntry = new FileEntry();
         final String id = entriesStream.readString();

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java Sun Mar 16 19:39:10 2014
@@ -60,10 +60,10 @@ final class CompoundFileWriter implement
   static final String ENTRY_CODEC = "CompoundFileWriterEntries";
 
   private final Directory directory;
-  private final Map<String, FileEntry> entries = new HashMap<String, FileEntry>();
-  private final Set<String> seenIDs = new HashSet<String>();
+  private final Map<String, FileEntry> entries = new HashMap<>();
+  private final Set<String> seenIDs = new HashSet<>();
   // all entries that are written to a sep. file but not yet moved into CFS
-  private final Queue<FileEntry> pendingEntries = new LinkedList<FileEntry>();
+  private final Queue<FileEntry> pendingEntries = new LinkedList<>();
   private boolean closed = false;
   private IndexOutput dataOut;
   private final AtomicBoolean outputTaken = new AtomicBoolean(false);

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/DataInput.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/DataInput.java Sun Mar 16 19:39:10 2014
@@ -212,7 +212,7 @@ public abstract class DataInput implemen
   /** Reads a Map&lt;String,String&gt; previously written
    *  with {@link DataOutput#writeStringStringMap(Map)}. */
   public Map<String,String> readStringStringMap() throws IOException {
-    final Map<String,String> map = new HashMap<String,String>();
+    final Map<String,String> map = new HashMap<>();
     final int count = readInt();
     for(int i=0;i<count;i++) {
       final String key = readString();
@@ -226,7 +226,7 @@ public abstract class DataInput implemen
   /** Reads a Set&lt;String&gt; previously written
    *  with {@link DataOutput#writeStringSet(Set)}. */
   public Set<String> readStringSet() throws IOException {
-    final Set<String> set = new HashSet<String>();
+    final Set<String> set = new HashSet<>();
     final int count = readInt();
     for(int i=0;i<count;i++) {
       set.add(readString());

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Sun Mar 16 19:39:10 2014
@@ -295,7 +295,7 @@ public abstract class FSDirectory extend
   @Override
   public void sync(Collection<String> names) throws IOException {
     ensureOpen();
-    Set<String> toSync = new HashSet<String>(names);
+    Set<String> toSync = new HashSet<>(names);
     toSync.retainAll(staleFiles);
 
     for (String name : toSync)

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java Sun Mar 16 19:39:10 2014
@@ -77,7 +77,7 @@ public class FileSwitchDirectory extends
   
   @Override
   public String[] listAll() throws IOException {
-    Set<String> files = new HashSet<String>();
+    Set<String> files = new HashSet<>();
     // LUCENE-3380: either or both of our dirs could be FSDirs,
     // but if one underlying delegate is an FSDir and mkdirs() has not
     // yet been called, because so far everything is written to the other,
@@ -154,8 +154,8 @@ public class FileSwitchDirectory extends
 
   @Override
   public void sync(Collection<String> names) throws IOException {
-    List<String> primaryNames = new ArrayList<String>();
-    List<String> secondaryNames = new ArrayList<String>();
+    List<String> primaryNames = new ArrayList<>();
+    List<String> secondaryNames = new ArrayList<>();
 
     for (String name : names)
       if (primaryExtensions.contains(getExtension(name)))

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java Sun Mar 16 19:39:10 2014
@@ -120,7 +120,7 @@ public class NRTCachingDirectory extends
 
   @Override
   public synchronized String[] listAll() throws IOException {
-    final Set<String> files = new HashSet<String>();
+    final Set<String> files = new HashSet<>();
     for(String f : cache.listAll()) {
       files.add(f);
     }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java Sun Mar 16 19:39:10 2014
@@ -146,7 +146,7 @@ class NativeFSLock extends Lock {
    * (same FileChannel instance or not), so we may want to 
    * change this when Lucene moves to Java 1.6.
    */
-  private static HashSet<String> LOCK_HELD = new HashSet<String>();
+  private static HashSet<String> LOCK_HELD = new HashSet<>();
 
   public NativeFSLock(File lockDir, String lockFileName) {
     this.lockDir = lockDir;

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java Sun Mar 16 19:39:10 2014
@@ -46,7 +46,7 @@ import java.util.concurrent.atomic.Atomi
  * operating system, so copying data to Java heap space is not useful.
  */
 public class RAMDirectory extends BaseDirectory {
-  protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
+  protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<>();
   protected final AtomicLong sizeInBytes = new AtomicLong();
   
   // *****
@@ -113,7 +113,7 @@ public class RAMDirectory extends BaseDi
     // NOTE: fileMap.keySet().toArray(new String[0]) is broken in non Sun JDKs,
     // and the code below is resilient to map changes during the array population.
     Set<String> fileNames = fileMap.keySet();
-    List<String> names = new ArrayList<String>(fileNames.size());
+    List<String> names = new ArrayList<>(fileNames.size());
     for (String name : fileNames) names.add(name);
     return names.toArray(new String[names.size()]);
   }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMFile.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/RAMFile.java Sun Mar 16 19:39:10 2014
@@ -23,7 +23,7 @@ import java.util.ArrayList;
  * Represents a file in RAM as a list of byte[] buffers.
  * @lucene.internal */
 public class RAMFile {
-  protected ArrayList<byte[]> buffers = new ArrayList<byte[]>();
+  protected ArrayList<byte[]> buffers = new ArrayList<>();
   long length;
   RAMDirectory directory;
   protected long sizeInBytes;

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java Sun Mar 16 19:39:10 2014
@@ -33,7 +33,7 @@ import java.util.HashSet;
 
 public class SingleInstanceLockFactory extends LockFactory {
 
-  private HashSet<String> locks = new HashSet<String>();
+  private HashSet<String> locks = new HashSet<>();
 
   @Override
   public Lock makeLock(String lockName) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/AttributeSource.java Sun Mar 16 19:39:10 2014
@@ -145,8 +145,8 @@ public class AttributeSource {
    * An AttributeSource using the supplied {@link AttributeFactory} for creating new {@link Attribute} instances.
    */
   public AttributeSource(AttributeFactory factory) {
-    this.attributes = new LinkedHashMap<Class<? extends Attribute>, AttributeImpl>();
-    this.attributeImpls = new LinkedHashMap<Class<? extends AttributeImpl>, AttributeImpl>();
+    this.attributes = new LinkedHashMap<>();
+    this.attributeImpls = new LinkedHashMap<>();
     this.currentState = new State[1];
     this.factory = factory;
   }
@@ -207,7 +207,7 @@ public class AttributeSource {
     LinkedList<WeakReference<Class<? extends Attribute>>> foundInterfaces = knownImplClasses.get(clazz);
     if (foundInterfaces == null) {
       // we have the slight chance that another thread may do the same, but who cares?
-      foundInterfaces = new LinkedList<WeakReference<Class<? extends Attribute>>>();
+      foundInterfaces = new LinkedList<>();
       // find all interfaces that this attribute instance implements
       // and that extend the Attribute interface
       Class<?> actClazz = clazz;

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java Sun Mar 16 19:39:10 2014
@@ -55,11 +55,11 @@ import java.util.concurrent.atomic.Atomi
 
 public class CloseableThreadLocal<T> implements Closeable {
 
-  private ThreadLocal<WeakReference<T>> t = new ThreadLocal<WeakReference<T>>();
+  private ThreadLocal<WeakReference<T>> t = new ThreadLocal<>();
 
   // Use a WeakHashMap so that if a Thread exits and is
   // GC'able, its entry may be removed:
-  private Map<Thread,T> hardRefs = new WeakHashMap<Thread,T>();
+  private Map<Thread,T> hardRefs = new WeakHashMap<>();
   
   // Increase this to decrease frequency of purging in get:
   private static int PURGE_MULTIPLIER = 20;
@@ -92,7 +92,7 @@ public class CloseableThreadLocal<T> imp
 
   public void set(T object) {
 
-    t.set(new WeakReference<T>(object));
+    t.set(new WeakReference<>(object));
 
     synchronized(hardRefs) {
       hardRefs.put(Thread.currentThread(), object);

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Constants.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Constants.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Constants.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Constants.java Sun Mar 16 19:39:10 2014
@@ -53,11 +53,6 @@ public final class Constants {
   public static final String OS_VERSION = System.getProperty("os.version");
   public static final String JAVA_VENDOR = System.getProperty("java.vendor");
 
-  /** @deprecated With Lucene 5.0, we are always on Java 7 */
-  @Deprecated
-  public static final boolean JRE_IS_MINIMUM_JAVA7 =
-    new Boolean(true).booleanValue(); // prevent inlining in foreign class files
-
   public static final boolean JRE_IS_MINIMUM_JAVA8;
   
   /** True iff running on a 64bit JVM */

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java Sun Mar 16 19:39:10 2014
@@ -59,8 +59,8 @@ final public class DoubleBarrelLRUCache<
   public DoubleBarrelLRUCache(int maxSize) {
     this.maxSize = maxSize;
     countdown = new AtomicInteger(maxSize);
-    cache1 = new ConcurrentHashMap<K,V>();
-    cache2 = new ConcurrentHashMap<K,V>();
+    cache1 = new ConcurrentHashMap<>();
+    cache2 = new ConcurrentHashMap<>();
   }
 
   @SuppressWarnings("unchecked") 

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java Sun Mar 16 19:39:10 2014
@@ -109,13 +109,13 @@ public final class FieldCacheSanityCheck
     //
     // maps the (valId) identityhashCode of cache values to 
     // sets of CacheEntry instances
-    final MapOfSets<Integer, CacheEntry> valIdToItems = new MapOfSets<Integer, CacheEntry>(new HashMap<Integer, Set<CacheEntry>>(17));
+    final MapOfSets<Integer, CacheEntry> valIdToItems = new MapOfSets<>(new HashMap<Integer, Set<CacheEntry>>(17));
     // maps ReaderField keys to Sets of ValueIds
-    final MapOfSets<ReaderField, Integer> readerFieldToValIds = new MapOfSets<ReaderField, Integer>(new HashMap<ReaderField, Set<Integer>>(17));
+    final MapOfSets<ReaderField, Integer> readerFieldToValIds = new MapOfSets<>(new HashMap<ReaderField, Set<Integer>>(17));
     //
 
     // any keys that we know result in more then one valId
-    final Set<ReaderField> valMismatchKeys = new HashSet<ReaderField>();
+    final Set<ReaderField> valMismatchKeys = new HashSet<>();
 
     // iterate over all the cacheEntries to get the mappings we'll need
     for (int i = 0; i < cacheEntries.length; i++) {
@@ -144,7 +144,7 @@ public final class FieldCacheSanityCheck
       }
     }
 
-    final List<Insanity> insanity = new ArrayList<Insanity>(valMismatchKeys.size() * 3);
+    final List<Insanity> insanity = new ArrayList<>(valMismatchKeys.size() * 3);
 
     insanity.addAll(checkValueMismatch(valIdToItems, 
                                        readerFieldToValIds, 
@@ -166,7 +166,7 @@ public final class FieldCacheSanityCheck
                                         MapOfSets<ReaderField, Integer> readerFieldToValIds,
                                         Set<ReaderField> valMismatchKeys) {
 
-    final List<Insanity> insanity = new ArrayList<Insanity>(valMismatchKeys.size() * 3);
+    final List<Insanity> insanity = new ArrayList<>(valMismatchKeys.size() * 3);
 
     if (! valMismatchKeys.isEmpty() ) { 
       // we have multiple values for some ReaderFields
@@ -174,7 +174,7 @@ public final class FieldCacheSanityCheck
       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);
+        final List<CacheEntry> badEntries = new ArrayList<>(valMismatchKeys.size() * 2);
         for(final Integer value: rfMap.get(rf)) {
           for (final CacheEntry cacheEntry : valMap.get(value)) {
             badEntries.add(cacheEntry);
@@ -203,15 +203,15 @@ public final class FieldCacheSanityCheck
   private Collection<Insanity> checkSubreaders( MapOfSets<Integer, CacheEntry>  valIdToItems,
                                       MapOfSets<ReaderField, Integer> readerFieldToValIds) {
 
-    final List<Insanity> insanity = new ArrayList<Insanity>(23);
+    final List<Insanity> insanity = new ArrayList<>(23);
 
-    Map<ReaderField, Set<ReaderField>> badChildren = new HashMap<ReaderField, Set<ReaderField>>(17);
-    MapOfSets<ReaderField, ReaderField> badKids = new MapOfSets<ReaderField, ReaderField>(badChildren); // wrapper
+    Map<ReaderField, Set<ReaderField>> badChildren = new HashMap<>(17);
+    MapOfSets<ReaderField, ReaderField> badKids = new MapOfSets<>(badChildren); // wrapper
 
     Map<Integer, Set<CacheEntry>> viToItemSets = valIdToItems.getMap();
     Map<ReaderField, Set<Integer>> rfToValIdSets = readerFieldToValIds.getMap();
 
-    Set<ReaderField> seen = new HashSet<ReaderField>(17);
+    Set<ReaderField> seen = new HashSet<>(17);
 
     Set<ReaderField> readerFields = rfToValIdSets.keySet();
     for (final ReaderField rf : readerFields) {
@@ -242,7 +242,7 @@ public final class FieldCacheSanityCheck
     for (final ReaderField parent : badChildren.keySet()) {
       Set<ReaderField> kids = badChildren.get(parent);
 
-      List<CacheEntry> badEntries = new ArrayList<CacheEntry>(kids.size() * 2);
+      List<CacheEntry> badEntries = new ArrayList<>(kids.size() * 2);
 
       // put parent entr(ies) in first
       {
@@ -277,7 +277,7 @@ public final class FieldCacheSanityCheck
    * returned by {@code seed.getCoreCacheKey()}
    */
   private List<Object> getAllDescendantReaderKeys(Object seed) {
-    List<Object> all = new ArrayList<Object>(17); // will grow as we iter
+    List<Object> all = new ArrayList<>(17); // will grow as we iter
     all.add(seed);
     for (int i = 0; i < all.size(); i++) {
       final Object obj = all.get(i);

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java Sun Mar 16 19:39:10 2014
@@ -55,7 +55,7 @@ public class MapOfSets<K, V> {
     if (theMap.containsKey(key)) {
       theSet = theMap.get(key);
     } else {
-      theSet = new HashSet<V>(23);
+      theSet = new HashSet<>(23);
       theMap.put(key, theSet);
     }
     theSet.add(val);
@@ -72,7 +72,7 @@ public class MapOfSets<K, V> {
     if (theMap.containsKey(key)) {
       theSet = theMap.get(key);
     } else {
-      theSet = new HashSet<V>(23);
+      theSet = new HashSet<>(23);
       theMap.put(key, theSet);
     }
     theSet.addAll(vals);

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java Sun Mar 16 19:39:10 2014
@@ -59,12 +59,12 @@ public final class MergedIterator<T exte
   @SuppressWarnings({"unchecked","rawtypes"})
   public MergedIterator(boolean removeDuplicates, Iterator<T>... iterators) {
     this.removeDuplicates = removeDuplicates;
-    queue = new TermMergeQueue<T>(iterators.length);
+    queue = new TermMergeQueue<>(iterators.length);
     top = new SubIterator[iterators.length];
     int index = 0;
     for (Iterator<T> iterator : iterators) {
       if (iterator.hasNext()) {
-        SubIterator<T> sub = new SubIterator<T>();
+        SubIterator<T> sub = new SubIterator<>();
         sub.current = iterator.next();
         sub.iterator = iterator;
         sub.index = index++;

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/NamedSPILoader.java Sun Mar 16 19:39:10 2014
@@ -59,7 +59,7 @@ public final class NamedSPILoader<S exte
    * of new service providers on the given classpath/classloader!</em>
    */
   public synchronized void reload(ClassLoader classloader) {
-    final LinkedHashMap<String,S> services = new LinkedHashMap<String,S>(this.services);
+    final LinkedHashMap<String,S> services = new LinkedHashMap<>(this.services);
     final SPIClassIterator<S> loader = SPIClassIterator.get(clazz, classloader);
     while (loader.hasNext()) {
       final Class<? extends S> c = loader.next();

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java Sun Mar 16 19:39:10 2014
@@ -225,7 +225,7 @@ public final class OfflineSorter {
 
     output.delete();
 
-    ArrayList<File> merges = new ArrayList<File>();
+    ArrayList<File> merges = new ArrayList<>();
     boolean success2 = false;
     try {
       ByteSequencesReader is = new ByteSequencesReader(input);

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PagedBytes.java Sun Mar 16 19:39:10 2014
@@ -35,9 +35,9 @@ import org.apache.lucene.store.IndexInpu
 // TODO: refactor this, byteblockpool, fst.bytestore, and any
 // other "shift/mask big arrays". there are too many of these classes!
 public final class PagedBytes {
-  private final List<byte[]> blocks = new ArrayList<byte[]>();
+  private final List<byte[]> blocks = new ArrayList<>();
   // TODO: these are unused?
-  private final List<Integer> blockEnd = new ArrayList<Integer>();
+  private final List<Integer> blockEnd = new ArrayList<>();
   private final int blockSize;
   private final int blockBits;
   private final int blockMask;

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java Sun Mar 16 19:39:10 2014
@@ -29,7 +29,7 @@ package org.apache.lucene.util;
  * @lucene.internal
 */
 public abstract class PriorityQueue<T> {
-  private int size;
+  private int size = 0;
   private final int maxSize;
   private final T[] heap;
 
@@ -37,10 +37,8 @@ public abstract class PriorityQueue<T> {
     this(maxSize, true);
   }
 
-  @SuppressWarnings("unchecked")
   public PriorityQueue(int maxSize, boolean prepopulate) {
-    size = 0;
-    int heapSize;
+    final int heapSize;
     if (0 == maxSize) {
       // We allocate 1 extra to avoid if statement in top()
       heapSize = 2;
@@ -62,7 +60,9 @@ public abstract class PriorityQueue<T> {
         heapSize = maxSize + 1;
       }
     }
-    heap = (T[]) new Object[heapSize]; // T is unbounded type, so this unchecked cast works always
+    // T is unbounded type, so this unchecked cast works always:
+    @SuppressWarnings("unchecked") final T[] h = (T[]) new Object[heapSize];
+    this.heap = h;
     this.maxSize = maxSize;
     
     if (prepopulate) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java Sun Mar 16 19:39:10 2014
@@ -305,7 +305,7 @@ public class QueryBuilder {
           // phrase query:
           MultiPhraseQuery mpq = newMultiPhraseQuery();
           mpq.setSlop(phraseSlop);
-          List<Term> multiTerms = new ArrayList<Term>();
+          List<Term> multiTerms = new ArrayList<>();
           int position = -1;
           for (int i = 0; i < numTokens; i++) {
             int positionIncrement = 1;

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/RamUsageEstimator.java Sun Mar 16 19:39:10 2014
@@ -105,7 +105,7 @@ public final class RamUsageEstimator {
    */
   private static final Map<Class<?>,Integer> primitiveSizes;
   static {
-    primitiveSizes = new IdentityHashMap<Class<?>,Integer>();
+    primitiveSizes = new IdentityHashMap<>();
     primitiveSizes.put(boolean.class, Integer.valueOf(NUM_BYTES_BOOLEAN));
     primitiveSizes.put(byte.class, Integer.valueOf(NUM_BYTES_BYTE));
     primitiveSizes.put(char.class, Integer.valueOf(NUM_BYTES_CHAR));
@@ -403,11 +403,11 @@ public final class RamUsageEstimator {
    */
   private static long measureObjectSize(Object root) {
     // Objects seen so far.
-    final IdentityHashSet<Object> seen = new IdentityHashSet<Object>();
+    final IdentityHashSet<Object> seen = new IdentityHashSet<>();
     // Class cache with reference Field and precalculated shallow size. 
-    final IdentityHashMap<Class<?>, ClassCache> classCache = new IdentityHashMap<Class<?>, ClassCache>();
+    final IdentityHashMap<Class<?>, ClassCache> classCache = new IdentityHashMap<>();
     // Stack of objects pending traversal. Recursion caused stack overflows. 
-    final ArrayList<Object> stack = new ArrayList<Object>();
+    final ArrayList<Object> stack = new ArrayList<>();
     stack.add(root);
 
     long totalSize = 0;
@@ -486,7 +486,7 @@ public final class RamUsageEstimator {
   private static ClassCache createCacheEntry(final Class<?> clazz) {
     ClassCache cachedInfo;
     long shallowInstanceSize = NUM_BYTES_OBJECT_HEADER;
-    final ArrayList<Field> referenceFields = new ArrayList<Field>(32);
+    final ArrayList<Field> referenceFields = new ArrayList<>(32);
     for (Class<?> c = clazz; c != null; c = c.getSuperclass()) {
       final Field[] fields = c.getDeclaredFields();
       for (final Field f : fields) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SPIClassIterator.java Sun Mar 16 19:39:10 2014
@@ -47,11 +47,11 @@ public final class SPIClassIterator<S> i
   private Iterator<String> linesIterator;
   
   public static <S> SPIClassIterator<S> get(Class<S> clazz) {
-    return new SPIClassIterator<S>(clazz, Thread.currentThread().getContextClassLoader());
+    return new SPIClassIterator<>(clazz, Thread.currentThread().getContextClassLoader());
   }
   
   public static <S> SPIClassIterator<S> get(Class<S> clazz, ClassLoader loader) {
-    return new SPIClassIterator<S>(clazz, loader);
+    return new SPIClassIterator<>(clazz, loader);
   }
   
   /** Utility method to check if some class loader is a (grand-)parent of or the same as another one.
@@ -84,7 +84,7 @@ public final class SPIClassIterator<S> i
       if (lines != null) {
         lines.clear();
       } else {
-        lines = new ArrayList<String>();
+        lines = new ArrayList<>();
       }
       final URL url = profilesEnum.nextElement();
       try {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SetOnce.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SetOnce.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SetOnce.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/SetOnce.java Sun Mar 16 19:39:10 2014
@@ -77,7 +77,7 @@ public final class SetOnce<T> implements
   
   @Override
   public SetOnce<T> clone() {
-    return obj == null ? new SetOnce<T>() : new SetOnce<T>(obj);
+    return obj == null ? new SetOnce<T>() : new SetOnce<>(obj);
   }
   
 }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Version.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/Version.java Sun Mar 16 19:39:10 2014
@@ -88,6 +88,13 @@ public enum Version { 
    */
   @Deprecated
   LUCENE_47,
+  
+  /**
+   * Match settings and bugs in Lucene's 4.8 release.
+   * @deprecated (5.0) Use latest
+   */
+  @Deprecated
+  LUCENE_48,
 
   /** Match settings and bugs in Lucene's 5.0 release.
    *  <p>

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java Sun Mar 16 19:39:10 2014
@@ -62,7 +62,7 @@ import java.util.concurrent.ConcurrentHa
  * @lucene.internal
  */
 public final class WeakIdentityMap<K,V> {
-  private final ReferenceQueue<Object> queue = new ReferenceQueue<Object>();
+  private final ReferenceQueue<Object> queue = new ReferenceQueue<>();
   private final Map<IdentityWeakReference, V> backingStore;
   private final boolean reapOnRead;
 
@@ -80,7 +80,7 @@ public final class WeakIdentityMap<K,V> 
    * @param reapOnRead controls if the map <a href="#reapInfo">cleans up the reference queue on every read operation</a>.
    */
   public static <K,V> WeakIdentityMap<K,V> newHashMap(boolean reapOnRead) {
-    return new WeakIdentityMap<K,V>(new HashMap<IdentityWeakReference,V>(), reapOnRead);
+    return new WeakIdentityMap<>(new HashMap<IdentityWeakReference,V>(), reapOnRead);
   }
 
   /**
@@ -96,7 +96,7 @@ public final class WeakIdentityMap<K,V> 
    * @param reapOnRead controls if the map <a href="#reapInfo">cleans up the reference queue on every read operation</a>.
    */
   public static <K,V> WeakIdentityMap<K,V> newConcurrentHashMap(boolean reapOnRead) {
-    return new WeakIdentityMap<K,V>(new ConcurrentHashMap<IdentityWeakReference,V>(), reapOnRead);
+    return new WeakIdentityMap<>(new ConcurrentHashMap<IdentityWeakReference,V>(), reapOnRead);
   }
 
   /** Private only constructor, to create use the static factory methods. */

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java Sun Mar 16 19:39:10 2014
@@ -269,8 +269,8 @@ public class Automaton implements Clonea
   public State[] getNumberedStates() {
     if (numberedStates == null) {
       expandSingleton();
-      final Set<State> visited = new HashSet<State>();
-      final LinkedList<State> worklist = new LinkedList<State>();
+      final Set<State> visited = new HashSet<>();
+      final LinkedList<State> worklist = new LinkedList<>();
       numberedStates = new State[4];
       int upto = 0;
       worklist.add(initial);
@@ -333,9 +333,9 @@ public class Automaton implements Clonea
    */
   public Set<State> getAcceptStates() {
     expandSingleton();
-    HashSet<State> accepts = new HashSet<State>();
-    HashSet<State> visited = new HashSet<State>();
-    LinkedList<State> worklist = new LinkedList<State>();
+    HashSet<State> accepts = new HashSet<>();
+    HashSet<State> visited = new HashSet<>();
+    LinkedList<State> worklist = new LinkedList<>();
     worklist.add(initial);
     visited.add(initial);
     while (worklist.size() > 0) {
@@ -399,7 +399,7 @@ public class Automaton implements Clonea
    */
   int[] getStartPoints() {
     final State[] states = getNumberedStates();
-    Set<Integer> pointset = new HashSet<Integer>();
+    Set<Integer> pointset = new HashSet<>();
     pointset.add(Character.MIN_CODE_POINT);
     for (State s : states) {
       for (Transition t : s.getTransitions()) {
@@ -423,7 +423,7 @@ public class Automaton implements Clonea
    */
   private State[] getLiveStates() {
     final State[] states = getNumberedStates();
-    Set<State> live = new HashSet<State>();
+    Set<State> live = new HashSet<>();
     for (State q : states) {
       if (q.isAccept()) {
         live.add(q);
@@ -432,13 +432,13 @@ public class Automaton implements Clonea
     // map<state, set<state>>
     @SuppressWarnings({"rawtypes","unchecked"}) Set<State> map[] = new Set[states.length];
     for (int i = 0; i < map.length; i++)
-      map[i] = new HashSet<State>();
+      map[i] = new HashSet<>();
     for (State s : states) {
       for(int i=0;i<s.numTransitions;i++) {
         map[s.transitionsArray[i].to.number].add(s);
       }
     }
-    LinkedList<State> worklist = new LinkedList<State>(live);
+    LinkedList<State> worklist = new LinkedList<>(live);
     while (worklist.size() > 0) {
       State s = worklist.removeFirst();
       for (State p : map[s.number])
@@ -639,7 +639,7 @@ public class Automaton implements Clonea
     try {
       Automaton a = (Automaton) super.clone();
       if (!isSingleton()) {
-        HashMap<State,State> m = new HashMap<State,State>();
+        HashMap<State,State> m = new HashMap<>();
         State[] states = getNumberedStates();
         for (State s : states)
           m.put(s, new State());

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicAutomata.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicAutomata.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicAutomata.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicAutomata.java Sun Mar 16 19:39:10 2014
@@ -216,10 +216,10 @@ final public class BasicAutomata {
       by.append('0');
     by.append(y);
     y = by.toString();
-    Collection<State> initials = new ArrayList<State>();
+    Collection<State> initials = new ArrayList<>();
     a.initial = between(x, y, 0, initials, digits <= 0);
     if (digits <= 0) {
-      ArrayList<StatePair> pairs = new ArrayList<StatePair>();
+      ArrayList<StatePair> pairs = new ArrayList<>();
       for (State p : initials)
         if (a.initial != p) pairs.add(new StatePair(a.initial, p));
       BasicOperations.addEpsilons(a, pairs);

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/BasicOperations.java Sun Mar 16 19:39:10 2014
@@ -106,7 +106,7 @@ final public class BasicOperations {
     } else {
       for (Automaton a : l)
         if (BasicOperations.isEmpty(a)) return BasicAutomata.makeEmpty();
-      Set<Integer> ids = new HashSet<Integer>();
+      Set<Integer> ids = new HashSet<>();
       for (Automaton a : l)
         ids.add(System.identityHashCode(a));
       boolean has_aliases = ids.size() != l.size();
@@ -187,7 +187,7 @@ final public class BasicOperations {
    */
   static public Automaton repeat(Automaton a, int min) {
     if (min == 0) return repeat(a);
-    List<Automaton> as = new ArrayList<Automaton>();
+    List<Automaton> as = new ArrayList<>();
     while (min-- > 0)
       as.add(a);
     as.add(repeat(a));
@@ -210,7 +210,7 @@ final public class BasicOperations {
     if (min == 0) b = BasicAutomata.makeEmptyString();
     else if (min == 1) b = a.clone();
     else {
-      List<Automaton> as = new ArrayList<Automaton>();
+      List<Automaton> as = new ArrayList<>();
       while (min-- > 0)
         as.add(a);
       b = concatenate(as);
@@ -287,8 +287,8 @@ final public class BasicOperations {
     Transition[][] transitions1 = a1.getSortedTransitions();
     Transition[][] transitions2 = a2.getSortedTransitions();
     Automaton c = new Automaton();
-    LinkedList<StatePair> worklist = new LinkedList<StatePair>();
-    HashMap<StatePair,StatePair> newstates = new HashMap<StatePair,StatePair>();
+    LinkedList<StatePair> worklist = new LinkedList<>();
+    HashMap<StatePair,StatePair> newstates = new HashMap<>();
     StatePair p = new StatePair(c.initial, a1.initial, a2.initial);
     worklist.add(p);
     newstates.put(p, p);
@@ -356,8 +356,8 @@ final public class BasicOperations {
     a2.determinize();
     Transition[][] transitions1 = a1.getSortedTransitions();
     Transition[][] transitions2 = a2.getSortedTransitions();
-    LinkedList<StatePair> worklist = new LinkedList<StatePair>();
-    HashSet<StatePair> visited = new HashSet<StatePair>();
+    LinkedList<StatePair> worklist = new LinkedList<>();
+    HashSet<StatePair> visited = new HashSet<>();
     StatePair p = new StatePair(a1.initial, a2.initial);
     worklist.add(p);
     visited.add(p);
@@ -431,7 +431,7 @@ final public class BasicOperations {
    * Complexity: linear in number of states.
    */
   public static Automaton union(Collection<Automaton> l) {
-    Set<Integer> ids = new HashSet<Integer>();
+    Set<Integer> ids = new HashSet<>();
     for (Automaton a : l)
       ids.add(System.identityHashCode(a));
     boolean has_aliases = ids.size() != l.size();
@@ -500,7 +500,7 @@ final public class BasicOperations {
     PointTransitions[] points = new PointTransitions[5];
 
     private final static int HASHMAP_CUTOVER = 30;
-    private final HashMap<Integer,PointTransitions> map = new HashMap<Integer,PointTransitions>();
+    private final HashMap<Integer,PointTransitions> map = new HashMap<>();
     private boolean useHash = false;
 
     private PointTransitions next(int point) {
@@ -597,8 +597,8 @@ final public class BasicOperations {
     a.initial = new State();
     SortedIntSet.FrozenIntSet initialset = new SortedIntSet.FrozenIntSet(initNumber, a.initial);
 
-    LinkedList<SortedIntSet.FrozenIntSet> worklist = new LinkedList<SortedIntSet.FrozenIntSet>();
-    Map<SortedIntSet.FrozenIntSet,State> newstate = new HashMap<SortedIntSet.FrozenIntSet,State>();
+    LinkedList<SortedIntSet.FrozenIntSet> worklist = new LinkedList<>();
+    Map<SortedIntSet.FrozenIntSet,State> newstate = new HashMap<>();
 
     worklist.add(initialset);
 
@@ -713,25 +713,25 @@ final public class BasicOperations {
    */
   public static void addEpsilons(Automaton a, Collection<StatePair> pairs) {
     a.expandSingleton();
-    HashMap<State,HashSet<State>> forward = new HashMap<State,HashSet<State>>();
-    HashMap<State,HashSet<State>> back = new HashMap<State,HashSet<State>>();
+    HashMap<State,HashSet<State>> forward = new HashMap<>();
+    HashMap<State,HashSet<State>> back = new HashMap<>();
     for (StatePair p : pairs) {
       HashSet<State> to = forward.get(p.s1);
       if (to == null) {
-        to = new HashSet<State>();
+        to = new HashSet<>();
         forward.put(p.s1, to);
       }
       to.add(p.s2);
       HashSet<State> from = back.get(p.s2);
       if (from == null) {
-        from = new HashSet<State>();
+        from = new HashSet<>();
         back.put(p.s2, from);
       }
       from.add(p.s1);
     }
     // calculate epsilon closure
-    LinkedList<StatePair> worklist = new LinkedList<StatePair>(pairs);
-    HashSet<StatePair> workset = new HashSet<StatePair>(pairs);
+    LinkedList<StatePair> worklist = new LinkedList<>(pairs);
+    HashSet<StatePair> workset = new HashSet<>(pairs);
     while (!worklist.isEmpty()) {
       StatePair p = worklist.removeFirst();
       workset.remove(p);
@@ -817,12 +817,12 @@ final public class BasicOperations {
       return p.accept;
     } else {
       State[] states = a.getNumberedStates();
-      LinkedList<State> pp = new LinkedList<State>();
-      LinkedList<State> pp_other = new LinkedList<State>();
+      LinkedList<State> pp = new LinkedList<>();
+      LinkedList<State> pp_other = new LinkedList<>();
       BitSet bb = new BitSet(states.length);
       BitSet bb_other = new BitSet(states.length);
       pp.add(a.initial);
-      ArrayList<State> dest = new ArrayList<State>();
+      ArrayList<State> dest = new ArrayList<>();
       boolean accept = a.initial.accept;
       for (int i = 0, c = 0; i < s.length(); i += Character.charCount(c)) {
         c = s.codePointAt(i);

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Sun Mar 16 19:39:10 2014
@@ -272,7 +272,7 @@ public class CompiledAutomaton {
       }
     }
 
-    final List<Integer> stack = new ArrayList<Integer>();
+    final List<Integer> stack = new ArrayList<>();
 
     int idx = 0;
     while (true) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/DaciukMihovAutomatonBuilder.java Sun Mar 16 19:39:10 2014
@@ -187,7 +187,7 @@ final class DaciukMihovAutomatonBuilder 
   /**
    * A "registry" for state interning.
    */
-  private HashMap<State,State> stateRegistry = new HashMap<State,State>();
+  private HashMap<State,State> stateRegistry = new HashMap<>();
   
   /**
    * Root automaton state.

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java Sun Mar 16 19:39:10 2014
@@ -63,7 +63,7 @@ public class LevenshteinAutomata {
     this.alphaMax = alphaMax;
 
     // calculate the alphabet
-    SortedSet<Integer> set = new TreeSet<Integer>();
+    SortedSet<Integer> set = new TreeSet<>();
     for (int i = 0; i < word.length; i++) {
       int v = word[i];
       if (v > alphaMax) {

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java Sun Mar 16 19:39:10 2014
@@ -83,13 +83,13 @@ final public class MinimizationOperation
     final int[] block = new int[statesLen];
     final StateList[][] active = new StateList[statesLen][sigmaLen];
     final StateListNode[][] active2 = new StateListNode[statesLen][sigmaLen];
-    final LinkedList<IntPair> pending = new LinkedList<IntPair>();
+    final LinkedList<IntPair> pending = new LinkedList<>();
     final BitSet pending2 = new BitSet(sigmaLen*statesLen);
     final BitSet split = new BitSet(statesLen), 
       refine = new BitSet(statesLen), refine2 = new BitSet(statesLen);
     for (int q = 0; q < statesLen; q++) {
-      splitblock[q] = new ArrayList<State>();
-      partition[q] = new HashSet<State>();
+      splitblock[q] = new ArrayList<>();
+      partition[q] = new HashSet<>();
       for (int x = 0; x < sigmaLen; x++) {
         active[q][x] = new StateList();
       }
@@ -104,7 +104,7 @@ final public class MinimizationOperation
         final ArrayList<State>[] r =
           reverse[qq.step(sigma[x]).number];
         if (r[x] == null)
-          r[x] = new ArrayList<State>();
+          r[x] = new ArrayList<>();
         r[x].add(qq);
       }
     }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java Sun Mar 16 19:39:10 2014
@@ -486,14 +486,14 @@ public class RegExp {
     Automaton a = null;
     switch (kind) {
       case REGEXP_UNION:
-        list = new ArrayList<Automaton>();
+        list = new ArrayList<>();
         findLeaves(exp1, Kind.REGEXP_UNION, list, automata, automaton_provider);
         findLeaves(exp2, Kind.REGEXP_UNION, list, automata, automaton_provider);
         a = BasicOperations.union(list);
         MinimizationOperations.minimize(a);
         break;
       case REGEXP_CONCATENATION:
-        list = new ArrayList<Automaton>();
+        list = new ArrayList<>();
         findLeaves(exp1, Kind.REGEXP_CONCATENATION, list, automata,
             automaton_provider);
         findLeaves(exp2, Kind.REGEXP_CONCATENATION, list, automata,
@@ -664,7 +664,7 @@ public class RegExp {
    * Returns set of automaton identifiers that occur in this regular expression.
    */
   public Set<String> getIdentifiers() {
-    HashSet<String> set = new HashSet<String>();
+    HashSet<String> set = new HashSet<>();
     getIdentifiers(set);
     return set;
   }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SortedIntSet.java Sun Mar 16 19:39:10 2014
@@ -35,7 +35,7 @@ final class SortedIntSet {
   // O(N^2) linear ops to O(N log(N)) TreeMap
   private final static int TREE_MAP_CUTOVER = 30;
 
-  private final Map<Integer,Integer> map = new TreeMap<Integer,Integer>();
+  private final Map<Integer,Integer> map = new TreeMap<>();
 
   private boolean useTreeMap;
 

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java Sun Mar 16 19:39:10 2014
@@ -95,7 +95,7 @@ final public class SpecialOperations {
   public static String getCommonPrefix(Automaton a) {
     if (a.isSingleton()) return a.singleton;
     StringBuilder b = new StringBuilder();
-    HashSet<State> visited = new HashSet<State>();
+    HashSet<State> visited = new HashSet<>();
     State s = a.initial;
     boolean done;
     do {
@@ -119,7 +119,7 @@ final public class SpecialOperations {
   public static BytesRef getCommonPrefixBytesRef(Automaton a) {
     if (a.isSingleton()) return new BytesRef(a.singleton);
     BytesRef ref = new BytesRef(10);
-    HashSet<State> visited = new HashSet<State>();
+    HashSet<State> visited = new HashSet<>();
     State s = a.initial;
     boolean done;
     do {
@@ -185,9 +185,9 @@ final public class SpecialOperations {
   public static Set<State> reverse(Automaton a) {
     a.expandSingleton();
     // reverse all edges
-    HashMap<State, HashSet<Transition>> m = new HashMap<State, HashSet<Transition>>();
+    HashMap<State, HashSet<Transition>> m = new HashMap<>();
     State[] states = a.getNumberedStates();
-    Set<State> accept = new HashSet<State>();
+    Set<State> accept = new HashSet<>();
     for (State s : states)
       if (s.isAccept())
         accept.add(s);
@@ -223,7 +223,7 @@ final public class SpecialOperations {
    * the limit is infinite.
    */
   public static Set<IntsRef> getFiniteStrings(Automaton a, int limit) {
-    HashSet<IntsRef> strings = new HashSet<IntsRef>();
+    HashSet<IntsRef> strings = new HashSet<>();
     if (a.isSingleton()) {
       if (limit > 0) {
         strings.add(Util.toUTF32(a.singleton, new IntsRef()));

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java Sun Mar 16 19:39:10 2014
@@ -261,7 +261,7 @@ public final class UTF32ToUTF8 {
     }
 
     State[] map = new State[utf32.getNumberedStates().length];
-    List<State> pending = new ArrayList<State>();
+    List<State> pending = new ArrayList<>();
     State utf32State = utf32.getInitialState();
     pending.add(utf32State);
     Automaton utf8 = new Automaton();

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java Sun Mar 16 19:39:10 2014
@@ -160,9 +160,9 @@ public class Builder<T> {
     this.shareMaxTailLength = shareMaxTailLength;
     this.doPackFST = doPackFST;
     this.acceptableOverheadRatio = acceptableOverheadRatio;
-    fst = new FST<T>(inputType, outputs, doPackFST, acceptableOverheadRatio, allowArrayArcs, bytesPageBits);
+    fst = new FST<>(inputType, outputs, doPackFST, acceptableOverheadRatio, allowArrayArcs, bytesPageBits);
     if (doShareSuffix) {
-      dedupHash = new NodeHash<T>(fst, fst.bytes.getReverseReader(false));
+      dedupHash = new NodeHash<>(fst, fst.bytes.getReverseReader(false));
     } else {
       dedupHash = null;
     }
@@ -172,7 +172,7 @@ public class Builder<T> {
         (UnCompiledNode<T>[]) new UnCompiledNode[10];
     frontier = f;
     for(int idx=0;idx<frontier.length;idx++) {
-      frontier[idx] = new UnCompiledNode<T>(this, idx);
+      frontier[idx] = new UnCompiledNode<>(this, idx);
     }
   }
 
@@ -301,7 +301,7 @@ public class Builder<T> {
             // undecided on whether to prune it.  later, it
             // will be either compiled or pruned, so we must
             // allocate a new node:
-            frontier[idx] = new UnCompiledNode<T>(this, idx);
+            frontier[idx] = new UnCompiledNode<>(this, idx);
           }
         }
       }
@@ -384,7 +384,7 @@ public class Builder<T> {
         new UnCompiledNode[ArrayUtil.oversize(input.length+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       System.arraycopy(frontier, 0, next, 0, frontier.length);
       for(int idx=frontier.length;idx<next.length;idx++) {
-        next[idx] = new UnCompiledNode<T>(this, idx);
+        next[idx] = new UnCompiledNode<>(this, idx);
       }
       frontier = next;
     }
@@ -553,7 +553,7 @@ public class Builder<T> {
     public UnCompiledNode(Builder<T> owner, int depth) {
       this.owner = owner;
       arcs = (Arc<T>[]) new Arc[1];
-      arcs[0] = new Arc<T>();
+      arcs[0] = new Arc<>();
       output = owner.NO_OUTPUT;
       this.depth = depth;
     }
@@ -587,7 +587,7 @@ public class Builder<T> {
           new Arc[ArrayUtil.oversize(numArcs+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
         System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
         for(int arcIdx=numArcs;arcIdx<newArcs.length;arcIdx++) {
-          newArcs[arcIdx] = new Arc<T>();
+          newArcs[arcIdx] = new Arc<>();
         }
         arcs = newArcs;
       }

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesRefFSTEnum.java Sun Mar 16 19:39:10 2014
@@ -30,7 +30,7 @@ import org.apache.lucene.util.BytesRef;
 
 public final class BytesRefFSTEnum<T> extends FSTEnum<T> {
   private final BytesRef current = new BytesRef(10);
-  private final InputOutput<T> result = new InputOutput<T>();
+  private final InputOutput<T> result = new InputOutput<>();
   private BytesRef target;
 
   /** Holds a single input (BytesRef) + output pair. */

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/BytesStore.java Sun Mar 16 19:39:10 2014
@@ -29,7 +29,7 @@ import org.apache.lucene.store.DataOutpu
 
 class BytesStore extends DataOutput {
 
-  private final List<byte[]> blocks = new ArrayList<byte[]>();
+  private final List<byte[]> blocks = new ArrayList<>();
 
   private final int blockSize;
   private final int blockBits;

Modified: lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1578144&r1=1578143&r2=1578144&view=diff
==============================================================================
--- lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene5376_2/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Sun Mar 16 19:39:10 2014
@@ -437,7 +437,7 @@ public final class FST<T> {
   }
   
   public void readRootArcs(Arc<T>[] arcs) throws IOException {
-    final Arc<T> arc = new Arc<T>();
+    final Arc<T> arc = new Arc<>();
     getFirstArc(arc);
     final BytesReader in = getBytesReader();
     if (targetHasArcs(arc)) {
@@ -592,7 +592,7 @@ public final class FST<T> {
     InputStream is = new BufferedInputStream(new FileInputStream(file));
     boolean success = false;
     try {
-      FST<T> fst = new FST<T>(new InputStreamDataInput(is), outputs);
+      FST<T> fst = new FST<>(new InputStreamDataInput(is), outputs);
       success = true;
       return fst;
     } finally {
@@ -1349,7 +1349,7 @@ public final class FST<T> {
     // TODO: must assert this FST was built with
     // "willRewrite"
 
-    final List<ArcAndState<T>> queue = new ArrayList<ArcAndState<T>>();
+    final List<ArcAndState<T>> queue = new ArrayList<>();
 
     // TODO: use bitset to not revisit nodes already
     // visited
@@ -1358,7 +1358,7 @@ public final class FST<T> {
     int saved = 0;
 
     queue.add(new ArcAndState<T>(getFirstArc(new Arc<T>()), new IntsRef()));
-    Arc<T> scratchArc = new Arc<T>();
+    Arc<T> scratchArc = new Arc<>();
     while(queue.size() > 0) {
       //System.out.println("cycle size=" + queue.size());
       //for(ArcAndState<T> ent : queue) {
@@ -1499,7 +1499,7 @@ public final class FST<T> {
       throw new IllegalArgumentException("this FST was not built with willPackFST=true");
     }
 
-    Arc<T> arc = new Arc<T>();
+    Arc<T> arc = new Arc<>();
 
     final BytesReader r = getBytesReader();
 
@@ -1526,7 +1526,7 @@ public final class FST<T> {
     // Free up RAM:
     inCounts = null;
 
-    final Map<Integer,Integer> topNodeMap = new HashMap<Integer,Integer>();
+    final Map<Integer,Integer> topNodeMap = new HashMap<>();
     for(int downTo=q.size()-1;downTo>=0;downTo--) {
       NodeAndInCount n = q.pop();
       topNodeMap.put(n.node, downTo);
@@ -1558,7 +1558,7 @@ public final class FST<T> {
       // for assert:
       boolean negDelta = false;
 
-      fst = new FST<T>(inputType, outputs, bytes.getBlockBits());
+      fst = new FST<>(inputType, outputs, bytes.getBlockBits());
       
       final BytesStore writer = fst.bytes;