You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2011/11/26 02:03:19 UTC

svn commit: r1206387 [4/5] - in /lucene/dev/branches/solrcloud: ./ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ dev-tools/maven/lucene/contrib/demo/ dev-tools/maven/lucene/contrib/highlighter/ dev-tools/maven/lucene/contrib/memory/ dev-tools/ma...

Modified: lucene/dev/branches/solrcloud/solr/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/build.xml?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/build.xml (original)
+++ lucene/dev/branches/solrcloud/solr/build.xml Sat Nov 26 01:03:02 2011
@@ -130,11 +130,11 @@
   <target name="compile" description="Compile the source code."
           depends="compile-core, compile-contrib"/>
   <target name="test" description="Validate, then run core, solrj, and contrib unit tests."
-          depends="validate-solr, test-core, test-contrib"/>
+          depends="validate-solr, test-jsp, test-core, test-contrib"/>
   <target name="test-core" description="Runs the core and solrj unit tests."
           depends="test-solr-core, test-solrj"/>
   <target name="compile-test" description="Compile unit tests."
-          depends="compile-solr-test-framework, compile-test-solr-core, compile-test-solrj, compile-test-contrib, test-jsp"/>
+          depends="compile-solr-test-framework, compile-test-solr-core, compile-test-solrj, compile-test-contrib"/>
   <target name="javadocs" description="Calls javadocs-all, javadocs-solrj, and javadocs-test-framework"
           depends="javadocs-all,javadocs-solrj,javadocs-test-framework"/>
   <target name="compile-core" depends="compile-solr-core" unless="solr.core.compiled"/>

Modified: lucene/dev/branches/solrcloud/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java Sat Nov 26 01:03:02 2011
@@ -218,7 +218,7 @@ public class ICUCollationField extends F
       throw new RuntimeException("Unable to end & close TokenStream after analyzing range part: " + part, e);
     }
       
-    return new BytesRef(bytes);
+    return BytesRef.deepCopyOf(bytes);
   }
   
   @Override

