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/01/18 23:38:59 UTC

svn commit: r1435384 [4/4] - in /lucene/dev/branches/lucene4547: ./ dev-tools/ dev-tools/maven/ dev-tools/maven/solr/core/src/test/ dev-tools/maven/solr/solrj/src/java/ dev-tools/maven/solr/solrj/src/test/ dev-tools/maven/solr/webapp/ dev-tools/scripts...

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/params/MultiCategoryListIteratorTest.java Fri Jan 18 22:38:56 2013
@@ -9,8 +9,7 @@ import org.apache.lucene.facet.index.Fac
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.PerDimensionIndexingParams;
 import org.apache.lucene.facet.search.CategoryListIterator;
-import org.apache.lucene.facet.search.PayloadCategoryListIteraor;
-import org.apache.lucene.facet.search.cache.CategoryListCache;
+import org.apache.lucene.facet.search.DocValuesCategoryListIterator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
@@ -20,7 +19,6 @@ import org.apache.lucene.facet.util.Mult
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
@@ -60,7 +58,7 @@ public class MultiCategoryListIteratorTe
     HashMap<CategoryPath,CategoryListParams> clps = new HashMap<CategoryPath,CategoryListParams>();
     for (String dim : dimensions) {
       CategoryPath cp = new CategoryPath(dim);
-      CategoryListParams clp = new CategoryListParams(new Term("$" + dim, CategoryListParams.DEFAULT_TERM.bytes()));
+      CategoryListParams clp = new CategoryListParams("$" + dim);
       clps.put(cp, clp);
     }
     PerDimensionIndexingParams indexingParams = new PerDimensionIndexingParams(clps);
@@ -86,23 +84,13 @@ public class MultiCategoryListIteratorTe
     IOUtils.close(indexWriter, taxoWriter);
     
     // test the multi iterator
-    CategoryListCache clCache = null;
-    if (random.nextBoolean()) {
-      clCache = new CategoryListCache();
-    }
-    
     DirectoryReader indexReader = DirectoryReader.open(indexDir);
     TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
     CategoryListIterator[] iterators = new CategoryListIterator[numDimensions];
     for (int i = 0; i < iterators.length; i++) {
       CategoryListParams clp = indexingParams.getCategoryListParams(new CategoryPath(dimensions[i]));
       IntDecoder decoder = clp.createEncoder().createMatchingDecoder();
-      if (clCache != null && random.nextBoolean()) {
-        clCache.loadAndRegister(clp, indexReader, taxoReader, indexingParams);
-        iterators[i] = clCache.get(clp).iterator(0); // no partitions
-      } else {
-        iterators[i] = new PayloadCategoryListIteraor(clp.getTerm(), decoder);
-      }
+      iterators[i] = new DocValuesCategoryListIterator(clp.field, decoder);
     }
     MultiCategoryListIterator cli = new MultiCategoryListIterator(iterators);
     for (AtomicReaderContext context : indexReader.leaves()) {

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCategoryPath.java Fri Jan 18 22:38:56 2013
@@ -163,16 +163,22 @@ public class TestCategoryPath extends Lu
     CategoryPath p = new CategoryPath("a/b/c/d", '/');
     CategoryPath pother = new CategoryPath("a/b/c/d", '/');
     assertEquals(0, pother.compareTo(p));
+    assertEquals(0, p.compareTo(pother));
     pother = new CategoryPath("", '/');
     assertTrue(pother.compareTo(p) < 0);
+    assertTrue(p.compareTo(pother) > 0);
     pother = new CategoryPath("a/b_/c/d", '/');
     assertTrue(pother.compareTo(p) > 0);
+    assertTrue(p.compareTo(pother) < 0);
     pother = new CategoryPath("a/b/c", '/');
     assertTrue(pother.compareTo(p) < 0);
+    assertTrue(p.compareTo(pother) > 0);
     pother = new CategoryPath("a/b/c/e", '/');
     assertTrue(pother.compareTo(p) > 0);
+    assertTrue(p.compareTo(pother) < 0);
     pother = new CategoryPath("a/b/c//e", '/');
     assertTrue(pother.compareTo(p) < 0);
+    assertTrue(p.compareTo(pother) > 0);
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java Fri Jan 18 22:38:56 2013
@@ -77,6 +77,7 @@ public class EncodingSpeed {
     encoderTest(new VInt8IntEncoder(), facetIDs, loopFactor);
     encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new VInt8IntEncoder())), facetIDs, loopFactor);
     encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))), facetIDs, loopFactor);
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapVInt8IntEncoder())), facetIDs, loopFactor);
     encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new EightFlagsIntEncoder()))), facetIDs, loopFactor);
     encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new FourFlagsIntEncoder()))), facetIDs, loopFactor);
     encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(3)))), facetIDs, loopFactor);

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java Fri Jan 18 22:38:56 2013
@@ -64,9 +64,12 @@ public class EncodingTest extends Lucene
     
     BytesRef bytes = new BytesRef(100); // some initial capacity - encoders should grow the byte[]
     IntsRef values = new IntsRef(100); // some initial capacity - decoders should grow the int[]
