You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2014/12/16 12:11:51 UTC

svn commit: r1645921 [2/2] - in /lucene/dev/branches/lucene2878: ./ lucene/ lucene/analysis/ lucene/analysis/kuromoji/src/resources/org/apache/lucene/analysis/ja/dict/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/ lucene/co...

Modified: lucene/dev/branches/lucene2878/solr/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/README.txt?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/README.txt (original)
+++ lucene/dev/branches/lucene2878/solr/README.txt Tue Dec 16 11:11:50 2014
@@ -87,7 +87,7 @@ To add documents to the index, use the p
 the example/exampledocs subdirectory (while Solr is running), for example:
 
      cd example/exampledocs
-     java -jar post.jar *.xml
+     java -jar -Dc=<collection_name>post.jar *.xml
 Or:  sh post.sh *.xml
 
 For more information about Solr examples please read...

Modified: lucene/dev/branches/lucene2878/solr/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/build.xml?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/build.xml (original)
+++ lucene/dev/branches/lucene2878/solr/build.xml Tue Dec 16 11:11:50 2014
@@ -295,6 +295,9 @@
     <delete dir="package" />
     <delete dir="server/solr/lib" />
     <delete dir="example/solr/lib" />
+    <delete dir="example/cloud" />
+    <delete dir="example/techproducts" />
+    <delete dir="example/schemaless" />
     <delete includeemptydirs="true">
       <fileset dir="example">
         <include name="**/data/**/*" />

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java Tue Dec 16 11:11:50 2014
@@ -342,7 +342,7 @@ public class SuggestComponent extends Se
   }
   
   @Override
-  public Iterable<? extends Accountable> getChildResources() {
+  public Iterable<Accountable> getChildResources() {
     return Accountables.namedAccountables("field", suggesters);
   }
   

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java Tue Dec 16 11:11:50 2014
@@ -213,7 +213,7 @@ public class SolrSuggester implements Ac
   }
   
   @Override
