You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/08/31 00:43:59 UTC

svn commit: r1379200 [11/11] - in /lucene/dev/branches/lucene3312: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/core/ dev-tools/maven/lucene/test-framework/ dev-tools/scripts/ lucene/ lucene/a...

Modified: lucene/dev/branches/lucene3312/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene3312/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Thu Aug 30 22:43:41 2012
@@ -183,15 +183,9 @@ public class DistributedUpdateProcessor 
       // set num nodes
       numNodes = zkController.getClusterState().getLiveNodes().size();
       
-      // the leader is...
-      // TODO: if there is no leader, wait and look again
-      // TODO: we are reading the leader from zk every time - we should cache
-      // this and watch for changes?? Just pull it from ZkController cluster state probably?
       String shardId = getShard(hash, collection, zkController.getClusterState()); // get the right shard based on the hash...
 
       try {
-        // TODO: if we find out we cannot talk to zk anymore, we should probably realize we are not
-        // a leader anymore - we shouldn't accept updates at all??
         ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(zkController.getZkStateReader().getLeaderProps(
             collection, shardId));
         
@@ -201,7 +195,10 @@ public class DistributedUpdateProcessor 
         isLeader = coreNodeName.equals(leaderNodeName);
         
         DistribPhase phase = 
-          DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
+            DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
+       
+        doDefensiveChecks(shardId, phase);
+     
 
         if (DistribPhase.FROMLEADER == phase) {
           // we are coming from the leader, just go local - add no urls
@@ -251,6 +248,36 @@ public class DistributedUpdateProcessor 
     return nodes;
   }
 
+  private void doDefensiveChecks(String shardId, DistribPhase phase) {
+    String from = req.getParams().get("distrib.from");
+    boolean localIsLeader = req.getCore().getCoreDescriptor().getCloudDescriptor().isLeader();
+    if (DistribPhase.FROMLEADER == phase && localIsLeader && from != null) { // from will be null on log replay
+      log.error("Request says it is coming from leader, but we are the leader: " + req.getParamString());
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Request says it is coming from leader, but we are the leader");
+    }
+
+    if (DistribPhase.FROMLEADER == phase && from != null) { // from will be null on log replay
+     
+      ZkCoreNodeProps clusterStateLeader = new ZkCoreNodeProps(zkController
+          .getClusterState().getLeader(collection, shardId));
+    
+      if (clusterStateLeader.getNodeProps() == null
+          || !clusterStateLeader.getCoreUrl().equals(from)) {
+        String coreUrl = null;
+        if (clusterStateLeader.getNodeProps() != null) {
+          coreUrl = clusterStateLeader.getCoreUrl();
+        }
+        log.error("We got a request from the leader, but it's not who our cluster state says is the leader :"
+            + req.getParamString()
+            + " : "
+            + coreUrl);
+
+        new SolrException(ErrorCode.BAD_REQUEST, "We got a request from the leader, but it's not who our cluster state says is the leader.");
+      }
+ 
+    }
+  }
+
 
   private String getShard(int hash, String collection, ClusterState clusterState) {
     // ranges should be part of the cloud state and eventually gotten from zk
@@ -329,6 +356,8 @@ public class DistributedUpdateProcessor 
                   DistribPhase.FROMLEADER.toString() : 
                   DistribPhase.TOLEADER.toString()));
       params.remove("commit"); // this will be distributed from the local commit
+      params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
+          zkController.getBaseUrl(), req.getCore().getName()));
       cmdDistrib.distribAdd(cmd, nodes, params);
     }
     
@@ -378,9 +407,11 @@ public class DistributedUpdateProcessor 
 
     // TODO: we should do this in the background it would seem
     for (SolrCmdDistributor.Error error : response.errors) {
-      if (error.node instanceof RetryNode) {
+      if (error.node instanceof RetryNode || error.e instanceof SolrException) {
         // we don't try to force a leader to recover
         // when we cannot forward to it
+        // and we assume SolrException means
+        // the node went down
         continue;
       }
       // TODO: we should force their state to recovering ??
@@ -658,6 +689,10 @@ public class DistributedUpdateProcessor 
                  (isLeader ? 
                   DistribPhase.FROMLEADER.toString() : 
                   DistribPhase.TOLEADER.toString()));
+      if (isLeader) {
+        params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
+            zkController.getBaseUrl(), req.getCore().getName()));
+      }
       params.remove("commit"); // we already will have forwarded this from our local commit
       cmdDistrib.distribDelete(cmd, nodes, params);
     }
@@ -819,6 +854,8 @@ public class DistributedUpdateProcessor 
       ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
       params.set(VERSION_FIELD, Long.toString(cmd.getVersion()));
       params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString());
+      params.set("update.from", ZkCoreNodeProps.getCoreUrl(
+          zkController.getBaseUrl(), req.getCore().getName()));
       cmdDistrib.distribDelete(cmd, replicas, params);
       cmdDistrib.finish();
     }

