You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2015/09/28 14:10:39 UTC
svn commit: r1705662 - in /lucene/dev/trunk/solr:
core/src/java/org/apache/solr/core/
core/src/java/org/apache/solr/handler/admin/
core/src/java/org/apache/solr/response/
core/src/java/org/apache/solr/servlet/
solrj/src/java/org/apache/solr/common/para...
Author: noble
Date: Mon Sep 28 12:10:39 2015
New Revision: 1705662
URL: http://svn.apache.org/viewvc?rev=1705662&view=rev
Log:
SOLR-8083: convert the ZookeeperInfoServlet to a request handler at /admin/zookeeper
Added:
lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
- copied, changed from r1705023, lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java
Removed:
lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java
Modified:
lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
lucene/dev/trunk/solr/webapp/web/WEB-INF/web.xml
lucene/dev/trunk/solr/webapp/web/js/angular/services.js
lucene/dev/trunk/solr/webapp/web/js/scripts/cloud.js
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=1705662&r1=1705661&r2=1705662&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 Mon Sep 28 12:10:39 2015
@@ -49,6 +49,7 @@ import org.apache.solr.handler.admin.Con
import org.apache.solr.handler.admin.CoreAdminHandler;
import org.apache.solr.handler.admin.InfoHandler;
import org.apache.solr.handler.admin.SecurityConfHandler;
+import org.apache.solr.handler.admin.ZookeeperInfoHandler;
import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.handler.component.ShardHandlerFactory;
import org.apache.solr.logging.LogWatcher;
@@ -398,6 +399,7 @@ public class CoreContainer {
initializeAuthorizationPlugin((Map<String, Object>) securityConfig.data.get("authorization"));
initializeAuthenticationPlugin((Map<String, Object>) securityConfig.data.get("authentication"));
+ containerHandlers.put(ZK_PATH, new ZookeeperInfoHandler(this));
securityConfHandler = new SecurityConfHandler(this);
collectionsHandler = createHandler(cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
containerHandlers.put(COLLECTIONS_HANDLER_PATH, collectionsHandler);
Copied: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java (from r1705023, lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java?p2=lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java&p1=lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java&r1=1705023&r2=1705662&rev=1705662&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java Mon Sep 28 12:10:39 2015
@@ -15,13 +15,12 @@
* limitations under the License.
*/
-package org.apache.solr.servlet;
+package org.apache.solr.handler.admin;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
+import java.io.InputStream;
import java.io.OutputStreamWriter;
+import java.io.Reader;
import java.io.Writer;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
@@ -29,6 +28,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
+import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@@ -46,21 +46,33 @@ import org.apache.solr.common.cloud.OnRe
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
-import org.apache.solr.util.FastWriter;
+import org.apache.solr.handler.ReplicationHandler;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.JSONResponseWriter;
+import org.apache.solr.response.RawResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.SimplePostTool.BAOS;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.ByteBufferInputStream;
import org.noggit.CharArr;
import org.noggit.JSONWriter;
import org.noggit.ObjectBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.solr.common.params.CommonParams.OMIT_HEADER;
import static org.apache.solr.common.params.CommonParams.PATH;
+import static org.apache.solr.common.params.CommonParams.WT;
/**
@@ -68,20 +80,32 @@ import static org.apache.solr.common.par
*
* @since solr 4.0
*/
-public final class ZookeeperInfoServlet extends BaseSolrServlet {
- static final Logger log = LoggerFactory.getLogger(ZookeeperInfoServlet.class);
-
+public final class ZookeeperInfoHandler extends RequestHandlerBase {
+ private final CoreContainer cores;
+
+ static final Logger log = LoggerFactory.getLogger(ZookeeperInfoHandler.class);
+
// used for custom sorting collection names looking like prefix##
// only go out to 7 digits (which safely fits in an int)
private static final Pattern endsWithDigits = Pattern.compile("^(\\D*)(\\d{1,7}?)$");
-
+
+ public ZookeeperInfoHandler(CoreContainer cc) {
+ this.cores = cc;
+ }
+
+
+ @Override
+ public String getDescription() {
+ return "Fetch Zookeeper contents";
+ }
+
/**
* Enumeration of ways to filter collections on the graph panel.
*/
static enum FilterType {
- none, name, status
+ none, name, status
}
-
+
/**
* Holds state of a single page of collections requested from the cloud panel.
*/
@@ -92,84 +116,84 @@ public final class ZookeeperInfoServlet
int rows = -1;
FilterType filterType;
String filter;
-
+
PageOfCollections(int start, int rows, FilterType filterType, String filter) {
this.start = start;
this.rows = rows;
this.filterType = filterType;
this.filter = filter;
}
-
+
void selectPage(List<String> collections) {
numFound = collections.size();
// start with full set and then find the sublist for the desired selected
selected = collections;
-
+
if (rows > 0) { // paging desired
if (start > numFound)
start = 0; // this might happen if they applied a new filter
-
- int lastIndex = Math.min(start+rows, numFound);
+
+ int lastIndex = Math.min(start + rows, numFound);
if (start > 0 || lastIndex < numFound)
selected = collections.subList(start, lastIndex);
- }
+ }
}
-
+
/**
- * Filters a list of collections by name if applicable.
+ * Filters a list of collections by name if applicable.
*/
List<String> applyNameFilter(List<String> collections) {
-
+
if (filterType != FilterType.name || filter == null)
return collections; // name filter doesn't apply
-
+
// typically, a user will type a prefix and then *, e.g. tj*
// when they really mean tj.*
- String regexFilter = (!filter.endsWith(".*") && filter.endsWith("*"))
- ? filter.substring(0,filter.length()-1)+".*" : filter;
-
+ String regexFilter = (!filter.endsWith(".*") && filter.endsWith("*"))
+ ? filter.substring(0, filter.length() - 1) + ".*" : filter;
+
// case-insensitive
if (!regexFilter.startsWith("(?i)"))
- regexFilter = "(?i)"+regexFilter;
-
- Pattern filterRegex = Pattern.compile(regexFilter);
+ regexFilter = "(?i)" + regexFilter;
+
+ Pattern filterRegex = Pattern.compile(regexFilter);
List<String> filtered = new ArrayList<String>();
for (String next : collections) {
if (matches(filterRegex, next))
filtered.add(next);
}
-
+
return filtered;
}
-
+
/**
* Walk the collection state JSON object to see if it has any replicas that match
- * the state the user is filtering by.
+ * the state the user is filtering by.
*/
@SuppressWarnings("unchecked")
- final boolean matchesStatusFilter(Map<String,Object> collectionState, Set<String> liveNodes) {
-
+ final boolean matchesStatusFilter(Map<String, Object> collectionState, Set<String> liveNodes) {
+
if (filterType != FilterType.status || filter == null || filter.length() == 0)
return true; // no status filter, so all match
-
+
boolean isHealthy = true; // means all replicas for all shards active
boolean hasDownedShard = false; // means one or more shards is down
boolean replicaInRecovery = false;
-
- Map<String,Object> shards = (Map<String,Object>)collectionState.get("shards");
+
+ Map<String, Object> shards = (Map<String, Object>) collectionState.get("shards");
for (String shardId : shards.keySet()) {
boolean hasActive = false;
- Map<String,Object> shard = (Map<String,Object>)shards.get(shardId);
- Map<String,Object> replicas = (Map<String,Object>)shard.get("replicas");
+ Map<String, Object> shard = (Map<String, Object>) shards.get(shardId);
+ Map<String, Object> replicas = (Map<String, Object>) shard.get("replicas");
for (String replicaId : replicas.keySet()) {
- Map<String,Object> replicaState = (Map<String,Object>)replicas.get(replicaId);
- Replica.State coreState = Replica.State.getState((String)replicaState.get(ZkStateReader.STATE_PROP));
- String nodeName = (String)replicaState.get("node_name");
-
+ Map<String, Object> replicaState = (Map<String, Object>) replicas.get(replicaId);
+ Replica.State coreState = Replica.State.getState((String) replicaState.get(ZkStateReader.STATE_PROP));
+ String nodeName = (String) replicaState.get("node_name");
+
// state can lie to you if the node is offline, so need to reconcile with live_nodes too
if (!liveNodes.contains(nodeName))
coreState = Replica.State.DOWN; // not on a live node, so must be down
-
+
if (coreState == Replica.State.ACTIVE) {
hasActive = true; // assumed no replicas active and found one that is for this shard
} else {
@@ -177,13 +201,13 @@ public final class ZookeeperInfoServlet
replicaInRecovery = true;
}
isHealthy = false; // assumed healthy and found one replica that is not
- }
+ }
}
-
+
if (!hasActive)
hasDownedShard = true; // this is bad
}
-
+
if ("healthy".equals(filter)) {
return isHealthy;
} else if ("degraded".equals(filter)) {
@@ -193,24 +217,24 @@ public final class ZookeeperInfoServlet
} else if (Replica.State.getState(filter) == Replica.State.RECOVERING) {
return !isHealthy && replicaInRecovery;
}
-
+
return true;
}
-
+
final boolean matches(final Pattern filter, final String collName) {
return filter.matcher(collName).matches();
}
-
+
String getPagingHeader() {
- return start+"|"+rows+"|"+numFound+"|"+(filterType != null ? filterType.toString() : "")+"|"+(filter != null ? filter : "");
+ return start + "|" + rows + "|" + numFound + "|" + (filterType != null ? filterType.toString() : "") + "|" + (filter != null ? filter : "");
}
-
+
public String toString() {
return getPagingHeader();
}
}
-
+
/**
* Supports paged navigation of collections on the cloud panel. To avoid serving
* stale collection data, this object watches the /collections znode, which will
@@ -230,42 +254,42 @@ public final class ZookeeperInfoServlet
cachedCollections = null;
}
}
-
+
/**
- * Create a merged view of all collections (internal from /clusterstate.json and external from /collections/?/state.json
+ * Create a merged view of all collections (internal from /clusterstate.json and external from /collections/?/state.json
*/
private synchronized List<String> getCollections(SolrZkClient zkClient) throws KeeperException, InterruptedException {
if (cachedCollections == null) {
// cache is stale, rebuild the full list ...
cachedCollections = new ArrayList<String>();
-
+
List<String> fromZk = zkClient.getChildren("/collections", this, true);
if (fromZk != null)
cachedCollections.addAll(fromZk);
-
+
// sort the final merged set of collections
Collections.sort(cachedCollections, this);
}
-
+
return cachedCollections;
}
-
+
/**
- * Gets the requested page of collections after applying filters and offsets.
+ * Gets the requested page of collections after applying filters and offsets.
*/
- public PageOfCollections fetchPage(PageOfCollections page, SolrZkClient zkClient)
+ public PageOfCollections fetchPage(PageOfCollections page, SolrZkClient zkClient)
throws KeeperException, InterruptedException {
List<String> children = getCollections(zkClient);
page.selected = children; // start with the page being the full list
-
+
// activate paging (if disabled) for large collection sets
if (page.start == 0 && page.rows == -1 && page.filter == null && children.size() > 10) {
page.rows = 20;
page.start = 0;
}
-
+
// apply the name filter if supplied (we don't need to pull state
// data from ZK to do name filtering
if (page.filterType == FilterType.name && page.filter != null)
@@ -274,19 +298,19 @@ public final class ZookeeperInfoServlet
// a little hacky ... we can't select the page when filtering by
// status until reading all status objects from ZK
if (page.filterType != FilterType.status)
- page.selectPage(children);
-
+ page.selectPage(children);
+
return page;
}
-
+
@Override
public int compare(String left, String right) {
if (left == null)
return -1;
-
+
if (left.equals(right))
return 0;
-
+
// sort lexically unless the two collection names start with the same base prefix
// and end in a number (which is a common enough naming scheme to have direct
// support for it)
@@ -300,7 +324,7 @@ public final class ZookeeperInfoServlet
// both start with the same prefix ... compare indexes
// using longs here as we don't know how long the 2nd group is
int leftGroup2 = Integer.parseInt(leftMatcher.group(2));
- int rightGroup2 = Integer.parseInt(rightMatcher.group(2));
+ int rightGroup2 = Integer.parseInt(rightMatcher.group(2));
return (leftGroup2 > rightGroup2) ? 1 : ((leftGroup2 == rightGroup2) ? 0 : -1);
}
}
@@ -319,19 +343,16 @@ public final class ZookeeperInfoServlet
}
}
}
-
+
private PagedCollectionSupport pagingSupport;
@Override
- public void doGet(HttpServletRequest request,
- HttpServletResponse response)
- throws ServletException,IOException {
- // This attribute is set by the SolrDispatchFilter
- CoreContainer cores = (CoreContainer) request.getAttribute("org.apache.solr.CoreContainer");
- if (cores == null) {
- throw new ServletException("Missing request attribute org.apache.solr.CoreContainer.");
- }
-
+ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+ final SolrParams params = req.getParams();
+ Map<String, String> map = new HashMap<>(1);
+ map.put(WT, "raw");
+ map.put(OMIT_HEADER, "true");
+ req.setParams(SolrParams.wrapDefaults(new MapSolrParams(map), params));
synchronized (this) {
if (pagingSupport == null) {
pagingSupport = new PagedCollectionSupport();
@@ -342,18 +363,6 @@ public final class ZookeeperInfoServlet
}
}
}
-
- final SolrParams params;
- try {
- params = SolrRequestParsers.DEFAULT.parse(null, request.getServletPath(), request).getParams();
- } catch (Exception e) {
- int code=500;
- if (e instanceof SolrException) {
- code = Math.min(599, Math.max(100, ((SolrException)e).code()));
- }
- response.sendError(code, e.toString());
- return;
- }
String path = params.get(PATH);
String addr = params.get("addr");
@@ -367,10 +376,10 @@ public final class ZookeeperInfoServlet
String dumpS = params.get("dump");
boolean dump = dumpS != null && dumpS.equals("true");
-
- int start = paramAsInt("start", params, 0);
- int rows = paramAsInt("rows", params, -1);
-
+
+ int start = params.getInt("start", 0);
+ int rows = params.getInt("rows", -1);
+
String filterType = params.get("filterType");
if (filterType != null) {
filterType = filterType.trim().toLowerCase(Locale.ROOT);
@@ -378,20 +387,15 @@ public final class ZookeeperInfoServlet
filterType = null;
}
FilterType type = (filterType != null) ? FilterType.valueOf(filterType) : FilterType.none;
-
+
String filter = (type != FilterType.none) ? params.get("filter") : null;
if (filter != null) {
filter = filter.trim();
if (filter.length() == 0)
filter = null;
}
-
- response.setCharacterEncoding("UTF-8");
- response.setContentType("application/json");
-
- Writer out = new FastWriter(new OutputStreamWriter(response.getOutputStream(), StandardCharsets.UTF_8));
- ZKPrinter printer = new ZKPrinter(response, out, cores.getZkController(), addr);
+ ZKPrinter printer = new ZKPrinter(cores.getZkController(), addr);
printer.detail = detail;
printer.dump = dump;
boolean isGraphView = "graph".equals(params.get("view"));
@@ -404,65 +408,39 @@ public final class ZookeeperInfoServlet
} finally {
printer.close();
}
-
- out.flush();
- }
-
- @Override
- public void doPost(HttpServletRequest request,
- HttpServletResponse response)
- throws ServletException,IOException {
- doGet(request, response);
- }
-
- protected int paramAsInt(final String paramName, final SolrParams params, final int defaultVal) {
- int val = defaultVal;
- String paramS = params.get(paramName);
- if (paramS != null) {
- String trimmed = paramS.trim();
- if (trimmed.length() > 0) {
- try {
- val = Integer.parseInt(trimmed);
- } catch (NumberFormatException nfe) {
- log.warn("Invalid value "+paramS+" passed for parameter "+paramName+"; expected integer!");
- }
- }
- }
- return val;
+ rsp.getValues().add(RawResponseWriter.CONTENT,printer);
}
//--------------------------------------------------------------------------------------
//
//--------------------------------------------------------------------------------------
- static class ZKPrinter {
+ static class ZKPrinter implements ContentStream {
static boolean FULLPATH_DEFAULT = false;
boolean indent = true;
boolean fullpath = FULLPATH_DEFAULT;
boolean detail = false;
boolean dump = false;
-
+
String addr; // the address passed to us
String keeperAddr; // the address we're connected to
boolean doClose; // close the client after done if we opened it
- final HttpServletResponse response;
- final Writer out;
+ final BAOS baos = new BAOS();
+ final Writer out = new OutputStreamWriter(baos, StandardCharsets.UTF_8);
SolrZkClient zkClient;
int level;
int maxData = 95;
-
+
PageOfCollections page;
PagedCollectionSupport pagingSupport;
ZkController zkController;
- public ZKPrinter(HttpServletResponse response, Writer out, ZkController controller, String addr) throws IOException {
+ public ZKPrinter(ZkController controller, String addr) throws IOException {
this.zkController = controller;
- this.response = response;
- this.out = out;
this.addr = addr;
if (addr == null) {
@@ -500,6 +478,11 @@ public final class ZookeeperInfoServlet
if (doClose) {
zkClient.close();
}
+ try {
+ out.flush();
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
}
// main entry point
@@ -551,7 +534,8 @@ public final class ZookeeperInfoServlet
}
void writeError(int code, String msg) throws IOException {
- response.setStatus(code);
+ throw new SolrException(ErrorCode.getErrorCode(code), msg);
+ /*response.setStatus(code);
CharArr chars = new CharArr();
JSONWriter w = new JSONWriter(chars, 2);
@@ -567,7 +551,7 @@ public final class ZookeeperInfoServlet
w.writeString(msg);
w.endObject();
- out.write(chars.toString());
+ out.write(chars.toString());*/
}
@@ -588,7 +572,7 @@ public final class ZookeeperInfoServlet
json.writeString("attr");
json.writeNameSeparator();
json.startObject();
- writeKeyValue(json, "href", "zookeeper?detail=true&path=" + URLEncoder.encode(path, "UTF-8"), true);
+ writeKeyValue(json, "href", "admin/zookeeper?detail=true&path=" + URLEncoder.encode(path, "UTF-8"), true);
json.endObject();
json.endObject();
@@ -679,12 +663,12 @@ public final class ZookeeperInfoServlet
@SuppressWarnings("unchecked")
boolean printZnode(JSONWriter json, String path) throws IOException {
- try {
+ try {
String dataStr = null;
String dataStrErr = null;
Stat stat = new Stat();
// Trickily, the call to zkClient.getData fills in the stat variable
- byte[] data = zkClient.getData(path, null, stat, true);
+ byte[] data = zkClient.getData(path, null, stat, true);
if (null != data) {
try {
dataStr = (new BytesRef(data)).utf8ToString();
@@ -697,7 +681,7 @@ public final class ZookeeperInfoServlet
// we've already pulled the data for /clusterstate.json from ZooKeeper above,
// but it needs to be parsed into a map so we can lookup collection states before
// trying to find them in the /collections/?/state.json znode
- Map<String,Object> clusterstateJsonMap = null;
+ Map<String, Object> clusterstateJsonMap = null;
if (dataStr != null) {
try {
clusterstateJsonMap = (Map<String, Object>) ObjectBuilder.fromJSON(dataStr);
@@ -708,73 +692,73 @@ public final class ZookeeperInfoServlet
} else {
clusterstateJsonMap = Utils.makeMap();
}
-
+
// fetch the requested page of collections and then retrieve the state for each
page = pagingSupport.fetchPage(page, zkClient);
// keep track of how many collections match the filter
- boolean applyStatusFilter =
+ boolean applyStatusFilter =
(page.filterType == FilterType.status && page.filter != null);
- List<String> matchesStatusFilter = applyStatusFilter ? new ArrayList<String>() : null;
- Set<String> liveNodes = applyStatusFilter ?
+ List<String> matchesStatusFilter = applyStatusFilter ? new ArrayList<String>() : null;
+ Set<String> liveNodes = applyStatusFilter ?
zkController.getZkStateReader().getClusterState().getLiveNodes() : null;
-
- SortedMap<String,Object> collectionStates = new TreeMap<String,Object>(pagingSupport);
+
+ SortedMap<String, Object> collectionStates = new TreeMap<String, Object>(pagingSupport);
for (String collection : page.selected) {
Object collectionState = clusterstateJsonMap.get(collection);
- if (collectionState != null) {
+ if (collectionState != null) {
// collection state was in /clusterstate.json
if (applyStatusFilter) {
// verify this collection matches the status filter
- if (page.matchesStatusFilter((Map<String,Object>)collectionState,liveNodes)) {
+ if (page.matchesStatusFilter((Map<String, Object>) collectionState, liveNodes)) {
matchesStatusFilter.add(collection);
collectionStates.put(collection, collectionState);
}
} else {
- collectionStates.put(collection, collectionState);
- }
+ collectionStates.put(collection, collectionState);
+ }
} else {
// looks like an external collection ...
String collStatePath = String.format(Locale.ROOT, "/collections/%s/state.json", collection);
String childDataStr = null;
- try {
+ try {
byte[] childData = zkClient.getData(collStatePath, null, null, true);
if (childData != null)
childDataStr = (new BytesRef(childData)).utf8ToString();
} catch (KeeperException.NoNodeException nne) {
- log.warn("State for collection "+collection+
- " not found in /clusterstate.json or /collections/"+collection+"/state.json!");
+ log.warn("State for collection " + collection +
+ " not found in /clusterstate.json or /collections/" + collection + "/state.json!");
} catch (Exception childErr) {
- log.error("Failed to get "+collStatePath+" due to: "+childErr);
+ log.error("Failed to get " + collStatePath + " due to: " + childErr);
}
-
+
if (childDataStr != null) {
- Map<String,Object> extColl = (Map<String,Object>)ObjectBuilder.fromJSON(childDataStr);
+ Map<String, Object> extColl = (Map<String, Object>) ObjectBuilder.fromJSON(childDataStr);
collectionState = extColl.get(collection);
-
+
if (applyStatusFilter) {
// verify this collection matches the filtered state
- if (page.matchesStatusFilter((Map<String,Object>)collectionState,liveNodes)) {
+ if (page.matchesStatusFilter((Map<String, Object>) collectionState, liveNodes)) {
matchesStatusFilter.add(collection);
collectionStates.put(collection, collectionState);
}
} else {
- collectionStates.put(collection, collectionState);
- }
- }
- }
+ collectionStates.put(collection, collectionState);
+ }
+ }
+ }
}
-
+
if (applyStatusFilter) {
// update the paged navigation info after applying the status filter
page.selectPage(matchesStatusFilter);
-
+
// rebuild the Map of state data
- SortedMap<String,Object> map = new TreeMap<String,Object>(pagingSupport);
+ SortedMap<String, Object> map = new TreeMap<String, Object>(pagingSupport);
for (String next : page.selected)
map.put(next, collectionStates.get(next));
collectionStates = map;
- }
-
+ }
+
if (collectionStates != null) {
CharArr out = new CharArr();
new JSONWriter(out, 2).write(collectionStates);
@@ -826,5 +810,41 @@ public final class ZookeeperInfoServlet
}
return true;
}
+
+ /* @Override
+ public void write(OutputStream os) throws IOException {
+ ByteBuffer bytes = baos.getByteBuffer();
+ os.write(bytes.array(),0,bytes.limit());
+ }
+*/
+ @Override
+ public String getName() {
+ return null;
+ }
+
+ @Override
+ public String getSourceInfo() {
+ return null;
+ }
+
+ @Override
+ public String getContentType() {
+ return JSONResponseWriter.CONTENT_TYPE_JSON_UTF8;
+ }
+
+ @Override
+ public Long getSize() {
+ return null;
+ }
+
+ @Override
+ public InputStream getStream() throws IOException {
+ return new ByteBufferInputStream(baos.getByteBuffer());
+ }
+
+ @Override
+ public Reader getReader() throws IOException {
+ return null;
+ }
}
}
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java?rev=1705662&r1=1705661&r2=1705662&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/response/JSONResponseWriter.java Mon Sep 28 12:10:39 2015
@@ -42,7 +42,7 @@ import org.apache.solr.search.SolrReturn
*/
public class JSONResponseWriter implements QueryResponseWriter {
- static String CONTENT_TYPE_JSON_UTF8 = "application/json; charset=UTF-8";
+ public static String CONTENT_TYPE_JSON_UTF8 = "application/json; charset=UTF-8";
private String contentType = CONTENT_TYPE_JSON_UTF8;
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java?rev=1705662&r1=1705661&r2=1705662&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java Mon Sep 28 12:10:39 2015
@@ -169,6 +169,7 @@ public interface CommonParams {
public static final String CONFIGSETS_HANDLER_PATH = "/admin/configs";
public static final String AUTHZ_PATH = "/admin/authorization";
public static final String AUTHC_PATH = "/admin/authentication";
+ public static final String ZK_PATH = "/admin/zookeeper";
/** valid values for: <code>echoParams</code> */
public enum EchoParamStyle {
Modified: lucene/dev/trunk/solr/webapp/web/WEB-INF/web.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/webapp/web/WEB-INF/web.xml?rev=1705662&r1=1705661&r2=1705662&view=diff
==============================================================================
--- lucene/dev/trunk/solr/webapp/web/WEB-INF/web.xml (original)
+++ lucene/dev/trunk/solr/webapp/web/WEB-INF/web.xml Mon Sep 28 12:10:39 2015
@@ -76,11 +76,6 @@
</filter-mapping>
<servlet>
- <servlet-name>Zookeeper</servlet-name>
- <servlet-class>org.apache.solr.servlet.ZookeeperInfoServlet</servlet-class>
- </servlet>
-
- <servlet>
<servlet-name>LoadAdminUI</servlet-name>
<servlet-class>org.apache.solr.servlet.LoadAdminUiServlet</servlet-class>
</servlet>
@@ -101,10 +96,10 @@
<servlet-class>org.apache.solr.servlet.RedirectServlet</servlet-class>
<init-param>
<param-name>destination</param-name>
- <param-value>${context}/zookeeper</param-value>
+ <param-value>${context}/admin/zookeeper</param-value>
</init-param>
</servlet>
-
+
<servlet>
<servlet-name>RedirectLogging</servlet-name>
<servlet-class>org.apache.solr.servlet.RedirectServlet</servlet-class>
@@ -136,16 +131,14 @@
<url-pattern>/zookeeper.jsp</url-pattern>
</servlet-mapping>
<servlet-mapping>
+ <servlet-name>RedirectOldZookeeper</servlet-name>
+ <url-pattern>/zookeeper</url-pattern>
+ </servlet-mapping>
+ <servlet-mapping>
<servlet-name>RedirectLogging</servlet-name>
<url-pattern>/logging</url-pattern>
</servlet-mapping>
- <!-- Servlet Mapping -->
- <servlet-mapping>
- <servlet-name>Zookeeper</servlet-name>
- <url-pattern>/zookeeper</url-pattern>
- </servlet-mapping>
-
<servlet-mapping>
<servlet-name>LoadAdminUI</servlet-name>
<url-pattern>/admin.html</url-pattern>
Modified: lucene/dev/trunk/solr/webapp/web/js/angular/services.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/webapp/web/js/angular/services.js?rev=1705662&r1=1705661&r2=1705662&view=diff
==============================================================================
--- lucene/dev/trunk/solr/webapp/web/js/angular/services.js (original)
+++ lucene/dev/trunk/solr/webapp/web/js/angular/services.js Mon Sep 28 12:10:39 2015
@@ -60,7 +60,7 @@ solrAdminServices.factory('System',
}])
.factory('Zookeeper',
['$resource', function($resource) {
- return $resource('/solr/zookeeper', {wt:'json', _:Date.now()}, {
+ return $resource('/solr/admin/zookeeper', {wt:'json', _:Date.now()}, {
"simple": {},
"dump": {params: {dump: "true"}},
"liveNodes": {params: {path: '/live_nodes'}},
Modified: lucene/dev/trunk/solr/webapp/web/js/scripts/cloud.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/webapp/web/js/scripts/cloud.js?rev=1705662&r1=1705661&r2=1705662&view=diff
==============================================================================
--- lucene/dev/trunk/solr/webapp/web/js/scripts/cloud.js (original)
+++ lucene/dev/trunk/solr/webapp/web/js/scripts/cloud.js Mon Sep 28 12:10:39 2015
@@ -91,7 +91,7 @@ var init_debug = function( cloud_element
$.ajax
(
{
- url : app.config.solr_path + '/zookeeper?wt=json&dump=true',
+ url : app.config.solr_path + '/admin/zookeeper?wt=json&dump=true',
dataType : 'text',
context : debug_element,
beforeSend : function( xhr, settings )
@@ -455,7 +455,7 @@ var prepare_graph = function( graph_elem
$.ajax
(
{
- url : app.config.solr_path + '/zookeeper?wt=json&path=%2Flive_nodes',
+ url : app.config.solr_path + '/admin/zookeeper?wt=json&path=%2Flive_nodes',
dataType : 'json',
success : function( response, text_status, xhr )
{
@@ -467,7 +467,7 @@ var prepare_graph = function( graph_elem
var start = $( '#cloudGraphPagingStart' ).val();
var rows = $( '#cloudGraphPagingRows' ).val();
- var clusterStateUrl = app.config.solr_path + '/zookeeper?wt=json&detail=true&path=%2Fclusterstate.json&view=graph';
+ var clusterStateUrl = app.config.solr_path + '/admin/zookeeper?wt=json&detail=true&path=%2Fclusterstate.json&view=graph';
if (start && rows)
clusterStateUrl += ('&start='+start+'&rows='+rows);
@@ -591,7 +591,7 @@ var init_tree = function( tree_element )
$.ajax
(
{
- url : app.config.solr_path + '/zookeeper?wt=json',
+ url : app.config.solr_path + '/admin/zookeeper?wt=json',
dataType : 'json',
context : tree_element,
beforeSend : function( xhr, settings )
@@ -859,7 +859,7 @@ sammy.get
$.ajax
(
{
- url : app.config.solr_path + '/zookeeper?wt=json',
+ url : app.config.solr_path + '/admin/zookeeper?wt=json',
dataType : 'json',
context : cloud_element,
success : function( response, text_status, xhr )