You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/10/07 08:24:27 UTC

svn commit: r1529770 [2/2] - in /lucene/dev/trunk: lucene/analysis/common/src/java/org/apache/lucene/analysis/synonym/ lucene/analysis/common/src/test/org/apache/lucene/analysis/core/ lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellan...

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java Mon Oct  7 06:24:25 2013
@@ -449,252 +449,251 @@ public class FreeTextSuggester extends L
 
   /** Retrieve suggestions. */
   public List<LookupResult> lookup(final CharSequence key, int num) throws IOException {
-    TokenStream ts = queryAnalyzer.tokenStream("", key.toString());
-    TermToBytesRefAttribute termBytesAtt = ts.addAttribute(TermToBytesRefAttribute.class);
-    OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
-    PositionLengthAttribute posLenAtt = ts.addAttribute(PositionLengthAttribute.class);
-    PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
-    ts.reset();
-
-    BytesRef[] lastTokens = new BytesRef[grams];
-    //System.out.println("lookup: key='" + key + "'");
-
-    // Run full analysis, but save only the
-    // last 1gram, last 2gram, etc.:
-    BytesRef tokenBytes = termBytesAtt.getBytesRef();
-    int maxEndOffset = -1;
-    boolean sawRealToken = false;
-    while(ts.incrementToken()) {
-      termBytesAtt.fillBytesRef();
-      sawRealToken |= tokenBytes.length > 0;
-      // TODO: this is somewhat iffy; today, ShingleFilter
-      // sets posLen to the gram count; maybe we should make
-      // a separate dedicated att for this?
-      int gramCount = posLenAtt.getPositionLength();
-
-      assert gramCount <= grams;
-
-      // Safety: make sure the recalculated count "agrees":
-      if (countGrams(tokenBytes) != gramCount) {
-        throw new IllegalArgumentException("tokens must not contain separator byte; got token=" + tokenBytes + " but gramCount=" + gramCount + " does not match recalculated count=" + countGrams(tokenBytes));
-      }
-      maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset());
-      lastTokens[gramCount-1] = BytesRef.deepCopyOf(tokenBytes);
-    }
-    ts.end();
-
-    if (!sawRealToken) {
-      throw new IllegalArgumentException("no tokens produced by analyzer, or the only tokens were empty strings");
-    }
-
-    // Carefully fill last tokens with _ tokens;
-    // ShingleFilter appraently won't emit "only hole"
-    // tokens:
-    int endPosInc = posIncAtt.getPositionIncrement();
-
-    // Note this will also be true if input is the empty
-    // string (in which case we saw no tokens and
-    // maxEndOffset is still -1), which in fact works out OK
-    // because we fill the unigram with an empty BytesRef
-    // below:
-    boolean lastTokenEnded = offsetAtt.endOffset() > maxEndOffset || endPosInc > 0;
-    ts.close();
-    //System.out.println("maxEndOffset=" + maxEndOffset + " vs " + offsetAtt.endOffset());
-
-    if (lastTokenEnded) {
-      //System.out.println("  lastTokenEnded");
-      // If user hit space after the last token, then
-      // "upgrade" all tokens.  This way "foo " will suggest
-      // all bigrams starting w/ foo, and not any unigrams
-      // starting with "foo":
-      for(int i=grams-1;i>0;i--) {
-        BytesRef token = lastTokens[i-1];
-        if (token == null) {
-          continue;
+    try (TokenStream ts = queryAnalyzer.tokenStream("", key.toString())) {
+      TermToBytesRefAttribute termBytesAtt = ts.addAttribute(TermToBytesRefAttribute.class);
+      OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+      PositionLengthAttribute posLenAtt = ts.addAttribute(PositionLengthAttribute.class);
+      PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
+      ts.reset();
+      
+      BytesRef[] lastTokens = new BytesRef[grams];
+      //System.out.println("lookup: key='" + key + "'");
+      
+      // Run full analysis, but save only the
+      // last 1gram, last 2gram, etc.:
+      BytesRef tokenBytes = termBytesAtt.getBytesRef();
+      int maxEndOffset = -1;
+      boolean sawRealToken = false;
+      while(ts.incrementToken()) {
+        termBytesAtt.fillBytesRef();
+        sawRealToken |= tokenBytes.length > 0;
+        // TODO: this is somewhat iffy; today, ShingleFilter
+        // sets posLen to the gram count; maybe we should make
+        // a separate dedicated att for this?
+        int gramCount = posLenAtt.getPositionLength();
+        
+        assert gramCount <= grams;
+        
+        // Safety: make sure the recalculated count "agrees":
+        if (countGrams(tokenBytes) != gramCount) {
+          throw new IllegalArgumentException("tokens must not contain separator byte; got token=" + tokenBytes + " but gramCount=" + gramCount + " does not match recalculated count=" + countGrams(tokenBytes));
         }
-        token.grow(token.length+1);
-        token.bytes[token.length] = separator;
-        token.length++;
-        lastTokens[i] = token;
+        maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset());
+        lastTokens[gramCount-1] = BytesRef.deepCopyOf(tokenBytes);
       }
-      lastTokens[0] = new BytesRef();
-    }
-
-    Arc<Long> arc = new Arc<Long>();
-
-    BytesReader bytesReader = fst.getBytesReader();
-
-    // Try highest order models first, and if they return
-    // results, return that; else, fallback:
-    double backoff = 1.0;
-
-    List<LookupResult> results = new ArrayList<LookupResult>(num);
-
-    // We only add a given suffix once, from the highest
-    // order model that saw it; for subsequent lower order
-    // models we skip it:
-    final Set<BytesRef> seen = new HashSet<BytesRef>();
-
-    for(int gram=grams-1;gram>=0;gram--) {
-      BytesRef token = lastTokens[gram];
-      // Don't make unigram predictions from empty string:
-      if (token == null || (token.length == 0 && key.length() > 0)) {
-        // Input didn't have enough tokens:
-        //System.out.println("  gram=" + gram + ": skip: not enough input");
-        continue;
-      }
-
-      if (endPosInc > 0 && gram <= endPosInc) {
-        // Skip hole-only predictions; in theory we
-        // shouldn't have to do this, but we'd need to fix
-        // ShingleFilter to produce only-hole tokens:
-        //System.out.println("  break: only holes now");
-        break;
-      }
-
-      //System.out.println("try " + (gram+1) + " gram token=" + token.utf8ToString());
-
-      // TODO: we could add fuzziness here
-      // match the prefix portion exactly
-      //Pair<Long,BytesRef> prefixOutput = null;
-      Long prefixOutput = null;
-      try {
-        prefixOutput = lookupPrefix(fst, bytesReader, token, arc);
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
-      }
-      //System.out.println("  prefixOutput=" + prefixOutput);
-
-      if (prefixOutput == null) {
-        // This model never saw this prefix, e.g. the
-        // trigram model never saw context "purple mushroom"
-        backoff *= ALPHA;
-        continue;
+      ts.end();
+      
+      if (!sawRealToken) {
+        throw new IllegalArgumentException("no tokens produced by analyzer, or the only tokens were empty strings");
+      }
+      
+      // Carefully fill last tokens with _ tokens;
+      // ShingleFilter appraently won't emit "only hole"
+      // tokens:
+      int endPosInc = posIncAtt.getPositionIncrement();
+      
+      // Note this will also be true if input is the empty
+      // string (in which case we saw no tokens and
+      // maxEndOffset is still -1), which in fact works out OK
+      // because we fill the unigram with an empty BytesRef
+      // below:
+      boolean lastTokenEnded = offsetAtt.endOffset() > maxEndOffset || endPosInc > 0;
+      //System.out.println("maxEndOffset=" + maxEndOffset + " vs " + offsetAtt.endOffset());
+      
+      if (lastTokenEnded) {
+        //System.out.println("  lastTokenEnded");
+        // If user hit space after the last token, then
+        // "upgrade" all tokens.  This way "foo " will suggest
+        // all bigrams starting w/ foo, and not any unigrams
+        // starting with "foo":
+        for(int i=grams-1;i>0;i--) {
+          BytesRef token = lastTokens[i-1];
+          if (token == null) {
+            continue;
+          }
+          token.grow(token.length+1);
+          token.bytes[token.length] = separator;
+          token.length++;
+          lastTokens[i] = token;
+        }
+        lastTokens[0] = new BytesRef();
       }
-
-      // TODO: we could do this division at build time, and
-      // bake it into the FST?
-
-      // Denominator for computing scores from current
-      // model's predictions:
-      long contextCount = totTokens;
-
-      BytesRef lastTokenFragment = null;
-
-      for(int i=token.length-1;i>=0;i--) {
-        if (token.bytes[token.offset+i] == separator) {
-          BytesRef context = new BytesRef(token.bytes, token.offset, i);
-          Long output = Util.get(fst, Util.toIntsRef(context, new IntsRef()));
-          assert output != null;
-          contextCount = decodeWeight(output);
-          lastTokenFragment = new BytesRef(token.bytes, token.offset + i + 1, token.length - i - 1);
+      
+      Arc<Long> arc = new Arc<Long>();
+      
+      BytesReader bytesReader = fst.getBytesReader();
+      
+      // Try highest order models first, and if they return
+      // results, return that; else, fallback:
+      double backoff = 1.0;
+      
+      List<LookupResult> results = new ArrayList<LookupResult>(num);
+      
+      // We only add a given suffix once, from the highest
+      // order model that saw it; for subsequent lower order
+      // models we skip it:
+      final Set<BytesRef> seen = new HashSet<BytesRef>();
+      
+      for(int gram=grams-1;gram>=0;gram--) {
+        BytesRef token = lastTokens[gram];
+        // Don't make unigram predictions from empty string:
+        if (token == null || (token.length == 0 && key.length() > 0)) {
+          // Input didn't have enough tokens:
+          //System.out.println("  gram=" + gram + ": skip: not enough input");
+          continue;
+        }
+        
+        if (endPosInc > 0 && gram <= endPosInc) {
+          // Skip hole-only predictions; in theory we
+          // shouldn't have to do this, but we'd need to fix
+          // ShingleFilter to produce only-hole tokens:
+          //System.out.println("  break: only holes now");
           break;
         }
-      }
-
-      final BytesRef finalLastToken;
-
-      if (lastTokenFragment == null) {
-        finalLastToken = BytesRef.deepCopyOf(token);
-      } else {
-        finalLastToken = BytesRef.deepCopyOf(lastTokenFragment);
-      }
-      assert finalLastToken.offset == 0;
-
-      CharsRef spare = new CharsRef();
-
-      // complete top-N
-      MinResult<Long> completions[] = null;
-      try {
-
-        // Because we store multiple models in one FST
-        // (1gram, 2gram, 3gram), we must restrict the
-        // search so that it only considers the current
-        // model.  For highest order model, this is not
-        // necessary since all completions in the FST
-        // must be from this model, but for lower order
-        // models we have to filter out the higher order
-        // ones:
-
-        // Must do num+seen.size() for queue depth because we may
-        // reject up to seen.size() paths in acceptResult():
-        Util.TopNSearcher<Long> searcher = new Util.TopNSearcher<Long>(fst, num, num+seen.size(), weightComparator) {
-
-          BytesRef scratchBytes = new BytesRef();
-
-          @Override
-          protected void addIfCompetitive(Util.FSTPath<Long> path) {
-            if (path.arc.label != separator) {
-              //System.out.println("    keep path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
-              super.addIfCompetitive(path);
-            } else {
-              //System.out.println("    prevent path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
-            }
-          }
-
-          @Override
-          protected boolean acceptResult(IntsRef input, Long output) {
-            Util.toBytesRef(input, scratchBytes);
-            finalLastToken.grow(finalLastToken.length + scratchBytes.length);
-            int lenSav = finalLastToken.length;
-            finalLastToken.append(scratchBytes);
-            //System.out.println("    accept? input='" + scratchBytes.utf8ToString() + "'; lastToken='" + finalLastToken.utf8ToString() + "'; return " + (seen.contains(finalLastToken) == false));
-            boolean ret = seen.contains(finalLastToken) == false;
-
-            finalLastToken.length = lenSav;
-            return ret;
-          }
-        };
-
-        // since this search is initialized with a single start node 
-        // it is okay to start with an empty input path here
-        searcher.addStartPaths(arc, prefixOutput, true, new IntsRef());
-
-        completions = searcher.search();
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
-      }
-
-      int prefixLength = token.length;
-
-      BytesRef suffix = new BytesRef(8);
-      //System.out.println("    " + completions.length + " completions");
-
-      nextCompletion:
-      for (MinResult<Long> completion : completions) {
-        token.length = prefixLength;
-        // append suffix
-        Util.toBytesRef(completion.input, suffix);
-        token.append(suffix);
-
-        //System.out.println("    completion " + token.utf8ToString());
-
-        // Skip this path if a higher-order model already
-        // saw/predicted its last token:
-        BytesRef lastToken = token;
+        
+        //System.out.println("try " + (gram+1) + " gram token=" + token.utf8ToString());
+        
+        // TODO: we could add fuzziness here
+        // match the prefix portion exactly
+        //Pair<Long,BytesRef> prefixOutput = null;
+        Long prefixOutput = null;
+        try {
+          prefixOutput = lookupPrefix(fst, bytesReader, token, arc);
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+        //System.out.println("  prefixOutput=" + prefixOutput);
+        
+        if (prefixOutput == null) {
+          // This model never saw this prefix, e.g. the
+          // trigram model never saw context "purple mushroom"
+          backoff *= ALPHA;
+          continue;
+        }
+        
+        // TODO: we could do this division at build time, and
+        // bake it into the FST?
+        
+        // Denominator for computing scores from current
+        // model's predictions:
+        long contextCount = totTokens;
+        
+        BytesRef lastTokenFragment = null;
+        
         for(int i=token.length-1;i>=0;i--) {
           if (token.bytes[token.offset+i] == separator) {
-            assert token.length-i-1 > 0;
-            lastToken = new BytesRef(token.bytes, token.offset+i+1, token.length-i-1);
+            BytesRef context = new BytesRef(token.bytes, token.offset, i);
+            Long output = Util.get(fst, Util.toIntsRef(context, new IntsRef()));
+            assert output != null;
+            contextCount = decodeWeight(output);
+            lastTokenFragment = new BytesRef(token.bytes, token.offset + i + 1, token.length - i - 1);
             break;
           }
         }
-        if (seen.contains(lastToken)) {
-          //System.out.println("      skip dup " + lastToken.utf8ToString());
-          continue nextCompletion;
+        
+        final BytesRef finalLastToken;
+        
+        if (lastTokenFragment == null) {
+          finalLastToken = BytesRef.deepCopyOf(token);
+        } else {
+          finalLastToken = BytesRef.deepCopyOf(lastTokenFragment);
+        }
+        assert finalLastToken.offset == 0;
+        
+        CharsRef spare = new CharsRef();
+        
+        // complete top-N
+        MinResult<Long> completions[] = null;
+        try {
+          
+          // Because we store multiple models in one FST
+          // (1gram, 2gram, 3gram), we must restrict the
+          // search so that it only considers the current
+          // model.  For highest order model, this is not
+          // necessary since all completions in the FST
+          // must be from this model, but for lower order
+          // models we have to filter out the higher order
+          // ones:
+          
+          // Must do num+seen.size() for queue depth because we may
+          // reject up to seen.size() paths in acceptResult():
+          Util.TopNSearcher<Long> searcher = new Util.TopNSearcher<Long>(fst, num, num+seen.size(), weightComparator) {
+            
+            BytesRef scratchBytes = new BytesRef();
+            
+            @Override
+            protected void addIfCompetitive(Util.FSTPath<Long> path) {
+              if (path.arc.label != separator) {
+                //System.out.println("    keep path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
+                super.addIfCompetitive(path);
+              } else {
+                //System.out.println("    prevent path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
+              }
+            }
+            
+            @Override
+            protected boolean acceptResult(IntsRef input, Long output) {
+              Util.toBytesRef(input, scratchBytes);
+              finalLastToken.grow(finalLastToken.length + scratchBytes.length);
+              int lenSav = finalLastToken.length;
+              finalLastToken.append(scratchBytes);
+              //System.out.println("    accept? input='" + scratchBytes.utf8ToString() + "'; lastToken='" + finalLastToken.utf8ToString() + "'; return " + (seen.contains(finalLastToken) == false));
+              boolean ret = seen.contains(finalLastToken) == false;
+              
+              finalLastToken.length = lenSav;
+              return ret;
+            }
+          };
+          
+          // since this search is initialized with a single start node 
+          // it is okay to start with an empty input path here
+          searcher.addStartPaths(arc, prefixOutput, true, new IntsRef());
+          
+          completions = searcher.search();
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
         }
-        seen.add(BytesRef.deepCopyOf(lastToken));
-        spare.grow(token.length);
-        UnicodeUtil.UTF8toUTF16(token, spare);
-        LookupResult result = new LookupResult(spare.toString(), (long) (Long.MAX_VALUE * backoff * ((double) decodeWeight(completion.output)) / contextCount));
-        results.add(result);
-        assert results.size() == seen.size();
-        //System.out.println("  add result=" + result);
+        
+        int prefixLength = token.length;
+        
+        BytesRef suffix = new BytesRef(8);
+        //System.out.println("    " + completions.length + " completions");
+        
+        nextCompletion:
+          for (MinResult<Long> completion : completions) {
+            token.length = prefixLength;
+            // append suffix
+            Util.toBytesRef(completion.input, suffix);
+            token.append(suffix);
+            
+            //System.out.println("    completion " + token.utf8ToString());
+            
+            // Skip this path if a higher-order model already
+            // saw/predicted its last token:
+            BytesRef lastToken = token;
+            for(int i=token.length-1;i>=0;i--) {
+              if (token.bytes[token.offset+i] == separator) {
+                assert token.length-i-1 > 0;
+                lastToken = new BytesRef(token.bytes, token.offset+i+1, token.length-i-1);
+                break;
+              }
+            }
+            if (seen.contains(lastToken)) {
+              //System.out.println("      skip dup " + lastToken.utf8ToString());
+              continue nextCompletion;
+            }
+            seen.add(BytesRef.deepCopyOf(lastToken));
+            spare.grow(token.length);
+            UnicodeUtil.UTF8toUTF16(token, spare);
+            LookupResult result = new LookupResult(spare.toString(), (long) (Long.MAX_VALUE * backoff * ((double) decodeWeight(completion.output)) / contextCount));
+            results.add(result);
+            assert results.size() == seen.size();
+            //System.out.println("  add result=" + result);
+          }
+        backoff *= ALPHA;
       }
-      backoff *= ALPHA;
-    }
-
-    Collections.sort(results, new Comparator<LookupResult>() {
+      
+      Collections.sort(results, new Comparator<LookupResult>() {
         @Override
         public int compare(LookupResult a, LookupResult b) {
           if (a.value > b.value) {
@@ -707,12 +706,13 @@ public class FreeTextSuggester extends L
           }
         }
       });
-
-    if (results.size() > num) {
-      results.subList(num, results.size()).clear();
+      
+      if (results.size() > num) {
+        results.subList(num, results.size()).clear();
+      }
+      
+      return results;
     }
-
-    return results;
   }
 
   /** weight -> cost */

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java Mon Oct  7 06:24:25 2013
@@ -165,43 +165,43 @@ public class AnalyzingInfixSuggesterTest
 
         @Override
         protected Object highlight(String text, Set<String> matchedTokens, String prefixToken) throws IOException {
-          TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text));
-          CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
-          OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
-          ts.reset();
-          List<LookupHighlightFragment> fragments = new ArrayList<LookupHighlightFragment>();
-          int upto = 0;
-          while (ts.incrementToken()) {
-            String token = termAtt.toString();
-            int startOffset = offsetAtt.startOffset();
-            int endOffset = offsetAtt.endOffset();
-            if (upto < startOffset) {
-              fragments.add(new LookupHighlightFragment(text.substring(upto, startOffset), false));
-              upto = startOffset;
-            } else if (upto > startOffset) {
-              continue;
-            }
-
-            if (matchedTokens.contains(token)) {
-              // Token matches.
-              fragments.add(new LookupHighlightFragment(text.substring(startOffset, endOffset), true));
-              upto = endOffset;
-            } else if (prefixToken != null && token.startsWith(prefixToken)) {
-              fragments.add(new LookupHighlightFragment(text.substring(startOffset, startOffset+prefixToken.length()), true));
-              if (prefixToken.length() < token.length()) {
-                fragments.add(new LookupHighlightFragment(text.substring(startOffset+prefixToken.length(), startOffset+token.length()), false));
+          try (TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text))) {
+            CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+            OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+            ts.reset();
+            List<LookupHighlightFragment> fragments = new ArrayList<LookupHighlightFragment>();
+            int upto = 0;
+            while (ts.incrementToken()) {
+              String token = termAtt.toString();
+              int startOffset = offsetAtt.startOffset();
+              int endOffset = offsetAtt.endOffset();
+              if (upto < startOffset) {
+                fragments.add(new LookupHighlightFragment(text.substring(upto, startOffset), false));
+                upto = startOffset;
+              } else if (upto > startOffset) {
+                continue;
+              }
+              
+              if (matchedTokens.contains(token)) {
+                // Token matches.
+                fragments.add(new LookupHighlightFragment(text.substring(startOffset, endOffset), true));
+                upto = endOffset;
+              } else if (prefixToken != null && token.startsWith(prefixToken)) {
+                fragments.add(new LookupHighlightFragment(text.substring(startOffset, startOffset+prefixToken.length()), true));
+                if (prefixToken.length() < token.length()) {
+                  fragments.add(new LookupHighlightFragment(text.substring(startOffset+prefixToken.length(), startOffset+token.length()), false));
+                }
+                upto = endOffset;
               }
-              upto = endOffset;
             }
+            ts.end();
+            int endOffset = offsetAtt.endOffset();
+            if (upto < endOffset) {
+              fragments.add(new LookupHighlightFragment(text.substring(upto), false));
+            }
+            
+            return fragments;
           }
-          ts.end();
-          int endOffset = offsetAtt.endOffset();
-          if (upto < endOffset) {
-            fragments.add(new LookupHighlightFragment(text.substring(upto), false));
-          }
-          ts.close();
-
-          return fragments;
         }
       };
     suggester.build(new TermFreqPayloadArrayIterator(keys));

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java Mon Oct  7 06:24:25 2013
@@ -258,17 +258,17 @@ public abstract class CollationTestBase 
 
     for (int i = 0; i < numTestPoints; i++) {
       String term = _TestUtil.randomSimpleString(random());
-      TokenStream ts = analyzer.tokenStream("fake", term);
-      TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
-      BytesRef bytes = termAtt.getBytesRef();
-      ts.reset();
-      assertTrue(ts.incrementToken());
-      termAtt.fillBytesRef();
-      // ensure we make a copy of the actual bytes too
-      map.put(term, BytesRef.deepCopyOf(bytes));
-      assertFalse(ts.incrementToken());
-      ts.end();
-      ts.close();
+      try (TokenStream ts = analyzer.tokenStream("fake", term)) {
+        TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
+        BytesRef bytes = termAtt.getBytesRef();
+        ts.reset();
+        assertTrue(ts.incrementToken());
+        termAtt.fillBytesRef();
+        // ensure we make a copy of the actual bytes too
+        map.put(term, BytesRef.deepCopyOf(bytes));
+        assertFalse(ts.incrementToken());
+        ts.end();
+      }
     }
     
     Thread threads[] = new Thread[numThreads];
@@ -280,16 +280,16 @@ public abstract class CollationTestBase 
             for (Map.Entry<String,BytesRef> mapping : map.entrySet()) {
               String term = mapping.getKey();
               BytesRef expected = mapping.getValue();
-              TokenStream ts = analyzer.tokenStream("fake", term);
-              TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
-              BytesRef bytes = termAtt.getBytesRef();
-              ts.reset();
-              assertTrue(ts.incrementToken());
-              termAtt.fillBytesRef();
-              assertEquals(expected, bytes);
-              assertFalse(ts.incrementToken());
-              ts.end();
-              ts.close();
+              try (TokenStream ts = analyzer.tokenStream("fake", term)) {
+                TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
+                BytesRef bytes = termAtt.getBytesRef();
+                ts.reset();
+                assertTrue(ts.incrementToken());
+                termAtt.fillBytesRef();
+                assertEquals(expected, bytes);
+                assertFalse(ts.incrementToken());
+                ts.end();
+              }
             }
           } catch (IOException e) {
             throw new RuntimeException(e);

Modified: lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java (original)
+++ lucene/dev/trunk/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java Mon Oct  7 06:24:25 2013
@@ -234,36 +234,23 @@ public class ICUCollationField extends F
    * simple (we already have a threadlocal clone in the reused TS)
    */
   private BytesRef analyzeRangePart(String field, String part) {
-    TokenStream source;
-      
-    try {
-      source = analyzer.tokenStream(field, part);
+    try (TokenStream source = analyzer.tokenStream(field, part)) {
       source.reset();
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e);
-    }
       
-    TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
-    BytesRef bytes = termAtt.getBytesRef();
+      TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
+      BytesRef bytes = termAtt.getBytesRef();
 
-    // we control the analyzer here: most errors are impossible
-    try {
+      // we control the analyzer here: most errors are impossible
       if (!source.incrementToken())
         throw new IllegalArgumentException("analyzer returned no terms for range part: " + part);
       termAtt.fillBytesRef();
       assert !source.incrementToken();
-    } catch (IOException e) {
-      throw new RuntimeException("error analyzing range part: " + part, e);
-    }
       
-    try {
       source.end();
-      source.close();
+      return BytesRef.deepCopyOf(bytes);
     } catch (IOException e) {
-      throw new RuntimeException("Unable to end & close TokenStream after analyzing range part: " + part, e);
+      throw new RuntimeException("Unable analyze range part: " + part, e);
     }
-      
-    return BytesRef.deepCopyOf(bytes);
   }
   
   @Override

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java Mon Oct  7 06:24:25 2013
@@ -85,15 +85,13 @@ public abstract class AnalysisRequestHan
 
     if (!TokenizerChain.class.isInstance(analyzer)) {
 
-      TokenStream tokenStream = null;
-      try {
-        tokenStream = analyzer.tokenStream(context.getFieldName(), value);
+      try (TokenStream tokenStream = analyzer.tokenStream(context.getFieldName(), value)) {
+        NamedList<List<NamedList>> namedList = new NamedList<List<NamedList>>();
+        namedList.add(tokenStream.getClass().getName(), convertTokensToNamedLists(analyzeTokenStream(tokenStream), context));
+        return namedList;
       } catch (IOException e) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
       }
-      NamedList<List<NamedList>> namedList = new NamedList<List<NamedList>>();
-      namedList.add(tokenStream.getClass().getName(), convertTokensToNamedLists(analyzeTokenStream(tokenStream), context));
-      return namedList;
     }
 
     TokenizerChain tokenizerChain = (TokenizerChain) analyzer;
@@ -139,10 +137,8 @@ public abstract class AnalysisRequestHan
    * @param analyzer The analyzer to use.
    */
   protected Set<BytesRef> getQueryTokenSet(String query, Analyzer analyzer) {
-    TokenStream tokenStream = null;
-    try {
+    try (TokenStream tokenStream = analyzer.tokenStream("", query)){
       final Set<BytesRef> tokens = new HashSet<BytesRef>();
-      tokenStream = analyzer.tokenStream("", query);
       final TermToBytesRefAttribute bytesAtt = tokenStream.getAttribute(TermToBytesRefAttribute.class);
       final BytesRef bytes = bytesAtt.getBytesRef();
 
@@ -157,8 +153,6 @@ public abstract class AnalysisRequestHan
       return tokens;
     } catch (IOException ioe) {
       throw new RuntimeException("Error occured while iterating over tokenstream", ioe);
-    } finally {
-      IOUtils.closeWhileHandlingException(tokenStream);
     }
   }
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java Mon Oct  7 06:24:25 2013
@@ -344,16 +344,16 @@ public class QueryElevationComponent ext
       return query;
     }
     StringBuilder norm = new StringBuilder();
-    TokenStream tokens = analyzer.tokenStream("", query);
-    tokens.reset();
+    try (TokenStream tokens = analyzer.tokenStream("", query)) {
+      tokens.reset();
 
-    CharTermAttribute termAtt = tokens.addAttribute(CharTermAttribute.class);
-    while (tokens.incrementToken()) {
-      norm.append(termAtt.buffer(), 0, termAtt.length());
+      CharTermAttribute termAtt = tokens.addAttribute(CharTermAttribute.class);
+      while (tokens.incrementToken()) {
+        norm.append(termAtt.buffer(), 0, termAtt.length());
+      }
+      tokens.end();
+      return norm.toString();
     }
-    tokens.end();
-    tokens.close();
-    return norm.toString();
   }
 
   //---------------------------------------------------------------------------------

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java Mon Oct  7 06:24:25 2013
@@ -463,29 +463,29 @@ public class SpellCheckComponent extends
   private Collection<Token> getTokens(String q, Analyzer analyzer) throws IOException {
     Collection<Token> result = new ArrayList<Token>();
     assert analyzer != null;
-    TokenStream ts = analyzer.tokenStream("", q);
-    ts.reset();
-    // TODO: support custom attributes
-    CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
-    OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
-    TypeAttribute typeAtt = ts.addAttribute(TypeAttribute.class);
-    FlagsAttribute flagsAtt = ts.addAttribute(FlagsAttribute.class);
-    PayloadAttribute payloadAtt = ts.addAttribute(PayloadAttribute.class);
-    PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
-    
-    while (ts.incrementToken()){
-      Token token = new Token();
-      token.copyBuffer(termAtt.buffer(), 0, termAtt.length());
-      token.setOffset(offsetAtt.startOffset(), offsetAtt.endOffset());
-      token.setType(typeAtt.type());
-      token.setFlags(flagsAtt.getFlags());
-      token.setPayload(payloadAtt.getPayload());
-      token.setPositionIncrement(posIncAtt.getPositionIncrement());
-      result.add(token);
+    try (TokenStream ts = analyzer.tokenStream("", q)) {
+      ts.reset();
+      // TODO: support custom attributes
+      CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
+      OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+      TypeAttribute typeAtt = ts.addAttribute(TypeAttribute.class);
+      FlagsAttribute flagsAtt = ts.addAttribute(FlagsAttribute.class);
+      PayloadAttribute payloadAtt = ts.addAttribute(PayloadAttribute.class);
+      PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class);
+      
+      while (ts.incrementToken()){
+        Token token = new Token();
+        token.copyBuffer(termAtt.buffer(), 0, termAtt.length());
+        token.setOffset(offsetAtt.startOffset(), offsetAtt.endOffset());
+        token.setType(typeAtt.type());
+        token.setFlags(flagsAtt.getFlags());
+        token.setPayload(payloadAtt.getPayload());
+        token.setPositionIncrement(posIncAtt.getPositionIncrement());
+        result.add(token);
+      }
+      ts.end();
+      return result;
     }
-    ts.end();
-    ts.close();
-    return result;
   }
 
   protected SolrSpellChecker getSpellChecker(SolrParams params) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java Mon Oct  7 06:24:25 2013
@@ -403,58 +403,49 @@ public abstract class SolrQueryParserBas
     // Use the analyzer to get all the tokens, and then build a TermQuery,
     // PhraseQuery, or nothing based on the term count
 
-    TokenStream source;
-    try {
-      source = analyzer.tokenStream(field, queryText);
-      source.reset();
-    } catch (IOException e) {
-      throw new SyntaxError("Unable to initialize TokenStream to analyze query text", e);
-    }
-    CachingTokenFilter buffer = new CachingTokenFilter(source);
+    CachingTokenFilter buffer = null;
     TermToBytesRefAttribute termAtt = null;
     PositionIncrementAttribute posIncrAtt = null;
     int numTokens = 0;
-
-    buffer.reset();
-
-    if (buffer.hasAttribute(TermToBytesRefAttribute.class)) {
-      termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
-    }
-    if (buffer.hasAttribute(PositionIncrementAttribute.class)) {
-      posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
-    }
-
     int positionCount = 0;
     boolean severalTokensAtSamePosition = false;
+    
+    try (TokenStream source = analyzer.tokenStream(field, queryText)) {
+      source.reset();
+      buffer = new CachingTokenFilter(source);
+      buffer.reset();
 
-    boolean hasMoreTokens = false;
-    if (termAtt != null) {
-      try {
-        hasMoreTokens = buffer.incrementToken();
-        while (hasMoreTokens) {
-          numTokens++;
-          int positionIncrement = (posIncrAtt != null) ? posIncrAtt.getPositionIncrement() : 1;
-          if (positionIncrement != 0) {
-            positionCount += positionIncrement;
-          } else {
-            severalTokensAtSamePosition = true;
-          }
+      if (buffer.hasAttribute(TermToBytesRefAttribute.class)) {
+        termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
+      }
+      if (buffer.hasAttribute(PositionIncrementAttribute.class)) {
+        posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
+      }
+
+      boolean hasMoreTokens = false;
+      if (termAtt != null) {
+        try {
           hasMoreTokens = buffer.incrementToken();
+          while (hasMoreTokens) {
+            numTokens++;
+            int positionIncrement = (posIncrAtt != null) ? posIncrAtt.getPositionIncrement() : 1;
+            if (positionIncrement != 0) {
+              positionCount += positionIncrement;
+            } else {
+              severalTokensAtSamePosition = true;
+            }
+            hasMoreTokens = buffer.incrementToken();
+          }
+        } catch (IOException e) {
+          // ignore
         }
-      } catch (IOException e) {
-        // ignore
       }
+    } catch (IOException e) {
+      throw new SyntaxError("Error analyzing query text", e);
     }
-    try {
-      // rewind the buffer stream
-      buffer.reset();
-
-      // close original stream - all tokens buffered
-      source.close();
-    }
-    catch (IOException e) {
-      throw new SyntaxError("Cannot close TokenStream analyzing query text", e);
-    }
+    
+    // rewind the buffer stream
+    buffer.reset();
 
     BytesRef bytes = termAtt == null ? null : termAtt.getBytesRef();
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/CollationField.java Mon Oct  7 06:24:25 2013
@@ -209,37 +209,23 @@ public class CollationField extends Fiel
    * its just that all methods are synced), this keeps things 
    * simple (we already have a threadlocal clone in the reused TS)
    */
-  private BytesRef analyzeRangePart(String field, String part) {
-    TokenStream source;
-      
-    try {
-      source = analyzer.tokenStream(field, part);
-      source.reset();
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to initialize TokenStream to analyze range part: " + part, e);
-    }
-      
-    TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
-    BytesRef bytes = termAtt.getBytesRef();
+  private BytesRef analyzeRangePart(String field, String part) {     
+    try (TokenStream source = analyzer.tokenStream(field, part)) {
+      source.reset();    
+      TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
+      BytesRef bytes = termAtt.getBytesRef();
 
-    // we control the analyzer here: most errors are impossible
-    try {
+      // we control the analyzer here: most errors are impossible
       if (!source.incrementToken())
         throw new IllegalArgumentException("analyzer returned no terms for range part: " + part);
       termAtt.fillBytesRef();
       assert !source.incrementToken();
-    } catch (IOException e) {
-      throw new RuntimeException("error analyzing range part: " + part, e);
-    }
       
-    try {
       source.end();
-      source.close();
+      return BytesRef.deepCopyOf(bytes);
     } catch (IOException e) {
-      throw new RuntimeException("Unable to end & close TokenStream after analyzing range part: " + part, e);
+      throw new RuntimeException("Unable to analyze range part: " + part, e);
     }
-      
-    return BytesRef.deepCopyOf(bytes);
   }
   
   @Override

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TextField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TextField.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TextField.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/TextField.java Mon Oct  7 06:24:25 2013
@@ -138,35 +138,23 @@ public class TextField extends FieldType
   public static BytesRef analyzeMultiTerm(String field, String part, Analyzer analyzerIn) {
     if (part == null || analyzerIn == null) return null;
 
-    TokenStream source;
-    try {
-      source = analyzerIn.tokenStream(field, part);
+    try (TokenStream source = analyzerIn.tokenStream(field, part)){
       source.reset();
-    } catch (IOException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to initialize TokenStream to analyze multiTerm term: " + part, e);
-    }
 
-    TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
-    BytesRef bytes = termAtt.getBytesRef();
+      TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
+      BytesRef bytes = termAtt.getBytesRef();
 
-    try {
       if (!source.incrementToken())
         throw  new SolrException(SolrException.ErrorCode.BAD_REQUEST,"analyzer returned no terms for multiTerm term: " + part);
       termAtt.fillBytesRef();
       if (source.incrementToken())
         throw  new SolrException(SolrException.ErrorCode.BAD_REQUEST,"analyzer returned too many terms for multiTerm term: " + part);
-    } catch (IOException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,"error analyzing range part: " + part, e);
-    }
 
-    try {
       source.end();
-      source.close();
+      return BytesRef.deepCopyOf(bytes);
     } catch (IOException e) {
-      throw new RuntimeException("Unable to end & close TokenStream after analyzing multiTerm term: " + part, e);
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,"error analyzing range part: " + part, e);
     }
-
-    return BytesRef.deepCopyOf(bytes);
   }
 
 
@@ -178,58 +166,50 @@ public class TextField extends FieldType
     // Use the analyzer to get all the tokens, and then build a TermQuery,
     // PhraseQuery, or nothing based on the term count
 
-    TokenStream source;
-    try {
-      source = analyzer.tokenStream(field, queryText);
-      source.reset();
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to initialize TokenStream to analyze query text", e);
-    }
-    CachingTokenFilter buffer = new CachingTokenFilter(source);
+    CachingTokenFilter buffer = null;
     CharTermAttribute termAtt = null;
     PositionIncrementAttribute posIncrAtt = null;
     int numTokens = 0;
-
-    buffer.reset();
-
-    if (buffer.hasAttribute(CharTermAttribute.class)) {
-      termAtt = buffer.getAttribute(CharTermAttribute.class);
-    }
-    if (buffer.hasAttribute(PositionIncrementAttribute.class)) {
-      posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
-    }
-
     int positionCount = 0;
     boolean severalTokensAtSamePosition = false;
 
-    boolean hasMoreTokens = false;
-    if (termAtt != null) {
-      try {
-        hasMoreTokens = buffer.incrementToken();
-        while (hasMoreTokens) {
-          numTokens++;
-          int positionIncrement = (posIncrAtt != null) ? posIncrAtt.getPositionIncrement() : 1;
-          if (positionIncrement != 0) {
-            positionCount += positionIncrement;
-          } else {
-            severalTokensAtSamePosition = true;
-          }
+    try (TokenStream source = analyzer.tokenStream(field, queryText)) {
+      source.reset();
+      buffer = new CachingTokenFilter(source);
+      
+      buffer.reset();
+      
+      if (buffer.hasAttribute(CharTermAttribute.class)) {
+        termAtt = buffer.getAttribute(CharTermAttribute.class);
+      }
+      if (buffer.hasAttribute(PositionIncrementAttribute.class)) {
+        posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
+      }
+      
+      boolean hasMoreTokens = false;
+      if (termAtt != null) {
+        try {
           hasMoreTokens = buffer.incrementToken();
+          while (hasMoreTokens) {
+            numTokens++;
+            int positionIncrement = (posIncrAtt != null) ? posIncrAtt.getPositionIncrement() : 1;
+            if (positionIncrement != 0) {
+              positionCount += positionIncrement;
+            } else {
+              severalTokensAtSamePosition = true;
+            }
+            hasMoreTokens = buffer.incrementToken();
+          }
+        } catch (IOException e) {
+          // ignore
         }
-      } catch (IOException e) {
-        // ignore
       }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
     }
-    try {
-      // rewind the buffer stream
-      buffer.reset();
 
-      // close original stream - all tokens buffered
-      source.close();
-    }
-    catch (IOException e) {
-      // ignore
-    }
+    // rewind the buffer stream
+    buffer.reset();
 
     if (numTokens == 0)
       return null;

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java?rev=1529770&r1=1529769&r2=1529770&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SimpleQueryConverter.java Mon Oct  7 06:24:25 2013
@@ -40,10 +40,10 @@ class SimpleQueryConverter extends Spell
 
   @Override
   public Collection<Token> convert(String origQuery) {
-    try {
-      Collection<Token> result = new HashSet<Token>();
-      WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer(Version.LUCENE_40);
-      TokenStream ts = analyzer.tokenStream("", origQuery);
+    Collection<Token> result = new HashSet<Token>();
+    WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer(Version.LUCENE_40);
+    
+    try (TokenStream ts = analyzer.tokenStream("", origQuery)) {
       // TODO: support custom attributes
       CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
       OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
@@ -64,9 +64,7 @@ class SimpleQueryConverter extends Spell
         tok.setType(typeAtt.type());
         result.add(tok);
       }
-      ts.end();
-      ts.close();
-      
+      ts.end();      
       return result;
     } catch (IOException e) {
       throw new RuntimeException(e);