Modified: lucene/dev/branches/solrcloud/solr/contrib/clustering/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/clustering/CHANGES.txt?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/clustering/CHANGES.txt (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/clustering/CHANGES.txt Sat Nov 26 01:03:02 2011
@@ -11,6 +11,10 @@ $Id$
 
 (No Changes)
 
+================== Release 3.6.0 ==================
+
+(No Changes)
+
 ================== Release 3.5.0 ==================
 
 (No Changes)

Modified: lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/extraction/CHANGES.txt Sat Nov 26 01:03:02 2011
@@ -28,6 +28,10 @@ $Id$
 
 (No Changes)
 
+================== Release 3.6.0 ==================
+
+(No Changes)
+
 ================== Release 3.5.0 ==================
 
 * SOLR-2372: Upgrade Solr to Tika 0.10 (janhoy)

Modified: lucene/dev/branches/solrcloud/solr/contrib/langid/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/langid/CHANGES.txt?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/langid/CHANGES.txt (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/langid/CHANGES.txt Sat Nov 26 01:03:02 2011
@@ -7,6 +7,10 @@ See http://wiki.apache.org/solr/Language
 
 $Id$
 
+================== Release 3.6.0 ==================
+
+(No Changes)
+
 ================== Release 3.5.0 ==================
 
 Initial release.  See README.txt.

Modified: lucene/dev/branches/solrcloud/solr/contrib/uima/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/contrib/uima/CHANGES.txt?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/contrib/uima/CHANGES.txt (original)
+++ lucene/dev/branches/solrcloud/solr/contrib/uima/CHANGES.txt Sat Nov 26 01:03:02 2011
@@ -23,6 +23,10 @@ $Id$
 
 (No Changes)
 
+==================  3.6.0 ==================
+
+(No Changes)
+
 ==================  3.5.0 ==================
 
 Other Changes

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/AnalysisRequestHandlerBase.java Sat Nov 26 01:03:02 2011
@@ -150,7 +150,7 @@ public abstract class AnalysisRequestHan
 
       while (tokenStream.incrementToken()) {
         bytesAtt.fillBytesRef();
-        tokens.add(new BytesRef(bytes));
+        tokens.add(BytesRef.deepCopyOf(bytes));
       }
 
       tokenStream.end();

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java Sat Nov 26 01:03:02 2011
@@ -24,12 +24,14 @@ import org.apache.solr.common.SolrInputD
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.FastInputStream;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.DeleteUpdateCommand;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
@@ -66,30 +68,49 @@ public class BinaryUpdateRequestHandler 
   private void parseAndLoadDocs(final SolrQueryRequest req, SolrQueryResponse rsp, InputStream stream,
                                 final UpdateRequestProcessor processor) throws IOException {
     UpdateRequest update = null;
-    update = new JavaBinUpdateRequestCodec().unmarshal(stream,
-            new JavaBinUpdateRequestCodec.StreamingDocumentHandler() {
-              private AddUpdateCommand addCmd = null;
-
-              public void document(SolrInputDocument document, UpdateRequest updateRequest) {
-                if (addCmd == null) {
-                  addCmd = getAddCommand(req, updateRequest.getParams());
-                }
-                addCmd.solrDoc = document;
-                try {
-                  processor.processAdd(addCmd);
-                  addCmd.clear();
-                } catch (IOException e) {
-                  throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ERROR adding document " + document);
-                }
-              }
-            });
-    if (update.getDeleteById() != null) {
-      delete(req, update.getDeleteById(), processor, true);
-    }
-    if (update.getDeleteQuery() != null) {
-      delete(req, update.getDeleteQuery(), processor, false);
-    }
+    JavaBinUpdateRequestCodec.StreamingUpdateHandler handler = new JavaBinUpdateRequestCodec.StreamingUpdateHandler() {
+      private AddUpdateCommand addCmd = null;
 
+      public void update(SolrInputDocument document, UpdateRequest updateRequest) {
+        if (document == null) {
+          // Perhaps commit from the parameters
+          try {
+            RequestHandlerUtils.handleCommit(req, processor, updateRequest.getParams(), false);
+            RequestHandlerUtils.handleRollback(req, processor, updateRequest.getParams(), false);
+          } catch (IOException e) {
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ERROR handling commit/rollback");
+          }
+          return;
+        }
+        if (addCmd == null) {
+          addCmd = getAddCommand(req, updateRequest.getParams());
+        }
+        addCmd.solrDoc = document;
+        try {
+          processor.processAdd(addCmd);
+          addCmd.clear();
+        } catch (IOException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ERROR adding document " + document);
+        }
+      }
+    };
+    FastInputStream in = FastInputStream.wrap(stream);
+    for (; ; ) {
+      try {
+        update = new JavaBinUpdateRequestCodec().unmarshal(in, handler);
+      } catch (EOFException e) {
+        break; // this is expected
+      } catch (Exception e) {
+        log.error("Exception while processing update request", e);
+        break;
+      }
+      if (update.getDeleteById() != null) {
+        delete(req, update.getDeleteById(), processor, true);
+      }
+      if (update.getDeleteQuery() != null) {
+        delete(req, update.getDeleteQuery(), processor, false);
+      }
+    }
   }
 
   private AddUpdateCommand getAddCommand(SolrQueryRequest req, SolrParams params) {

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java Sat Nov 26 01:03:02 2011
@@ -48,6 +48,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.analysis.CharFilterFactory;
 import org.apache.solr.analysis.TokenFilterFactory;
 import org.apache.solr.analysis.TokenizerChain;
@@ -273,7 +274,8 @@ public class LukeRequestHandler extends 
             BytesRef text;
             while((text = termsEnum.next()) != null) {
               final int freq = (int) termsEnum.totalTermFreq();
-              tfv.add( text.utf8ToChars(spare).toString(), freq );
+              UnicodeUtil.UTF8toUTF16(text, spare);
+              tfv.add(spare.toString(), freq);
             }
             f.add( "termVector", tfv );
           }
@@ -649,7 +651,8 @@ public class LukeRequestHandler extends 
         TermsEnum termsEnum = terms.iterator(null);
         BytesRef text;
         while((text = termsEnum.next()) != null) {
-          String t = text.utf8ToChars(spare).toString();
+          UnicodeUtil.UTF8toUTF16(text, spare);
+          String t = spare.toString();
   
           // Compute distinct terms for every field
           TopTermQueue tiq = info.get( field );

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java Sat Nov 26 01:03:02 2011
@@ -206,11 +206,14 @@ public class ShowFileRequestHandler exte
         return ""; // ignore it...
       }
     }
+    InputStream input = null;
     try {
-      InputStream input = core.getResourceLoader().openResource(path);
+      input = core.getResourceLoader().openResource(path);
       return IOUtils.toString( input, "UTF-8" );
+    } catch( Exception ex ) {
+    } finally {
+      IOUtils.closeQuietly(input);
     }
-    catch( Exception ex ) {} // ignore it
     return "";
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java Sat Nov 26 01:03:02 2011
@@ -30,6 +30,7 @@ import org.apache.lucene.search.grouping
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrDocument;
@@ -607,7 +608,8 @@ public class QueryComponent extends Sear
           // String field in Lucene, which returns the terms
           // data as BytesRef:
           if (val instanceof BytesRef) {
-            field.setValue(((BytesRef)val).utf8ToChars(spare).toString());
+            UnicodeUtil.UTF8toUTF16((BytesRef)val, spare);
+            field.setValue(spare.toString());
             val = ft.toObject(field);
           }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java Sat Nov 26 01:03:02 2011
@@ -207,7 +207,7 @@ public class TermsComponent extends Sear
         if (docFreq >= freqmin && docFreq <= freqmax) {
           // add the term to the list
           if (sort) {
-            queue.add(new CountPair<BytesRef, Integer>(new BytesRef(term), docFreq));
+            queue.add(new CountPair<BytesRef, Integer>(BytesRef.deepCopyOf(term), docFreq));
           } else {
 
             // TODO: handle raw somehow

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java Sat Nov 26 01:03:02 2011
@@ -366,7 +366,8 @@ class CountSortedFacetCollector extends 
       // NOTE: we use c>min rather than c>=min as an optimization because we are going in
       // index order, so we already know that the keys are ordered.  This can be very
       // important if a lot of the counts are repeated (like zero counts would be).
-      queue.add(new SimpleFacets.CountPair<String,Integer>(term.utf8ToChars(spare).toString(), count));
+      UnicodeUtil.UTF8toUTF16(term, spare);
+      queue.add(new SimpleFacets.CountPair<String,Integer>(spare.toString(), count));
       if (queue.size()>=maxsize) min=queue.last().val;
     }
     return false;
@@ -414,7 +415,8 @@ class IndexSortedFacetCollector extends 
     }
 
     if (limit > 0) {
-      res.add(term.utf8ToChars(spare).toString(), count);
+      UnicodeUtil.UTF8toUTF16(term, spare);
+      res.add(spare.toString(), count);
       limit--;
     }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Sat Nov 26 01:03:02 2011
@@ -729,7 +729,7 @@ public class SimpleFacets {
 
           if (sortByCount) {
             if (c>min) {
-              BytesRef termCopy = new BytesRef(term);
+              BytesRef termCopy = BytesRef.deepCopyOf(term);
               queue.add(new CountPair<BytesRef,Integer>(termCopy, c));
               if (queue.size()>=maxsize) min=queue.last().val;
             }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/UnInvertedField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/UnInvertedField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/request/UnInvertedField.java Sat Nov 26 01:03:02 2011
@@ -118,7 +118,7 @@ public class UnInvertedField extends Doc
 
     if (te.docFreq() > maxTermDocFreq) {
       TopTerm topTerm = new TopTerm();
-      topTerm.term = new BytesRef(term);
+      topTerm.term = BytesRef.deepCopyOf(term);
       topTerm.termNum = termNum;
       bigTerms.put(topTerm.termNum, topTerm);
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java Sat Nov 26 01:03:02 2011
@@ -40,9 +40,13 @@ import org.apache.solr.search.ReturnFiel
  */
 
 public class JSONResponseWriter implements QueryResponseWriter {
-  static String CONTENT_TYPE_JSON_UTF8="text/x-json; charset=UTF-8";
+  static String CONTENT_TYPE_JSON_UTF8="application/json; charset=UTF-8";
 
-  public void init(NamedList n) {
+  private String contentType;
+
+  public void init(NamedList namedList) {
+    String contentType = (String) namedList.get("content-type");
+    this.contentType = (contentType != null) ? contentType : CONTENT_TYPE_JSON_UTF8;
   }
 
   public void write(Writer writer, SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
@@ -55,12 +59,10 @@ public class JSONResponseWriter implemen
   }
 
   public String getContentType(SolrQueryRequest request, SolrQueryResponse response) {
-    // using the text/plain allows this to be viewed in the browser easily
-    return CONTENT_TYPE_TEXT_UTF8;
+    return contentType;
   }
 }
 
-
 class JSONWriter extends TextResponseWriter {
   private String namedListStyle;
   private String wrapperFunction;

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/BoolField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/BoolField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/BoolField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/BoolField.java Sat Nov 26 01:03:02 2011
@@ -142,9 +142,9 @@ public class BoolField extends FieldType
   @Override
   public CharsRef indexedToReadable(BytesRef input, CharsRef charsRef) {
     if (input.length > 0 && input.bytes[input.offset] == 'T') {
-      charsRef.copy(TRUE);
+      charsRef.copyChars(TRUE);
     } else {
-      charsRef.copy(FALSE);
+      charsRef.copyChars(FALSE);
     }
     return charsRef;
   }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/CollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/CollationField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/CollationField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/CollationField.java Sat Nov 26 01:03:02 2011
@@ -240,7 +240,7 @@ public class CollationField extends Fiel
       throw new RuntimeException("Unable to end & close TokenStream after analyzing range part: " + part, e);
     }
       
-    return new BytesRef(bytes);
+    return BytesRef.deepCopyOf(bytes);
   }
   
   @Override

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/DateField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/DateField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/DateField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/DateField.java Sat Nov 26 01:03:02 2011
@@ -28,6 +28,7 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.valuesource.FieldCacheSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.DateUtil;
 import org.apache.solr.request.SolrQueryRequest;
@@ -205,7 +206,7 @@ public class DateField extends FieldType
 
   @Override
   public CharsRef indexedToReadable(BytesRef input, CharsRef charsRef) {
-    input.utf8ToChars(charsRef);
+    UnicodeUtil.UTF8toUTF16(input, charsRef);
     charsRef.append(Z_ARRAY, 0, 1);
     return charsRef;
   }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldProperties.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldProperties.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldProperties.java Sat Nov 26 01:03:02 2011
@@ -48,13 +48,15 @@ public abstract class FieldProperties {
   
   protected final static int REQUIRED            = 0x00001000;
   protected final static int OMIT_POSITIONS      = 0x00002000;
+  protected final static int LEGACY_MULTITERM    = 0x00004000;
   
   static final String[] propertyNames = {
           "indexed", "tokenized", "stored",
           "binary", "omitNorms", "omitTermFreqAndPositions",
           "termVectors", "termPositions", "termOffsets",
           "multiValued",
-          "sortMissingFirst","sortMissingLast","required", "omitPositions"
+          "sortMissingFirst","sortMissingLast","required", "omitPositions" ,
+          "legacyMultiTerm"
   };
 
   static final Map<String,Integer> propertyMap = new HashMap<String,Integer>();

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldType.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldType.java Sat Nov 26 01:03:02 2011
@@ -349,7 +349,7 @@ public abstract class FieldType extends 
 
   /** Given an indexed term, append the human readable representation*/
   public CharsRef indexedToReadable(BytesRef input, CharsRef output) {
-    input.utf8ToChars(output);
+    UnicodeUtil.UTF8toUTF16(input, output);
     return output;
   }
 
@@ -429,6 +429,21 @@ public abstract class FieldType extends 
   protected Analyzer queryAnalyzer=analyzer;
 
   /**
+   * Analyzer set by schema for text types to use when searching fields
+   * of this type, subclasses can set analyzer themselves or override
+   * getAnalyzer()
+   * This analyzer is used to process wildcard, prefix, regex and other multiterm queries. It
+   * assembles a list of tokenizer +filters that "make sense" for this, primarily accent folding and
+   * lowercasing filters, and charfilters.
+   *
+   * If users require old-style behavior, they can specify 'legacyMultiterm="true" ' in the schema file
+   * @see #getMultiTermAnalyzer
+   * @see #setMultiTermAnalyzer
+   */
+  protected Analyzer multiTermAnalyzer=null;
+
+
+  /**
    * Returns the Analyzer to be used when indexing fields of this type.
    * <p>
    * This method may be called many times, at any time.
@@ -450,6 +465,17 @@ public abstract class FieldType extends 
     return queryAnalyzer;
   }
 
+  /**
+   * Returns the Analyzer to be used when searching fields of this type when mult-term queries are specified.
+   * <p>
+   * This method may be called many times, at any time.
+   * </p>
+   * @see #getAnalyzer
+   */
+  public Analyzer getMultiTermAnalyzer() {
+    return multiTermAnalyzer;
+  }
+
   private final String analyzerError = 
     "FieldType: " + this.getClass().getSimpleName() + 
     " (" + typeName + ") does not support specifying an analyzer";
@@ -498,6 +524,28 @@ public abstract class FieldType extends 
     throw e;
   }
 
+  /**
+   * Sets the Analyzer to be used when querying fields of this type.
+   *
+   * <p>
+   *
+   * Subclasses that override this method need to ensure the behavior
+   * of the analyzer is consistent with the implementation of toInternal.
+   * </p>
+   *
+   * @see #toInternal
+   * @see #setAnalyzer
+   * @see #getQueryAnalyzer
+   */
+  public void setMultiTermAnalyzer(Analyzer analyzer) {
+    SolrException e = new SolrException
+      (ErrorCode.SERVER_ERROR,
+       "FieldType: " + this.getClass().getSimpleName() +
+       " (" + typeName + ") does not support specifying an analyzer");
+    SolrException.logOnce(log,null,e);
+    throw e;
+  }
+
   /** @lucene.internal */
   protected Similarity similarity;
   

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java Sat Nov 26 01:03:02 2011
@@ -18,19 +18,15 @@
 package org.apache.solr.schema;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Version;
+import org.apache.solr.analysis.*;
 import org.apache.solr.common.ResourceLoader;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.DOMUtil;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.Config;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.analysis.CharFilterFactory;
-import org.apache.solr.analysis.TokenFilterFactory;
-import org.apache.solr.analysis.TokenizerChain;
-import org.apache.solr.analysis.TokenizerFactory;
 import org.apache.solr.util.plugin.AbstractPluginLoader;
 import org.w3c.dom.*;
 
@@ -88,12 +84,16 @@ public final class FieldTypePluginLoader
     String expression = "./analyzer[@type='query']";
     Node anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
     Analyzer queryAnalyzer = readAnalyzer(anode);
-    
+
+    expression = "./analyzer[@type='multiterm']";
+    anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
+    Analyzer multiAnalyzer = readAnalyzer(anode);
+
     // An analyzer without a type specified, or with type="index"
     expression = "./analyzer[not(@type)] | ./analyzer[@type='index']";
     anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
     Analyzer analyzer = readAnalyzer(anode);
-    
+
     // a custom similarity[Factory]
     expression = "./similarity";
     anode = (Node)xpath.evaluate(expression, node, XPathConstants.NODE);
@@ -101,9 +101,16 @@ public final class FieldTypePluginLoader
     
     if (queryAnalyzer==null) queryAnalyzer=analyzer;
     if (analyzer==null) analyzer=queryAnalyzer;
+    if (multiAnalyzer == null) {
+      Boolean legacyMatch = ! schema.getDefaultLuceneMatchVersion().onOrAfter(Version.LUCENE_36);
+      legacyMatch = (DOMUtil.getAttr(node, "legacyMultiTerm", null) == null) ? legacyMatch :
+          Boolean.parseBoolean(DOMUtil.getAttr(node, "legacyMultiTerm", null));
+      multiAnalyzer = constructMultiTermAnalyzer(queryAnalyzer, legacyMatch);
+    }
     if (analyzer!=null) {
       ft.setAnalyzer(analyzer);
       ft.setQueryAnalyzer(queryAnalyzer);
+      ft.setMultiTermAnalyzer(multiAnalyzer);
     }
     if (similarity!=null) {
       ft.setSimilarity(similarity);
@@ -130,6 +137,42 @@ public final class FieldTypePluginLoader
     return fieldTypes.put( name, plugin );
   }
 
+  // The point here is that, if no multitermanalyzer was specified in the schema file, do one of several things:
+  // 1> If legacyMultiTerm == false, assemble a new analyzer composed of all of the charfilters,
+  //    lowercase filters and asciifoldingfilter.
+  // 2> If letacyMultiTerm == true just construct the analyzer from a KeywordTokenizer. That should mimic current behavior.
+  //    Do the same if they've specified that the old behavior is required (legacyMultiTerm="true")
+
+  private Analyzer constructMultiTermAnalyzer(Analyzer queryAnalyzer, Boolean legacyMultiTerm) {
+    if (queryAnalyzer == null) return null;
+
+    if (legacyMultiTerm || (!(queryAnalyzer instanceof TokenizerChain))) {
+      return new KeywordAnalyzer();
+    }
+
+    TokenizerChain tc = (TokenizerChain) queryAnalyzer;
+
+    // we know it'll never be longer than this unless the code below is explicitly changed
+    TokenFilterFactory[] filters = new TokenFilterFactory[2];
+    int idx = 0;
+    for (TokenFilterFactory factory : tc.getTokenFilterFactories()) {
+      if (factory instanceof LowerCaseFilterFactory) {
+        filters[idx] = new LowerCaseFilterFactory();
+        filters[idx++].init(factory.getArgs());
+      }
+      if (factory instanceof ASCIIFoldingFilterFactory) {
+        filters[idx] = new ASCIIFoldingFilterFactory();
+        filters[idx++].init(factory.getArgs());
+      }
+    }
+    WhitespaceTokenizerFactory white = new WhitespaceTokenizerFactory();
+    white.init(tc.getTokenizerFactory().getArgs());
+
+    return new TokenizerChain(tc.getCharFilterFactories(),
+        white,
+        Arrays.copyOfRange(filters, 0, idx));
+  }
+
   //
   // <analyzer><tokenizer class="...."/><tokenizer class="...." arg="....">
   //

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SchemaField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SchemaField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SchemaField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SchemaField.java Sat Nov 26 01:03:02 2011
@@ -97,6 +97,9 @@ public final class SchemaField extends F
   boolean isTokenized() { return (properties & TOKENIZED)!=0; }
   boolean isBinary() { return (properties & BINARY)!=0; }
 
+  boolean legacyMultiTerm() {
+    return (properties & LEGACY_MULTITERM) != 0;
+  }
 
   public IndexableField createField(Object val, float boost) {
     return type.createField(this,val,boost);

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableDoubleField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableDoubleField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableDoubleField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableDoubleField.java Sat Nov 26 01:03:02 2011
@@ -24,6 +24,7 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueDouble;
 import org.apache.solr.search.QParser;
@@ -79,9 +80,9 @@ public class SortableDoubleField extends
   @Override
   public CharsRef indexedToReadable(BytesRef input, CharsRef charsRef) {
     // TODO: this could be more efficient, but the sortable types should be deprecated instead
-    input.utf8ToChars(charsRef);
+    UnicodeUtil.UTF8toUTF16(input, charsRef);
     final char[] indexedToReadable = indexedToReadable(charsRef.toString()).toCharArray();
-    charsRef.copy(indexedToReadable, 0, indexedToReadable.length);
+    charsRef.copyChars(indexedToReadable, 0, indexedToReadable.length);
     return charsRef;
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableFloatField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableFloatField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableFloatField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableFloatField.java Sat Nov 26 01:03:02 2011
@@ -24,6 +24,7 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueFloat;
 import org.apache.solr.search.QParser;
@@ -78,8 +79,9 @@ public class SortableFloatField extends 
 
   public CharsRef indexedToReadable(BytesRef input, CharsRef charsRef) {
     // TODO: this could be more efficient, but the sortable types should be deprecated instead
-    final char[] indexedToReadable = indexedToReadable(input.utf8ToChars(charsRef).toString()).toCharArray();
-    charsRef.copy(indexedToReadable, 0, indexedToReadable.length);
+    UnicodeUtil.UTF8toUTF16(input, charsRef);
+    final char[] indexedToReadable = indexedToReadable(charsRef.toString()).toCharArray();
+    charsRef.copyChars(indexedToReadable, 0, indexedToReadable.length);
     return charsRef;
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableIntField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableIntField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableIntField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableIntField.java Sat Nov 26 01:03:02 2011
@@ -24,6 +24,7 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueInt;
 import org.apache.solr.search.QParser;
@@ -76,8 +77,9 @@ public class SortableIntField extends Fi
 
   public CharsRef indexedToReadable(BytesRef input, CharsRef charsRef) {
     // TODO: this could be more efficient, but the sortable types should be deprecated instead
-    final char[] indexedToReadable = indexedToReadable(input.utf8ToChars(charsRef).toString()).toCharArray();
-    charsRef.copy(indexedToReadable, 0, indexedToReadable.length);
+    UnicodeUtil.UTF8toUTF16(input, charsRef);
+    final char[] indexedToReadable = indexedToReadable(charsRef.toString()).toCharArray();
+    charsRef.copyChars(indexedToReadable, 0, indexedToReadable.length);
     return charsRef;
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableLongField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableLongField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableLongField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/SortableLongField.java Sat Nov 26 01:03:02 2011
@@ -24,6 +24,7 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueLong;
 import org.apache.solr.search.QParser;
@@ -68,8 +69,9 @@ public class SortableLongField extends F
 
   public CharsRef indexedToReadable(BytesRef input, CharsRef charsRef) {
     // TODO: this could be more efficient, but the sortable types should be deprecated instead
-    final char[] indexedToReadable = indexedToReadable(input.utf8ToChars(charsRef).toString()).toCharArray();
-    charsRef.copy(indexedToReadable, 0, indexedToReadable.length);
+    UnicodeUtil.UTF8toUTF16(input, charsRef);
+    final char[] indexedToReadable = indexedToReadable(charsRef.toString()).toCharArray();
+    charsRef.copyChars(indexedToReadable, 0, indexedToReadable.length);
     return charsRef;
   }
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/TextField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/TextField.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/TextField.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/schema/TextField.java Sat Nov 26 01:03:02 2011
@@ -98,6 +98,11 @@ public class TextField extends FieldType
     this.queryAnalyzer = analyzer;
   }
 
+  @Override
+  public void setMultiTermAnalyzer(Analyzer analyzer) {
+    this.multiTermAnalyzer = analyzer;
+  }
+
   static Query parseFieldQuery(QParser parser, Analyzer analyzer, String field, String queryText) {
     int phraseSlop = 0;
     boolean enablePositionIncrements = true;

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Sat Nov 26 01:03:02 2011
@@ -877,7 +877,7 @@ public class SolrIndexSearcher extends I
     TermQuery key = null;
 
     if (useCache) {
-      key = new TermQuery(new Term(deState.fieldName, new BytesRef(deState.termsEnum.term())));
+      key = new TermQuery(new Term(deState.fieldName, BytesRef.deepCopyOf(deState.termsEnum.term())));
       DocSet result = filterCache.get(key);
       if (result != null) return result;
     }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java Sat Nov 26 01:03:02 2011
@@ -26,7 +26,6 @@ import org.apache.lucene.queryparser.cla
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.search.*;
 import org.apache.lucene.util.ToStringUtils;
-import org.apache.lucene.util.Version;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.BasicAutomata;
 import org.apache.lucene.util.automaton.BasicOperations;
@@ -71,7 +70,6 @@ public class SolrQueryParser extends Que
     this.schema = parser.getReq().getSchema();
     this.parser = parser;
     this.defaultField = defaultField;
-    setLowercaseExpandedTerms(false);
     setEnablePositionIncrements(true);
     checkAllowLeadingWildcards();
   }
@@ -106,6 +104,14 @@ public class SolrQueryParser extends Que
     }
   }
 
+  protected String analyzeIfMultitermTermText(String field, String part, Analyzer analyzer) {
+    if (part == null) return part;
+
+    SchemaField sf = schema.getFieldOrNull((field));
+    if (sf == null || ! (sf.getType() instanceof TextField)) return part;
+    return analyzeMultitermTerm(field, part, analyzer).utf8ToString();
+  }
+
   @Override
   protected Query getFieldQuery(String field, String queryText, boolean quoted) throws ParseException {
     checkNullField(field);
@@ -137,6 +143,8 @@ public class SolrQueryParser extends Que
   @Override
   protected Query getRangeQuery(String field, String part1, String part2, boolean startInclusive, boolean endInclusive) throws ParseException {
     checkNullField(field);
+    part1 = analyzeIfMultitermTermText(field, part1, schema.getFieldType(field).getMultiTermAnalyzer());
+    part2 = analyzeIfMultitermTermText(field, part2, schema.getFieldType(field).getMultiTermAnalyzer());
     SchemaField sf = schema.getField(field);
     return sf.getType().getRangeQuery(parser, sf, part1, part2, startInclusive, endInclusive);
   }
@@ -144,9 +152,8 @@ public class SolrQueryParser extends Que
   @Override
   protected Query getPrefixQuery(String field, String termStr) throws ParseException {
     checkNullField(field);
-    if (getLowercaseExpandedTerms()) {
-      termStr = termStr.toLowerCase();
-    }
+
+    termStr = analyzeIfMultitermTermText(field, termStr, schema.getFieldType(field).getMultiTermAnalyzer());
 
     // TODO: toInternal() won't necessarily work on partial
     // values, so it looks like we need a getPrefix() function
@@ -162,14 +169,13 @@ public class SolrQueryParser extends Que
     PrefixQuery prefixQuery = new PrefixQuery(t);
     return prefixQuery;
   }
-
   @Override
   protected Query getWildcardQuery(String field, String termStr) throws ParseException {
     // *:* -> MatchAllDocsQuery
     if ("*".equals(field) && "*".equals(termStr)) {
       return newMatchAllDocsQuery();
     }
-    
+    termStr = analyzeIfMultitermTermText(field, termStr, schema.getFieldType(field).getMultiTermAnalyzer());
     // can we use reversed wildcards in this field?
     String type = schema.getFieldType(field).getTypeName();
     ReversedWildcardFilterFactory factory = leadingWildcards.get(type);
@@ -213,4 +219,11 @@ public class SolrQueryParser extends Que
     }
     return q;
   }
+
+
+  protected Query getRegexpQuery(String field, String termStr) throws ParseException
+  {
+    termStr = analyzeIfMultitermTermText(field, termStr, schema.getFieldType(field).getMultiTermAnalyzer());
+    return super.getRegexpQuery(field, termStr);
+  }
 }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/TermQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/TermQParserPlugin.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/TermQParserPlugin.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/TermQParserPlugin.java Sat Nov 26 01:03:02 2011
@@ -59,7 +59,7 @@ public class TermQParserPlugin extends Q
         if (ft != null) {
           ft.readableToIndexed(val, term);
         } else {
-          term.copy(val);
+          term.copyChars(val);
         }
         return new TermQuery(new Term(fname, term));
       }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java Sat Nov 26 01:03:02 2011
@@ -21,6 +21,7 @@ import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
@@ -99,7 +100,8 @@ public class SearchGroupsResultTransform
         if (field != null) {
           FieldType fieldType = field.getType();
           if (sortValue instanceof BytesRef) {
-            String indexedValue = ((BytesRef) sortValue).utf8ToChars(spare).toString();
+            UnicodeUtil.UTF8toUTF16((BytesRef)sortValue, spare);
+            String indexedValue = spare.toString();
             sortValue = (Comparable) fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 0.0f));
           } else if (sortValue instanceof String) {
             sortValue = (Comparable) fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 0.0f));

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java Sat Nov 26 01:03:02 2011
@@ -27,6 +27,7 @@ import org.apache.lucene.search.grouping
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.handler.component.ShardDoc;
@@ -200,7 +201,8 @@ public class TopGroupsResultTransformer 
           if (field != null) {
             FieldType fieldType = field.getType();
             if (sortValue instanceof BytesRef) {
-              String indexedValue = ((BytesRef) sortValue).utf8ToChars(spare).toString();
+              UnicodeUtil.UTF8toUTF16((BytesRef)sortValue, spare);
+              String indexedValue = spare.toString();
               sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 0.0f));
             } else if (sortValue instanceof String) {
               sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 0.0f));