Modified: lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java (original)
+++ lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java Thu Aug 30 22:43:41 2012
@@ -42,7 +42,7 @@ import org.slf4j.LoggerFactory;
 public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase {
   public static Logger log = LoggerFactory.getLogger(ChaosMonkeyNothingIsSafeTest.class);
   
-  private static final int BASE_RUN_LENGTH = 45000;
+  private static final int BASE_RUN_LENGTH = 20000;
 
   @BeforeClass
   public static void beforeSuperClass() {
@@ -56,8 +56,8 @@ public class ChaosMonkeyNothingIsSafeTes
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    // TODO use @Noisy annotation as we expect lots of exceptions
-    //ignoreException(".*");
+    // can help to hide this when testing and looking at logs
+    //ignoreException("shard update error");
     System.setProperty("numShards", Integer.toString(sliceCount));
   }
   
@@ -71,8 +71,8 @@ public class ChaosMonkeyNothingIsSafeTes
   
   public ChaosMonkeyNothingIsSafeTest() {
     super();
-    sliceCount = 3;
-    shardCount = 12;
+    sliceCount = 1;
+    shardCount = 7;
   }
   
   @Override
@@ -83,9 +83,16 @@ public class ChaosMonkeyNothingIsSafeTes
       handle.put("QTime", SKIPVAL);
       handle.put("timestamp", SKIPVAL);
       
+      // make sure we have leaders for each shard
+      for (int j = 1; j < sliceCount; j++) {
+        zkStateReader.getLeaderProps(DEFAULT_COLLECTION, "shard" + j, 10000);
+      }      // make sure we again have leaders for each shard
+      
+      waitForRecoveriesToFinish(false);
+      
       // we cannot do delete by query
       // as it's not supported for recovery
-      // del("*:*");
+       del("*:*");
       
       List<StopableThread> threads = new ArrayList<StopableThread>();
       int threadCount = 1;
@@ -152,6 +159,7 @@ public class ChaosMonkeyNothingIsSafeTes
       zkStateReader.updateClusterState(true);
       assertTrue(zkStateReader.getClusterState().getLiveNodes().size() > 0);
       
+      
       // we dont't current check vs control because the full throttle thread can
       // have request fails
       checkShardConsistency(false, true);

Modified: lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java (original)
+++ lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java Thu Aug 30 22:43:41 2012
@@ -50,11 +50,6 @@ public class ChaosMonkeySafeLeaderTest e
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    // we expect this time of exception as shards go up and down...
-    //ignoreException(".*");
-    
-    // sometimes we cannot get the same port
-    ignoreException("java\\.net\\.BindException: Address already in use");
     
     System.setProperty("numShards", Integer.toString(sliceCount));
   }

Modified: lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java (original)
+++ lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java Thu Aug 30 22:43:41 2012
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -125,7 +124,7 @@ public class OverseerTest extends SolrTe
         q.offer(ZkStateReader.toJSON(m));
       }
       
-      for (int i = 0; i < 30; i++) {
+      for (int i = 0; i < 120; i++) {
         String shardId = getShardId(coreName);
         if (shardId != null) {
           try {
@@ -143,7 +142,7 @@ public class OverseerTest extends SolrTe
           elector.joinElection(ctx);
           return shardId;
         }
-        Thread.sleep(200);
+        Thread.sleep(500);
       }
       return null;
     }
@@ -297,21 +296,25 @@ public class OverseerTest extends SolrTe
       }
       
       // make sure all cores have been assigned a id in cloudstate
+      int cloudStateSliceCount = 0;
       for (int i = 0; i < 40; i++) {
+        cloudStateSliceCount = 0;
         reader.updateClusterState(true);
         ClusterState state = reader.getClusterState();
         Map<String,Slice> slices = state.getSlices("collection1");
-        int count = 0;
         for (String name : slices.keySet()) {
-          count += slices.get(name).getShards().size();
+          cloudStateSliceCount += slices.get(name).getShards().size();
         }
-        if (coreCount == count) break;
+        if (coreCount == cloudStateSliceCount) break;
         Thread.sleep(200);
       }
+      assertEquals("Unable to verify all cores have been assigned an id in cloudstate", 
+                   coreCount, cloudStateSliceCount);
 