-  public Iterable<? extends Accountable> getChildResources() {
+  public Iterable<Accountable> getChildResources() {
     return lookup.getChildResources();
   }
   

Modified: lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java Tue Dec 16 11:11:50 2014
@@ -17,6 +17,19 @@ package org.apache.solr.util;
  * limitations under the License.
  */
 
+import org.w3c.dom.Document;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.xml.bind.DatatypeConverter;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpression;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -53,20 +66,6 @@ import java.util.zip.GZIPInputStream;
 import java.util.zip.Inflater;
 import java.util.zip.InflaterInputStream;
 
-import javax.xml.bind.DatatypeConverter;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpression;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
-
-import org.w3c.dom.Document;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
 /**
  * A simple utility class for posting raw updates to a Solr server, 
  * has a main method so it can be run on the command line.
@@ -77,7 +76,6 @@ import org.xml.sax.SAXException;
 public class SimplePostTool {
   private static final String DEFAULT_POST_HOST = "localhost";
   private static final String DEFAULT_POST_PORT = "8983";
-  private static final String DEFAULT_POST_CORE = "collection1";
   private static final String VERSION_OF_THIS_TOOL = "1.5";
 
   private static final String DEFAULT_COMMIT = "yes";
@@ -217,13 +215,22 @@ public class SimplePostTool {
       if (! DATA_MODES.contains(mode)) {
         fatal("System Property 'data' is not valid for this tool: " + mode);
       }
+      
       String params = System.getProperty("params", "");
 
       String host = System.getProperty("host", DEFAULT_POST_HOST);
       String port = System.getProperty("port", DEFAULT_POST_PORT);
-      String core = System.getProperty("c", DEFAULT_POST_CORE);
-
-      urlStr = System.getProperty("url", String.format(Locale.ROOT, "http://%s:%s/solr/%s/update", host, port, core));
+      String core = System.getProperty("c");
+      
+      urlStr = System.getProperty("url");
+      
+      if (urlStr == null && core == null) {
+        fatal("Specifying either url or core/collection is mandatory.\n" + USAGE_STRING_SHORT);
+      }
+      
+      if(urlStr == null) {
+        urlStr = String.format(Locale.ROOT, "http://%s:%s/solr/%s/update", host, port, core);
+      }
       urlStr = SimplePostTool.appendParam(urlStr, params);
       URL url = new URL(urlStr);
       boolean auto = isOn(System.getProperty("auto", DEFAULT_AUTO));
@@ -372,12 +379,12 @@ public class SimplePostTool {
     System.out.println
     (USAGE_STRING_SHORT+"\n\n" +
      "Supported System Properties and their defaults:\n"+
+     "  -Dc=<core/collection>\n"+
+     "  -Durl=<solr-update-url> \n"+
      "  -Ddata=files|web|args|stdin (default=" + DEFAULT_DATA_MODE + ")\n"+
      "  -Dtype=<content-type> (default=" + DEFAULT_CONTENT_TYPE + ")\n"+
-     "  -Durl=<solr-update-url> (default=" + String.format(Locale.ROOT, "http://%s:%s/solr/%s/update", DEFAULT_POST_HOST, DEFAULT_POST_PORT, DEFAULT_POST_CORE) + ")\n"+
      "  -Dhost=<host> (default: " + DEFAULT_POST_HOST+ ")\n"+
      "  -Dport=<port> (default: " + DEFAULT_POST_PORT+ ")\n"+
-     "  -Dc=<core/collection> (default: " + DEFAULT_POST_CORE+ ")\n"+
      "  -Dauto=yes|no (default=" + DEFAULT_AUTO + ")\n"+
      "  -Drecursive=yes|no|<depth> (default=" + DEFAULT_RECURSIVE + ")\n"+
      "  -Ddelay=<seconds> (default=0 for files, 10 for web)\n"+
@@ -386,26 +393,27 @@ public class SimplePostTool {
      "  -Dcommit=yes|no (default=" + DEFAULT_COMMIT + ")\n"+
      "  -Doptimize=yes|no (default=" + DEFAULT_OPTIMIZE + ")\n"+
      "  -Dout=yes|no (default=" + DEFAULT_OUT + ")\n\n"+
-     "This is a simple command line tool for POSTing raw data to a Solr\n"+
-     "port.  Data can be read from files specified as commandline args,\n"+
+     "This is a simple command line tool for POSTing raw data to a Solr port.\n"+
+     "NOTE: Specifying the url/core/collection name is mandatory.\n" +
+     "Data can be read from files specified as commandline args,\n"+
      "URLs specified as args, as raw commandline arg strings or via STDIN.\n"+
      "Examples:\n"+
      "  java -jar post.jar *.xml\n"+
-     "  java -Ddata=args  -jar post.jar '<delete><id>42</id></delete>'\n"+
-     "  java -Ddata=stdin -jar post.jar < hd.xml\n"+
-     "  java -Ddata=web -jar post.jar http://example.com/\n"+
-     "  java -Dtype=text/csv -jar post.jar *.csv\n"+
-     "  java -Dtype=application/json -jar post.jar *.json\n"+
+     "  java -Ddata=args -Dc=gettingstarted -jar post.jar '<delete><id>42</id></delete>'\n"+
+     "  java -Ddata=stdin -Dc=gettingstarted -jar post.jar < hd.xml\n"+
+     "  java -Ddata=web -Dc=gettingstarted -jar post.jar http://example.com/\n"+
+     "  java -Dtype=text/csv -Dc=gettingstarted -jar post.jar *.csv\n"+
+     "  java -Dtype=application/json -Dc=gettingstarted -jar post.jar *.json\n"+
      "  java -Durl=http://localhost:8983/solr/update/extract -Dparams=literal.id=a -Dtype=application/pdf -jar post.jar a.pdf\n"+
-     "  java -Dauto -jar post.jar *\n"+
-     "  java -Dauto -Drecursive -jar post.jar afolder\n"+
-     "  java -Dauto -Dfiletypes=ppt,html -jar post.jar afolder\n"+
+     "  java -Dauto -Dc=gettingstarted -jar post.jar *\n"+
+     "  java -Dauto -Dc=gettingstarted -Drecursive -jar post.jar afolder\n"+
+     "  java -Dauto -Dc=gettingstarted -Dfiletypes=ppt,html -jar post.jar afolder\n"+
      "The options controlled by System Properties include the Solr\n"+
      "URL to POST to, the Content-Type of the data, whether a commit\n"+
      "or optimize should be executed, and whether the response should\n"+
      "be written to STDOUT. If auto=yes the tool will try to set type\n"+
-     "and url automatically from file name. When posting rich documents\n"+
-     "the file name will be propagated as \"resource.name\" and also used\n"+
+     "automatically from file name. When posting rich documents the\n"+
+     "file name will be propagated as \"resource.name\" and also used\n"+
      "as \"literal.id\". You may override these or any other request parameter\n"+
      "through the -Dparams property. To do a commit only, use \"-\" as argument.\n"+
      "The web mode is a simple crawler following links within domain, default delay=10s.");

Modified: lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java Tue Dec 16 11:11:50 2014
@@ -17,14 +17,16 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.internal.matchers.StringContains.containsString;
+
 /**
  * Verify that remote (proxied) queries return proper error messages
  */
@@ -60,7 +62,7 @@ public class RemoteQueryErrorTest extend
         solrServer.add(emptyDoc);
         fail("Expected unique key exceptoin");
       } catch (SolrException ex) {
-        assertEquals("Document is missing mandatory uniqueKey field: id", ex.getMessage());
+        assertThat(ex.getMessage(), containsString("Document is missing mandatory uniqueKey field: id"));
       } catch(Exception ex) {
         fail("Expected a SolrException to occur, instead received: " + ex.getClass());
       } finally {

Modified: lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/TestModifyConfFiles.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/TestModifyConfFiles.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/TestModifyConfFiles.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/cloud/TestModifyConfFiles.java Tue Dec 16 11:11:50 2014
@@ -28,6 +28,8 @@ import org.apache.solr.common.util.Simpl
 import java.io.File;
 import java.nio.charset.StandardCharsets;
 
+import static org.junit.internal.matchers.StringContains.containsString;
+
 public class TestModifyConfFiles extends AbstractFullDistribZkTestBase {
 
   public TestModifyConfFiles() {
@@ -48,7 +50,7 @@ public class TestModifyConfFiles extends
       client.request(request);
       fail("Should have caught exception");
     } catch (Exception e) {
-      assertEquals(e.getMessage(), "Input stream list was null for admin file write operation.");
+      assertThat(e.getMessage(), containsString("Input stream list was null for admin file write operation."));
     }
 
     params.remove("file");
@@ -60,7 +62,7 @@ public class TestModifyConfFiles extends
       client.request(request);
       fail("Should have caught exception");
     } catch (Exception e) {
-      assertEquals(e.getMessage(), "No file name specified for write operation.");
+      assertThat(e.getMessage(), containsString("No file name specified for write operation."));
     }
 
     params.set("op", "write");
@@ -71,7 +73,7 @@ public class TestModifyConfFiles extends
       client.request(request);
       fail("Should have caught exception");
     } catch (Exception e) {
-      assertEquals(e.getMessage(), "Can not access: bogus.txt");
+      assertThat(e.getMessage(), containsString("Can not access: bogus.txt"));
     }
 
     try {

Modified: lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java (original)
+++ lucene/dev/branches/lucene2878/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java Tue Dec 16 11:11:50 2014
@@ -17,6 +17,12 @@ package org.apache.solr.util;
  * limitations under the License.
  */
 
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.util.SimplePostTool.PageFetcher;
+import org.apache.solr.util.SimplePostTool.PageFetcherResult;
+import org.junit.Before;
+import org.junit.Test;
+
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.IOException;
@@ -29,12 +35,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.util.SimplePostTool.PageFetcher;
-import org.apache.solr.util.SimplePostTool.PageFetcherResult;
-import org.junit.Before;
-import org.junit.Test;
-
 /**
  * NOTE: do *not* use real hostnames, not even "example.com", in this test.
  *
@@ -49,6 +49,11 @@ public class SimplePostToolTest extends
   @Before
   public void initVariousPostTools() throws Exception {
     String[] args = {"-"};
+    
+    // Add a dummy core/collection property so that the SimplePostTool
+    // doesn't fail fast. 
+    System.setProperty("c", "testcollection");
+    
     System.setProperty("data", "files");
     t_file = SimplePostTool.parseArgsAndInit(args);
 

Modified: lucene/dev/branches/lucene2878/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml (original)
+++ lucene/dev/branches/lucene2878/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml Tue Dec 16 11:11:50 2014
@@ -1359,8 +1359,13 @@
     </arr>
   </requestHandler>
 
-  <!-- This causes long startup times on big indexes, even when never used.  See SOLR-6679
-  <searchComponent name="suggest" class="solr.SuggestComponent">
+  <!-- This is disabled by default because it currently causes long startup times on 
+       big indexes, even when never used.  See SOLR-6679 for background.
+
+       To use this suggester, set the "solr.suggester.enabled=true" system property
+    -->
+  <searchComponent name="suggest" class="solr.SuggestComponent" 
+                   enable="${solr.suggester.enabled:false}"     >
     <lst name="suggester">
       <str name="name">mySuggester</str>
       <str name="lookupImpl">FuzzyLookupFactory</str>      
@@ -1371,7 +1376,8 @@
     </lst>
   </searchComponent>
 
-  <requestHandler name="/suggest" class="solr.SearchHandler" startup="lazy">
+  <requestHandler name="/suggest" class="solr.SearchHandler" 
+                  startup="lazy" enable="${solr.suggester.enabled:false}" >
     <lst name="defaults">
       <str name="suggest">true</str>
       <str name="suggest.count">10</str>

Modified: lucene/dev/branches/lucene2878/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java (original)
+++ lucene/dev/branches/lucene2878/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java Tue Dec 16 11:11:50 2014
@@ -16,24 +16,6 @@
  */
 package org.apache.solr.client.solrj.impl;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.ConnectException;
-import java.net.SocketTimeoutException;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Locale;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.http.Header;
 import org.apache.http.HttpResponse;
@@ -48,7 +30,6 @@ import org.apache.http.client.methods.Ht
 import org.apache.http.client.methods.HttpPut;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.client.methods.HttpUriRequest;
-import org.apache.http.client.params.ClientPNames;
 import org.apache.http.conn.ClientConnectionManager;
 import org.apache.http.entity.ContentType;
 import org.apache.http.entity.InputStreamEntity;
@@ -80,6 +61,23 @@ import org.apache.solr.common.util.Solrj
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ConnectException;
+import java.net.SocketTimeoutException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
 public class HttpSolrServer extends SolrServer {
   private static final String UTF_8 = StandardCharsets.UTF_8.name();
   private static final String DEFAULT_PATH = "/select";
@@ -490,8 +488,7 @@ public class HttpSolrServer extends Solr
           break;
         default:
           if (processor == null) {
-            throw new RemoteSolrException(httpStatus, "Server at "
-                + getBaseURL() + " returned non ok status:" + httpStatus
+            throw new RemoteSolrException(baseUrl, httpStatus, "non ok status: " + httpStatus
                 + ", message:" + response.getStatusLine().getReasonPhrase(),
                 null);
           }
@@ -524,9 +521,9 @@ public class HttpSolrServer extends Solr
           try {
             msg = msg + " " + IOUtils.toString(respBody, encoding);
           } catch (IOException e) {
-            throw new RemoteSolrException(httpStatus, "Could not parse response with encoding " + encoding, e);
+            throw new RemoteSolrException(baseUrl, httpStatus, "Could not parse response with encoding " + encoding, e);
           }
-          RemoteSolrException e = new RemoteSolrException(httpStatus, msg, null);
+          RemoteSolrException e = new RemoteSolrException(baseUrl, httpStatus, msg, null);
           throw e;
         }
       }
@@ -544,7 +541,7 @@ public class HttpSolrServer extends Solr
       try {
         rsp = processor.processResponse(respBody, charset);
       } catch (Exception e) {
-        throw new RemoteSolrException(httpStatus, e.getMessage(), e);
+        throw new RemoteSolrException(baseUrl, httpStatus, e.getMessage(), e);
       }
       if (httpStatus != HttpStatus.SC_OK) {
         NamedList<String> metadata = null;
@@ -566,7 +563,7 @@ public class HttpSolrServer extends Solr
           msg.append("request: " + method.getURI());
           reason = java.net.URLDecoder.decode(msg.toString(), UTF_8);
         }
-        RemoteSolrException rss = new RemoteSolrException(httpStatus, reason, null);
+        RemoteSolrException rss = new RemoteSolrException(baseUrl, httpStatus, reason, null);
         if (metadata != null) rss.setMetadata(metadata);
         throw rss;
       }
