SOLR-5555: CloudSolrServer and LBHttpSolrServer shouldn't throw MUE from constructors

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1550824 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alan Woodward 2013-12-13 19:47:06 +00:00
parent f590fe62a6
commit d22cffebb7
5 changed files with 39 additions and 49 deletions

View File

@ -74,6 +74,8 @@ Apache ZooKeeper 3.4.5
Upgrading from Solr 4.6.0
----------------------
* CloudSolrServer and LBHttpSolrServer no longer declare MalfurmedURLException
as thrown from their constructors.
Detailed Change List
----------------------
@ -248,6 +250,9 @@ Other Changes
* SOLR-5548: Give DistributedSearchTestCase / JettySolrRunner the ability to
specify extra filters. (Greg Chanan via Mark Miller)
* SOLR-5555: LBHttpSolrServer and CloudSolrServer constructors don't need to
declare MalformedURLExceptions (Sushil Bajracharya, Alan Woodward)
================== 4.6.0 ==================
Versions of Major Components

View File

@ -16,12 +16,15 @@
*/
package org.apache.solr.morphlines.solr;
import java.io.File;
import java.io.IOException;
import java.net.MalformedURLException;
import javax.xml.parsers.ParserConfigurationException;
import com.cloudera.cdk.morphline.api.MorphlineCompilationException;
import com.cloudera.cdk.morphline.api.MorphlineContext;
import com.cloudera.cdk.morphline.api.MorphlineRuntimeException;
import com.cloudera.cdk.morphline.base.Configs;
import com.google.common.base.Preconditions;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import com.typesafe.config.ConfigRenderOptions;
import com.typesafe.config.ConfigUtil;
import org.apache.solr.client.solrj.SolrServer;
import org.apache.solr.client.solrj.impl.CloudSolrServer;
import org.apache.solr.common.cloud.SolrZkClient;
@ -35,15 +38,9 @@ import org.slf4j.LoggerFactory;
import org.xml.sax.InputSource;
import org.xml.sax.SAXException;
import com.cloudera.cdk.morphline.api.MorphlineCompilationException;
import com.cloudera.cdk.morphline.api.MorphlineContext;
import com.cloudera.cdk.morphline.api.MorphlineRuntimeException;
import com.cloudera.cdk.morphline.base.Configs;
import com.google.common.base.Preconditions;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import com.typesafe.config.ConfigRenderOptions;
import com.typesafe.config.ConfigUtil;
import javax.xml.parsers.ParserConfigurationException;
import java.io.File;
import java.io.IOException;
/**
* Set of configuration parameters that identify the location and schema of a Solr server or
@ -94,14 +91,10 @@ public class SolrLocator {
if (collectionName == null || collectionName.length() == 0) {
throw new MorphlineCompilationException("Parameter 'zkHost' requires that you also pass parameter 'collection'", config);
}
try {
CloudSolrServer cloudSolrServer = new CloudSolrServer(zkHost);
cloudSolrServer.setDefaultCollection(collectionName);
cloudSolrServer.connect();
return new SolrServerDocumentLoader(cloudSolrServer, batchSize);
} catch (MalformedURLException e) {
throw new MorphlineRuntimeException(e);
}
CloudSolrServer cloudSolrServer = new CloudSolrServer(zkHost);
cloudSolrServer.setDefaultCollection(collectionName);
cloudSolrServer.connect();
return new SolrServerDocumentLoader(cloudSolrServer, batchSize);
} else {
if (solrUrl == null || solrUrl.length() == 0) {
throw new MorphlineCompilationException("Missing parameter 'solrUrl'", config);

View File

@ -16,19 +16,6 @@ package org.apache.solr.handler.component;
* limitations under the License.
*/
import java.io.IOException;
import java.net.MalformedURLException;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
@ -44,6 +31,18 @@ import org.apache.solr.util.DefaultSolrThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.apache.solr.util.plugin.PluginInfoInitialized {
protected static Logger log = LoggerFactory.getLogger(HttpShardHandlerFactory.class);
@ -158,12 +157,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
}
protected LBHttpSolrServer createLoadbalancer(HttpClient httpClient){
try {
return new LBHttpSolrServer(httpClient);
} catch (MalformedURLException e) {
// should be impossible since we're not passing any URLs here
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
return new LBHttpSolrServer(httpClient);
}
protected <T> T getParameter(NamedList initArgs, String configKey, T defaultValue) {

View File

@ -119,7 +119,7 @@ public class CloudSolrServer extends SolrServer {
* @param zkHost The client endpoint of the zookeeper quorum containing the cloud state,
* in the form HOST:PORT.
*/
public CloudSolrServer(String zkHost) throws MalformedURLException {
public CloudSolrServer(String zkHost) {
this.zkHost = zkHost;
this.myClient = HttpClientUtil.createClient(null);
this.lbServer = new LBHttpSolrServer(myClient);

View File

@ -192,14 +192,12 @@ public class LBHttpSolrServer extends SolrServer {
}
/** The provided httpClient should use a multi-threaded connection manager */
public LBHttpSolrServer(HttpClient httpClient, String... solrServerUrl)
throws MalformedURLException {
public LBHttpSolrServer(HttpClient httpClient, String... solrServerUrl) {
this(httpClient, new BinaryResponseParser(), solrServerUrl);
}
/** The provided httpClient should use a multi-threaded connection manager */
public LBHttpSolrServer(HttpClient httpClient, ResponseParser parser, String... solrServerUrl)
throws MalformedURLException {
public LBHttpSolrServer(HttpClient httpClient, ResponseParser parser, String... solrServerUrl) {
clientIsInternal = (httpClient == null);
this.parser = parser;
if (httpClient == null) {
@ -234,7 +232,7 @@ public class LBHttpSolrServer extends SolrServer {
return server;
}
protected HttpSolrServer makeServer(String server) throws MalformedURLException {
protected HttpSolrServer makeServer(String server) {
HttpSolrServer s = new HttpSolrServer(server, httpClient, parser);
if (requestWriter != null) {
s.setRequestWriter(requestWriter);