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 )