@@ -814,12 +811,13 @@ public class HttpSolrServer extends Solr
    */
   public static class RemoteSolrException extends SolrException {
     /**
+     * @param remoteHost the host the error was received from
      * @param code Arbitrary HTTP status code
      * @param msg Exception Message
      * @param th Throwable to wrap with this Exception
      */
-    public RemoteSolrException(int code, String msg, Throwable th) {
-      super(code, msg, th);
+    public RemoteSolrException(String remoteHost, int code, String msg, Throwable th) {
+      super(code, "Error from server at " + remoteHost + ": " + msg, th);
     }
   }
 }

Modified: lucene/dev/branches/lucene2878/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java?rev=1645921&r1=1645920&r2=1645921&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java (original)
+++ lucene/dev/branches/lucene2878/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java Tue Dec 16 11:11:50 2014
@@ -61,6 +61,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
+import static org.junit.internal.matchers.StringContains.containsString;
+
 /**
  * This should include tests against the example solr config
  * 
@@ -408,7 +410,7 @@ abstract public class SolrExampleTests e
     }
     catch(SolrException ex) {
       assertEquals(400, ex.code());
-      assertEquals("Invalid Number: ignore_exception", ex.getMessage());  // The reason should get passed through
+      assertThat(ex.getMessage(), containsString("Invalid Number: ignore_exception"));
     }
     catch(Throwable t) {
       t.printStackTrace();