You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/07/07 13:40:25 UTC
lucene-solr:jira/solr-10996: SOLR-10996 Reuse existing listeners on
config change. Unit test.
Repository: lucene-solr
Updated Branches:
refs/heads/jira/solr-10996 1f02ae325 -> 11f5568b7
SOLR-10996 Reuse existing listeners on config change. Unit test.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/11f5568b
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/11f5568b
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/11f5568b
Branch: refs/heads/jira/solr-10996
Commit: 11f5568b76d4b95363ee61df7684446eff2456cc
Parents: 1f02ae3
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Fri Jul 7 15:39:54 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Fri Jul 7 15:39:54 2017 +0200
----------------------------------------------------------------------
.../cloud/autoscaling/HttpTriggerListener.java | 45 ++++-
.../cloud/autoscaling/ScheduledTriggers.java | 78 ++++++--
.../solr/cloud/autoscaling/TriggerListener.java | 6 +-
.../cloud/autoscaling/TriggerListenerBase.java | 2 +-
.../autoscaling/HttpTriggerListenerTest.java | 199 +++++++++++++++++++
.../autoscaling/TriggerIntegrationTest.java | 1 -
6 files changed, 302 insertions(+), 29 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11f5568b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
index d53e444..b5a7122 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/HttpTriggerListener.java
@@ -21,12 +21,15 @@ import java.lang.invoke.MethodHandles;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
+import java.util.StringJoiner;
import org.apache.http.HttpEntity;
import org.apache.http.HttpResponse;
import org.apache.http.client.HttpClient;
import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.protocol.HttpClientContext;
import org.apache.http.entity.StringEntity;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.PropertiesUtil;
@@ -35,7 +38,7 @@ import org.slf4j.LoggerFactory;
/**
* Simple HTTP callback that POSTs event data to a URL.
- * <p>URL and payload may contain property substitution patterns, with the following properties available:
+ * <p>URL, payload and headers may contain property substitution patterns, with the following properties available:
* <ul>
* <li>config.* - listener configuration</li>
* <li>event.* - event properties</li>
@@ -49,7 +52,7 @@ import org.slf4j.LoggerFactory;
* The following listener configuration is supported:
* <ul>
* <li>url - a URL template</li>
- * <li>payload - optional payload template. If absent a JSON string of all properties listed above will be used.</li>
+ * <li>payload - optional payload template. If absent a JSON map of all properties listed above will be used.</li>
* <li>contentType - optional payload content type. If absent then <code>application/json</code> will be used.</li>
* <li>header.* - optional header template(s). The name of the property without "header." prefix defines the literal header name.</li>
* </ul>
@@ -81,9 +84,12 @@ public class HttpTriggerListener extends TriggerListenerBase {
public void onEvent(TriggerEvent event, AutoScaling.EventProcessorStage stage, String actionName, ActionContext context, Throwable error, String message) {
Properties properties = new Properties();
properties.setProperty("stage", stage.toString());
- if (actionName != null) {
- properties.setProperty("actionName", actionName);
+ // if configuration used "actionName" but we're in a non-action related stage then PropertiesUtil will
+ // throws an exception on missing value - so replace it with an empty string
+ if (actionName == null) {
+ actionName = "";
}
+ properties.setProperty("actionName", actionName);
if (context != null) {
context.getProperties().forEach((k, v) -> {
properties.setProperty("context." + k, String.valueOf(v));
@@ -91,9 +97,13 @@ public class HttpTriggerListener extends TriggerListenerBase {
}
if (error != null) {
properties.setProperty("error", error.toString());
+ } else {
+ properties.setProperty("error", "");
}
if (message != null) {
properties.setProperty("message", message);
+ } else {
+ properties.setProperty("message", "");
}
// add event properties
properties.setProperty("event.id", event.getId());
@@ -104,8 +114,16 @@ public class HttpTriggerListener extends TriggerListenerBase {
properties.setProperty("event.properties." + k, String.valueOf(v));
});
// add config properties
+ properties.setProperty("config.name", config.name);
+ properties.setProperty("config.trigger", config.trigger);
+ properties.setProperty("config.listenerClass", config.listenerClass);
+ properties.setProperty("config.beforeActions", String.join(",", config.beforeActions));
+ properties.setProperty("config.afterActions", String.join(",", config.afterActions));
+ StringJoiner joiner = new StringJoiner(",");
+ config.stages.forEach(s -> joiner.add(s.toString()));
+ properties.setProperty("config.stages", joiner.toString());
config.properties.forEach((k, v) -> {
- properties.setProperty("config." + k, String.valueOf(v));
+ properties.setProperty("config.properties." + k, String.valueOf(v));
});
String url = PropertiesUtil.substituteProperty(urlTemplate, properties);
String payload;
@@ -131,12 +149,27 @@ public class HttpTriggerListener extends TriggerListenerBase {
});
post.setEntity(entity);
post.setHeader("Content-Type", type);
+ org.apache.http.client.config.RequestConfig.Builder requestConfigBuilder = HttpClientUtil.createDefaultRequestConfigBuilder();
+// if (soTimeout != null) {
+// requestConfigBuilder.setSocketTimeout(soTimeout);
+// }
+// if (connectionTimeout != null) {
+// requestConfigBuilder.setConnectTimeout(connectionTimeout);
+// }
+// if (followRedirects != null) {
+// requestConfigBuilder.setRedirectsEnabled(followRedirects);
+// }
+
+ post.setConfig(requestConfigBuilder.build());
try {
- HttpResponse rsp = httpClient.execute(post);
+ HttpClientContext httpClientRequestContext = HttpClientUtil.createNewHttpClientRequestContext();
+ HttpResponse rsp = httpClient.execute(post, httpClientRequestContext);
int statusCode = rsp.getStatusLine().getStatusCode();
if (statusCode != 200) {
LOG.warn("Error sending request for event " + event + ", HTTP response: " + rsp.toString());
}
+ HttpEntity responseEntity = rsp.getEntity();
+ Utils.consumeFully(responseEntity);
} catch (IOException e) {
LOG.warn("Exception sending request for event " + event, e);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11f5568b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
index d2f03f5..4b7c0d0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTriggers.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@@ -36,6 +37,7 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.cloud.ActionThrottle;
@@ -369,35 +371,75 @@ public class ScheduledTriggers implements Closeable {
}
private class TriggerListeners {
- Map<String, Map<AutoScaling.EventProcessorStage, List<TriggerListener>>> listenerPerStage = new HashMap<>();
- List<TriggerListener> listeners = new ArrayList<>();
+ Map<String, Map<AutoScaling.EventProcessorStage, List<TriggerListener>>> listenersPerStage = new HashMap<>();
+ Map<String, TriggerListener> listenersPerName = new HashMap<>();
ReentrantLock updateLock = new ReentrantLock();
void setAutoScalingConfig(AutoScalingConfig autoScalingConfig) {
updateLock.lock();
- reset();
+ // we will recreate this from scratch
+ listenersPerStage.clear();
try {
- // instantiate only those for existing triggers
Set<String> triggerNames = autoScalingConfig.getTriggerConfigs().keySet();
Map<String, AutoScalingConfig.TriggerListenerConfig> configs = autoScalingConfig.getTriggerListenerConfigs();
+ Set<String> listenerNames = configs.entrySet().stream().map(entry -> entry.getValue().name).collect(Collectors.toSet());
+ // close those for non-existent triggers and nonexistent listener configs
+ for (Iterator<Map.Entry<String, TriggerListener>> it = listenersPerName.entrySet().iterator(); it.hasNext(); ) {
+ Map.Entry<String, TriggerListener> entry = it.next();
+ String name = entry.getKey();
+ TriggerListener listener = entry.getValue();
+ if (!triggerNames.contains(listener.getConfig().trigger) || !listenerNames.contains(name)) {
+ try {
+ listener.close();
+ } catch (Exception e) {
+ log.warn("Exception closing old listener " + listener.getConfig(), e);
+ }
+ it.remove();
+ }
+ }
for (Map.Entry<String, AutoScalingConfig.TriggerListenerConfig> entry : configs.entrySet()) {
AutoScalingConfig.TriggerListenerConfig config = entry.getValue();
if (!triggerNames.contains(config.trigger)) {
log.debug("-- skipping listener for non-existent trigger: {}", config);
continue;
}
- String clazz = config.listenerClass;
+ // find previous instance and reuse if possible
+ TriggerListener oldListener = listenersPerName.get(config.name);
TriggerListener listener = null;
- try {
- listener = coreContainer.getResourceLoader().newInstance(clazz, TriggerListener.class);
- } catch (Exception e) {
- log.warn("Invalid TriggerListener class name '" + clazz + "', skipping...", e);
+ if (oldListener != null) {
+ if (!oldListener.getConfig().equals(config)) { // changed config
+ try {
+ oldListener.close();
+ } catch (Exception e) {
+ log.warn("Exception closing old listener " + listener.getConfig(), e);
+ }
+ } else {
+ listener = oldListener; // reuse
+ }
+ }
+ if (listener == null) { // create new instance
+ String clazz = config.listenerClass;
+ try {
+ listener = coreContainer.getResourceLoader().newInstance(clazz, TriggerListener.class);
+ } catch (Exception e) {
+ log.warn("Invalid TriggerListener class name '" + clazz + "', skipping...", e);
+ }
+ try {
+ listener.init(coreContainer, config);
+ listenersPerName.put(config.name, listener);
+ } catch (Exception e) {
+ log.warn("Error initializing TriggerListener " + config, e);
+ try {
+ listener.close();
+ } catch (Exception e1) {
+ // ignore
+ }
+ listener = null;
+ }
}
if (listener == null) {
continue;
}
- listener.init(coreContainer, config);
- listeners.add(listener);
// add per stage
for (AutoScaling.EventProcessorStage stage : config.stages) {
addPerStage(config.trigger, stage, listener);
@@ -417,7 +459,7 @@ public class ScheduledTriggers implements Closeable {
private void addPerStage(String triggerName, AutoScaling.EventProcessorStage stage, TriggerListener listener) {
Map<AutoScaling.EventProcessorStage, List<TriggerListener>> perStage =
- listenerPerStage.computeIfAbsent(triggerName, k -> new HashMap<>());
+ listenersPerStage.computeIfAbsent(triggerName, k -> new HashMap<>());
List<TriggerListener> lst = perStage.computeIfAbsent(stage, k -> new ArrayList<>(3));
lst.add(listener);
}
@@ -425,11 +467,11 @@ public class ScheduledTriggers implements Closeable {
void reset() {
updateLock.lock();
try {
- listenerPerStage.clear();
- for (TriggerListener listener : listeners) {
+ listenersPerStage.clear();
+ for (TriggerListener listener : listenersPerName.values()) {
IOUtils.closeQuietly(listener);
}
- listeners.clear();
+ listenersPerName.clear();
} finally {
updateLock.unlock();
}
@@ -440,7 +482,7 @@ public class ScheduledTriggers implements Closeable {
}
List<TriggerListener> getTriggerListeners(String trigger, AutoScaling.EventProcessorStage stage) {
- Map<AutoScaling.EventProcessorStage, List<TriggerListener>> perStage = listenerPerStage.get(trigger);
+ Map<AutoScaling.EventProcessorStage, List<TriggerListener>> perStage = listenersPerStage.get(trigger);
if (perStage == null) {
return Collections.emptyList();
}
@@ -471,7 +513,7 @@ public class ScheduledTriggers implements Closeable {
try {
for (TriggerListener listener : getTriggerListeners(trigger, stage)) {
if (actionName != null) {
- AutoScalingConfig.TriggerListenerConfig config = listener.getTriggerListenerConfig();
+ AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
if (stage == AutoScaling.EventProcessorStage.BEFORE_ACTION) {
if (!config.beforeActions.contains(actionName)) {
continue;
@@ -485,7 +527,7 @@ public class ScheduledTriggers implements Closeable {
try {
listener.onEvent(event, stage, actionName, context, error, message);
} catch (Exception e) {
- log.warn("Exception running listener " + listener.getTriggerListenerConfig(), e);
+ log.warn("Exception running listener " + listener.getConfig(), e);
}
}
} finally {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11f5568b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
index e67366c..479de49 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListener.java
@@ -26,9 +26,9 @@ import org.apache.solr.core.CoreContainer;
*/
public interface TriggerListener extends Closeable {
- void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config);
+ void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) throws Exception;
- AutoScalingConfig.TriggerListenerConfig getTriggerListenerConfig();
+ AutoScalingConfig.TriggerListenerConfig getConfig();
/**
* This method is called when either a particular <code>stage</code> or
@@ -41,5 +41,5 @@ public interface TriggerListener extends Closeable {
* @param message optional message
*/
void onEvent(TriggerEvent event, AutoScaling.EventProcessorStage stage, String actionName, ActionContext context,
- Throwable error, String message);
+ Throwable error, String message) throws Exception;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11f5568b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
index 2887cec..1cefa0e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/TriggerListenerBase.java
@@ -35,7 +35,7 @@ public abstract class TriggerListenerBase implements TriggerListener {
}
@Override
- public AutoScalingConfig.TriggerListenerConfig getTriggerListenerConfig() {
+ public AutoScalingConfig.TriggerListenerConfig getConfig() {
return config;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11f5568b/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
new file mode 100644
index 0000000..12a1677
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/HttpTriggerListenerTest.java
@@ -0,0 +1,199 @@
+/*
+ * 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.
+ */
+package org.apache.solr.cloud.autoscaling;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.LogLevel;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.handler.AbstractHandler;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
+
+/**
+ *
+ */
+@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
+@SolrTestCaseJ4.SuppressSSL
+public class HttpTriggerListenerTest extends SolrCloudTestCase {
+
+ private static CountDownLatch triggerFiredLatch;
+
+ private MockService mockService;
+
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ configureCluster(2)
+ .addConfig("conf", configset("cloud-minimal"))
+ .configure();
+ }
+
+ @Before
+ public void setupTest() throws Exception {
+ mockService = new MockService();
+ mockService.start();
+ triggerFiredLatch = new CountDownLatch(1);
+ }
+
+ @After
+ public void teardownTest() throws Exception {
+ if (mockService != null) {
+ mockService.close();
+ }
+ }
+
+ @Test
+ public void testHttpListenerIntegration() throws Exception {
+ CloudSolrClient solrClient = cluster.getSolrClient();
+ String setTriggerCommand = "{" +
+ "'set-trigger' : {" +
+ "'name' : 'node_added_trigger'," +
+ "'event' : 'nodeAdded'," +
+ "'waitFor' : '0s'," +
+ "'enabled' : true," +
+ "'actions' : [" +
+ "{'name':'test','class':'" + TestDummyAction.class.getName() + "'}" +
+ "]" +
+ "}}";
+ SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
+ NamedList<Object> response = solrClient.request(req);
+ assertEquals(response.get("result").toString(), "success");
+
+ String setListenerCommand = "{" +
+ "'set-listener' : " +
+ "{" +
+ "'name' : 'foo'," +
+ "'trigger' : 'node_added_trigger'," +
+ "'stage' : ['WAITING', 'STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
+ "'beforeAction' : 'test'," +
+ "'afterAction' : ['test']," +
+ "'class' : '" + HttpTriggerListener.class.getName() + "'," +
+ "'url' : '" + mockService.server.getURI().toString() + "/${config.name:invalid}/${config.properties.xyz:invalid}/${stage}'," +
+ "'payload': 'actionName=${actionName}, source=${event.source}, type=${event.eventType}'," +
+ "'header.X-Foo' : '${config.name:invalid}'," +
+ "'xyz': 'foo'" +
+ "}" +
+ "}";
+ req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
+ response = solrClient.request(req);
+ assertEquals(response.get("result").toString(), "success");
+
+ assertEquals(requests.toString(), 0, requests.size());
+
+ cluster.startJettySolrRunner();
+ boolean await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
+ assertTrue("The trigger did not fire at all", await);
+
+ Thread.sleep(5000);
+
+ assertEquals(requests.toString(), 4, requests.size());
+ requests.forEach(s -> assertTrue(s.contains("Content-Type: application/json")));
+ requests.forEach(s -> assertTrue(s.contains("X-Foo: foo")));
+ requests.forEach(s -> assertTrue(s.contains("source=node_added_trigger")));
+ requests.forEach(s -> assertTrue(s.contains("type=NODEADDED")));
+
+ String request = requests.get(0);
+ assertTrue(request, request.startsWith("/foo/foo/STARTED"));
+ assertTrue(request, request.contains("actionName=,")); // empty actionName
+
+ request = requests.get(1);
+ assertTrue(request, request.startsWith("/foo/foo/BEFORE_ACTION"));
+ assertTrue(request, request.contains("actionName=test,")); // actionName
+
+ request = requests.get(2);
+ assertTrue(request, request.startsWith("/foo/foo/AFTER_ACTION"));
+ assertTrue(request, request.contains("actionName=test,")); // actionName
+
+ request = requests.get(3);
+ assertTrue(request, request.startsWith("/foo/foo/SUCCEEDED"));
+ assertTrue(request, request.contains("actionName=,")); // empty actionName
+ }
+
+ public static class TestDummyAction extends TriggerActionBase {
+
+ @Override
+ public void process(TriggerEvent event, ActionContext context) {
+ triggerFiredLatch.countDown();
+ }
+ }
+
+
+ static List<String> requests = new ArrayList<>();
+
+ private static class MockService extends Thread {
+ Server server;
+
+ public void start() {
+ server = new Server(0);
+ server.setHandler(new AbstractHandler() {
+ @Override
+ public void handle(String s, Request request, HttpServletRequest httpServletRequest, HttpServletResponse httpServletResponse) throws IOException, ServletException {
+ StringBuilder stringBuilder = new StringBuilder();
+ stringBuilder.append(httpServletRequest.getRequestURI());
+ Enumeration<String> headerNames = httpServletRequest.getHeaderNames();
+ while (headerNames.hasMoreElements()) {
+ stringBuilder.append('\n');
+ String name = headerNames.nextElement();
+ stringBuilder.append(name);
+ stringBuilder.append(": ");
+ stringBuilder.append(httpServletRequest.getHeader(name));
+ }
+ stringBuilder.append("\n\n");
+ ServletInputStream is = request.getInputStream();
+ byte[] httpInData = new byte[request.getContentLength()];
+ int len = -1;
+ while ((len = is.read(httpInData)) != -1) {
+ stringBuilder.append(new String(httpInData, 0, len, "UTF-8"));
+ }
+ requests.add(stringBuilder.toString());
+ httpServletResponse.setStatus(HttpServletResponse.SC_OK);
+ request.setHandled(true);
+ }
+ });
+ try {
+ server.start();
+ } catch (Exception e) {
+ throw new RuntimeException("Exception starting MockService", e);
+ }
+ }
+
+ void close() throws Exception {
+ if (server != null) {
+ server.stop();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/11f5568b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
index 07a8e60..6e162f5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TriggerIntegrationTest.java
@@ -17,7 +17,6 @@
package org.apache.solr.cloud.autoscaling;
-import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.HashMap;