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 2013/04/26 16:32:45 UTC

svn commit: r1476227 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/test-files/old-solr-example/ core/src/test/org/apache/solr/cloud/ example/solr/ example/solr/collection1/ solrj/src/t...

Author: markrmiller
Date: Fri Apr 26 14:32:35 2013
New Revision: 1476227

URL: http://svn.apache.org/r1476227
Log:
SOLR-4757: Change the example to use the new solr.xml format and core discovery by directory structure.

Added:
    lucene/dev/trunk/solr/core/src/test-files/old-solr-example/
    lucene/dev/trunk/solr/core/src/test-files/old-solr-example/README.txt   (with props)
    lucene/dev/trunk/solr/core/src/test-files/old-solr-example/solr.xml   (with props)
    lucene/dev/trunk/solr/example/solr/collection1/core.properties   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCoreDiscoverer.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
    lucene/dev/trunk/solr/example/solr/solr.xml
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Fri Apr 26 14:32:35 2013
@@ -94,6 +94,9 @@ Other Changes
 * SOLR-4547: Move logging of filenames on commit from INFO to DEBUG.
   (Shawn Heisey, hossman)
 
+* SOLR-4757: Change the example to use the new solr.xml format and core 
+  discovery by directory structure. (Mark Miller)
+
 ==================  4.3.0 ==================
 
 Versions of Major Components

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java Fri Apr 26 14:32:35 2013
@@ -187,7 +187,7 @@ public class ZkCLI {
             Config config = new Config(loader, null, new InputSource(is), null, false);
             
             boolean oldStyle = (config.getNode("solr/cores", false) != null);
-            // cfg = new ConfigSolrXml(loader, null, is, null, false, this);
+
              if (oldStyle) {
                cfg = new ConfigSolrXmlOld(config, null);
              } else {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Fri Apr 26 14:32:35 2013
@@ -1441,12 +1441,15 @@ public final class ZkController {
   public static void bootstrapConf(SolrZkClient zkClient, ConfigSolr cfg, String solrHome) throws IOException,
       KeeperException, InterruptedException {
 
-    log.info("bootstraping config into ZooKeeper using solr.xml");
     List<String> allCoreNames = cfg.getAllCoreNames();
+    
+    log.info("bootstraping config for " + allCoreNames.size() + " cores into ZooKeeper using solr.xml from " + solrHome);
+
     for (String coreName : allCoreNames) {
       String rawName = PropertiesUtil.substituteProperty(cfg.getProperty(coreName, "name", null), new Properties());
       String instanceDir = cfg.getProperty(coreName, "instanceDir", null);
       File idir = new File(instanceDir);
+      System.out.println("idir:" + idir);
       if (!idir.isAbsolute()) {
         idir = new File(solrHome, instanceDir);
       }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java Fri Apr 26 14:32:35 2013
@@ -49,15 +49,15 @@ public class ConfigSolrXml extends Confi
   public ConfigSolrXml(Config config, CoreContainer container)
       throws ParserConfigurationException, IOException, SAXException {
     super(config);
-    checkForIllegalConfig(container);
+    checkForIllegalConfig();
     
     fillPropMap();
     
-    String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY, (container == null ? null : container.getSolrHome()));
+    String coreRoot = get(CfgProp.SOLR_COREROOTDIRECTORY, (container == null ? config.getResourceLoader().getInstanceDir() : container.getSolrHome()));
     coreDescriptorMap = solrCoreDiscoverer.discover(container, new File(coreRoot));
   }
   
-  private void checkForIllegalConfig(CoreContainer container) throws IOException {
+  private void checkForIllegalConfig() throws IOException {
     
     // Do sanity checks - we don't want to find old style config
     failIfFound("solr/@coreLoadThreads");

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Fri Apr 26 14:32:35 2013
@@ -183,6 +183,9 @@ public class CoreContainer
       log.info("looking for solr config file: " + fconf.getAbsolutePath());
       cores = new CoreContainer(solrHome);
 
+      // first we find zkhost, then we check for solr.xml in zk
+      // 1. look for zkhost from sys prop 2. look for zkhost in {solr.home}/solr.properties
+      
       // Either we have a config file or not.
       
       if (fconf.exists()) {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java Fri Apr 26 14:32:35 2013
@@ -78,6 +78,7 @@ public class CoreDescriptor {
     coreProperties.put(CORE_TRANSIENT, "false");
 
   }
+  
   public CoreDescriptor(CoreContainer container, String name, String instanceDir) {
     this(container);
     doInit(name, instanceDir);
@@ -202,7 +203,7 @@ public class CoreDescriptor {
    */
   public String getInstanceDir() {
     String instDir = coreProperties.getProperty(CORE_INSTDIR);
-    if (instDir == null) return null; // No worse than before.
+    if (instDir == null) return null;
 
     if (new File(instDir).isAbsolute()) {
       return SolrResourceLoader.normalizeDir(

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCoreDiscoverer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCoreDiscoverer.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCoreDiscoverer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCoreDiscoverer.java Fri Apr 26 14:32:35 2013
@@ -37,9 +37,9 @@ public class SolrCoreDiscoverer {
   
   public Map<String, CoreDescriptor> discover(CoreContainer container, File root) throws IOException {
     Map<String, CoreDescriptor> coreDescriptorMap = new HashMap<String, CoreDescriptor>();
-    if (container != null) { 
-      walkFromHere(root, container, coreDescriptorMap);
-    }
+
+    walkFromHere(root, container, coreDescriptorMap);
+    
     return coreDescriptorMap;
   }
   
@@ -83,6 +83,7 @@ public class SolrCoreDiscoverer {
 
     // Too much of the code depends on this value being here, but it is NOT supported in discovery mode, so
     // ignore it if present in the core.properties file.
+    System.out.println("SET INST DIR:" + childFile.getPath());
     props.setProperty(CoreDescriptor.CORE_INSTDIR, childFile.getPath());
 
     if (props.getProperty(CoreDescriptor.CORE_NAME) == null) {

Added: lucene/dev/trunk/solr/core/src/test-files/old-solr-example/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/old-solr-example/README.txt?rev=1476227&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/old-solr-example/README.txt (added)
+++ lucene/dev/trunk/solr/core/src/test-files/old-solr-example/README.txt Fri Apr 26 14:32:35 2013
@@ -0,0 +1 @@
+This is around for back compat testing purposes and should be able to be removed in Solr 5.0
\ No newline at end of file

Added: lucene/dev/trunk/solr/core/src/test-files/old-solr-example/solr.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/old-solr-example/solr.xml?rev=1476227&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/old-solr-example/solr.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/old-solr-example/solr.xml Fri Apr 26 14:32:35 2013
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!--
+   This is an example of a simple "solr.xml" file for configuring one or 
+   more Solr Cores, as well as allowing Cores to be added, removed, and 
+   reloaded via HTTP requests.
+
+   More information about options available in this configuration file, 
+   and Solr Core administration can be found online:
+   http://wiki.apache.org/solr/CoreAdmin
+-->
+
+<!--
+ All (relative) paths are relative to the Solr Home Directory
+  
+  persistent: Save changes made via the API to this file
+  sharedLib: path to a lib directory that will be shared across all cores
+-->
+<solr persistent="true">
+	<!-- by default, this is 50 @ WARN
+  <logging enabled="true">
+  	<watcher size="100" threshold="INFO" />
+  </logging>
+   -->
+
+  <!--
+  adminPath: RequestHandler path to manage cores.  
+    If 'null' (or absent), cores will not be manageable via request handler
+  defaultCoreName: (optional) core to use when no core name is specified in an access url
+    
+  All of the attributes in cores after defaultCoreName only apply when running in SolrCloud mode.
+  You can read more about SolrCloud mode at http://wiki.apache.org/solr/SolrCloud
+  -->
+  <cores adminPath="/admin/cores" defaultCoreName="collection1" host="${host:}" hostPort="${jetty.port:8983}" hostContext="${hostContext:solr}" zkClientTimeout="${zkClientTimeout:15000}">
+    <core name="collection1" instanceDir="collection1" />
+  </cores>
+</solr>

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java Fri Apr 26 14:32:35 2013
@@ -20,6 +20,7 @@ package org.apache.solr.cloud;
 import java.io.File;
 import java.util.List;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -42,6 +43,8 @@ public class ZkCLITest extends SolrTestC
   protected ZkTestServer zkServer;
   
   protected String zkDir;
+  
+  private String solrHome;
 
   private SolrZkClient zkClient;
   
@@ -62,6 +65,18 @@ public class ZkCLITest extends SolrTestC
     log.info("####SETUP_START " + getTestName());
     createTempDir();
     
+    boolean useNewSolrXml = random().nextBoolean();
+    
+    if (useNewSolrXml) {
+      solrHome = ExternalPaths.EXAMPLE_HOME;
+    } else {
+      File tmpSolrHome = new File(dataDir, "tmp-solr-home");
+      FileUtils.copyDirectory(new File(ExternalPaths.EXAMPLE_HOME), tmpSolrHome);
+      FileUtils.copyFile(new File(ExternalPaths.SOURCE_HOME, "core/src/test-files/old-solr-example/solr.xml"), new File(tmpSolrHome, "solr.xml"));
+      solrHome = tmpSolrHome.getAbsolutePath();
+    }
+    
+    
     zkDir = dataDir.getAbsolutePath() + File.separator
         + "zookeeper/server1/data";
     log.info("ZooKeeper dataDir:" + zkDir);
@@ -83,7 +98,7 @@ public class ZkCLITest extends SolrTestC
   public void testBootstrap() throws Exception {
     // test bootstrap_conf
     String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",
-        "bootstrap", "-solrhome", ExternalPaths.EXAMPLE_HOME};
+        "bootstrap", "-solrhome", this.solrHome};
     ZkCLI.main(args);
     
     assertTrue(zkClient.exists(ZkController.CONFIGS_ZKNODE + "/collection1", true));
@@ -102,7 +117,7 @@ public class ZkCLITest extends SolrTestC
     assertFalse(zkClient.exists(chroot, true));
     
     String[] args = new String[] {"-zkhost", zkServer.getZkAddress() + chroot,
-        "-cmd", "bootstrap", "-solrhome", ExternalPaths.EXAMPLE_HOME};
+        "-cmd", "bootstrap", "-solrhome", this.solrHome};
     
     ZkCLI.main(args);
     

Added: lucene/dev/trunk/solr/example/solr/collection1/core.properties
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/example/solr/collection1/core.properties?rev=1476227&view=auto
==============================================================================
--- lucene/dev/trunk/solr/example/solr/collection1/core.properties (added)
+++ lucene/dev/trunk/solr/example/solr/collection1/core.properties Fri Apr 26 14:32:35 2013
@@ -0,0 +1 @@
+name=collection1
\ No newline at end of file

Modified: lucene/dev/trunk/solr/example/solr/solr.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/example/solr/solr.xml?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/example/solr/solr.xml (original)
+++ lucene/dev/trunk/solr/example/solr/solr.xml Fri Apr 26 14:32:35 2013
@@ -26,28 +26,19 @@
    http://wiki.apache.org/solr/CoreAdmin
 -->
 
-<!--
- All (relative) paths are relative to the Solr Home Directory
-  
-  persistent: Save changes made via the API to this file
-  sharedLib: path to a lib directory that will be shared across all cores
--->
-<solr persistent="true">
-	<!-- by default, this is 50 @ WARN
-  <logging enabled="true">
-  	<watcher size="100" threshold="INFO" />
-  </logging>
-   -->
+<solr>
+
+  <solrcloud>
+    <str name="host">${host:}</str>
+    <int name="hostPort">${jetty.port:8983}</int>
+    <str name="hostContext">${hostContext:solr}</str>
+    <int name="zkClientTimeout">${zkClientTimeout:15000}</int>
+  </solrcloud>
+
+  <shardHandlerFactory name="shardHandlerFactory"
+    class="HttpShardHandlerFactory">
+    <int name="socketTimeout">${socketTimeout:0}</int>
+    <int name="connTimeout">${connTimeout:0}</int>
+  </shardHandlerFactory>
 
-  <!--
-  adminPath: RequestHandler path to manage cores.  
-    If 'null' (or absent), cores will not be manageable via request handler
-  defaultCoreName: (optional) core to use when no core name is specified in an access url
-    
-  All of the attributes in cores after defaultCoreName only apply when running in SolrCloud mode.
-  You can read more about SolrCloud mode at http://wiki.apache.org/solr/SolrCloud
-  -->
-  <cores adminPath="/admin/cores" defaultCoreName="collection1" host="${host:}" hostPort="${jetty.port:8983}" hostContext="${hostContext:solr}" zkClientTimeout="${zkClientTimeout:15000}">
-    <core name="collection1" instanceDir="collection1" />
-  </cores>
 </solr>

Modified: lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java (original)
+++ lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleBinaryTest.java Fri Apr 26 14:32:35 2013
@@ -41,7 +41,7 @@ public class SolrExampleBinaryTest exten
   {
     try {
       // setup the server...
-      String url = jetty.getBaseUrl().toString();
+      String url = jetty.getBaseUrl().toString() + "/collection1";
       HttpSolrServer s = new HttpSolrServer( url );
       s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
       s.setDefaultMaxConnectionsPerHost(100);

Modified: lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java (original)
+++ lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleXMLTest.java Fri Apr 26 14:32:35 2013
@@ -36,7 +36,7 @@ public class SolrExampleXMLTest extends 
   @Override
   public SolrServer createNewSolrServer() {
     try {
-      String url = jetty.getBaseUrl().toString();
+      String url = jetty.getBaseUrl().toString() + "/collection1";
       HttpSolrServer s = new HttpSolrServer(url);
       s.setUseMultiPartPost(random().nextBoolean());
       s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);

Modified: lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java (original)
+++ lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java Fri Apr 26 14:32:35 2013
@@ -55,7 +55,7 @@ public class SolrExampleStreamingTest ex
   {
     try {
       // setup the server...
-      String url = jetty.getBaseUrl().toString();
+      String url = jetty.getBaseUrl().toString() + "/collection1";
       // smaller queue size hits locks more often
       ConcurrentUpdateSolrServer s = new ConcurrentUpdateSolrServer( url, 2, 5 ) {
         
@@ -80,7 +80,7 @@ public class SolrExampleStreamingTest ex
     // SOLR-3903
     final List<Throwable> failures = new ArrayList<Throwable>();
     ConcurrentUpdateSolrServer s = new ConcurrentUpdateSolrServer
-      (jetty.getBaseUrl().toString(), 2, 2) {
+      (jetty.getBaseUrl().toString() + "/collection1", 2, 2) {
         @Override
         public void handleError(Throwable ex) {
           failures.add(ex);

Modified: lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java (original)
+++ lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java Fri Apr 26 14:32:35 2013
@@ -57,7 +57,7 @@ public class BasicHttpSolrServerTest ext
     @Override
     protected void doGet(HttpServletRequest req, HttpServletResponse resp)
         throws ServletException, IOException {
-      resp.sendRedirect("/solr/select?" + req.getQueryString());
+      resp.sendRedirect("/solr/collection1/select?" + req.getQueryString());
     }
   }
   
@@ -439,7 +439,7 @@ public class BasicHttpSolrServerTest ext
     assertNull(DebugServlet.headers.get("Accept-Encoding"));
     
     // verify server compresses output
-    HttpGet get = new HttpGet(jetty.getBaseUrl().toString() + 
+    HttpGet get = new HttpGet(jetty.getBaseUrl().toString() + "/collection1" +
                               "/select?q=foo&wt=xml");
     get.setHeader("Accept-Encoding", "gzip");
     HttpClient client = HttpClientUtil.createClient(null);
@@ -458,7 +458,7 @@ public class BasicHttpSolrServerTest ext
     }
     
     // verify compressed response can be handled
-    server = new HttpSolrServer(jetty.getBaseUrl().toString());
+    server = new HttpSolrServer(jetty.getBaseUrl().toString() + "/collection1");
     server.setAllowCompression(true);
     q = new SolrQuery("foo");
     QueryResponse response = server.query(q);
@@ -485,7 +485,7 @@ public class BasicHttpSolrServerTest ext
   @Test
   public void testGetRawStream() throws SolrServerException, IOException{
     HttpClient client = HttpClientUtil.createClient(null);
-    HttpSolrServer server = new HttpSolrServer(jetty.getBaseUrl().toString(), 
+    HttpSolrServer server = new HttpSolrServer(jetty.getBaseUrl().toString() + "/collection1", 
                                                client, null);
     QueryRequest req = new QueryRequest();
     NamedList response = server.request(req);

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java?rev=1476227&r1=1476226&r2=1476227&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java Fri Apr 26 14:32:35 2013
@@ -17,25 +17,19 @@ package org.apache.solr;
  * limitations under the License.
  */
 
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.util.ExternalPaths;
-
-import java.io.File;
-import java.util.Collections;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.solr.util.RESTfulServerProvider;
-import org.apache.solr.util.RestTestHarness;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.restlet.ext.servlet.ServerServlet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -159,7 +153,7 @@ abstract public class SolrJettyTestBase 
     if (jetty != null) {
       try {
         // setup the server...
-        String url = jetty.getBaseUrl().toString();
+        String url = jetty.getBaseUrl().toString() + "/" + "collection1";
         HttpSolrServer s = new HttpSolrServer( url );
         s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
         s.setDefaultMaxConnectionsPerHost(100);