@@ -252,7 +254,8 @@ public class TopGroupsResultTransformer 
         if (field != null) {
           FieldType fieldType = field.getType();
           if (sortValue instanceof BytesRef) {
-            String indexedValue = ((BytesRef) sortValue).utf8ToChars(spare).toString();
+            UnicodeUtil.UTF8toUTF16((BytesRef)sortValue, spare);
+            String indexedValue = spare.toString();
             sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 0.0f));
           } else if (sortValue instanceof String) {
             sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 0.0f));

Modified: lucene/dev/branches/solrcloud/solr/example/solr/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/example/solr/conf/schema.xml?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/example/solr/conf/schema.xml (original)
+++ lucene/dev/branches/solrcloud/solr/example/solr/conf/schema.xml Sat Nov 26 01:03:02 2011
@@ -427,6 +427,42 @@
       </analyzer>
     </fieldType>
 
+    <!-- Illustrates the new "multiterm" analyzer definition the <fieldType> can take a new
+         parameter legacyMultiTerm="true" if the old behvaior is desired. The new default
+         behavior as of 3.6+ is to automatically define a multiterm analyzer
+    -->
+    <fieldType name="text_multiterm" class="solr.TextField" positionIncrementGap="100">
+      <analyzer type="index">
+        <tokenizer class="solr.StandardTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+      <analyzer type="query">
+        <tokenizer class="solr.StandardTokenizerFactory"/>
+        <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+      </analyzer>
+      <!-- Illustrates the use of a new analyzer type "multiterm". See the Wiki page "Multiterm 
+           Query Analysis" and SOLR-2438 for full details. The short form is that this analyzer is
+           applied to wildcard terms (prefix, wildcard range) if specified. This allows, among other
+           things, not having to lowercase wildcard terms on the client.
+           
+           In the absence of this section, the new default behavior (3.6, 4.0) is to construct
+           one of these from the query analyzer that incorporates any defined charfilters, a
+           WhitespaceTokenizer, a LowerCaseFilter (if defined), and an ASCIIFoldingFilter 
+           (if defined).
+           
+           Arguably, this is an expert-level analyzer, most cases will be handled by an instance
+           of this being automatically constructed from the queryanalyzer.
+           
+      -->
+      <analyzer type="multiterm"> 
+        <tokenizer class="solr.WhitespaceTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
+        <filter class="solr.ASCIIFoldingFilterFactory"/>
+      </analyzer>
+    </fieldType>
+
     <!-- since fields of this type are by default not stored or indexed,
          any data added to them will be ignored outright.  --> 
     <fieldtype name="ignored" stored="false" indexed="false" multiValued="true" class="solr.StrField" />