-      // make sure all cores have been returned a id
+      // make sure all cores have been returned an id
+      int assignedCount = 0;
       for (int i = 0; i < 90; i++) {
-        int assignedCount = 0;
+        assignedCount = 0;
         for (int j = 0; j < coreCount; j++) {
           if (ids[j] != null) {
             assignedCount++;
@@ -322,6 +325,8 @@ public class OverseerTest extends SolrTe
         }
         Thread.sleep(500);
       }
+      assertEquals("Unable to verify all cores have been returned an id", 
+                   coreCount, assignedCount);
       
       final HashMap<String, AtomicInteger> counters = new HashMap<String,AtomicInteger>();
       for (int i = 1; i < sliceCount+1; i++) {
@@ -347,7 +352,7 @@ public class OverseerTest extends SolrTe
       
       //make sure leaders are in cloud state
       for (int i = 0; i < sliceCount; i++) {
-        assertNotNull(reader.getLeaderUrl("collection1", "shard" + (i + 1)), 15000);
+        assertNotNull(reader.getLeaderUrl("collection1", "shard" + (i + 1), 15000));
       }
 
     } finally {
@@ -574,7 +579,6 @@ public class OverseerTest extends SolrTe
     public void run() {
       try {
         overseerClient = electNewOverseer(zkAddress);
-        Random rnd = random();
         while (run) {
           if (killCounter.get()>0) {
             try {
@@ -876,4 +880,4 @@ public class OverseerTest extends SolrTe
     return zkClient;
   }
   
-}
\ No newline at end of file
+}

Modified: lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java (original)
+++ lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java Thu Aug 30 22:43:41 2012
@@ -27,12 +27,25 @@ public class TestHashPartitioner extends
   
   public void testMapHashes() throws Exception {
     HashPartitioner hp = new HashPartitioner();
-    
-    for (int i = 1; i <= 30000; i++) {
-      List<Range> ranges = hp.partitionRange(i);
-      
+    List<Range> ranges;
+
+    // make sure the partitioner uses the "natural" boundaries and doesn't suffer from an off-by-one
+    ranges = hp.partitionRange(2, Integer.MIN_VALUE, Integer.MAX_VALUE);
+    assertEquals(Integer.MIN_VALUE, ranges.get(0).min);
+    assertEquals(0x80000000, ranges.get(0).min);
+    assertEquals(0xffffffff, ranges.get(0).max);
+    assertEquals(0x00000000, ranges.get(1).min);
+    assertEquals(0x7fffffff, ranges.get(1).max);
+
+    ranges = hp.partitionRange(2, 0, 0x7fffffff);
+    assertEquals(0x00000000, ranges.get(0).min);
+    assertEquals(0x3fffffff, ranges.get(0).max);
+    assertEquals(0x40000000, ranges.get(1).min);
+    assertEquals(0x7fffffff, ranges.get(1).max);
+
+    for (int i = 1; i <= 30000; i += 13) {
+      ranges = hp.partitionRange(i, Integer.MIN_VALUE, Integer.MAX_VALUE);
       assertEquals(i, ranges.size());
-      
       assertTrue("First range does not start before " + Integer.MIN_VALUE
           + " it is:" + ranges.get(0).min,
           ranges.get(0).min <= Integer.MIN_VALUE);

Modified: lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java (original)
+++ lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/cloud/TestMultiCoreConfBootstrap.java Thu Aug 30 22:43:41 2012
@@ -25,9 +25,7 @@ import org.apache.solr.core.CoreContaine
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.util.ExternalPaths;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,27 +34,25 @@ public class TestMultiCoreConfBootstrap 
   protected static Logger log = LoggerFactory.getLogger(TestMultiCoreConfBootstrap.class);
   protected CoreContainer cores = null;
   private String home;
-
-  protected static ZkTestServer zkServer;
-  protected static String zkDir;
-  
-  @BeforeClass
-  public static void beforeClass() {
-    createTempDir();
-  }
   
-  @AfterClass
-  public static void afterClass() {
-    zkServer = null;
-    zkDir = null;
-  }
+  protected File dataDir2;
+  protected ZkTestServer zkServer;
+  protected String zkDir;
   
   @Override
   @Before
   public void setUp() throws Exception {
     super.setUp();
+    
+    createTempDir();
+    dataDir2 = new File(TEMP_DIR, getSimpleClassName() + "-core1-"
+        + System.currentTimeMillis());
+    dataDir2.mkdirs();
+
     home = ExternalPaths.EXAMPLE_MULTICORE_HOME;
     System.setProperty("solr.solr.home", home);
+    System.setProperty( "solr.core0.data.dir", dataDir.getCanonicalPath() ); 
+    System.setProperty( "solr.core1.data.dir", dataDir2.getCanonicalPath() ); 
     
     zkDir = dataDir.getAbsolutePath() + File.separator
         + "zookeeper/server1/data";
@@ -82,14 +78,11 @@ public class TestMultiCoreConfBootstrap 
     
     zkServer.shutdown();
     
-    File dataDir1 = new File(home + File.separator + "core0","data");
-    File dataDir2 = new File(home + File.separator + "core1","data");
-
     String skip = System.getProperty("solr.test.leavedatadir");
     if (null != skip && 0 != skip.trim().length()) {
       log.info("NOTE: per solr.test.leavedatadir, dataDir will not be removed: " + dataDir.getAbsolutePath());
     } else {
-      if (!AbstractSolrTestCase.recurseDelete(dataDir1)) {
+      if (!AbstractSolrTestCase.recurseDelete(dataDir)) {
         log.warn("!!!! WARNING: best effort to remove " + dataDir.getAbsolutePath() + " FAILED !!!!!");
       }
       if (!AbstractSolrTestCase.recurseDelete(dataDir2)) {
@@ -97,6 +90,9 @@ public class TestMultiCoreConfBootstrap 
       }
     }
 
+    zkServer = null;
+    zkDir = null;
+
     super.tearDown();
   }
 

Modified: lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java (original)
+++ lucene/dev/branches/lucene3312/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java Thu Aug 30 22:43:41 2012
@@ -17,6 +17,7 @@
 package org.apache.solr.core;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrConfig.JmxConfiguration;
 import org.junit.After;
@@ -30,10 +31,12 @@ import javax.management.Query;
 import javax.management.remote.JMXConnector;
 import javax.management.remote.JMXConnectorFactory;
 import javax.management.remote.JMXServiceURL;
+import javax.management.remote.rmi.RMIConnectorServer;
+import java.io.IOException;
 import java.net.ServerSocket;
 import java.net.URL;
-import java.rmi.RemoteException;
 import java.rmi.registry.LocateRegistry;
+import java.rmi.server.RMIServerSocketFactory;
 import java.util.Set;
 
 import static org.hamcrest.CoreMatchers.allOf;
@@ -59,35 +62,36 @@ public class TestJmxMonitoredMap extends
   @Override
   @Before
   public void setUp() throws Exception {
-
     super.setUp();
-
-    int retries = 5;
-    for (int i = 0; i < retries; i++) {
-      try {
-        ServerSocket server = new ServerSocket(0);
-        try {
-          port = server.getLocalPort();
-        } finally {
-          server.close();
-        }
-        // System.out.println("Using port: " + port);
-        try {
-          LocateRegistry.createRegistry(port);
-        } catch (RemoteException e) {
-          throw e;
-        }
-        String url = "service:jmx:rmi:///jndi/rmi://:" + port + "/solrjmx";
-        JmxConfiguration config = new JmxConfiguration(true, null, url, null);
-        monitoredMap = new JmxMonitoredMap<String, SolrInfoMBean>("", "", config);
-        JMXServiceURL u = new JMXServiceURL(url);
-        connector = JMXConnectorFactory.connect(u);
-        mbeanServer = connector.getMBeanServerConnection();
-        break;
-      } catch (Exception e) {
-        if(retries == (i + 1)) {
-          throw e;
+    String oldHost = System.getProperty("java.rmi.server.hostname");
+    try {
+      // this stupid sysprop thing is needed, because remote stubs use an
+      // arbitrary local ip to connect
+      // See: http://weblogs.java.net/blog/emcmanus/archive/2006/12/multihomed_comp.html
+      System.setProperty("java.rmi.server.hostname", "127.0.0.1");
+      class LocalhostRMIServerSocketFactory implements RMIServerSocketFactory {
+        ServerSocket socket;
+        
+        @Override
+        public ServerSocket createServerSocket(int port) throws IOException {
+          return socket = new ServerSocket(port);
         }
+      };
+      LocalhostRMIServerSocketFactory factory = new LocalhostRMIServerSocketFactory();
+      LocateRegistry.createRegistry(0, null, factory);
+      port = factory.socket.getLocalPort();
+      AbstractSolrTestCase.log.info("Using port: " + port);
+      String url = "service:jmx:rmi:///jndi/rmi://127.0.0.1:"+port+"/solrjmx";
+      JmxConfiguration config = new JmxConfiguration(true, null, url, null);
+      monitoredMap = new JmxMonitoredMap<String, SolrInfoMBean>("", "", config);
+      JMXServiceURL u = new JMXServiceURL(url);
+      connector = JMXConnectorFactory.connect(u);
+      mbeanServer = connector.getMBeanServerConnection();
+    } finally {
+      if (oldHost == null) {
+        System.clearProperty("java.rmi.server.hostname");
+      } else {
+        System.setProperty("java.rmi.server.hostname", oldHost);
       }
     }
   }

Modified: lucene/dev/branches/lucene3312/solr/example/cloud-scripts/zkcli.bat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/example/cloud-scripts/zkcli.bat?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/example/cloud-scripts/zkcli.bat (original)
+++ lucene/dev/branches/lucene3312/solr/example/cloud-scripts/zkcli.bat Thu Aug 30 22:43:41 2012
@@ -1,12 +1,12 @@
-REM You can override pass the following parameters to this script:
-REM 
-
-set JVM=java
-
-REM  Find location of this script
-
-set SDIR=%~dp0
-if "%SDIR:~-1%"=="\" set SDIR=%SDIR:~0,-1%
-
-     
-"%JVM%" -classpath "%SDIR%\..\solr-webapp\webapp\WEB-INF\lib\*" org.apache.solr.cloud.ZkCLI %*
+REM You can override pass the following parameters to this script:
+REM 
+
+set JVM=java
+
+REM  Find location of this script
+
+set SDIR=%~dp0
+if "%SDIR:~-1%"=="\" set SDIR=%SDIR:~0,-1%
+
+     
+"%JVM%" -classpath "%SDIR%\..\solr-webapp\webapp\WEB-INF\lib\*" org.apache.solr.cloud.ZkCLI %*

Modified: lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/schema.xml?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/schema.xml Thu Aug 30 22:43:41 2012
@@ -47,14 +47,20 @@
 
 <schema name="example" version="1.5">
   <!-- attribute "name" is the name of this schema and is only used for display purposes.
-       version="x.y" is Solr's version number for the schema syntax and semantics.  It should
-       not normally be changed by applications.
-       1.0: multiValued attribute did not exist, all fields are multiValued by nature
+       version="x.y" is Solr's version number for the schema syntax and 
+       semantics.  It should not normally be changed by applications.
+
+       1.0: multiValued attribute did not exist, all fields are multiValued 
+            by nature
        1.1: multiValued attribute introduced, false by default 
-       1.2: omitTermFreqAndPositions attribute introduced, true by default except for text fields.
+       1.2: omitTermFreqAndPositions attribute introduced, true by default 
+            except for text fields.
        1.3: removed optional field compress feature
-       1.4: default auto-phrase (QueryParser feature) to off
-       1.5: omitNorms defaults to true for primitive field types (int, float, boolean, string...)
+       1.4: autoGeneratePhraseQueries attribute introduced to drive QueryParser
+            behavior when a single string produces multiple tokens.  Defaults 
+            to off for version >= 1.4
+       1.5: omitNorms defaults to true for primitive field types 
+            (int, float, boolean, string...)
      -->
 
  <fields>

Modified: lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/solrconfig.xml?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/solrconfig.xml (original)
+++ lucene/dev/branches/lucene3312/solr/example/solr/collection1/conf/solrconfig.xml Thu Aug 30 22:43:41 2012
@@ -1104,30 +1104,33 @@
 
        http://wiki.apache.org/solr/SolrReplication 
 
-       In the example below, remove the <lst name="master"> section if
-       this is just a slave and remove  the <lst name="slave"> section
-       if this is just a master.
+       It is also neccessary for SolrCloud to function (in Cloud mode, the 
+       replication handler is used to bulk transfer segments when nodes 
+       are added or need to recover).
+
+       https://wiki.apache.org/solr/SolrCloud/
     -->
-  <!--
-     <requestHandler name="/replication" class="solr.ReplicationHandler" >
+  <requestHandler name="/replication" class="solr.ReplicationHandler" > 
+    <!--
+       To enable simple master/slave replication, uncomment one of the 
+       sections below, depending on wether this solr instance should be 
+       the "master" or a "slave".  If this instance is a "slave" you will 
+       also need to fill in the masterUrl to point to a real machine.
+    -->
+    <!--
        <lst name="master">
          <str name="replicateAfter">commit</str>
          <str name="replicateAfter">startup</str>
          <str name="confFiles">schema.xml,stopwords.txt</str>
        </lst>
+    -->
+    <!--
        <lst name="slave">
-         <str name="masterUrl">http://localhost:8983/solr</str>
+         <str name="masterUrl">http://your-master-hostname:8983/solr</str>
          <str name="pollInterval">00:00:60</str>
        </lst>
-     </requestHandler>
-    -->
-    
-    <!-- Solr Replication for SolrCloud Recovery
-    
-         This is the config need for SolrCloud's recovery replication.
     -->
-	<requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" /> 
-
+  </requestHandler>
 
   <!-- Search Components
 

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java Thu Aug 30 22:43:41 2012
@@ -242,14 +242,16 @@ public class CloudSolrServer extends Sol
       theUrlList.addAll(urlList);
     }
     Collections.shuffle(theUrlList, rand);
-    if (replicas != null) {
+    if (sendToLeaders) {
       ArrayList<String> theReplicas = new ArrayList<String>(replicasList.size());
       theReplicas.addAll(replicasList);
       Collections.shuffle(theReplicas, rand);
-
+    //  System.out.println("leaders:" + theUrlList);
+    //  System.out.println("replicas:" + theReplicas);
       theUrlList.addAll(theReplicas);
     }
-    //System.out.println("########################## MAKING REQUEST TO " + theUrlList);
+ 
+   // System.out.println("########################## MAKING REQUEST TO " + theUrlList);
  
     LBHttpSolrServer.Req req = new LBHttpSolrServer.Req(request, theUrlList);
     LBHttpSolrServer.Rsp rsp = lbServer.request(req);

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java Thu Aug 30 22:43:41 2012
@@ -121,7 +121,9 @@ public class CoreAdminRequest extends So
     protected String state;
     protected Boolean checkLive;
     protected Integer pauseFor;
+    protected Boolean onlyIfLeader;
     
+
     public WaitForState() {
       action = CoreAdminAction.PREPRECOVERY;
     }
@@ -166,6 +168,14 @@ public class CoreAdminRequest extends So
       this.pauseFor = pauseFor;
     }
     
+    public boolean isOnlyIfLeader() {
+      return onlyIfLeader;
+    }
+
+    public void setOnlyIfLeader(boolean onlyIfLeader) {
+      this.onlyIfLeader = onlyIfLeader;
+    }
+    
     @Override
     public SolrParams getParams() {
       if( action == null ) {
@@ -195,6 +205,10 @@ public class CoreAdminRequest extends So
       if (pauseFor != null) {
         params.set( "pauseFor", pauseFor);
       }
+      
+      if (onlyIfLeader != null) {
+        params.set( "onlyIfLeader", onlyIfLeader);
+      }
 
       return params;
     }

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java Thu Aug 30 22:43:41 2012
@@ -226,7 +226,7 @@ public class ClusterState implements JSO
     shardList.addAll(shards);
     Collections.sort(shardList);
     
-    ranges = hp.partitionRange(shards.size());
+    ranges = hp.partitionRange(shards.size(), Integer.MIN_VALUE, Integer.MAX_VALUE);
     
     rangeInfo.ranges = ranges;
     rangeInfo.shardList = shardList;
@@ -243,7 +243,7 @@ public class ClusterState implements JSO
     
     int cnt = 0;
     for (Range range : rangInfo.ranges) {
-      if (hash < range.max) {
+      if (range.includes(hash)) {
         return rangInfo.shardList.get(cnt);
       }
       cnt++;

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java Thu Aug 30 22:43:41 2012
@@ -38,6 +38,8 @@ class ConnectionManager implements Watch
   private boolean connected;
 
   private final ZkClientConnectionStrategy connectionStrategy;
+  
+  private Object connectionUpdateLock = new Object();
 
   private String zkServerAddress;
 
@@ -72,6 +74,7 @@ class ConnectionManager implements Watch
     }
     
     if (isClosed) {
+      log.info("Client->ZooKeeper status change trigger but we are already closed");
       return;
     }
 
@@ -79,28 +82,25 @@ class ConnectionManager implements Watch
     if (state == KeeperState.SyncConnected) {
       connected = true;
       clientConnected.countDown();
+      connectionStrategy.connected();
     } else if (state == KeeperState.Expired) {
       connected = false;
-      log.info("Attempting to reconnect to recover relationship with ZooKeeper...");
-
+      log.info("Our previous ZooKeeper session was expired. Attempting to reconnect to recover relationship with ZooKeeper...");
+      
       try {
         connectionStrategy.reconnect(zkServerAddress, zkClientTimeout, this,
             new ZkClientConnectionStrategy.ZkUpdate() {
               @Override
               public void update(SolrZooKeeper keeper) {
                 // if keeper does not replace oldKeeper we must be sure to close it
-                synchronized (connectionStrategy) {
+                synchronized (connectionUpdateLock) {
                   try {
                     waitForConnected(SolrZkClient.DEFAULT_CLIENT_CONNECT_TIMEOUT);
-                  } catch (InterruptedException e1) {
-                    closeKeeper(keeper);
-                    Thread.currentThread().interrupt();
-                    throw new RuntimeException("Giving up on connecting - we were interrupted", e1);
                   } catch (Exception e1) {
                     closeKeeper(keeper);
                     throw new RuntimeException(e1);
                   }
-                  
+                  log.info("Connection with ZooKeeper reestablished.");
                   try {
                     client.updateKeeper(keeper);
                   } catch (InterruptedException e) {
@@ -129,7 +129,9 @@ class ConnectionManager implements Watch
       }
       log.info("Connected:" + connected);
     } else if (state == KeeperState.Disconnected) {
+      log.info("zkClient has disconnected");
       connected = false;
+      connectionStrategy.disconnected();
     } else {
       connected = false;
     }
@@ -151,19 +153,26 @@ class ConnectionManager implements Watch
   }
 
   public synchronized void waitForConnected(long waitForConnection)
-      throws InterruptedException, TimeoutException {
+      throws TimeoutException {
+    log.info("Waiting for client to connect to ZooKeeper");
     long expire = System.currentTimeMillis() + waitForConnection;
     long left = 1;
     while (!connected && left > 0) {
       if (isClosed) {
         break;
       }
-      wait(500);
+      try {
+        wait(500);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(e);
+      }
       left = expire - System.currentTimeMillis();
     }
     if (!connected) {
       throw new TimeoutException("Could not connect to ZooKeeper " + zkServerAddress + " within " + waitForConnection + " ms");
     }
+    log.info("Client is connected to ZooKeeper");
   }
 
   public synchronized void waitForDisconnected(long timeout)

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java Thu Aug 30 22:43:41 2012
@@ -25,39 +25,57 @@ import java.util.List;
  * 
  */
 public class HashPartitioner {
-  
+
+  // Hash ranges can't currently "wrap" - i.e. max must be greater or equal to min.
   public static class Range {
-    public long min;
-    public long max;
+    public int min;  // inclusive
+    public int max;  // inclusive
     
-    public Range(long min, long max) {
+    public Range(int min, int max) {
       this.min = min;
       this.max = max;
     }
+
+    public boolean includes(int hash) {
+      return hash >= min && hash <= max;
+    }
+
+    public String toString() {
+      return Integer.toHexString(min) + '-' + Integer.toHexString(max);
+    }
+
+    public static Range fromString(String range) {
+      return null; // TODO
+    }
   }
   
+
   /**
-   * works up to 65537 before requested num of ranges is one short
-   * 
+   *
    * @param partitions
    * @return Range for each partition
    */
-  public List<Range> partitionRange(int partitions) {
-    // some hokey code to partition the int space
-    long range = Integer.MAX_VALUE + (Math.abs((long) Integer.MIN_VALUE));
-    long srange = range / partitions;
-    
+  public List<Range> partitionRange(int partitions, int min, int max) {
+    assert max >= min;
+    long range = (long)max - (long)min;
+    long srange = Math.max(1, range / partitions);
+
     List<Range> ranges = new ArrayList<Range>(partitions);
-    
-    long end = 0;
-    long start = Integer.MIN_VALUE;
-    
-    while (end < Integer.MAX_VALUE) {
+
+    long start = min;
+    long end = start;
+
+    while (end < max) {
       end = start + srange;
-      ranges.add(new Range(start, end));
+      // make last range always end exactly on MAX_VALUE
+      if (ranges.size() == partitions - 1) {
+        end = max;
+      }
+      ranges.add(new Range((int)start, (int)end));
       start = end + 1L;
     }
-    
+
     return ranges;
   }
+
 }

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java Thu Aug 30 22:43:41 2012
@@ -74,6 +74,7 @@ public class SolrZkClient {
   private ZkCmdExecutor zkCmdExecutor = new ZkCmdExecutor();
 
   private volatile boolean isClosed = false;
+  private ZkClientConnectionStrategy zkClientConnectionStrategy;
   
   /**
    * @param zkServerAddress
@@ -116,6 +117,7 @@ public class SolrZkClient {
    */
   public SolrZkClient(String zkServerAddress, int zkClientTimeout,
       ZkClientConnectionStrategy strat, final OnReconnect onReconnect, int clientConnectTimeout) {
+    this.zkClientConnectionStrategy = strat;
     connManager = new ConnectionManager("ZooKeeperConnection Watcher:"
         + zkServerAddress, this, zkServerAddress, zkClientTimeout, strat, onReconnect);
     try {
@@ -135,29 +137,24 @@ public class SolrZkClient {
               }
             }
           });
-    } catch (IOException e) {
-      connManager.close();
-      throw new RuntimeException();
-    } catch (InterruptedException e) {
-      connManager.close();
-      throw new RuntimeException();
-    } catch (TimeoutException e) {
+    } catch (Throwable e) {
       connManager.close();
       throw new RuntimeException();
     }
+    
     try {
       connManager.waitForConnected(clientConnectTimeout);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      connManager.close();
-      throw new RuntimeException();
-    } catch (TimeoutException e) {
+    } catch (Throwable e) {
       connManager.close();
       throw new RuntimeException();
     }
     numOpens.incrementAndGet();
   }
 
+  public ZkClientConnectionStrategy getZkClientConnectionStrategy() {
+    return zkClientConnectionStrategy;
+  }
+
   /**
    * @return true if client is connected
    */

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java Thu Aug 30 22:43:41 2012
@@ -18,18 +18,65 @@ package org.apache.solr.common.cloud;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.solr.common.SolrException;
 import org.apache.zookeeper.SolrZooKeeper;
 import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *
  */
 public abstract class ZkClientConnectionStrategy {
+  private static Logger log = LoggerFactory.getLogger(ZkClientConnectionStrategy.class);
+  
+  private List<DisconnectedListener> disconnectedListeners = new ArrayList<DisconnectedListener>();
+  private List<ConnectedListener> connectedListeners = new ArrayList<ConnectedListener>();
+  
   public abstract void connect(String zkServerAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException;
   public abstract void reconnect(String serverAddress, int zkClientTimeout, Watcher watcher, ZkUpdate updater) throws IOException, InterruptedException, TimeoutException;
   
+  public synchronized void disconnected() {
+    for (DisconnectedListener listener : disconnectedListeners) {
+      try {
+        listener.disconnected();
+      } catch (Throwable t) {
+        SolrException.log(log, "", t);
+      }
+    }
+  }
+  
+  public synchronized void connected() {
+    for (ConnectedListener listener : connectedListeners) {
+      try {
+        listener.connected();
+      } catch (Throwable t) {
+        SolrException.log(log, "", t);
+      }
+    }
+  }
+  
+  public interface DisconnectedListener {
+    public void disconnected();
+  };
+  
+  public interface ConnectedListener {
+    public void connected();
+  };
+  
+  
+  public synchronized void addDisconnectedListener(DisconnectedListener listener) {
+    disconnectedListeners.add(listener);
+  }
+  
+  public synchronized void addConnectedListener(ConnectedListener listener) {
+    connectedListeners.add(listener);
+  }
+  
   public static abstract class ZkUpdate {
     public abstract void update(SolrZooKeeper zooKeeper) throws InterruptedException, TimeoutException, IOException;
   }

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCoreNodeProps.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCoreNodeProps.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCoreNodeProps.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCoreNodeProps.java Thu Aug 30 22:43:41 2012
@@ -24,7 +24,6 @@ public class ZkCoreNodeProps {
     this.nodeProps = nodeProps;
   }
   
-  // may return null
   public String getCoreUrl() {
     return getCoreUrl(nodeProps.get(ZkStateReader.BASE_URL_PROP), nodeProps.get(ZkStateReader.CORE_NAME_PROP));
   }
@@ -51,13 +50,10 @@ public class ZkCoreNodeProps {
   
   public static String getCoreUrl(String baseUrl, String coreName) {
     StringBuilder sb = new StringBuilder();
-    if (baseUrl == null) return null;
-    if (coreName == null) return null;
     sb.append(baseUrl);
     if (!baseUrl.endsWith("/")) sb.append("/");
-    sb.append(coreName == null ? "" : coreName);
+    sb.append(coreName);
     if (!(sb.substring(sb.length() - 1).equals("/"))) sb.append("/");
-    
     return sb.toString();
   }
 

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Thu Aug 30 22:43:41 2012
@@ -385,13 +385,6 @@ public class ZkStateReader {
 
 	}
   
-  /**
-   * Get shard leader url.
-   */
-  public String getLeaderUrl(String collection, String shard) throws InterruptedException, KeeperException {
-    return getLeaderUrl(collection, shard, 1000);
-  }
-  
   public String getLeaderUrl(String collection, String shard, int timeout)
       throws InterruptedException, KeeperException {
     ZkCoreNodeProps props = new ZkCoreNodeProps(getLeaderProps(collection,

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java Thu Aug 30 22:43:41 2012
@@ -92,7 +92,8 @@ public interface CoreAdminParams 
     SWAP,
     RENAME,
     MERGEINDEXES,
-    PREPRECOVERY, 
+    SPLIT,
+    PREPRECOVERY,
     REQUESTRECOVERY, 
     REQUESTSYNCSHARD;
     

Modified: lucene/dev/branches/lucene3312/solr/test-framework/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/test-framework/ivy.xml?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/test-framework/ivy.xml (original)
+++ lucene/dev/branches/lucene3312/solr/test-framework/ivy.xml Thu Aug 30 22:43:41 2012
@@ -30,7 +30,6 @@
 
     <dependencies defaultconf="default">
       <dependency org="org.apache.ant" name="ant" rev="1.8.2" transitive="false" />
-      <dependency org="org.apache.ant" name="ant-junit" rev="1.8.2" transitive="false" />
 
       <dependency org="junit" name="junit" rev="4.10" transitive="false" conf="default->*;junit4-stdalone->*" />
       <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="2.0.0.rc5" transitive="false" conf="default->*;junit4-stdalone->*" />

Modified: lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java Thu Aug 30 22:43:41 2012
@@ -203,8 +203,6 @@ public abstract class AbstractDistribZkT
     System.clearProperty("solr.test.sys.prop2");
     resetExceptionIgnores();
     super.tearDown();
-    
-    JettySolrRunner.assertStoppedJetties();
   }
   
   protected void printLayout() throws Exception {

Modified: lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Thu Aug 30 22:43:41 2012
@@ -801,7 +801,7 @@ public abstract class AbstractFullDistri
             SolrDocumentList lst1 = lastJetty.client.solrClient.query(query).getResults();
             SolrDocumentList lst2 = cjetty.client.solrClient.query(query).getResults();
 
-            showDiff(lst1, lst2, lastJetty.toString(), cjetty.client.solrClient.toString());
+            showDiff(lst1, lst2, lastJetty.url, cjetty.url);
           }
 
         }
@@ -1130,7 +1130,8 @@ public abstract class AbstractFullDistri
       
       try {
         commit();
-      } catch (Exception e) {
+      } catch (Throwable t) {
+        t.printStackTrace();
         // we don't care if this commit fails on some nodes
       }
       
@@ -1146,8 +1147,8 @@ public abstract class AbstractFullDistri
         retry  = true;
       }
       cnt++;
-      if (cnt > 2) break;
-      Thread.sleep(4000);
+      if (cnt > 4) break;
+      Thread.sleep(2000);
     } while (retry);
   }
   

Modified: lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java (original)
+++ lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java Thu Aug 30 22:43:41 2012
@@ -17,7 +17,6 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
-import java.net.BindException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -174,16 +173,10 @@ public class ChaosMonkey {
   public static void kill(CloudJettyRunner cjetty) throws Exception {
     JettySolrRunner jetty = cjetty.jetty;
     monkeyLog("kill shard! " + jetty.getLocalPort());
-    FilterHolder fh = jetty.getDispatchFilter();
-    SolrDispatchFilter sdf = null;
-    if (fh != null) {
-      sdf = (SolrDispatchFilter) fh.getFilter();
-    }
+    
     jetty.stop();
     
-    if (sdf != null) {
-      sdf.destroy();
-    }
+    stop(jetty);
     
     if (!jetty.isStopped()) {
       throw new RuntimeException("could not kill jetty");
@@ -441,6 +434,7 @@ public class ChaosMonkey {
   }
   
   public static boolean start(JettySolrRunner jetty) throws Exception {
+    
     try {
       jetty.start();
     } catch (Exception e) {
@@ -454,7 +448,7 @@ public class ChaosMonkey {
         try {
           jetty.start();
         } catch (Exception e3) {
-          log.error("", e3);
+          log.error("Could not get the port to start jetty again", e3);
           // we coud not get the port
           jetty.stop();
           return false;

Modified: lucene/dev/branches/lucene3312/solr/webapp/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/webapp/build.xml?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/webapp/build.xml (original)
+++ lucene/dev/branches/lucene3312/solr/webapp/build.xml Thu Aug 30 22:43:41 2012
@@ -40,10 +40,6 @@
   <target name="dist"
           description="Creates the Solr WAR Distribution file."
           depends="test, init-dist, dist-core, dist-solrj, lucene-jars-to-solr">
-    <exec dir="." executable="${svnversion.exe}"
-          outputproperty="svnversion" failifexecutionfails="false">
-      <arg line="."/>
-    </exec>
     <build-manifest title="Apache Solr Search Server"
                     implementation.title="org.apache.solr"
                     spec.version="${solr.spec.version}"/>

Modified: lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js?rev=1379200&r1=1379199&r2=1379200&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js (original)
+++ lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js Thu Aug 30 22:43:41 2012
@@ -258,8 +258,7 @@ var solr_admin = function( app_config )
           {
             show_global_error
             (
-              '<div class="message">There are no SolrCores running — for the current functionality ' +
-              'we require at least one SolrCore, sorry :)</div>'
+              '<div class="message">There are no SolrCores running. <br/> Using the Solr Admin UI currently requires at least one SolrCore.</div>'
             );
             return;
           } // else: we have at least one core....