-    encoding(encoder, data, bytes);
-    decoding(bytes, values, encoder.createMatchingDecoder());
-    assertTrue(expected.intsEquals(values));
+    for (int i = 0; i < 2; i++) {
+      // run 2 iterations to catch encoders/decoders which don't reset properly
+      encoding(encoder, data, bytes);
+      decoding(bytes, values, encoder.createMatchingDecoder());
+      assertTrue(expected.intsEquals(values));
+    }
   }
 
   private static void encoding(IntEncoder encoder, IntsRef data, BytesRef bytes) throws IOException {
@@ -147,5 +150,10 @@ public class EncodingTest extends Lucene
   public void testSortingUniqueDGapNOnes3() throws Exception {
     encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(3)))), data, uniqueSortedData);
   }
+  
+  @Test
+  public void testSortingUniqueDGapVInt() throws Exception {
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapVInt8IntEncoder())), data, uniqueSortedData);
+  }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java (original)
+++ lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java Fri Jan 18 22:38:56 2013
@@ -40,11 +40,13 @@ import java.util.Arrays;
 import java.util.Comparator;
 
 /**
- * 
  * <code>HighFreqTerms</code> class extracts the top n most frequent terms
- * (by document frequency ) from an existing Lucene index and reports their document frequencey.
- * If the -t flag is  and reports both their document frequency and their total tf (total number of occurences) 
- * in order of highest total tf
+ * (by document frequency) from an existing Lucene index and reports their
+ * document frequency.
+ * <p>
+ * If the -t flag is given, both document frequency and total tf (total
+ * number of occurrences) are reported, ordered by descending total tf.
+ *
  */
 public class HighFreqTerms {
   

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java Fri Jan 18 22:38:56 2013
@@ -17,20 +17,12 @@ package org.apache.lucene.search.spell;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Locale;
-import java.util.PriorityQueue;
-
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.search.FuzzyTermsEnum;
 import org.apache.lucene.search.BoostAttribute;
+import org.apache.lucene.search.FuzzyTermsEnum;
 import org.apache.lucene.search.MaxNonCompetitiveBoostAttribute;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
@@ -39,6 +31,14 @@ import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.PriorityQueue;
+
 /**
  * Simple automaton-based spellchecker.
  * <p>
@@ -384,9 +384,22 @@ public class DirectSpellChecker {
     }
     return suggestions;
   }
-  
-  private Collection<ScoreTerm> suggestSimilar(Term term, int numSug, 
-      IndexReader ir, int docfreq, int editDistance, float accuracy, final CharsRef spare) throws IOException {
+
+  /**
+   * Provide spelling corrections based on several parameters.
+   *
+   * @param term The term to suggest spelling corrections for
+   * @param numSug The maximum number of spelling corrections
+   * @param ir The index reader to fetch the candidate spelling corrections from
+   * @param docfreq The minimum document frequency a potential suggestion need to have in order to be included
+   * @param editDistance The maximum edit distance candidates are allowed to have
+   * @param accuracy The minimum accuracy a suggested spelling correction needs to have in order to be included
+   * @param spare a chars scratch
+   * @return a collection of spelling corrections sorted by <code>ScoreTerm</code>'s natural order.
+   * @throws IOException If I/O related errors occur
+   */
+  protected Collection<ScoreTerm> suggestSimilar(Term term, int numSug, IndexReader ir, int docfreq, int editDistance,
+                                                 float accuracy, final CharsRef spare) throws IOException {
     
     AttributeSource atts = new AttributeSource();
     MaxNonCompetitiveBoostAttribute maxBoostAtt =
@@ -449,15 +462,43 @@ public class DirectSpellChecker {
       
     return stQueue;
   }
-  
-  private static class ScoreTerm implements Comparable<ScoreTerm> {
+
+  /**
+   * Holds a spelling correction for internal usage inside {@link DirectSpellChecker}.
+   */
+  protected static class ScoreTerm implements Comparable<ScoreTerm> {
+
+    /**
+     * The actual spellcheck correction.
+     */
     public BytesRef term;
+
+    /**
+     * The boost representing the similarity from the FuzzyTermsEnum (internal similarity score)
+     */
     public float boost;
+
+    /**
+     * The df of the spellcheck correction.
+     */
     public int docfreq;
-    
+
+    /**
+     * The spellcheck correction represented as string, can be <code>null</code>.
+     */
     public String termAsString;
+
+    /**
+     * The similarity score.
+     */
     public float score;
-    
+
+    /**
+     * Constructor.
+     */
+    public ScoreTerm() {
+    }
+
     @Override
     public int compareTo(ScoreTerm other) {
       if (term.bytesEquals(other.term))

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Fri Jan 18 22:38:56 2013
@@ -587,7 +587,7 @@ public class AnalyzingSuggester extends 
 
       //System.out.println("  prefixPaths: " + prefixPaths.size());
 
-      BytesReader bytesReader = fst.getBytesReader(0);
+      BytesReader bytesReader = fst.getBytesReader();
 
       FST.Arc<Pair<Long,BytesRef>> scratchArc = new FST.Arc<Pair<Long,BytesRef>>();
 

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java Fri Jan 18 22:38:56 2013
@@ -77,7 +77,7 @@ public class FSTUtil {
         new IntsRef()));
     
     final FST.Arc<T> scratchArc = new FST.Arc<T>();
-    final FST.BytesReader fstReader = fst.getBytesReader(0);
+    final FST.BytesReader fstReader = fst.getBytesReader();
     
     while (queue.size() != 0) {
       final Path<T> path = queue.remove(queue.size() - 1);

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java Fri Jan 18 22:38:56 2013
@@ -139,7 +139,7 @@ public class FSTCompletion {
     try {
       List<Arc<Object>> rootArcs = new ArrayList<Arc<Object>>();
       Arc<Object> arc = automaton.getFirstArc(new Arc<Object>());
-      FST.BytesReader fstReader = automaton.getBytesReader(0);
+      FST.BytesReader fstReader = automaton.getBytesReader();
       automaton.readFirstTargetArc(arc, arc, fstReader);
       while (true) {
         rootArcs.add(new Arc<Object>().copyFrom(arc));
@@ -173,7 +173,7 @@ public class FSTCompletion {
     // Get the UTF-8 bytes representation of the input key.
     try {
       final FST.Arc<Object> scratch = new FST.Arc<Object>();
-      FST.BytesReader fstReader = automaton.getBytesReader(0);
+      FST.BytesReader fstReader = automaton.getBytesReader();
       for (; rootArcIndex < rootArcs.length; rootArcIndex++) {
         final FST.Arc<Object> rootArc = rootArcs[rootArcIndex];
         final FST.Arc<Object> arc = scratch.copyFrom(rootArc);
@@ -338,7 +338,7 @@ public class FSTCompletion {
     final int max = utf8.offset + utf8.length;
     // Cannot save as instance var since multiple threads
     // can use FSTCompletion at once...
-    final FST.BytesReader fstReader = automaton.getBytesReader(0);
+    final FST.BytesReader fstReader = automaton.getBytesReader();
     for (int i = utf8.offset; i < max; i++) {
       if (automaton.findTargetArc(utf8.bytes[i] & 0xff, arc, arc, fstReader) == null) {
         // No matching prefixes, return an empty result.
@@ -362,7 +362,7 @@ public class FSTCompletion {
     }
     assert output.offset == 0;
     output.bytes[output.length++] = (byte) arc.label;
-    FST.BytesReader fstReader = automaton.getBytesReader(0);
+    FST.BytesReader fstReader = automaton.getBytesReader();
     automaton.readFirstTargetArc(arc, arc, fstReader);
     while (true) {
       if (arc.label == FST.END_LABEL) {

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Fri Jan 18 22:38:56 2013
@@ -200,7 +200,7 @@ public class WFSTCompletionLookup extend
   private Long lookupPrefix(BytesRef scratch, Arc<Long> arc) throws /*Bogus*/IOException {
     assert 0 == fst.outputs.getNoOutput().longValue();
     long output = 0;
-    BytesReader bytesReader = fst.getBytesReader(0);
+    BytesReader bytesReader = fst.getBytesReader();
     
     fst.getFirstArc(arc);
     

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/fst/FSTTester.java Fri Jan 18 22:38:56 2013
@@ -204,7 +204,7 @@ public class FSTTester<T> {
     final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
     final T NO_OUTPUT = fst.outputs.getNoOutput();
     T output = NO_OUTPUT;
-    final FST.BytesReader fstReader = fst.getBytesReader(0);
+    final FST.BytesReader fstReader = fst.getBytesReader();
 
     for(int i=0;i<=term.length;i++) {
       final int label;
@@ -241,7 +241,7 @@ public class FSTTester<T> {
     in.offset = 0;
     final T NO_OUTPUT = fst.outputs.getNoOutput();
     T output = NO_OUTPUT;
-    final FST.BytesReader fstReader = fst.getBytesReader(0);
+    final FST.BytesReader fstReader = fst.getBytesReader();
 
     while(true) {
       // read all arcs:

Modified: lucene/dev/branches/lucene4547/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/CHANGES.txt?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene4547/solr/CHANGES.txt Fri Jan 18 22:38:56 2013
@@ -42,6 +42,40 @@ Other Changes
 ----------------------
 
 
+==================  4.2.0 ==================
+
+Versions of Major Components
+---------------------
+Apache Tika 1.2
+Carrot2 3.6.2
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.5
+
+Upgrading from Solr 4.1.0
+----------------------
+
+(No upgrade instructions yet)
+
+Detailed Change List
+----------------------
+
+New Features
+----------------------
+
+Bug Fixes
+----------------------
+
+* SOLR-4309: /browse: Improve JQuery autosuggest behavior (janhoy)
+
+Optimizations
+----------------------
+
+Other Changes
+----------------------
+
+
+
 ==================  4.1.0 ==================
 
 Versions of Major Components
@@ -210,6 +244,30 @@ New Features
 * SOLR-4302: New parameter 'indexInfo' (defaults to true) in CoreAdmin STATUS
   command can be used to omit index specific information (Shahar Davidson via shalin)
 
+* SOLR-2592: Collection specific document routing.  The "compositeId"
+  router is the default for collections with hash based routing (i.e. when
+  numShards=N is specified on collection creation).  Documents with ids sharing
+  the same domain (prefix) will be routed to the same shard, allowing for
+  efficient querying.
+
+  Example:
+    The following two documents will be indexed to the same shard
+    since they share the same domain "customerB!". 
+    <code>
+       {"id" : "customerB!doc1" [...] }
+       {"id" : "customerB!doc2" [...] }
+    </code>
+    At query time, one can specify a "shard.keys" parameter that lists what
+    shards the query should cover.
+       http://.../query?q=my_query&shard.keys=customerB!
+
+  Collections that do not specify numShards at collection creation time
+  use custom sharding and default to the "implicit" router.  Document updates
+  received by a shard will be indexed to that shard, unless a "_shard_" parameter
+  or document field names a different shard.
+  (Michael Garski, Dan Rosher, yonik)
+
+
 Optimizations
 ----------------------
 
@@ -258,6 +316,9 @@ Optimizations
 Bug Fixes
 ----------------------
 
+* SOLR-4288: Improve logging for FileDataSource (basePath, relative 
+  resources). (Dawid Weiss)
+
 * SOLR-4007: Morfologik dictionaries not available in Solr field type
   due to class loader lookup problems. (Lance Norskog, Dawid Weiss)
 
@@ -348,7 +409,7 @@ Bug Fixes
 
 * SOLR-4064: When there is an unexpected exception while trying to run the new
   leader process, the SolrCore will not correctly rejoin the election.
-  (Po Rui Via Mark Miller)
+  (Po Rui via Mark Miller)
 
 * SOLR-3989: SolrZkClient constructor dropped exception cause when throwing
   a new RuntimeException. (Colin Bartolome, yonik)
@@ -518,6 +579,9 @@ Bug Fixes
 
 * SOLR-4303: On replication, if the generation of the master is lower than the
   slave we need to force a full copy of the index. (Mark Miller, Gregg Donovan)
+  
+* SOLR-4266: HttpSolrServer does not release connection properly on exception
+  when no response parser is used. (Steve Molloy via Mark Miller)
 
 Other Changes
 ----------------------
@@ -606,6 +670,9 @@ Other Changes
   disallow atomic update requests which change signature generating fields.
   (Joel Nothman, yonik, shalin)
 
+* SOLR-4308: Remove the problematic and now unnecessary log4j-over-slf4j.
+  (Mark Miller)
+
 ==================  4.0.0 ==================
 
 Versions of Major Components

Modified: lucene/dev/branches/lucene4547/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java (original)
+++ lucene/dev/branches/lucene4547/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileDataSource.java Fri Jan 18 22:38:56 2013
@@ -92,22 +92,35 @@ public class FileDataSource extends Data
 
   static File getFile(String basePath, String query) {
     try {
-      File file0 = new File(query);
-      File file = file0;
+      File file = new File(query);
 
-      if (!file.isAbsolute())
-        file = new File(basePath + query);
+      // If it's not an absolute path, try relative from basePath. 
+      if (!file.isAbsolute()) {
+        // Resolve and correct basePath.
+        File basePathFile;
+        if (basePath == null) {
+          basePathFile = new File(".").getAbsoluteFile(); 
+          LOG.warn("FileDataSource.basePath is empty. " +
+              "Resolving to: " + basePathFile.getAbsolutePath());
+        } else {
+          basePathFile = new File(basePath);
+          if (!basePathFile.isAbsolute()) {
+            basePathFile = basePathFile.getAbsoluteFile();
+            LOG.warn("FileDataSource.basePath is not absolute. Resolving to: "
+                + basePathFile.getAbsolutePath());
+          }
+        }
+
+        file = new File(basePathFile, query).getAbsoluteFile();
+      }
 
       if (file.isFile() && file.canRead()) {
-        LOG.debug("Accessing File: " + file.toString());
+        LOG.debug("Accessing File: " + file.getAbsolutePath());
         return file;
-      } else if (file != file0)
-        if (file0.isFile() && file0.canRead()) {
-          LOG.debug("Accessing File0: " + file0.toString());
-          return  file0;
-        }
-
-      throw new FileNotFoundException("Could not find file: " + query);
+      } else {
+        throw new FileNotFoundException("Could not find file: " + query + 
+            " (resolved to: " + file.getAbsolutePath());
+      }
     } catch (FileNotFoundException e) {
       throw new RuntimeException(e);
     }

Modified: lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/core/CoreContainer.java Fri Jan 18 22:38:56 2013
@@ -128,6 +128,8 @@ public class CoreContainer 
 
   protected final Map<String, CoreDescriptor> dynamicDescriptors = new LinkedHashMap<String, CoreDescriptor>();
 
+  protected final Set<String> pendingDynamicCoreLoads = new HashSet<String>();
+
   protected final Map<String,Exception> coreInitFailures = 
     Collections.synchronizedMap(new LinkedHashMap<String,Exception>());
   
@@ -1245,17 +1247,8 @@ public class CoreContainer 
       }
     }
   }
-  
-  /** Gets a core by name and increase its refcount.
-   * @see SolrCore#close() 
-   * @param name the core name
-   * @return the core if found
-   */
-  public SolrCore getCore(String name) {
-    name = checkDefault(name);
-    // Do this in two phases since we don't want to lock access to the cores over a load.
+  private SolrCore getCoreFromAnyList(String name) {
     SolrCore core;
-
     synchronized (cores) {
       core = cores.get(name);
       if (core != null) {
@@ -1274,20 +1267,64 @@ public class CoreContainer 
         return core;
       }
     }
+    return null;
+  }
+  /** Gets a core by name and increase its refcount.
+   * @see SolrCore#close() 
+   * @param name the core name
+   * @return the core if found
+   */
+  public SolrCore getCore(String name) {
+    name = checkDefault(name);
+    // Do this in two phases since we don't want to lock access to the cores over a load.
+    SolrCore core = getCoreFromAnyList(name);
+
+    if (core != null) return core;
+
+    // OK, it's not presently in any list, is it in the list of dynamic cores but not loaded yet? If so, load it.
     CoreDescriptor desc =  dynamicDescriptors.get(name);
     if (desc == null) { //Nope, no transient core with this name
       return null;
     }
+
+    // Keep multiple threads from loading the same core at the same time.
     try {
-      core = create(desc); // This should throw an error if it fails.
-      core.open();
-      if (desc.isTransient()) {
-        registerLazyCore(name, core, false);    // This is a transient core
-      } else {
-        register(name, core, false); // This is a "permanent", although deferred-load core
+      boolean isPending;
+      synchronized (pendingDynamicCoreLoads) {
+        isPending = pendingDynamicCoreLoads.contains(name);
+        if (! isPending) {
+          pendingDynamicCoreLoads.add(name);
+        }
       }
-    } catch (Exception ex) {
-      throw recordAndThrow(name, "Unable to create core" + name, ex);
+
+      while (isPending) {
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          return null; // Seems best not to do anything at all if the thread is interrupted
+        }
+
+        synchronized (pendingDynamicCoreLoads) {
+          if (!pendingDynamicCoreLoads.contains(name)) {
+            // NOTE: If, for some reason, the load failed, we'll return null here and presumably the log will show
+            // why. We'll fail all over again next time if the problem isn't corrected.
+            return getCoreFromAnyList(name);
+          }
+        }
+      }
+      try {
+        core = create(desc); // This should throw an error if it fails.
+        core.open();
+        if (desc.isTransient()) {
+          registerLazyCore(name, core, false);    // This is a transient core
+        } else {
+          register(name, core, false); // This is a "permanent", although deferred-load core
+        }
+      } catch (Exception ex) {
+        throw recordAndThrow(name, "Unable to create core" + name, ex);
+      }
+    } finally {
+      pendingDynamicCoreLoads.remove(name);
     }
     return core;
   }

Modified: lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene4547/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Fri Jan 18 22:38:56 2013
@@ -227,7 +227,7 @@ public class DistributedUpdateProcessor 
         DistribPhase phase =
             DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
 
-        doDefensiveChecks(shardId, phase);
+        doDefensiveChecks(phase);
 
 
         if (DistribPhase.FROMLEADER == phase) {
@@ -279,7 +279,7 @@ public class DistributedUpdateProcessor 
   }
 
 
-  private void doDefensiveChecks(String shardId, DistribPhase phase) {
+  private void doDefensiveChecks(DistribPhase phase) {
     boolean isReplayOrPeersync = (updateCommand.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
     if (isReplayOrPeersync) return;
 

Modified: lucene/dev/branches/lucene4547/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/core/src/test/org/apache/solr/core/TestLazyCores.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/core/src/test/org/apache/solr/core/TestLazyCores.java (original)
+++ lucene/dev/branches/lucene4547/solr/core/src/test/org/apache/solr/core/TestLazyCores.java Fri Jan 18 22:38:56 2013
@@ -34,8 +34,10 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 
 public class TestLazyCores extends SolrTestCaseJ4 {
 
@@ -246,6 +248,44 @@ public class TestLazyCores extends SolrT
     }
   }
 
+  // Test case for SOLR-4300
+  @Test
+  public void testRace() throws Exception {
+    final List<SolrCore> _theCores = new ArrayList<SolrCore>();
+    final CoreContainer cc = init();
+    try {
+
+      Thread[] threads = new Thread[15];
+      for (int idx = 0; idx < threads.length; idx++) {
+        threads[idx] = new Thread() {
+          @Override
+          public void run() {
+            SolrCore core = cc.getCore("collectionLazy3");
+            synchronized (_theCores) {
+              _theCores.add(core);
+            }
+          }
+        };
+        threads[idx].start();
+      }
+
+      for (Thread thread : threads) {
+        thread.join();
+      }
+
+      for (int idx = 0; idx < _theCores.size() - 1; ++idx) {
+        assertEquals("Cores should be the same!", _theCores.get(idx), _theCores.get(idx + 1));
+      }
+
+      for (SolrCore core : _theCores) {
+        core.close();
+      }
+
+    } finally {
+      cc.shutdown();
+    }
+  }
+
   private void checkNotInCores(CoreContainer cc, String... nameCheck) {
     Collection<String> names = cc.getCoreNames();
     for (String name : nameCheck) {

Modified: lucene/dev/branches/lucene4547/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml (original)
+++ lucene/dev/branches/lucene4547/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml Fri Jan 18 22:38:56 2013
@@ -5,22 +5,22 @@
                 pk="link"
                 url="http://rss.slashdot.org/Slashdot/slashdot"
                 processor="XPathEntityProcessor"
-                forEach="/RDF/channel | /RDF/item"
+                forEach="/rss/channel | /rss/item"
                 transformer="DateFormatTransformer">
 				
-            <field column="source" xpath="/RDF/channel/title" commonField="true" />
-            <field column="source-link" xpath="/RDF/channel/link" commonField="true" />
-            <field column="subject" xpath="/RDF/channel/subject" commonField="true" />
+            <field column="source" xpath="/rss/channel/title" commonField="true" />
+            <field column="source-link" xpath="/rss/channel/link" commonField="true" />
+            <field column="subject" xpath="/rss/channel/subject" commonField="true" />
 			
-            <field column="title" xpath="/RDF/item/title" />
-            <field column="link" xpath="/RDF/item/link" />
-            <field column="description" xpath="/RDF/item/description" />
-            <field column="creator" xpath="/RDF/item/creator" />
-            <field column="item-subject" xpath="/RDF/item/subject" />
-            <field column="date" xpath="/RDF/item/date" dateTimeFormat="yyyy-MM-dd'T'HH:mm:ss" />
-            <field column="slash-department" xpath="/RDF/item/department" />
-            <field column="slash-section" xpath="/RDF/item/section" />
-            <field column="slash-comments" xpath="/RDF/item/comments" />
+            <field column="title" xpath="/rss/item/title" />
+            <field column="link" xpath="/rss/item/link" />
+            <field column="description" xpath="/rss/item/description" />
+            <field column="creator" xpath="/rss/item/creator" />
+            <field column="item-subject" xpath="/rss/item/subject" />
+            <field column="date" xpath="/rss/item/date" dateTimeFormat="yyyy-MM-dd'T'HH:mm:ss" />
+            <field column="slash-department" xpath="/rss/item/department" />
+            <field column="slash-section" xpath="/rss/item/section" />
+            <field column="slash-comments" xpath="/rss/item/comments" />
         </entity>
     </document>
 </dataConfig>

Modified: lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/solrconfig.xml?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/solrconfig.xml (original)
+++ lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/solrconfig.xml Fri Jan 18 22:38:56 2013
@@ -239,12 +239,9 @@
       -->
 
     <!-- Commit Deletion Policy
-
          Custom deletion policies can be specified here. The class must
          implement org.apache.lucene.index.IndexDeletionPolicy.
 
-         http://lucene.apache.org/java/3_5_0/api/core/org/apache/lucene/index/IndexDeletionPolicy.html
-
          The default Solr IndexDeletionPolicy implementation supports
          deleting index commit points on number of commits, age of
          commit point and optimized status.
@@ -304,6 +301,16 @@
   <!-- The default high-performance update handler -->
   <updateHandler class="solr.DirectUpdateHandler2">
 
+    <!-- Enables a transaction log, used for real-time get, durability, and
+         and solr cloud replica recovery.  The log can grow as big as
+         uncommitted changes to the index, so use of a hard autoCommit
+         is recommended (see below).
+         "dir" - the target directory for transaction logs, defaults to the
+                solr data directory.  --> 
+    <updateLog>
+      <str name="dir">${solr.ulog.dir:}</str>
+    </updateLog>
+ 
     <!-- AutoCommit
 
          Perform a hard commit automatically under certain conditions.
@@ -319,8 +326,11 @@
                    since a document was added before automaticly
                    triggering a new commit. 
          openSearcher - if false, the commit causes recent index changes
-         to be flushed to stable storage, but does not cause a new
-         searcher to be opened to make those changes visible.
+           to be flushed to stable storage, but does not cause a new
+           searcher to be opened to make those changes visible.
+
+         If the updateLog is enabled, then it's highly recommended to
+         have some sort of hard autoCommit to limit the log size.
       -->
      <autoCommit> 
        <maxTime>15000</maxTime> 
@@ -370,14 +380,6 @@
        </listener>
       -->
 
-    <!-- Enables a transaction log, currently used for real-time get.
-         "dir" - the target directory for transaction logs, defaults to the
-            solr data directory.  --> 
-    <updateLog>
-      <str name="dir">${solr.ulog.dir:}</str>
-    </updateLog>
-   
-
   </updateHandler>
   
   <!-- IndexReaderFactory

Modified: lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/head.vm
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/head.vm?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/head.vm (original)
+++ lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/head.vm Fri Jan 18 22:38:56 2013
@@ -20,7 +20,11 @@
              'v.template': 'suggest'
            }
          }
-      );
+      ).keydown(function(e){
+        if (e.keyCode === 13){
+          $("#query-form").trigger('submit');
+        }
+      });
 
       // http://localhost:8983/solr/terms?terms.fl=name&terms.prefix=i&terms.sort=count
     });

Modified: lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/jquery.autocomplete.js
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/jquery.autocomplete.js?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/jquery.autocomplete.js (original)
+++ lucene/dev/branches/lucene4547/solr/example/solr/collection1/conf/velocity/jquery.autocomplete.js Fri Jan 18 22:38:56 2013
@@ -325,6 +325,7 @@ $.Autocompleter = function(input, option
 		if (!options.matchCase)
 			term = term.toLowerCase();
 		var data = cache.load(term);
+		data = null; // Avoid buggy cache and go to Solr every time 
 		// recieve the cached data
 		if (data && data.length) {
 			success(term, data);
@@ -398,7 +399,7 @@ $.Autocompleter.defaults = {
 	max: 100,
 	mustMatch: false,
 	extraParams: {},
-	selectFirst: true,
+	selectFirst: false,
 	formatItem: function(row) { return row[0]; },
 	formatMatch: null,
 	autoFill: false,

Modified: lucene/dev/branches/lucene4547/solr/solrj/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/solrj/ivy.xml?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/solrj/ivy.xml (original)
+++ lucene/dev/branches/lucene4547/solr/solrj/ivy.xml Fri Jan 18 22:38:56 2013
@@ -21,7 +21,6 @@
 
     <dependencies>
       <dependency org="org.apache.zookeeper" name="zookeeper" rev="3.4.5" transitive="false"/>
-      <dependency org="org.slf4j" name="log4j-over-slf4j" rev="1.6.4" transitive="false"/>
       <dependency org="org.apache.httpcomponents" name="httpcore" rev="4.1.4" transitive="false"/>
       <dependency org="org.apache.httpcomponents" name="httpclient" rev="4.1.3" transitive="false"/>
       <dependency org="org.apache.httpcomponents" name="httpmime" rev="4.1.3" transitive="false"/>

Modified: lucene/dev/branches/lucene4547/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java?rev=1435384&r1=1435383&r2=1435384&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java (original)
+++ lucene/dev/branches/lucene4547/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java Fri Jan 18 22:38:56 2013
@@ -346,6 +346,7 @@ public class HttpSolrServer extends Solr
     method.addHeader("User-Agent", AGENT);
     
     InputStream respBody = null;
+    boolean shouldClose = true;
     
     try {
       // Execute the method.
@@ -378,6 +379,8 @@ public class HttpSolrServer extends Solr
         // no processor specified, return raw stream
         NamedList<Object> rsp = new NamedList<Object>();
         rsp.add("stream", respBody);
+        // Only case where stream should not be closed
+        shouldClose = false;
         return rsp;
       }
       String charset = EntityUtils.getContentCharSet(response.getEntity());
@@ -413,7 +416,7 @@ public class HttpSolrServer extends Solr
       throw new SolrServerException(
           "IOException occured when talking to server at: " + getBaseURL(), e);
     } finally {
-      if (respBody != null && processor!=null) {
+      if (respBody != null && shouldClose) {
         try {
           respBody.close();
         } catch (Throwable t) {} // ignore