You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2010/10/13 19:01:33 UTC

svn commit: r1022188 [4/4] - in /lucene/dev/trunk/solr: ./ lib/ src/common/org/apache/solr/common/cloud/ src/common/org/apache/solr/common/params/ src/java/org/apache/solr/cloud/ src/java/org/apache/solr/core/ src/java/org/apache/solr/handler/admin/ sr...

Added: lucene/dev/trunk/solr/src/test/org/apache/solr/cloud/ZkTestServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/org/apache/solr/cloud/ZkTestServer.java?rev=1022188&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/test/org/apache/solr/cloud/ZkTestServer.java (added)
+++ lucene/dev/trunk/solr/src/test/org/apache/solr/cloud/ZkTestServer.java Wed Oct 13 17:01:13 2010
@@ -0,0 +1,319 @@
+package org.apache.solr.cloud;
+
+/**
+ * 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.
+ */
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.management.JMException;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.zookeeper.jmx.ManagedUtil;
+import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerConfig;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.SessionTracker.Session;
+import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
+
+public class ZkTestServer {
+
+  protected final ZKServerMain zkServer = new ZKServerMain();
+
+  private String zkDir;
+
+  private int clientPort;
+
+  private Thread zooThread;
+
+  class ZKServerMain {
+
+    private NIOServerCnxn.Factory cnxnFactory;
+    private ZooKeeperServer zooKeeperServer;
+    
+    protected void initializeAndRun(String[] args) throws ConfigException,
+        IOException {
+      try {
+        ManagedUtil.registerLog4jMBeans();
+      } catch (JMException e) {
+
+      }
+
+      ServerConfig config = new ServerConfig();
+      if (args.length == 1) {
+        config.parse(args[0]);
+      } else {
+        config.parse(args);
+      }
+
+      runFromConfig(config);
+    }
+
+    /**
+     * Run from a ServerConfig.
+     * 
+     * @param config ServerConfig to use.
+     * @throws IOException
+     */
+    public void runFromConfig(ServerConfig config) throws IOException {
+      try {
+        // Note that this thread isn't going to be doing anything else,
+        // so rather than spawning another thread, we will just call
+        // run() in this thread.
+        // create a file logger url from the command line args
+        zooKeeperServer = new ZooKeeperServer();
+
+        FileTxnSnapLog ftxn = new FileTxnSnapLog(new File(config
+            .getDataLogDir()), new File(config.getDataDir()));
+        zooKeeperServer.setTxnLogFactory(ftxn);
+        zooKeeperServer.setTickTime(config.getTickTime());
+        cnxnFactory = new NIOServerCnxn.Factory(config.getClientPortAddress(), config
+            .getMaxClientCnxns());
+        cnxnFactory.startup(zooKeeperServer);
+        cnxnFactory.join();
+        if (zooKeeperServer.isRunning()) {
+          zooKeeperServer.shutdown();
+        }
+      } catch (InterruptedException e) {
+      }
+    }
+
+    /**
+     * Shutdown the serving instance
+     * @throws IOException 
+     */
+    protected void shutdown() throws IOException {
+      zooKeeperServer.shutdown();
+      zooKeeperServer.getZKDatabase().close();
+      waitForServerDown(getZkHost() + ":" + getPort(), 5000);
+      cnxnFactory.shutdown();
+    }
+
+    public int getLocalPort() {
+      if (cnxnFactory == null) {
+        throw new IllegalStateException("A port has not yet been selected");
+      }
+      int port = cnxnFactory.getLocalPort();
+      if (port == 0) {
+        throw new IllegalStateException("A port has not yet been selected");
+      }
+      return port;
+    }
+  }
+
+  public ZkTestServer(String zkDir) {
+    this.zkDir = zkDir;
+  }
+
+  public ZkTestServer(String zkDir, int port) {
+    this.zkDir = zkDir;
+    this.clientPort = port;
+  }
+
+  public String getZkHost() {
+    return "127.0.0.1:" + zkServer.getLocalPort();
+  }
+
+  public String getZkAddress() {
+    return "127.0.0.1:" + zkServer.getLocalPort() + "/solr";
+  }
+
+  public int getPort() {
+    return zkServer.getLocalPort();
+  }
+  
+  public void expire(final long sessionId) {
+    zkServer.zooKeeperServer.expire(new Session() {
+      @Override
+      public long getSessionId() {
+        return sessionId;
+      }
+      @Override
+      public int getTimeout() {
+        return 4000;
+      }});
+  }
+
+  public void run() throws InterruptedException {
+    // we don't call super.setUp
+    zooThread = new Thread() {
+      
+      @Override
+      public void run() {
+        ServerConfig config = new ServerConfig() {
+
+          {
+            setClientPort(ZkTestServer.this.clientPort);
+            this.dataDir = zkDir;
+            this.dataLogDir = zkDir;
+            this.tickTime = 1500;
+          }
+          
+          public void setClientPort(int clientPort) {
+            if (clientPortAddress != null) {
+              try {
+                this.clientPortAddress = new InetSocketAddress(
+                        InetAddress.getByName(clientPortAddress.getHostName()), clientPort);
+              } catch (UnknownHostException e) {
+                throw new RuntimeException(e);
+              }
+            } else {
+              this.clientPortAddress = new InetSocketAddress(clientPort);
+            }
+          }
+        };
+
+        try {
+          zkServer.runFromConfig(config);
+        } catch (Throwable e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+
+    zooThread.setDaemon(true);
+    zooThread.start();
+
+    int cnt = 0;
+    int port = -1;
+    try {
+       port = getPort();
+    } catch(IllegalStateException e) {
+      
+    }
+    while (port < 1) {
+      Thread.sleep(100);
+      try {
+        port = getPort();
+      } catch(IllegalStateException e) {
+        
+      }
+      if (cnt == 40) {
+        throw new RuntimeException("Could not get the port for ZooKeeper server");
+      }
+      cnt++;
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  public void shutdown() throws IOException {
+    SolrTestCaseJ4.ignoreException("java.nio.channels.ClosedChannelException");
+    // TODO: this can log an exception while trying to unregister a JMX MBean
+    try {
+      zkServer.shutdown();
+    } finally {
+      SolrTestCaseJ4.resetExceptionIgnores();
+    }
+  }
+ 
+  
+  public static boolean waitForServerDown(String hp, long timeout) {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        HostPort hpobj = parseHostPortList(hp).get(0);
+        send4LetterWord(hpobj.host, hpobj.port, "stat");
+      } catch (IOException e) {
+        return true;
+      }
+      
+      if (System.currentTimeMillis() > start + timeout) {
+        break;
+      }
+      try {
+        Thread.sleep(250);
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+    return false;
+  }
+  
+  public static class HostPort {
+    String host;
+    int port;
+    
+    HostPort(String host, int port) {
+      this.host = host;
+      this.port = port;
+    }
+  }
+  
+  /**
+   * Send the 4letterword
+   * @param host the destination host
+   * @param port the destination port
+   * @param cmd the 4letterword
+   * @return
+   * @throws IOException
+   */
+  public static String send4LetterWord(String host, int port, String cmd)
+      throws IOException
+  {
+
+      Socket sock = new Socket(host, port);
+      BufferedReader reader = null;
+      try {
+          OutputStream outstream = sock.getOutputStream();
+          outstream.write(cmd.getBytes());
+          outstream.flush();
+          // this replicates NC - close the output stream before reading
+          sock.shutdownOutput();
+
+          reader =
+              new BufferedReader(
+                      new InputStreamReader(sock.getInputStream()));
+          StringBuilder sb = new StringBuilder();
+          String line;
+          while((line = reader.readLine()) != null) {
+              sb.append(line + "\n");
+          }
+          return sb.toString();
+      } finally {
+          sock.close();
+          if (reader != null) {
+              reader.close();
+          }
+      }
+  }
+  
+  public static List<HostPort> parseHostPortList(String hplist) {
+    ArrayList<HostPort> alist = new ArrayList<HostPort>();
+    for (String hp : hplist.split(",")) {
+      int idx = hp.lastIndexOf(':');
+      String host = hp.substring(0, idx);
+      int port;
+      try {
+        port = Integer.parseInt(hp.substring(idx + 1));
+      } catch (RuntimeException e) {
+        throw new RuntimeException("Problem parsing " + hp + e.toString());
+      }
+      alist.add(new HostPort(host, port));
+    }
+    return alist;
+  }
+}

Modified: lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java?rev=1022188&r1=1022187&r2=1022188&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java (original)
+++ lucene/dev/trunk/solr/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java Wed Oct 13 17:01:13 2010
@@ -1,6 +1,7 @@
 package org.apache.solr.handler.component;
 
 import org.apache.solr.BaseDistributedSearchTestCase;
+import org.junit.Test;
 
 /**
  * Test for TermsComponent distributed querying
@@ -12,6 +13,7 @@ public class DistributedTermsComponentTe
 
   @Override
   public void doTest() throws Exception {
+    del("*:*");
     index(id, 18, "b_t", "snake spider shark snail slug seal");
     index(id, 19, "b_t", "snake spider shark snail slug");
     index(id, 20, "b_t", "snake spider shark snail");

Added: lucene/dev/trunk/solr/src/test/test-files/solr/solr.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/test-files/solr/solr.xml?rev=1022188&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/test/test-files/solr/solr.xml (added)
+++ lucene/dev/trunk/solr/src/test/test-files/solr/solr.xml Wed Oct 13 17:01:13 2010
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!--
+ All (relative) paths are relative to the installation path
+  
+  persistent: Save changes made via the API to this file
+  sharedLib: path to a lib directory that will be shared across all cores
+-->
+<solr persistent="false">
+
+  <!--
+  adminPath: RequestHandler path to manage cores.  
+    If 'null' (or absent), cores will not be manageable via request handler
+  -->
+  <cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" hostContext="solr" zkClientTimeout="8000">
+    <core name="collection1" shard="${shard:}" collection="${collection:collection1}" config="${solrconfig:solrconfig.xml}" instanceDir="."/>
+  </cores>
+</solr>

Modified: lucene/dev/trunk/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java?rev=1022188&r1=1022187&r2=1022188&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java (original)
+++ lucene/dev/trunk/solr/src/webapp/src/org/apache/solr/client/solrj/embedded/JettySolrRunner.java Wed Oct 13 17:01:13 2010
@@ -18,8 +18,6 @@
 package org.apache.solr.client.solrj.embedded;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
 import java.util.Random;
 
 import javax.servlet.http.HttpServlet;
@@ -27,9 +25,11 @@ import javax.servlet.http.HttpServletReq
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.solr.servlet.SolrDispatchFilter;
+import org.mortbay.component.LifeCycle;
+import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Handler;
 import org.mortbay.jetty.Server;
-import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.FilterHolder;
@@ -41,43 +41,30 @@ import org.mortbay.log.Logger;
  * 
  * @since solr 1.3
  */
-public class JettySolrRunner 
-{
+public class JettySolrRunner {
   Server server;
+
   FilterHolder dispatchFilter;
+
   String context;
-  
-  public JettySolrRunner( String context, int port )
-  {
-    this.init( context, port );
-  }
-
-  public JettySolrRunner( String context, int port, String solrConfigFilename )
-  {
-    this.init( context, port );
-    if (solrConfigFilename != null)
-      dispatchFilter.setInitParameter("solrconfig-filename", solrConfigFilename);
-  }
-  
-//  public JettySolrRunner( String context, String home, String dataDir, int port, boolean log )
-//  {
-//    if(!log) {
-//      System.setProperty("org.mortbay.log.class", NoLog.class.getName() );
-//      System.setProperty("java.util.logging.config.file", home+"/conf/logging.properties");
-//      NoLog noLogger = new NoLog();
-//      org.mortbay.log.Log.setLog(noLogger);
-//    }
-//
-//    // Initalize JNDI
-//    Config.setInstanceDir(home);
-//    new SolrCore(dataDir, new IndexSchema(home+"/conf/schema.xml"));
-//    this.init( context, port );
-//  }
-  
-  private void init( String context, int port )
-  {
+
+  private String solrConfigFilename;
+
+  private boolean waitOnSolr = false;
+
+  public JettySolrRunner(String context, int port) {
+    this.init(context, port);
+  }
+
+  public JettySolrRunner(String context, int port, String solrConfigFilename) {
+    this.init(context, port);
+    this.solrConfigFilename = solrConfigFilename;
+  }
+
+  private void init(String context, int port) {
     this.context = context;
-    server = new Server( port );    
+    server = new Server(port);
+    server.setStopAtShutdown(true);
     if (System.getProperty("jetty.testMode") != null) {
       SelectChannelConnector connector = new SelectChannelConnector();
       connector.setPort(port);
@@ -85,72 +72,76 @@ public class JettySolrRunner 
       server.setConnectors(new Connector[] { connector });
       server.setSessionIdManager(new HashSessionIdManager(new Random()));
     }
-    server.setStopAtShutdown( true );
-    
+
     // Initialize the servlets
-    Context root = new Context( server, context, Context.SESSIONS );
-    
+    final Context root = new Context(server, context, Context.SESSIONS);
+    server.addLifeCycleListener(new LifeCycle.Listener() {
+
+      public void lifeCycleStopping(LifeCycle arg0) {
+        System.clearProperty("hostPort");
+      }
+
+      public void lifeCycleStopped(LifeCycle arg0) {}
+
+      public void lifeCycleStarting(LifeCycle arg0) {
+        synchronized (JettySolrRunner.this) {
+          waitOnSolr = true;
+          JettySolrRunner.this.notify();
+        }
+      }
+
+      public void lifeCycleStarted(LifeCycle arg0) {
+        System.setProperty("hostPort", Integer.toString(getLocalPort()));
+        if (solrConfigFilename != null)
+          System.setProperty("solrconfig", solrConfigFilename);
+        dispatchFilter = root.addFilter(SolrDispatchFilter.class, "*",
+            Handler.REQUEST);
+        if (solrConfigFilename != null)
+          System.clearProperty("solrconfig");
+      }
+
+      public void lifeCycleFailure(LifeCycle arg0, Throwable arg1) {
+        System.clearProperty("hostPort");
+      }
+    });
+
     // for some reason, there must be a servlet for this to get applied
-    root.addServlet( Servlet404.class, "/*" );
-    dispatchFilter = root.addFilter( SolrDispatchFilter.class, "*", Handler.REQUEST );
+    root.addServlet(Servlet404.class, "/*");
+
   }
 
-  //------------------------------------------------------------------------------------------------
-  //------------------------------------------------------------------------------------------------
-  
-  public void start() throws Exception
-  {
+  // ------------------------------------------------------------------------------------------------
+  // ------------------------------------------------------------------------------------------------
+
+  public void start() throws Exception {
     start(true);
   }
 
-  public void start(boolean waitForSolr) throws Exception
-  {
-    if(!server.isRunning() ) {
+  public void start(boolean waitForSolr) throws Exception {
+    if (!server.isRunning()) {
       server.start();
     }
-    if (waitForSolr) waitForSolr(context);
+    synchronized (JettySolrRunner.this) {
+      int cnt = 0;
+      while (!waitOnSolr) {
+        this.wait(100);
+        if (cnt++ == 5) {
+          throw new RuntimeException("Jetty/Solr unresponsive");
+        }
+      }
+    }
   }
 
-
-  public void stop() throws Exception
-  {
-    if( server.isRunning() ) {
+  public void stop() throws Exception {
+    if (server.isRunning()) {
       server.stop();
       server.join();
     }
   }
 
-  /** Waits until a ping query to the solr server succeeds,
-   * retrying every 200 milliseconds up to 2 minutes.
-   */
-  public void waitForSolr(String context) throws Exception
-  {
-    int port = getLocalPort();
-
-    // A raw term query type doesn't check the schema
-    URL url = new URL("http://localhost:"+port+context+"/select?q={!raw+f=junit_test_query}ping");
-
-    Exception ex = null;
-    // Wait for a total of 20 seconds: 100 tries, 200 milliseconds each
-    for (int i=0; i<600; i++) {
-      try {
-        InputStream stream = url.openStream();
-        stream.close();
-      } catch (IOException e) {
-        // e.printStackTrace();
-        ex = e;
-        Thread.sleep(200);
-        continue;
-      }
-
-      return;
-    }
-
-    throw new RuntimeException("Jetty/Solr unresponsive",ex);
-  }
-
   /**
    * Returns the Local Port of the first Connector found for the jetty Server.
+   * 
    * @exception RuntimeException if there is no Connector
    */
   public int getLocalPort() {
@@ -161,80 +152,78 @@ public class JettySolrRunner 
     return conns[0].getLocalPort();
   }
 
-  //--------------------------------------------------------------
-  //--------------------------------------------------------------
-    
-  /** 
+  // --------------------------------------------------------------
+  // --------------------------------------------------------------
+
+  /**
    * This is a stupid hack to give jetty something to attach to
    */
-  public static class Servlet404 extends HttpServlet
-  {
+  public static class Servlet404 extends HttpServlet {
     @Override
-    public void service(HttpServletRequest req, HttpServletResponse res ) throws IOException
-    {
-      res.sendError( 404, "Can not find: "+req.getRequestURI() );
+    public void service(HttpServletRequest req, HttpServletResponse res)
+        throws IOException {
+      res.sendError(404, "Can not find: " + req.getRequestURI());
     }
   }
-  
+
   /**
-   * A main class that starts jetty+solr 
-   * This is useful for debugging
+   * A main class that starts jetty+solr This is useful for debugging
    */
-  public static void main( String[] args )
-  {
+  public static void main(String[] args) {
     try {
-      JettySolrRunner jetty = new JettySolrRunner( "/solr", 3456 );
+      JettySolrRunner jetty = new JettySolrRunner("/solr", 3456);
       jetty.start();
-    }
-    catch( Exception ex ) {
+    } catch (Exception ex) {
       ex.printStackTrace();
     }
   }
 }
 
+class NoLog implements Logger {
+  private static boolean debug = System.getProperty("DEBUG", null) != null;
 
-class NoLog implements Logger
-{    
-  private static boolean debug = System.getProperty("DEBUG",null)!=null;
   private final String name;
-      
-  public NoLog()
-  {
+
+  public NoLog() {
     this(null);
   }
-  
-  public NoLog(String name)
-  {    
-    this.name=name==null?"":name;
-  }
-  
-  public boolean isDebugEnabled()
-  {
+
+  public NoLog(String name) {
+    this.name = name == null ? "" : name;
+  }
+
+  public boolean isDebugEnabled() {
     return debug;
   }
-  
-  public void setDebugEnabled(boolean enabled)
-  {
-    debug=enabled;
-  }
-  
-  public void info(String msg,Object arg0, Object arg1) {}
-  public void debug(String msg,Throwable th){}
-  public void debug(String msg,Object arg0, Object arg1){}
-  public void warn(String msg,Object arg0, Object arg1){}
-  public void warn(String msg, Throwable th){}
-
-  public Logger getLogger(String name)
-  {
-    if ((name==null && this.name==null) ||
-      (name!=null && name.equals(this.name)))
+
+  public void setDebugEnabled(boolean enabled) {
+    debug = enabled;
+  }
+
+  public void info(String msg, Object arg0, Object arg1) {
+  }
+
+  public void debug(String msg, Throwable th) {
+  }
+
+  public void debug(String msg, Object arg0, Object arg1) {
+  }
+
+  public void warn(String msg, Object arg0, Object arg1) {
+  }
+
+  public void warn(String msg, Throwable th) {
+  }
+
+  public Logger getLogger(String name) {
+    if ((name == null && this.name == null)
+        || (name != null && name.equals(this.name)))
       return this;
     return new NoLog(name);
   }
-  
+
   @Override
-  public String toString()
-  {
-    return "NOLOG["+name+"]";
+  public String toString() {
+    return "NOLOG[" + name + "]";
   }
 }

Modified: lucene/dev/trunk/solr/src/webapp/web/admin/index.jsp
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/webapp/web/admin/index.jsp?rev=1022188&r1=1022187&r2=1022188&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/webapp/web/admin/index.jsp (original)
+++ lucene/dev/trunk/solr/src/webapp/web/admin/index.jsp Wed Oct 13 17:01:13 2010
@@ -50,6 +50,7 @@
     [<a href="stats.jsp">Statistics</a>]
     [<a href="registry.jsp">Info</a>]
     [<a href="distributiondump.jsp">Distribution</a>]
+    [<a href="zookeeper.jsp">ZooKeeper</a>]
     [<a href="ping">Ping</a>]
     [<a href="logging">Logging</a>]
   </td>

Added: lucene/dev/trunk/solr/src/webapp/web/admin/zookeeper.jsp
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/webapp/web/admin/zookeeper.jsp?rev=1022188&view=auto
==============================================================================
--- lucene/dev/trunk/solr/src/webapp/web/admin/zookeeper.jsp (added)
+++ lucene/dev/trunk/solr/src/webapp/web/admin/zookeeper.jsp Wed Oct 13 17:01:13 2010
@@ -0,0 +1,478 @@
+<%@ page contentType="text/html; charset=utf-8" pageEncoding="UTF-8"%>
+<%--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+--%>
+<%@ page import="javax.servlet.jsp.JspWriter,java.io.IOException,org.apache.zookeeper.*,org.apache.zookeeper.data.Stat,org.apache.solr.core.*,org.apache.solr.cloud.*,org.apache.solr.common.cloud.*,org.apache.solr.common.util.*,java.util.concurrent.TimeoutException"%>
+<%@ page import="java.io.*"%>
+<%@ page import="java.util.*"%>
+<%@ page import="java.net.URLEncoder"%>
+
+<%@include file="header.jsp" %>
+
+<br clear="all">
+<h2>Zookeeper Browser</h2>
+
+<%
+  String path = request.getParameter("path");
+  String addr = request.getParameter("addr");
+  if (addr != null && addr.length() == 0)
+    addr = null;
+  String detailS = request.getParameter("detail");
+  boolean detail = detailS != null && detailS.equals("true");
+
+  ZKPrinter printer = new ZKPrinter(out, core, addr);
+  printer.detail = detail;
+  String tryAddr = printer.keeperAddr != null ? printer.keeperAddr
+      : "localhost:2181";
+%>
+
+<form method="GET" action="zookeeper.jsp" accept-charset="UTF-8">
+<table>
+<tr>
+  <td>
+     <strong>   <%
+     XML.escapeCharData(printer.zkClient == null ? "Disconnected"
+         : ("Connected to zookeeper " + printer.keeperAddr), out);
+   %>  </strong>
+  </td>
+  <td>
+        Connect to different zookeeper:
+	<input class="std" name="addr" type="text" value="<%XML.escapeCharData(tryAddr, out);%>">
+  </td>
+    <td>
+	<input class="stdbutton" type="submit" value="CONNECT">
+  </td>
+</tr>
+<tr>
+</table>
+</form>
+
+
+<%
+  try {
+    printer.print(path);
+  } finally {
+    printer.close();
+  }
+%>
+
+</body>
+</html>
+
+<%!static class ZKPrinter {
+    static boolean FULLPATH_DEFAULT = false;
+
+    boolean indent = true;
+    boolean fullpath = FULLPATH_DEFAULT;
+
+    boolean detail = false;
+
+    String addr; // the address passed to us
+
+    String keeperAddr; // the address we're connected to
+
+    SolrZkClient zkClient;
+    boolean doClose;  // close the client after done if we opened it
+
+    JspWriter out;
+
+    int level;
+
+    int maxData = 100;
+
+    private boolean levelchange;
+
+    public ZKPrinter(JspWriter out, SolrCore core, String addr)
+        throws IOException {
+      this.out = out;
+      this.addr = addr;
+
+      if (addr == null) {
+        ZkController controller = core.getCoreDescriptor().getCoreContainer().getZkController();
+        if (controller != null) {
+          // this core is zk enabled
+          keeperAddr = controller.getZkServerAddress();
+          zkClient = controller.getZkClient();
+          if (zkClient != null && zkClient.isConnected()) {
+            return;
+          } else {
+            // try a different client with this address
+            addr = keeperAddr;
+          }
+        }
+      }
+
+      keeperAddr = addr;
+      if (addr == null) {
+        out.println("Zookeeper is not configured for this Solr Core.  Please try connecting to an alternate zookeeper address.");
+        return;
+      }
+
+      try {
+        zkClient = new SolrZkClient(addr, 10000);
+        doClose = true;
+      } catch (TimeoutException e) {
+       out.println("Could not connect to zookeeper at " + addr);
+       zkClient = null;
+       return;
+      } catch (InterruptedException e) {
+        // Restore the interrupted status
+        Thread.currentThread().interrupt();
+        out.println("Could not connect to zookeeper at " + addr);
+        zkClient = null;
+        return;
+      }
+
+
+    }
+
+    public void close() {
+        try {
+          if (doClose) zkClient.close();
+        } catch (InterruptedException e) {
+            // ignore exception on close
+        }
+    }
+
+    // main entry point
+    void print(String path) throws IOException {
+      if (zkClient == null)
+        return;
+
+      out.print("<table>");
+      out.print("<tr><td>");
+      out.print("[");
+      url("ROOT", "/", false);
+      out.print("]");
+
+      // normalize path
+      if (path == null)
+        path = "/";
+      else {
+        path.trim();
+        if (path.length() == 0)
+          path = "/";
+      }
+      if (path.endsWith("/") && path.length() > 1) {
+        path = path.substring(0, path.length() - 1);
+      }
+
+      int idx = path.lastIndexOf('/');
+      String parent = idx >= 0 ? path.substring(0, idx) : path;
+      if (parent.length() == 0)
+        parent = "/";
+
+      out.print(" [");
+      url("PARENT", parent, detail);
+      out.print("]");
+      out.print("</td></tr>");
+
+      if (detail) {
+        out.print("<tr><td>");
+        printZnode(path);
+        out.print("</td></tr>");
+      }
+
+      out.print("<tr><td>");
+      printTree(path);
+      out.print("</td></tr>");
+
+      out.print("</table>");
+    }
+
+    void exception(Exception e) {
+      try {
+        out.println(e.toString());
+      } catch (IOException e1) {
+        // nothing we can do
+      }
+    }
+
+    void xmlescape(String s) {
+      try {
+        XML.escapeCharData(s, out);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    void up() throws IOException {
+      level++;
+      levelchange = true;
+    }
+
+    void down() throws IOException {
+      level--;
+      levelchange = true;
+    }
+
+    void indent() throws IOException {
+      // if we are using blockquote and just changed indent levels, don't output a break
+      // if (fullpath || !levelchange)
+      out.println("<br>");
+      levelchange = false;
+
+      for (int i=0; i<level; i++)
+        out.println("&nbsp;&nbsp;&nbsp;&nbsp;");
+
+      // if fullpath, no indent is needed
+      // if not, we are currently using blockquote which the browser
+      // will take care of indenting.
+    }
+
+    // collapse all whitespace to a single space or escaped newline
+    String compress(String str) {
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < str.length(); i++) {
+        char ch = str.charAt(i);
+        boolean whitespace = false;
+        boolean newline = false;
+        while (Character.isWhitespace(ch)) {
+          whitespace = true;
+          if (ch == '\n')
+            newline = true;
+          if (++i >= str.length())
+            return sb.toString();
+          ch = str.charAt(i);
+        }
+
+        if (newline) {
+          // sb.append("\\n");
+          sb.append("  ");  // collapse newline to two spaces
+        } else if (whitespace) {
+          sb.append(' ');
+        }
+
+        // TODO: handle non-printable chars
+        sb.append(ch);
+
+        if (sb.length() >= maxData)
+          return sb.toString() + "...";
+      }
+      return sb.toString();
+    }
+
+    void url(String label, String path, boolean detail) throws IOException {
+      try {
+        out.print("<a href=\"zookeeper.jsp?");
+        if (path != null) {
+          out.print("path=");
+          out.print(URLEncoder.encode(path, "UTF-8"));
+        }
+        if (detail) {
+          out.print("&detail=" + detail);
+        }
+        if (fullpath != FULLPATH_DEFAULT) {
+          out.print("&fullpath=" + fullpath);
+        }
+        if (addr != null) {
+          out.print("&addr=");
+          out.print(URLEncoder.encode(addr, "UTF-8"));
+        }
+
+        out.print("\">");
+        xmlescape(label);
+        out.print("</a>");
+
+      } catch (UnsupportedEncodingException e) {
+        exception(e);
+      }
+    }
+
+    void printTree(String path) throws IOException {
+
+      indent();
+
+      // TODO: make a link from the path
+
+      String label = path;
+      if (!fullpath) {
+        int idx = path.lastIndexOf('/');
+        label = idx > 0 ? path.substring(idx + 1) : path;
+      }
+
+      url(label, path, true);
+
+      out.print(" (");
+
+      Stat stat = new Stat();
+      try {
+        byte[] data = zkClient.getData(path, null, stat);
+
+        if (stat.getEphemeralOwner() != 0)
+          out.print("ephemeral ");
+        out.print("v=" + stat.getVersion());
+        if (stat.getNumChildren() != 0) {
+          out.print(" children=" + stat.getNumChildren());
+        }
+        out.print(")");
+
+        if (data != null) {
+
+          String str;
+          try {
+            str = new String(data, "UTF-8");
+            out.print(" \"");
+            xmlescape(compress(str));
+            out.print("\"");
+          } catch (UnsupportedEncodingException e) {
+            // not UTF8
+            StringBuilder sb = new StringBuilder("BIN(");
+            sb.append("len=" + data.length);
+            sb.append("hex=");
+            int limit = Math.min(data.length, maxData / 2);
+            for (int i = 0; i < limit; i++) {
+              byte b = data[i];
+              sb.append(StrUtils.HEX_DIGITS[(b >> 4) & 0xf]);
+              sb.append(StrUtils.HEX_DIGITS[b & 0xf]);
+            }
+            if (limit != data.length)
+              sb.append("...");
+            sb.append(")");
+            str = sb.toString();
+            out.print(str);
+          }
+
+        }
+
+      } catch (IllegalArgumentException e) {
+        // path doesn't exist (must have been removed)
+        out.println("(path gone)");
+      } catch (KeeperException e) {
+        e.printStackTrace();
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+
+      if (stat.getNumChildren() <= 0)
+        return;
+
+      List<String> children = null;
+      try {
+        children = zkClient.getChildren(path, null);
+      } catch (KeeperException e) {
+        exception(e);
+        return;
+      } catch (InterruptedException e) {
+        exception(e);
+      } catch (IllegalArgumentException e) {
+        // path doesn't exist (must have been removed)
+        out.println("(children gone)");
+      }
+
+      up();
+      for (String child : children) {
+        String childPath = path + (path.endsWith("/") ? "" : "/") + child;
+        printTree(childPath);
+      }
+      down();
+    }
+
+    String time(long ms) {
+      return (new Date(ms)).toString() + " (" + ms + ")";
+    }
+
+    void printZnode(String path) throws IOException {
+      try {
+
+        Stat stat = new Stat();
+        byte[] data = zkClient.getData(path, null, stat);
+
+        out.print("<h2>");
+        xmlescape(path);
+        out.print("</h2>");
+
+        up();
+        indent();
+        out.print("version = " + stat.getVersion());
+        indent();
+        out.print("aversion = " + stat.getAversion());
+        indent();
+        out.print("cversion = " + stat.getCversion());
+        indent();
+        out.print("ctime = " + time(stat.getCtime()));
+        indent();
+        out.print("mtime = " + time(stat.getMtime()));
+        indent();
+        out.print("czxid = " + stat.getCzxid());
+        indent();
+        out.print("mzxid = " + stat.getMzxid());
+        indent();
+        out.print("pzxid = " + stat.getPzxid());
+        indent();
+        out.print("numChildren = " + stat.getNumChildren());
+        indent();
+        out.print("ephemeralOwner = " + stat.getEphemeralOwner());
+        indent();
+        out.print("dataLength = " + stat.getDataLength());
+
+        if (data != null) {
+          boolean isBinary = false;
+          String str;
+          try {
+            str = new String(data, "UTF-8");
+          } catch (UnsupportedEncodingException e) {
+            // The results are unspecified
+            // when the bytes are not properly encoded.
+
+            // not UTF8
+            StringBuilder sb = new StringBuilder(data.length * 2);
+            for (int i = 0; i < data.length; i++) {
+              byte b = data[i];
+              sb.append(StrUtils.HEX_DIGITS[(b >> 4) & 0xf]);
+              sb.append(StrUtils.HEX_DIGITS[b & 0xf]);
+              if ((i & 0x3f) == 0x3f)
+                sb.append("\n");
+            }
+            str = sb.toString();
+          }
+
+          int nLines = 1;
+          int lineLen = 0;
+          int maxLineLen = 10; // the minimum
+          for (int i = 0; i < str.length(); i++) {
+            if (str.charAt(i) == '\n') {
+              nLines++;
+              maxLineLen = Math.max(maxLineLen, lineLen);
+              lineLen = 0;
+            } else {
+              lineLen++;
+            }
+          }
+
+          indent();
+          out.println("<form method='post' action=''>");
+          out.println("<textarea class='big' wrap='off' readonly rows='"
+              + Math.min(20, nLines)
+              //                  + "' cols='" + Math.min(80, maxLineLen+1)
+              //                  + "' cols='" + (maxLineLen+1)
+              + "' name='data'>");
+
+          xmlescape(str);
+
+          out.println("</textarea></form>");
+        }
+
+        down();
+
+      } catch (KeeperException e) {
+        exception(e);
+        return;
+      } catch (InterruptedException e) {
+        exception(e);
+      }
+    }
+  }%>
\ No newline at end of file