Modified: lucene/dev/branches/solrcloud/solr/example/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/example/solr/conf/solrconfig.xml?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/example/solr/conf/solrconfig.xml (original)
+++ lucene/dev/branches/solrcloud/solr/example/solr/conf/solrconfig.xml Sat Nov 26 01:03:02 2011
@@ -1570,6 +1570,15 @@
      <queryResponseWriter name="phps" class="solr.PHPSerializedResponseWriter"/>
      <queryResponseWriter name="csv" class="solr.CSVResponseWriter"/>
     -->
+
+  <queryResponseWriter name="json" class="solr.JSONResponseWriter">
+     <!-- For the purposes of the tutorial, JSON responses are written as
+      plain text so that they are easy to read in *any* browser.
+      If you expect a MIME type of "application/json" just remove this override.
+     -->
+    <str name="content-type">text/plain; charset=UTF-8</str>
+  </queryResponseWriter>
+  
   <!--
      Custom response writers can be declared as needed...
     -->

Modified: lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/index.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/index.xml?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/index.xml (original)
+++ lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/index.xml Sat Nov 26 01:03:02 2011
@@ -67,7 +67,36 @@ customization is required.
     <section id="news">
       <title>News</title>
 
-        <section><title>26 October 2011 - Java 7u1 fixes index corruption and crash bugs in Apache Lucene Core and Apache Solr</title>
+      <section>
+        <title>18 November 2011 - 2nd edition of the first book on Solr published!</title>
+        <p><img src="images/as3ess_book.jpg" class="float-right" alt="Apache Solr 3 Enterprise Search Server cover" />
+          David Smiley and Eric Pugh proudly announce the second edition of the first book on Solr,
+          <a title="PACKT: Apache Solr 3 Enterprise Search Server"
+             href="http://link.packtpub.com/2LjDxE">"Apache Solr 3 Enterprise Search Server"</a>
+          from Packt Publishing.</p>
+        <p>Apache Solr 3 Enterprise Search Server is a comprehensive reference guide for nearly every feature has to
+          offer. Through using a large set of metadata about artists, releases, and tracks courtesy of the
+          MusicBrainz.org project, you will have a testing ground for learning Solr. You'll learn how to design a
+          schema, use appropriate text analysis and then how to import this data in various
+          ways. Next, you'll learn how to search this data, how to use advanced relevancy tuning, and how to enhance
+          standard search results with highlighting, faceting, query auto-complete, and other features. The book,
+          supported with working code examples in various languages, shows how to use a wide selection of Solr
+          integration client libraries, frameworks and other software like web crawlers. The book wraps up with
+          deployment considerations, tuning Solr performance, and scaling Solr to multiple machines.
+        </p>
+        <p>This edition naturally covers the latest features in Solr as of version 3.4 like Result Grouping and
+          Geospatial, but this is not a small update to the first book. No chapter was untouched &mdash; some were
+          revamped significantly and the content was expanded by about 25% by page count. Each chapter has a tip in the
+          introduction that advises readers in a hurry on what parts should be read now or later. Finally, it includes a
+          2-page parameter quick-reference appendix that you will surely find useful printed on your desk.
+        </p>
+        <p>You can find further information at <a href="http://link.packtpub.com/2LjDxE">the publisher's site</a>
+          and at <a href="http://www.solrenterprisesearchserver.com">the authors' site</a>, including a free chapter and
+          search parameter quick-reference sheet (the appendix).
+        </p>
+      </section>
+      
+      <section><title>26 October 2011 - Java 7u1 fixes index corruption and crash bugs in Apache Lucene Core and Apache Solr</title>
           <p>Oracle released <a href="http://www.oracle.com/technetwork/java/javase/7u1-relnotes-507962.html">Java 7u1</a> on October 19.
           According to the release notes and tests done by the Lucene committers, all bugs reported on July 28 are fixed in this release,
           so code using Porter stemmer no longer crashes with <code>SIGSEGV</code>. We were not able to experience any index corruption anymore,

Modified: lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skinconf.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skinconf.xml?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skinconf.xml (original)
+++ lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skinconf.xml Sat Nov 26 01:03:02 2011
@@ -424,9 +424,9 @@ which will be used to configure the chos
     </credit>
 
     <credit box-location="alt2">
-      <name>Book: Solr 1.4 Enterprise Search Server</name>
-      <url>http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&amp;utm_medium=spons&amp;utm_content=pod&amp;utm_campaign=mdb_000275</url>
-      <image>images/solr-book-image.jpg</image>
+      <name>Book: Apache Solr 3 Enterprise Search Server</name>
+      <url>http://link.packtpub.com/2LjDxE</url>
+      <image>images/as3ess_book.jpg</image>
       <width>150</width>
       <height>185</height>
     </credit>

Modified: lucene/dev/branches/solrcloud/solr/site/features.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/features.html?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/features.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/features.html Sat Nov 26 01:03:02 2011
@@ -205,7 +205,7 @@ document.write("Last Published: " + docu
   |alternative credits
   +-->
 <div id="credit2">
-<a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&amp;utm_medium=spons&amp;utm_content=pod&amp;utm_campaign=mdb_000275"><img border="0" title="Book: Solr 1.4 Enterprise Search Server" alt="Book: Solr 1.4 Enterprise Search Server - logo" src="images/solr-book-image.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
+<a href="http://link.packtpub.com/2LjDxE"><img border="0" title="Book: Apache Solr 3 Enterprise Search Server" alt="Book: Apache Solr 3 Enterprise Search Server - logo" src="images/as3ess_book.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
 </div>
 </div>
 <!--+

Modified: lucene/dev/branches/solrcloud/solr/site/features.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/features.pdf?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/index.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/index.html?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/index.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/index.html Sat Nov 26 01:03:02 2011
@@ -208,7 +208,7 @@ document.write("Last Published: " + docu
   |alternative credits
   +-->
 <div id="credit2">
-<a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&amp;utm_medium=spons&amp;utm_content=pod&amp;utm_campaign=mdb_000275"><img border="0" title="Book: Solr 1.4 Enterprise Search Server" alt="Book: Solr 1.4 Enterprise Search Server - logo" src="images/solr-book-image.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
+<a href="http://link.packtpub.com/2LjDxE"><img border="0" title="Book: Apache Solr 3 Enterprise Search Server" alt="Book: Apache Solr 3 Enterprise Search Server - logo" src="images/as3ess_book.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
 </div>
 </div>
 <!--+
@@ -235,6 +235,9 @@ document.write("Last Published: " + docu
 <a href="#news">News</a>
 <ul class="minitoc">
 <li>
+<a href="#18+November+2011+-+2nd+edition+of+the+first+book+on+Solr+published%21">18 November 2011 - 2nd edition of the first book on Solr published!</a>
+</li>
+<li>
 <a href="#26+October+2011+-+Java+7u1+fixes+index+corruption+and+crash+bugs+in+Apache+Lucene+Core+and+Apache+Solr">26 October 2011 - Java 7u1 fixes index corruption and crash bugs in Apache Lucene Core and Apache Solr</a>
 </li>
 <li>
@@ -370,7 +373,34 @@ customization is required.
 <a name="N1005A"></a><a name="news"></a>
 <h2 class="boxed">News</h2>
 <div class="section">
-<a name="N10060"></a><a name="26+October+2011+-+Java+7u1+fixes+index+corruption+and+crash+bugs+in+Apache+Lucene+Core+and+Apache+Solr"></a>
+<a name="N10060"></a><a name="18+November+2011+-+2nd+edition+of+the+first+book+on+Solr+published%21"></a>
+<h3 class="boxed">18 November 2011 - 2nd edition of the first book on Solr published!</h3>
+<p>
+<img alt="Apache Solr 3 Enterprise Search Server cover" class="float-right" src="images/as3ess_book.jpg">
+          David Smiley and Eric Pugh proudly announce the second edition of the first book on Solr,
+          <a href="http://link.packtpub.com/2LjDxE" title="PACKT: Apache Solr 3 Enterprise Search Server">"Apache Solr 3 Enterprise Search Server"</a>
+          from Packt Publishing.</p>
+<p>Apache Solr 3 Enterprise Search Server is a comprehensive reference guide for nearly every feature has to
+          offer. Through using a large set of metadata about artists, releases, and tracks courtesy of the
+          MusicBrainz.org project, you will have a testing ground for learning Solr. You'll learn how to design a
+          schema, use appropriate text analysis and then how to import this data in various
+          ways. Next, you'll learn how to search this data, how to use advanced relevancy tuning, and how to enhance
+          standard search results with highlighting, faceting, query auto-complete, and other features. The book,
+          supported with working code examples in various languages, shows how to use a wide selection of Solr
+          integration client libraries, frameworks and other software like web crawlers. The book wraps up with
+          deployment considerations, tuning Solr performance, and scaling Solr to multiple machines.
+        </p>
+<p>This edition naturally covers the latest features in Solr as of version 3.4 like Result Grouping and
+          Geospatial, but this is not a small update to the first book. No chapter was untouched &mdash; some were
+          revamped significantly and the content was expanded by about 25% by page count. Each chapter has a tip in the
+          introduction that advises readers in a hurry on what parts should be read now or later. Finally, it includes a
+          2-page parameter quick-reference appendix that you will surely find useful printed on your desk.
+        </p>
+<p>You can find further information at <a href="http://link.packtpub.com/2LjDxE">the publisher's site</a>
+          and at<a href="http://www.solrenterprisesearchserver.com">the authors' site</a>, including a free chapter and
+          search parameter quick-reference sheet (the appendix).
+        </p>
+<a name="N10084"></a><a name="26+October+2011+-+Java+7u1+fixes+index+corruption+and+crash+bugs+in+Apache+Lucene+Core+and+Apache+Solr"></a>
 <h3 class="boxed">26 October 2011 - Java 7u1 fixes index corruption and crash bugs in Apache Lucene Core and Apache Solr</h3>
 <p>Oracle released <a href="http://www.oracle.com/technetwork/java/javase/7u1-relnotes-507962.html">Java 7u1</a> on October 19.
           According to the release notes and tests done by the Lucene committers, all bugs reported on July 28 are fixed in this release,
@@ -384,7 +414,7 @@ customization is required.
           version shipped with Java 7 changed and tokenization behaves differently
           (e.g. lowercasing). For more information, read <span class="codefrag">JRE_VERSION_MIGRATION.txt</span>
           in your distribution package!</p>
-<a name="N10089"></a><a name="14+September+2011+-+Solr+3.4.0+Released"></a>
+<a name="N100AD"></a><a name="14+September+2011+-+Solr+3.4.0+Released"></a>
 <h3 class="boxed">14 September 2011 - Solr 3.4.0 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.4.0</a>!
 	</p>
@@ -448,7 +478,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_4_0/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
 	</p>
-<a name="N10116"></a><a name="28+July+2011+-+WARNING%3A+Index+corruption+and+crashes+in+Apache+Lucene+Core+%2F+Apache+Solr+with+Java+7"></a>
+<a name="N1013A"></a><a name="28+July+2011+-+WARNING%3A+Index+corruption+and+crashes+in+Apache+Lucene+Core+%2F+Apache+Solr+with+Java+7"></a>
 <h3 class="boxed">28 July 2011 - WARNING: Index corruption and crashes in Apache Lucene Core / Apache Solr with Java 7</h3>
 <p>Oracle released <a href="http://www.oracle.com/technetwork/java/javase/jdk7-relnotes-429209.html">Java 7</a> today.
   Unfortunately it contains hotspot compiler optimizations, which miscompile some loops.
@@ -486,13 +516,13 @@ customization is required.
   version shipped with Java 7 changed and tokenization behaves differently
   (e.g. lowercasing). For more information, read <span class="codefrag">JRE_VERSION_MIGRATION.txt</span>
   in your distribution package!</p>
-<a name="N10166"></a><a name="22+July+2011+-+Solr+3.1+cookbook+published%21"></a>
+<a name="N1018A"></a><a name="22+July+2011+-+Solr+3.1+cookbook+published%21"></a>
 <h3 class="boxed">22 July 2011 - Solr 3.1 cookbook published!</h3>
 <p>
 <img alt="Solr Cookbook cover" class="float-right" src="images/solr_31_cookbook.jpg">Rafał Kuć is proud to introduce a new book on Solr, <a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book" title="PACKT:Apache Solr 3.1 Cookbook">"Apache Solr 3.1 Cookbook"</a> from Packt Publishing.</p>
 <p>The Solr 3.1 Cookbook will make your everyday work easier by using real-life examples that show you how to deal with the most common problems that can arise while using the Apache Solr search engine.</p>
 <p>This cookbook will show you how to get the most out of your search engine. Each chapter covers a different aspect of working with Solr from analyzing your text data through querying, performance improvement, and developing your own modules. The practical recipes will help you to quickly solve common problems with data analysis, show you how to use faceting to collect data and to speed up the performance of Solr. You will learn about functionalities that most newbies are unaware of, such as sorting results by a function value, highlighting matched words, and computing statistics to make your work with Solr easy and stress free.</p>
-<a name="N1017F"></a><a name="July+2011+-+Solr+3.3+Released"></a>
+<a name="N101A3"></a><a name="July+2011+-+Solr+3.3+Released"></a>
 <h3 class="boxed">July 2011 - Solr 3.3 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.3</a>!
   </p>
@@ -522,7 +552,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_3/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
  </p>
-<a name="N101B3"></a><a name="May+2011+-+Solr+3.2+Released"></a>
+<a name="N101D7"></a><a name="May+2011+-+Solr+3.2+Released"></a>
 <h3 class="boxed">May 2011 - Solr 3.2 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.2</a>!
   </p>
@@ -548,7 +578,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_2/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
  </p>
-<a name="N101E0"></a><a name="March+2011+-+Solr+3.1+Released"></a>
+<a name="N10204"></a><a name="March+2011+-+Solr+3.1+Released"></a>
 <h3 class="boxed">March 2011 - Solr 3.1 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.1</a>!
   </p>
@@ -596,7 +626,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_1/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
  </p>
-<a name="N1024B"></a><a name="25+June+2010+-+Solr+1.4.1+Released"></a>
+<a name="N1026F"></a><a name="25+June+2010+-+Solr+1.4.1+Released"></a>
 <h3 class="boxed">25 June 2010 - Solr 1.4.1 Released</h3>
 <p>
            Solr 1.4.1 has been released and is now available for public
@@ -609,7 +639,7 @@ customization is required.
           See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.4.1/CHANGES.txt">release notes</a>
           for more details.
           </p>
-<a name="N10260"></a><a name="7+May+2010+-+Apache+Lucene+Eurocon+2010+Coming+to+Prague+May+18-21"></a>
+<a name="N10284"></a><a name="7+May+2010+-+Apache+Lucene+Eurocon+2010+Coming+to+Prague+May+18-21"></a>
 <h3 class="boxed">7 May 2010 - Apache Lucene Eurocon 2010 Coming to Prague May 18-21</h3>
 <p>
           On May 18th to the 21st Prague will play host to the first
@@ -681,7 +711,7 @@ customization is required.
 </li>
         
 </ul>
-<a name="N102AC"></a><a name="10+November+2009+-+Solr+1.4+Released"></a>
+<a name="N102D0"></a><a name="10+November+2009+-+Solr+1.4+Released"></a>
 <h3 class="boxed">10 November 2009 - Solr 1.4 Released</h3>
 <p>
      Solr 1.4 has been released and is now available for public <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr/">download</a>!
@@ -713,7 +743,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.4.0/CHANGES.txt">release notes</a> for more details.
  </p>
-<a name="N102DC"></a><a name="20+August+2009+-+Solr%27s+first+book+is+published%21"></a>
+<a name="N10300"></a><a name="20+August+2009+-+Solr%27s+first+book+is+published%21"></a>
 <h3 class="boxed">20 August 2009 - Solr's first book is published!</h3>
 <p>
    
@@ -728,7 +758,7 @@ customization is required.
    </p>
 <p>Finally, this book covers various deployment considerations to include indexing strategies and performance-oriented configuration that will enable you to scale Solr to meet the needs of a high-volume site.
    </p>
-<a name="N102FC"></a><a name="18+August+2009+-+Lucene+at+US+ApacheCon"></a>
+<a name="N10320"></a><a name="18+August+2009+-+Lucene+at+US+ApacheCon"></a>
 <h3 class="boxed">18 August 2009 - Lucene at US ApacheCon</h3>
 <p>
  
@@ -804,7 +834,7 @@ Be sure not to miss:
     Search</a> - Jason Rutherglen @ 15:00</li>
 
 </ul>
-<a name="N10370"></a><a name="09+February+2009+-+Lucene+at+ApacheCon+Europe+2009+in+Amsterdam"></a>
+<a name="N10394"></a><a name="09+February+2009+-+Lucene+at+ApacheCon+Europe+2009+in+Amsterdam"></a>
 <h3 class="boxed">09 February 2009 - Lucene at ApacheCon Europe 2009 in Amsterdam</h3>
 <p>
                
@@ -842,23 +872,23 @@ Be sure not to miss:
 
 							              
 </ul>
-<a name="N103B9"></a><a name="19+December+2008+-+Solr+Logo+Contest+Results"></a>
+<a name="N103DD"></a><a name="19+December+2008+-+Solr+Logo+Contest+Results"></a>
 <h3 class="boxed">19 December 2008 - Solr Logo Contest Results</h3>
 <p>Many great logos were submitted, but only one could be chosen.  Congratulations Michiel,
 	the creator of the winning logo that is proudly displayed at the top of this page.  
 	</p>
-<a name="N103C2"></a><a name="03+October+2008+-+Solr+Logo+Contest"></a>
+<a name="N103E6"></a><a name="03+October+2008+-+Solr+Logo+Contest"></a>
 <h3 class="boxed">03 October 2008 - Solr Logo Contest</h3>
 <p>By popular demand, Solr is holding a contest to pick a new Solr logo.  Details about how to submit an entry can be found <a href="http://wiki.apache.org/solr/LogoContest">on the wiki</a>.  The Deadline for submissions is November 20th, 2008 @ 11:59PM GMT.
 	</p>
-<a name="N103CF"></a><a name="15+September+2008+-+Solr+1.3.0+Available"></a>
+<a name="N103F3"></a><a name="15+September+2008+-+Solr+1.3.0+Available"></a>
 <h3 class="boxed">15 September 2008 - Solr 1.3.0 Available</h3>
 <p>Solr 1.3.0 is available for public download.  This version contains many enhancements and bug fixes, including distributed search capabilities,
 			Lucene 2.3.x performance improvements and many others.
 		</p>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.3.0/CHANGES.txt">release notes</a> for more details.  Download is
     available from a <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr/">Apache Mirror</a>.</p>
-<a name="N103E3"></a><a name="28+August+2008+-+Lucene%2FSolr+at+ApacheCon+New+Orleans"></a>
+<a name="N10407"></a><a name="28+August+2008+-+Lucene%2FSolr+at+ApacheCon+New+Orleans"></a>
 <h3 class="boxed">28 August 2008 - Lucene/Solr at ApacheCon New Orleans</h3>
 <p>
 	         
@@ -880,7 +910,7 @@ Be sure not to miss:
 <li>An <a href="http://us.apachecon.com/c/acus2008/schedule/2008/11/05">entire day of Lucene sessions</a> on November 5th</li>
 	        
 </ul>
-<a name="N10411"></a><a name="03+September+2007+-+Lucene+at+ApacheCon+Atlanta"></a>
+<a name="N10435"></a><a name="03+September+2007+-+Lucene+at+ApacheCon+Atlanta"></a>
 <h3 class="boxed">03 September 2007 - Lucene at ApacheCon Atlanta</h3>
 <p>
 <a href="http://www.us.apachecon.com"><img alt="ApacheCon US logo" class="float-right" src="http://www.apache.org/ads/ApacheCon/2007-usa-125x125.png"></a>
@@ -900,7 +930,7 @@ Be sure not to miss:
 <li>November 16, 4:00 pm: <a href="http://us.apachecon.com/us2007/program/talk/2017"> Advanced Indexing Techniques with Apache Lucene</a> by Michael Busch. Information on payloads and advanced indexing techniques.</li>
               
 </ul>
-<a name="N1044A"></a><a name="06+June+2007%3A+Release+1.2+available"></a>
+<a name="N1046E"></a><a name="06+June+2007%3A+Release+1.2+available"></a>
 <h3 class="boxed">06 June 2007: Release 1.2 available</h3>
 <p>
         This is the first release since Solr graduated from the Incubator,
@@ -910,40 +940,40 @@ Be sure not to miss:
         and more flexible plugins.
       </p>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.2.0/CHANGES.txt">release notes</a> for more details.</p>
-<a name="N1045B"></a><a name="17+January+2007%3A+Solr+graduates+from+Incubator"></a>
+<a name="N1047F"></a><a name="17+January+2007%3A+Solr+graduates+from+Incubator"></a>
 <h3 class="boxed">17 January 2007: Solr graduates from Incubator</h3>
 <p>
         Solr has graduated from the Apache Incubator, and is now a sub-project of Lucene.
       </p>
-<a name="N10465"></a><a name="22+December+2006%3A+Release+1.1.0+available"></a>
+<a name="N10489"></a><a name="22+December+2006%3A+Release+1.1.0+available"></a>
 <h3 class="boxed">22 December 2006: Release 1.1.0 available</h3>
 <p>
         This is the first release since Solr joined the Incubator, and brings
         many new features and performance optimizations including highlighting,
         faceted search, and JSON/Python/Ruby response formats.
       </p>
-<a name="N1046F"></a><a name="15+August+2006%3A+Solr+at+ApacheCon+US"></a>
+<a name="N10493"></a><a name="15+August+2006%3A+Solr+at+ApacheCon+US"></a>
 <h3 class="boxed">15 August 2006: Solr at ApacheCon US</h3>
 <p>Chris Hostetter will be presenting
         <strong><a href="http://www.apachecon.com/2006/US/html/sessions.html#FR26">"Faceted Searching With Apache Solr"</a></strong>  
         at ApacheCon US 2006, on October 13th at 4:30pm.
         See the <a href="http://www.us.apachecon.com/">ApacheCon</a> website for more details.
       </p>
-<a name="N10482"></a><a name="21+April+2006%3A+Solr+at+ApacheCon"></a>
+<a name="N104A6"></a><a name="21+April+2006%3A+Solr+at+ApacheCon"></a>
 <h3 class="boxed">21 April 2006: Solr at ApacheCon</h3>
 <p>Yonik Seeley will be presenting
         <strong>"Apache Solr, a Full-Text Search Server based on Lucene"</strong>  
         at ApacheCon Europe 2006, on June 29th at 5:30pm.
         See the <a href="http://www.eu.apachecon.com/">ApacheCon</a> website for more details.
       </p>
-<a name="N10493"></a><a name="21+February+2006%3A+nightly+builds"></a>
+<a name="N104B7"></a><a name="21+February+2006%3A+nightly+builds"></a>
 <h3 class="boxed">21 February 2006: nightly builds</h3>
 <p>Solr now has nightly builds.  This automatically creates a
       <a href="http://people.apache.org/builds/lucene/solr/nightly/">downloadable version of Solr every
       night</a>.  All unit tests must pass, or a message is sent to
       the developers mailing list and no new version is created.  This
       also updates the <a href="api/index.html">javadoc</a>.</p>
-<a name="N104A5"></a><a name="17+January+2006%3A+Solr+Joins+Apache+Incubator"></a>
+<a name="N104C9"></a><a name="17+January+2006%3A+Solr+Joins+Apache+Incubator"></a>
 <h3 class="boxed">17 January 2006: Solr Joins Apache Incubator</h3>
 <p>Solr, a search server based on Lucene, has been accepted into the Apache Incubator.
             Solr was originally developed by CNET Networks, and is widely used within CNET

Modified: lucene/dev/branches/solrcloud/solr/site/index.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/index.pdf?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/issue_tracking.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/issue_tracking.html?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/issue_tracking.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/issue_tracking.html Sat Nov 26 01:03:02 2011
@@ -205,7 +205,7 @@ document.write("Last Published: " + docu
   |alternative credits
   +-->
 <div id="credit2">
-<a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&amp;utm_medium=spons&amp;utm_content=pod&amp;utm_campaign=mdb_000275"><img border="0" title="Book: Solr 1.4 Enterprise Search Server" alt="Book: Solr 1.4 Enterprise Search Server - logo" src="images/solr-book-image.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
+<a href="http://link.packtpub.com/2LjDxE"><img border="0" title="Book: Apache Solr 3 Enterprise Search Server" alt="Book: Apache Solr 3 Enterprise Search Server - logo" src="images/as3ess_book.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
 </div>
 </div>
 <!--+

Modified: lucene/dev/branches/solrcloud/solr/site/issue_tracking.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/issue_tracking.pdf?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/linkmap.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/linkmap.html?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/linkmap.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/linkmap.html Sat Nov 26 01:03:02 2011
@@ -205,7 +205,7 @@ document.write("Last Published: " + docu
   |alternative credits
   +-->
 <div id="credit2">
-<a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&amp;utm_medium=spons&amp;utm_content=pod&amp;utm_campaign=mdb_000275"><img border="0" title="Book: Solr 1.4 Enterprise Search Server" alt="Book: Solr 1.4 Enterprise Search Server - logo" src="images/solr-book-image.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
+<a href="http://link.packtpub.com/2LjDxE"><img border="0" title="Book: Apache Solr 3 Enterprise Search Server" alt="Book: Apache Solr 3 Enterprise Search Server - logo" src="images/as3ess_book.jpg" style="width: 150px;height: 185px;"></a><a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book"><img border="0" title="Book: Apache Solr 3.1 Cookbook" alt="Book: Apache Solr 3.1 Cookbook - logo" src="images/solr_31_cookbook.jpg" style="width: 150px;height: 180px;"></a><a href="http://www.lucidimagination.com/Downloads/LucidWorks-for-Solr/Reference-Guide?sc=AP"><img border="0" title="LucidWorks for Solr Certified Distribution Reference Guide" alt="LucidWorks for Solr Certified Distribution Reference Guide - logo" src="images/lucidworks_reference_guide.png" style="width: 150px;height: 185px;"></a>
 </div>
 </div>
 <!--+

Modified: lucene/dev/branches/solrcloud/solr/site/linkmap.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/linkmap.pdf?rev=1206387&r1=1206386&r2=1206387&view=diff
==============================================================================
Binary files - no diff available.