You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2015/01/23 17:13:34 UTC

svn commit: r1654256 [3/3] - in /lucene/dev/trunk/solr: ./ contrib/clustering/src/test/org/apache/solr/handler/clustering/ contrib/map-reduce/src/test/org/apache/solr/hadoop/ contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/ core/src/te...

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java?rev=1654256&r1=1654255&r2=1654256&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java Fri Jan 23 16:13:32 2015
@@ -39,12 +39,19 @@ import org.apache.solr.schema.TrieDateFi
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -63,6 +70,20 @@ import java.util.concurrent.atomic.Atomi
 /**
  * Helper base class for distributed search test cases
  *
+ * By default, all tests in sub-classes will be executed with
+ * 1, 2, ... DEFAULT_MAX_SHARD_COUNT number of shards set up repeatedly.
+ *
+ * In general, it's preferable to annotate the tests in sub-classes with a
+ * {@literal @}ShardsFixed(num = N) or a {@literal @}ShardsRepeat(min = M, max = N)
+ * to indicate whether the test should be called once, with a fixed number of shards,
+ * or called repeatedly for number of shards = M to N.
+ *
+ * In some cases though, if the number of shards has to be fixed, but the number
+ * itself is dynamic, or if it has to be set as a default for all sub-classes
+ * of a sub-class, there's a fixShardCount(N) available, which is identical to
+ * {@literal @}ShardsFixed(num = N) for all tests without annotations in that class
+ * hierarchy. Ideally this function should be retired in favour of better annotations..
+ *
  * @since solr 1.5
  */
 public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
@@ -174,16 +195,19 @@ public abstract class BaseDistributedSea
                                      "[ff01::213]:33332" + context};
   }
 
-  protected int shardCount = 4;      // the actual number of solr cores that will be created in the cluster
+  private final static int DEFAULT_MAX_SHARD_COUNT = 3;
 
-  /**
-   * Sub classes can set this flag in their constructor to true if they
-   * want to fix the number of shards to 'shardCount'
-   *
-   * The default is false which means that test will be executed with
-   * 1, 2, 3, ....shardCount number of shards repeatedly
-   */
-  protected boolean fixShardCount = false;
+  private int shardCount = -1;      // the actual number of solr cores that will be created in the cluster
+  public int getShardCount() {
+    return shardCount;
+  }
+
+  private boolean isShardCountFixed = false;
+
+  public void fixShardCount(int count) {
+    isShardCountFixed = true;
+    shardCount = count;
+  }
 
   protected JettySolrRunner controlJetty;
   protected List<SolrClient> clients = new ArrayList<>();
@@ -245,13 +269,6 @@ public abstract class BaseDistributedSea
 
   public static RandVal rdate = new RandDate();
 
-  /**
-   * Perform the actual tests here
-   *
-   * @throws Exception on error
-   */
-  public abstract void doTest() throws Exception;
-
   public static String[] fieldNames = new String[]{"n_ti1", "n_f1", "n_tf1", "n_d1", "n_td1", "n_l1", "n_tl1", "n_dt1", "n_tdt1"};
   public static RandVal[] randVals = new RandVal[]{rint, rfloat, rfloat, rdouble, rdouble, rlong, rlong, rdate, rdate};
 
@@ -270,20 +287,20 @@ public abstract class BaseDistributedSea
   public String getSolrHome() {
     return SolrTestCaseJ4.TEST_HOME();
   }
-  
-  @Override
-  public void setUp() throws Exception {
+
+  private boolean distribSetUpCalled = false;
+  public void distribSetUp() throws Exception {
+    distribSetUpCalled = true;
     SolrTestCaseJ4.resetExceptionIgnores();  // ignore anything with ignore_exception in it
-    super.setUp();
     System.setProperty("solr.test.sys.prop1", "propone");
     System.setProperty("solr.test.sys.prop2", "proptwo");
     testDir = createTempDir().toFile();
   }
 
-  @Override
-  public void tearDown() throws Exception {
+  private boolean distribTearDownCalled = false;
+  public void distribTearDown() throws Exception {
+    distribTearDownCalled = true;
     destroyServers();
-    super.tearDown();
   }
 
   protected JettySolrRunner createControlJetty() throws Exception {
@@ -870,23 +887,107 @@ public abstract class BaseDistributedSea
     compareSolrResponses(a, b);
   }
 
-  @Test
-  public void testDistribSearch() throws Exception {
-    if (fixShardCount) {
-      createServers(shardCount);
-      RandVal.uniqueValues = new HashSet(); //reset random values
-      doTest();
-      destroyServers();
-    } else {
-      for (int nServers = 1; nServers < shardCount; nServers++) {
-        createServers(nServers);
+  @Retention(RetentionPolicy.RUNTIME)
+  @Target(ElementType.METHOD)
+  public @interface ShardsRepeat {
+    public abstract int min() default 1;
+    public abstract int max() default DEFAULT_MAX_SHARD_COUNT;
+  }
+
+  @Retention(RetentionPolicy.RUNTIME)
+  @Target(ElementType.METHOD)
+  public @interface ShardsFixed {
+    public abstract int num();
+  }
+
+  public class ShardsRepeatRule implements TestRule {
+
+    private abstract class ShardsStatement extends Statement {
+      abstract protected void callStatement() throws Throwable;
+
+      @Override
+      public void evaluate() throws Throwable {
+        distribSetUp();
+        if (! distribSetUpCalled) {
+          Assert.fail("One of the overrides of distribSetUp does not propagate the call.");
+        }
+        try {
+          callStatement();
+        } finally {
+          distribTearDown();
+          if (! distribTearDownCalled) {
+            Assert.fail("One of the overrides of distribTearDown does not propagate the call.");
+          }
+        }
+      }
+    }
+
+    private class ShardsFixedStatement extends ShardsStatement {
+
+      private final int numShards;
+      private final Statement statement;
+
+      private ShardsFixedStatement(int numShards, Statement statement) {
+        this.numShards = numShards;
+        this.statement = statement;
+      }
+
+      @Override
+      public void callStatement() throws Throwable {
+        fixShardCount(numShards);
+        createServers(numShards);
         RandVal.uniqueValues = new HashSet(); //reset random values
-        doTest();
+        statement.evaluate();
         destroyServers();
       }
     }
+
+    private class ShardsRepeatStatement extends ShardsStatement {
+
+      private final int min;
+      private final int max;
+      private final Statement statement;
+
+      private ShardsRepeatStatement(int min, int max, Statement statement) {
+        this.min = min;
+        this.max = max;
+        this.statement = statement;
+      }
+
+      @Override
+      public void callStatement() throws Throwable {
+        for (shardCount = min; shardCount <= max; shardCount++) {
+          createServers(shardCount);
+          RandVal.uniqueValues = new HashSet(); //reset random values
+          statement.evaluate();
+          destroyServers();
+        }
+      }
+    }
+
+    @Override
+    public Statement apply(Statement statement, Description description) {
+      ShardsFixed fixed = description.getAnnotation(ShardsFixed.class);
+      ShardsRepeat repeat = description.getAnnotation(ShardsRepeat.class);
+      if (fixed != null && repeat != null) {
+        throw new RuntimeException("ShardsFixed and ShardsRepeat annotations can't coexist");
+      }
+      else if (fixed != null) {
+        return new ShardsFixedStatement(fixed.num(), statement);
+      }
+      else if (repeat != null) {
+        return new ShardsRepeatStatement(repeat.min(), repeat.max(), statement);
+      }
+      else {
+        return (isShardCountFixed ? new ShardsFixedStatement(shardCount, statement) :
+          new ShardsRepeatStatement(1, DEFAULT_MAX_SHARD_COUNT, statement));
+      }
+    }
   }
 
+  @Rule
+  public ShardsRepeatRule repeatRule = new ShardsRepeatRule();
+
   public static Object[] getRandFields(String[] fields, RandVal[] randVals) {
     Object[] o = new Object[fields.length * 2];
     for (int i = 0; i < fields.length; i++) {

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java?rev=1654256&r1=1654255&r2=1654256&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java Fri Jan 23 16:13:32 2015
@@ -33,8 +33,6 @@ import org.apache.solr.core.Diagnostics;
 import org.apache.solr.core.MockDirectoryFactory;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.zookeeper.KeeperException;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.BeforeClass;
 
 public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTestCase {
@@ -54,11 +52,9 @@ public abstract class AbstractDistribZkT
     //useFactory(null);
   }
 
-
-  @Before
   @Override
-  public void setUp() throws Exception {
-    super.setUp();
+  public void distribSetUp() throws Exception {
+    super.distribSetUp();
     
     String zkDir = testDir.getAbsolutePath() + File.separator
     + "zookeeper/server1/data";
@@ -212,8 +208,7 @@ public abstract class AbstractDistribZkT
   }
   
   @Override
-  @After
-  public void tearDown() throws Exception {
+  public void distribTearDown() throws Exception {
     if (DEBUG) {
       printLayout();
     }
@@ -228,7 +223,7 @@ public abstract class AbstractDistribZkT
     System.clearProperty(MockDirectoryFactory.SOLR_TESTS_ALLOW_READING_FILES_STILL_OPEN_FOR_WRITE);
     
     resetExceptionIgnores();
-    super.tearDown();
+    super.distribTearDown();
     zkServer.shutdown();
   }
   

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1654256&r1=1654255&r2=1654256&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Fri Jan 23 16:13:32 2015
@@ -55,9 +55,7 @@ import org.apache.solr.core.SolrResource
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.zookeeper.CreateMode;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.noggit.CharArr;
 import org.noggit.JSONWriter;
@@ -95,18 +93,18 @@ import static org.apache.solr.common.clo
 @Slow
 public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTestBase {
   static Logger log = LoggerFactory.getLogger(AbstractFullDistribZkTestBase.class);
-  
+
   @BeforeClass
   public static void beforeFullSolrCloudTest() {
     // shorten the log output more for this test type
     if (formatter != null) formatter.setShorterFormat();
   }
-  
+
   public static final String SHARD1 = "shard1";
   public static final String SHARD2 = "shard2";
-  
+
   protected boolean printLayoutOnTearDown = false;
-  
+
   String t1 = "a_t";
   String i1 = "a_i1";
   String tlong = "other_tl1";
@@ -122,14 +120,14 @@ public abstract class AbstractFullDistri
   protected Map<String,List<CloudJettyRunner>> shardToJetty = new HashMap<>();
   private AtomicInteger jettyIntCntr = new AtomicInteger(0);
   protected ChaosMonkey chaosMonkey;
-  
+
   protected Map<String,CloudJettyRunner> shardToLeaderJetty = new HashMap<>();
   private boolean cloudInit;
   protected boolean checkCreatedVsState;
   protected boolean useJettyDataDir = true;
 
   protected Map<URI,SocketProxy> proxies = new HashMap<URI,SocketProxy>();
-  
+
   public static class CloudJettyRunner {
     public JettySolrRunner jetty;
     public String nodeName;
@@ -160,19 +158,19 @@ public abstract class AbstractFullDistri
       return "CloudJettyRunner [url=" + url + "]";
     }
   }
-  
+
   static class CloudSolrServerClient {
     SolrClient solrClient;
     String shardName;
     int port;
     public ZkNodeProps info;
-    
+
     public CloudSolrServerClient() {}
     
     public CloudSolrServerClient(SolrClient client) {
       this.solrClient = client;
     }
-    
+
     @Override
     public int hashCode() {
       final int prime = 31;
@@ -180,7 +178,7 @@ public abstract class AbstractFullDistri
       result = prime * result + ((solrClient == null) ? 0 : solrClient.hashCode());
       return result;
     }
-    
+
     @Override
     public boolean equals(Object obj) {
       if (this == obj) return true;
@@ -192,69 +190,67 @@ public abstract class AbstractFullDistri
       } else if (!solrClient.equals(other.solrClient)) return false;
       return true;
     }
-    
+
   }
-  
-  @Before
+
   @Override
-  public void setUp() throws Exception {
-    super.setUp();
+  public void distribSetUp() throws Exception {
+    super.distribSetUp();
     // ignoreException(".*");
     if (sliceCount > 0) {
       System.setProperty("numShards", Integer.toString(sliceCount));
     } else {
       System.clearProperty("numShards");
     }
-    
+
     if (isSSLMode()) {
       System.clearProperty("urlScheme");
       ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
           AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT);
       try {
         zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS,
-          ZkStateReader.toJSON(Collections.singletonMap("urlScheme","https")), 
+          ZkStateReader.toJSON(Collections.singletonMap("urlScheme","https")),
           CreateMode.PERSISTENT, true);
       } finally {
         zkStateReader.close();
       }
     }
   }
-  
+
   @BeforeClass
   public static void beforeClass() {
     System.setProperty("solrcloud.update.delay", "0");
   }
-  
+
   @AfterClass
   public static void afterClass() throws Exception {
     System.clearProperty("solrcloud.update.delay");
     System.clearProperty("genericCoreNodeNames");
   }
-  
+
   public AbstractFullDistribZkTestBase() {
-    fixShardCount = true;
-    
-    shardCount = 4;
     sliceCount = 2;
+    fixShardCount(4);
+
     // TODO: for now, turn off stress because it uses regular clients, and we
     // need the cloud client because we kill servers
     stress = 0;
-    
+
     useExplicitNodeNames = random().nextBoolean();
   }
-  
+
   protected String getDataDir(String dataDir) throws IOException {
     return dataDir;
   }
-  
+
   protected void initCloud() throws Exception {
     assert(cloudInit == false);
     cloudInit = true;
     cloudClient = createCloudClient(DEFAULT_COLLECTION);
     cloudClient.connect();
-    
+
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
-    
+
     chaosMonkey = new ChaosMonkey(zkServer, zkStateReader, DEFAULT_COLLECTION,
         shardToJetty, shardToLeaderJetty);
   }
@@ -269,18 +265,18 @@ public abstract class AbstractFullDistri
     .setParameter(CoreConnectionPNames.SO_TIMEOUT, 60000);
     return client;
   }
-  
+
   @Override
   protected void createServers(int numServers) throws Exception {
-    
+
     System.setProperty("collection", "control_collection");
 
     // we want hashes by default for the control, so set to 1 shard as opposed to leaving unset
     String oldNumShards = System.getProperty(ZkStateReader.NUM_SHARDS_PROP);
     System.setProperty(ZkStateReader.NUM_SHARDS_PROP, "1");
-    
+
     try {
-      
+
       File controlJettyDir = createTempDir("control").toFile();
       setupJettySolrHome(controlJettyDir);
 
@@ -305,7 +301,7 @@ public abstract class AbstractFullDistri
                                           // cloudClient
         return;
       }
-      
+
     } finally {
       System.clearProperty("collection");
       if (oldNumShards != null) {
@@ -317,9 +313,9 @@ public abstract class AbstractFullDistri
 
 
     initCloud();
-    
+
     createJettys(numServers, checkCreatedVsState).size();
-    
+
     int cnt = getTotalReplicas(DEFAULT_COLLECTION);
     if (cnt > 0) {
       waitForCollection(cloudClient.getZkStateReader(), DEFAULT_COLLECTION, sliceCount);
@@ -348,7 +344,7 @@ public abstract class AbstractFullDistri
       Thread.sleep(500);
     }
   }
-  
+
   protected List<JettySolrRunner> createJettys(int numJettys) throws Exception {
     return createJettys(numJettys, false);
   }
@@ -402,22 +398,22 @@ public abstract class AbstractFullDistri
       SolrClient client = createNewSolrClient(j.getLocalPort());
       clients.add(client);
     }
-  
+
     this.jettys.addAll(jettys);
     this.clients.addAll(clients);
-    
+
     int numShards = getTotalReplicas(DEFAULT_COLLECTION);
     if (checkCreatedVsState) {
       // now wait until we see that the number of shards in the cluster state
       // matches what we expect
       int retries = 0;
-      while (numShards != shardCount) {
+      while (numShards != getShardCount()) {
         numShards = getTotalReplicas(DEFAULT_COLLECTION);
-        if (numShards == shardCount) break;
+        if (numShards == getShardCount()) break;
         if (retries++ == 60) {
           printLayoutOnTearDown = true;
           fail("Shards in the state does not match what we set:" + numShards
-              + " vs " + shardCount);
+              + " vs " + getShardCount());
         }
         Thread.sleep(500);
       }
@@ -432,7 +428,7 @@ public abstract class AbstractFullDistri
     if (numShards > 0) {
       updateMappingsFromZk(this.jettys, this.clients);
     }
-    
+
     // build the shard string
     for (int i = 1; i <= numJettys / 2; i++) {
       JettySolrRunner j = this.jettys.get(i);
@@ -442,7 +438,7 @@ public abstract class AbstractFullDistri
       sb.append("|").append(buildUrl(j2.getLocalPort()));
     }
     shards = sb.toString();
-    
+
     return jettys;
   }
 
@@ -494,30 +490,30 @@ public abstract class AbstractFullDistri
     }
     return cnt;
   }
-  
+
   public JettySolrRunner createJetty(String dataDir, String ulogDir, String shardList,
       String solrConfigOverride) throws Exception {
-    
+
     JettySolrRunner jetty = new JettySolrRunner(getSolrHome(), context, 0,
         solrConfigOverride, null, false, getExtraServlets(), sslConfig, getExtraRequestFilters());
     jetty.setShards(shardList);
     jetty.setDataDir(getDataDir(dataDir));
     jetty.start();
-    
+
     return jetty;
   }
-  
+
   public JettySolrRunner createJetty(File solrHome, String dataDir, String shardList, String solrConfigOverride, String schemaOverride) throws Exception {
     // randomly test a relative solr.home path
     if (random().nextBoolean()) {
       solrHome = getRelativeSolrHomePath(solrHome);
     }
-    
+
     JettySolrRunner jetty = new JettySolrRunner(solrHome.getPath(), context, 0, solrConfigOverride, schemaOverride, false, getExtraServlets(), sslConfig, getExtraRequestFilters());
     jetty.setShards(shardList);
     jetty.setDataDir(getDataDir(dataDir));
     jetty.start();
-    
+
     return jetty;
   }
 
@@ -598,7 +594,7 @@ public abstract class AbstractFullDistri
   private File getRelativeSolrHomePath(File solrHome) {
     String path = SolrResourceLoader.normalizeDir(new File(".").getAbsolutePath());
     String base = new File(solrHome.getPath()).getAbsolutePath();
-    
+
     if (base.startsWith(".")) {
       base = base.replaceFirst("\\.", new File(".").getName());
     }
@@ -606,14 +602,14 @@ public abstract class AbstractFullDistri
     if (path.endsWith(File.separator + ".")) {
       path = path.substring(0, path.length() - 2);
     }
-    
+
     int splits = path.split("\\" + File.separator).length;
-    
+
     StringBuilder p = new StringBuilder();
     for (int i = 0; i < splits - 2; i++) {
       p.append("..").append(File.separator);
-    }   
-    
+    }
+
     String prefix = FilenameUtils.getPrefix(path);
     if (base.startsWith(prefix)) {
       base = base.substring(prefix.length());
@@ -632,7 +628,7 @@ public abstract class AbstractFullDistri
     zkStateReader.updateClusterState(true);
     cloudJettys.clear();
     shardToJetty.clear();
-    
+
     ClusterState clusterState = zkStateReader.getClusterState();
     DocCollection coll = clusterState.getCollection(DEFAULT_COLLECTION);
 
@@ -645,28 +641,28 @@ public abstract class AbstractFullDistri
         for (Replica replica : slice.getReplicas()) {
           int port = new URI(((HttpSolrClient) client).getBaseURL())
               .getPort();
-          
+
           if (replica.getStr(ZkStateReader.BASE_URL_PROP).contains(":" + port)) {
             CloudSolrServerClient csc = new CloudSolrServerClient();
             csc.solrClient = client;
             csc.port = port;
             csc.shardName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
             csc.info = replica;
-            
+
             theClients .add(csc);
-            
+
             break nextClient;
           }
         }
       }
     }
- 
+
     for (JettySolrRunner jetty : jettys) {
       int port = jetty.getLocalPort();
       if (port == -1) {
         throw new RuntimeException("Cannot find the port for jetty");
       }
-      
+
       nextJetty:
       for (Slice slice : coll.getSlices()) {
         Set<Entry<String,Replica>> entries = slice.getReplicasMap().entrySet();
@@ -696,7 +692,7 @@ public abstract class AbstractFullDistri
         }
       }
     }
-    
+
     // # of jetties may not match replicas in shard here, because we don't map
     // jetties that are not running - every shard should have at least one
     // running jetty though
@@ -706,13 +702,13 @@ public abstract class AbstractFullDistri
       if (!allowOverSharding) {
         assertNotNull("Test setup problem: We found no jetties for shard: "
             + slice.getName() + " just:" + shardToJetty.keySet(), jetties);
-        
+
         assertEquals("slice:" + slice.getName(), slice.getReplicas().size(),
             jetties.size());
       }
     }
   }
-  
+
   private CloudSolrServerClient findClientByPort(int port, List<CloudSolrServerClient> theClients) {
     for (CloudSolrServerClient client : theClients) {
       if (client.port == port) {
@@ -724,7 +720,7 @@ public abstract class AbstractFullDistri
 
   @Override
   protected void setDistributedParams(ModifiableSolrParams params) {
-    
+
     if (r.nextBoolean()) {
       // don't set shards, let that be figured out from the cloud state
     } else {
@@ -737,19 +733,19 @@ public abstract class AbstractFullDistri
       params.set("shards", sb.toString());
     }
   }
-  
+
   @Override
   protected void indexDoc(SolrInputDocument doc) throws IOException,
       SolrServerException {
-    
+
     UpdateRequest req = new UpdateRequest();
     req.add(doc);
     req.setParam("CONTROL", "TRUE");
     req.process(controlClient);
-    
+
     // if we wanted to randomly pick a client - but sometimes they may be
     // down...
-    
+
     // boolean pick = random.nextBoolean();
     //
     // int which = (doc.getField(id).toString().hashCode() & 0x7fffffff) %
@@ -762,13 +758,13 @@ public abstract class AbstractFullDistri
     //
     // HttpSolrServer client = (HttpSolrServer)
     // clients.get(which);
-    
+
     UpdateRequest ureq = new UpdateRequest();
     ureq.add(doc);
     // ureq.setParam(UpdateParams.UPDATE_CHAIN, DISTRIB_UPDATE_CHAIN);
     ureq.process(cloudClient);
   }
-  
+
   @Override
   protected void index_specific(int serverNumber, Object... fields)
       throws Exception {
@@ -780,7 +776,7 @@ public abstract class AbstractFullDistri
     
     HttpSolrClient client = (HttpSolrClient) clients
         .get(serverNumber);
-    
+
     UpdateRequest ureq = new UpdateRequest();
     ureq.add(doc);
     // ureq.setParam("update.chain", DISTRIB_UPDATE_CHAIN);
@@ -793,16 +789,16 @@ public abstract class AbstractFullDistri
     for (int i = 0; i < fields.length; i += 2) {
       doc.addField((String) (fields[i]), fields[i + 1]);
     }
-    
+
     UpdateRequest ureq = new UpdateRequest();
     ureq.add(doc);
     // ureq.setParam("update.chain", DISTRIB_UPDATE_CHAIN);
     ureq.process(client);
-    
+
     // add to control second in case adding to shards fails
     controlClient.add(doc);
   }
-  
+
   @Override
   protected void del(String q) throws Exception {
     controlClient.deleteByQuery(q);
@@ -816,25 +812,25 @@ public abstract class AbstractFullDistri
     }
      ***/
   }// serial commit...
-  
+
   protected void waitForRecoveriesToFinish(boolean verbose)
       throws Exception {
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     super.waitForRecoveriesToFinish(DEFAULT_COLLECTION, zkStateReader, verbose);
   }
-  
+
   protected void waitForRecoveriesToFinish(String collection, boolean verbose)
       throws Exception {
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     super.waitForRecoveriesToFinish(collection, zkStateReader, verbose);
   }
-  
+
   protected void waitForRecoveriesToFinish(boolean verbose, int timeoutSeconds)
       throws Exception {
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     super.waitForRecoveriesToFinish(DEFAULT_COLLECTION, zkStateReader, verbose, true, timeoutSeconds);
   }
-  
+
   protected void checkQueries() throws Exception {
 
     handle.put("_version_", SKIPVAL);
@@ -858,7 +854,7 @@ public abstract class AbstractFullDistri
       query("q", "*:*", "sort", f + " desc");
       query("q", "*:*", "sort", f + " asc");
     }
-    
+
     // these queries should be exactly ordered and scores should exactly match
     query("q", "*:*", "sort", i1 + " desc");
     query("q", "*:*", "sort", i1 + " asc");
@@ -875,30 +871,30 @@ public abstract class AbstractFullDistri
     handle.remove("maxScore");
     query("q", "{!func}" + i1, "fl", "*,score"); // even scores should match
                                                  // exactly here
-    
+
     handle.put("highlighting", UNORDERED);
     handle.put("response", UNORDERED);
-    
+
     handle.put("maxScore", SKIPVAL);
     query("q", "quick");
     query("q", "all", "fl", "id", "start", "0");
     query("q", "all", "fl", "foofoofoo", "start", "0"); // no fields in returned
                                                         // docs
     query("q", "all", "fl", "id", "start", "100");
-    
+
     handle.put("score", SKIPVAL);
     query("q", "quick", "fl", "*,score");
     query("q", "all", "fl", "*,score", "start", "1");
     query("q", "all", "fl", "*,score", "start", "100");
-    
+
     query("q", "now their fox sat had put", "fl", "*,score", "hl", "true",
         "hl.fl", t1);
-    
+
     query("q", "now their fox sat had put", "fl", "foofoofoo", "hl", "true",
         "hl.fl", t1);
-    
+
     query("q", "matchesnothing", "fl", "*,score");
-    
+
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", t1);
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", t1,
         "facet.limit", -1, "facet.sort", "count");
@@ -916,11 +912,11 @@ public abstract class AbstractFullDistri
         "facet.offset", 1);
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", t1,
         "facet.mincount", 2);
-    
+
     // test faceting multiple things at once
     query("q", "*:*", "rows", 100, "facet", "true", "facet.query", "quick",
         "facet.query", "all", "facet.query", "*:*", "facet.field", t1);
-    
+
     // test filter tagging, facet exclusion, and naming (multi-select facet
     // support)
     query("q", "*:*", "rows", 100, "facet", "true", "facet.query",
@@ -933,16 +929,16 @@ public abstract class AbstractFullDistri
         "{!ex=t1}SubjectTerms_mfacet", "fq",
         "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10",
         "facet.mincount", "1");
-    
+
     // test field that is valid in schema but missing in all shards
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field",
         missingField, "facet.mincount", 2);
     // test field that is valid in schema and missing in some shards
     query("q", "*:*", "rows", 100, "facet", "true", "facet.field", oddField,
         "facet.mincount", 2);
-    
+
     query("q", "*:*", "sort", i1 + " desc", "stats", "true", "stats.field", i1);
-    
+
     // Try to get better coverage for refinement queries by turning off over
     // requesting.
     // This makes it much more likely that we may not get the top facet values
@@ -956,15 +952,15 @@ public abstract class AbstractFullDistri
         "{!key='a b/c \\' \\} foo'}" + t1, "facet.limit", 5,
         "facet.shard.limit", 5);
     handle.remove("facet_fields");
-    
+
     query("q", "*:*", "sort", "n_tl1 desc");
-    
+
     // index the same document to two shards and make sure things
     // don't blow up.
     // assumes first n clients are first n shards
     if (clients.size() >= 2) {
       index(id, 100, i1, 107, t1, "oh no, a duplicate!");
-      for (int i = 0; i < shardCount; i++) {
+      for (int i = 0; i < getShardCount(); i++) {
         index_specific(i, id, 100, i1, 107, t1, "oh no, a duplicate!");
       }
       commit();
@@ -973,7 +969,7 @@ public abstract class AbstractFullDistri
       query("q", "*:*", "rows", 100);
     }
   }
-  
+
   protected void indexAbunchOfDocs() throws Exception {
     indexr(id, 2, i1, 50, t1, "to come to the aid of their country.");
     indexr(id, 3, i1,  2, t1, "how now brown cow");
@@ -993,7 +989,7 @@ public abstract class AbstractFullDistri
         "Great works are performed, not by strength, but by perseverance.");
     indexr(id, 13, i1, 232, t1, "no eggs on wall, lesson learned",
         oddField, "odd man out");
-    
+
     indexr(id, 14, "SubjectTerms_mfacet", new String[] {"mathematical models",
         "mathematical analysis"});
     indexr(id, 15, "SubjectTerms_mfacet", new String[] {"test 1", "test 2",
@@ -1005,19 +1001,19 @@ public abstract class AbstractFullDistri
       vals[i] = "test " + i;
     }
     indexr(id, 17, "SubjectTerms_mfacet", vals);
-    
+
     for (int i = 100; i < 150; i++) {
       indexr(id, i);
     }
   }
-  
+
   /**
-   * Executes a query against each live and active replica of the specified shard 
+   * Executes a query against each live and active replica of the specified shard
    * and aserts that the results are identical.
    *
    * @see #queryAndCompare
    */
-  public QueryResponse queryAndCompareReplicas(SolrParams params, String shard) 
+  public QueryResponse queryAndCompareReplicas(SolrParams params, String shard)
     throws Exception {
 
     ArrayList<SolrClient> shardClients = new ArrayList<>(7);
@@ -1042,9 +1038,9 @@ public abstract class AbstractFullDistri
 
   /**
    * For each Shard, executes a query against each live and active replica of that shard
-   * and asserts that the results are identical for each replica of the same shard.  
-   * Because results are not compared between replicas of different shards, this method 
-   * should be safe for comparing the results of any query, even if it contains 
+   * and asserts that the results are identical for each replica of the same shard.
+   * Because results are not compared between replicas of different shards, this method
+   * should be safe for comparing the results of any query, even if it contains
    * "distrib=false", because the replicas should all be identical.
    *
    * @see AbstractFullDistribZkTestBase#queryAndCompareReplicas(SolrParams, String)
@@ -1058,25 +1054,25 @@ public abstract class AbstractFullDistri
     }
   }
 
-  /** 
-   * Returns a non-null string if replicas within the same shard do not have a 
-   * consistent number of documents. 
+  /**
+   * Returns a non-null string if replicas within the same shard do not have a
+   * consistent number of documents.
    */
   protected void checkShardConsistency(String shard) throws Exception {
     checkShardConsistency(shard, false, false);
   }
 
-  /** 
-   * Returns a non-null string if replicas within the same shard do not have a 
+  /**
+   * Returns a non-null string if replicas within the same shard do not have a
    * consistent number of documents.
-   * If expectFailure==false, the exact differences found will be logged since 
+   * If expectFailure==false, the exact differences found will be logged since
    * this would be an unexpected failure.
-   * verbose causes extra debugging into to be displayed, even if everything is 
+   * verbose causes extra debugging into to be displayed, even if everything is
    * consistent.
    */
   protected String checkShardConsistency(String shard, boolean expectFailure, boolean verbose)
       throws Exception {
-    
+
     List<CloudJettyRunner> solrJetties = shardToJetty.get(shard);
     if (solrJetties == null) {
       throw new RuntimeException("shard not found:" + shard + " keys:"
@@ -1112,7 +1108,7 @@ public abstract class AbstractFullDistri
             + e.getMessage() + "\n");
         continue;
       }
-      
+
       boolean live = false;
       String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
       if (zkStateReader.getClusterState().liveNodesContain(nodeName)) {
@@ -1149,19 +1145,19 @@ public abstract class AbstractFullDistri
       }
     }
     return failMessage;
-    
+
   }
-  
+
   public void showCounts() {
     Set<String> theShards = shardToJetty.keySet();
-    
+
     for (String shard : theShards) {
       List<CloudJettyRunner> solrJetties = shardToJetty.get(shard);
-      
+
       for (CloudJettyRunner cjetty : solrJetties) {
         ZkNodeProps props = cjetty.info;
         System.err.println("PROPS:" + props);
-        
+
         try {
           SolrParams query = params("q", "*:*", "rows", "0", "distrib",
               "false", "tests", "checkShardConsistency"); // "tests" is just a
@@ -1187,11 +1183,11 @@ public abstract class AbstractFullDistri
           live = true;
         }
         System.err.println(" live:" + live);
-        
+
       }
     }
   }
-  
+
   protected void randomlyEnableAutoSoftCommit() {
     if (r.nextBoolean()) {
       enableAutoSoftCommit(1000);
@@ -1199,7 +1195,7 @@ public abstract class AbstractFullDistri
       log.info("Not turning on auto soft commit");
     }
   }
-  
+
   protected void enableAutoSoftCommit(int time) {
     log.info("Turning on auto soft commit: " + time);
     for (List<CloudJettyRunner> jettyList : shardToJetty.values()) {
@@ -1228,7 +1224,7 @@ public abstract class AbstractFullDistri
       throws Exception {
     checkShardConsistency(checkVsControl, verbose, null, null);
   }
-  
+
   /* Checks shard consistency and optionally checks against the control shard.
    * The test will be failed if differences are found.
    */
@@ -1236,7 +1232,7 @@ public abstract class AbstractFullDistri
       throws Exception {
 
     updateMappingsFromZk(jettys, clients, true);
-    
+
     Set<String> theShards = shardToJetty.keySet();
     String failMessage = null;
     for (String shard : theShards) {
@@ -1245,7 +1241,7 @@ public abstract class AbstractFullDistri
         failMessage = shardFailMessage;
       }
     }
-    
+
     if (failMessage != null) {
       fail(failMessage);
     }
@@ -1260,7 +1256,7 @@ public abstract class AbstractFullDistri
     SolrDocumentList cloudDocList = cloudClient.query(q).getResults();
     long cloudClientDocs = cloudDocList.getNumFound();
 
-    
+
     // now check that the right # are on each shard
     theShards = shardToJetty.keySet();
     int cnt = 0;
@@ -1314,7 +1310,7 @@ public abstract class AbstractFullDistri
     }
     return null;
   }
-  
+
   protected void assertDocCounts(boolean verbose) throws Exception {
     // TODO: as we create the clients, we should build a map from shard to
     // node/client
@@ -1324,7 +1320,7 @@ public abstract class AbstractFullDistri
         + "\n\n");
     long controlCount = controlClient.query(new SolrQuery("*:*")).getResults()
         .getNumFound();
-    
+
     // do some really inefficient mapping...
     ZkStateReader zk = new ZkStateReader(zkServer.getZkAddress(), 10000,
         AbstractZkTestCase.TIMEOUT);
@@ -1337,12 +1333,12 @@ public abstract class AbstractFullDistri
     } finally {
       zk.close();
     }
-    
+
     if (slices == null) {
       throw new RuntimeException("Could not find collection "
           + DEFAULT_COLLECTION + " in " + clusterState.getCollections());
     }
-    
+
     for (CloudJettyRunner cjetty : cloudJettys) {
       CloudSolrServerClient client = cjetty.client;
       for (Map.Entry<String,Slice> slice : slices.entrySet()) {
@@ -1368,7 +1364,7 @@ public abstract class AbstractFullDistri
         query.set("distrib", false);
         count = client.solrClient.query(query).getResults().getNumFound();
       }
-      
+
       if (verbose) System.err.println("client docs:" + count + "\n\n");
     }
     if (verbose) System.err.println("control docs:"
@@ -1378,39 +1374,39 @@ public abstract class AbstractFullDistri
     assertEquals("Doc Counts do not add up", controlCount,
         cloudClient.query(query).getResults().getNumFound());
   }
-  
+
   @Override
   protected QueryResponse queryServer(ModifiableSolrParams params)
       throws SolrServerException {
-    
+
     if (r.nextBoolean()) params.set("collection", DEFAULT_COLLECTION);
-    
+
     QueryResponse rsp = cloudClient.query(params);
     return rsp;
   }
-  
+
   static abstract class StopableThread extends Thread {
     public StopableThread(String name) {
       super(name);
     }
     public abstract void safeStop();
   }
-  
+
   class StopableSearchThread extends StopableThread {
     private volatile boolean stop = false;
     protected final AtomicInteger queryFails = new AtomicInteger();
     private String[] QUERIES = new String[] {"to come","their country","aid","co*"};
-    
+
     public StopableSearchThread() {
       super("StopableSearchThread");
       setDaemon(true);
     }
-    
+
     @Override
     public void run() {
       Random random = random();
       int numSearches = 0;
-      
+
       while (true && !stop) {
         numSearches++;
         try {
@@ -1431,43 +1427,43 @@ public abstract class AbstractFullDistri
           Thread.currentThread().interrupt();
         }
       }
-      
+
       System.err.println("num searches done:" + numSearches + " with " + queryFails + " fails");
     }
-    
+
     @Override
     public void safeStop() {
       stop = true;
     }
-    
+
     public int getFails() {
       return queryFails.get();
     }
-    
+
   };
-  
+
   public void waitForThingsToLevelOut(int waitForRecTimeSeconds) throws Exception {
     log.info("Wait for recoveries to finish - wait " + waitForRecTimeSeconds + " for each attempt");
     int cnt = 0;
     boolean retry = false;
     do {
       waitForRecoveriesToFinish(VERBOSE, waitForRecTimeSeconds);
-      
+
       try {
         commit();
       } catch (Throwable t) {
         t.printStackTrace();
         // we don't care if this commit fails on some nodes
       }
-      
+
       updateMappingsFromZk(jettys, clients);
-      
+
       Set<String> theShards = shardToJetty.keySet();
       String failMessage = null;
       for (String shard : theShards) {
         failMessage = checkShardConsistency(shard, true, false);
       }
-      
+
       if (failMessage != null) {
         log.info("shard inconsistency - waiting ...");
         retry = true;
@@ -1479,8 +1475,8 @@ public abstract class AbstractFullDistri
       Thread.sleep(2000);
     } while (retry);
   }
-  
-  
+
+
   public void waitForNoShardInconsistency() throws Exception {
     log.info("Wait for no shard inconsistency");
     int cnt = 0;
@@ -1492,9 +1488,9 @@ public abstract class AbstractFullDistri
         t.printStackTrace();
         // we don't care if this commit fails on some nodes
       }
-      
+
       updateMappingsFromZk(jettys, clients);
-      
+
       Set<String> theShards = shardToJetty.keySet();
       String failMessage = null;
       for (String shard : theShards) {
@@ -1505,7 +1501,7 @@ public abstract class AbstractFullDistri
           failMessage="hit exception:" + e.getMessage();
         }
       }
-      
+
       if (failMessage != null) {
         log.info("shard inconsistency - waiting ...");
         retry = true;
@@ -1531,8 +1527,7 @@ public abstract class AbstractFullDistri
   }
 
   @Override
-  @After
-  public void tearDown() throws Exception {
+  public void distribTearDown() throws Exception {
     if (VERBOSE || printLayoutOnTearDown) {
       super.printLayout();
     }
@@ -1548,25 +1543,25 @@ public abstract class AbstractFullDistri
     if (controlClientCloud != null) {
       controlClientCloud.shutdown();
     }
-    super.tearDown();
-    
+    super.distribTearDown();
+
     System.clearProperty("zkHost");
     System.clearProperty("numShards");
 
-    // close socket proxies after super.tearDown
+    // close socket proxies after super.distribTearDown
     if (!proxies.isEmpty()) {
       for (SocketProxy proxy : proxies.values()) {
         proxy.close();
       }
     }
   }
-  
+
   @Override
   protected void commit() throws Exception {
     controlClient.commit();
     cloudClient.commit();
   }
-  
+
   @Override
   protected void destroyServers() throws Exception {
     if (controlJetty != null) {
@@ -1581,7 +1576,7 @@ public abstract class AbstractFullDistri
     }
     super.destroyServers();
   }
-  
+
   protected CollectionAdminResponse createCollection(String collectionName, int numShards, int replicationFactor, int maxShardsPerNode) throws SolrServerException, IOException {
     return createCollection(null, collectionName, numShards, replicationFactor, maxShardsPerNode, null, null);
   }
@@ -1653,7 +1648,7 @@ public abstract class AbstractFullDistri
         MAX_SHARDS_PER_NODE, maxShardsPerNode),
         client);
   }
-  
+
   protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
                                                      String collectionName, int numShards, int replicationFactor, int maxShardsPerNode, SolrClient client, String createNodeSetStr, String configName) throws SolrServerException, IOException {
 
@@ -1702,7 +1697,7 @@ public abstract class AbstractFullDistri
         0, client.getBaseURL().length()
             - DEFAULT_COLLECTION.length() - 1);
   }
-  
+
   protected SolrInputDocument getDoc(Object... fields) throws Exception {
     SolrInputDocument doc = new SolrInputDocument();
     addFields(doc, fields);
@@ -1716,7 +1711,7 @@ public abstract class AbstractFullDistri
     // The Math.min thing is here, because we expect replication-factor to be reduced to if there are not enough live nodes to spread all shards of a collection over different nodes
     int expectedShardsPerSlice = numShardsNumReplicaList.get(1);
     int expectedTotalShards = expectedSlices * expectedShardsPerSlice;
-    
+
 //      Map<String,DocCollection> collections = clusterState
 //          .getCollectionStates();
       if (clusterState.hasCollection(collectionName)) {
@@ -1735,14 +1730,14 @@ public abstract class AbstractFullDistri
         totalShards += slices.get(sliceName).getReplicas().size();
       }
       if (totalShards != expectedTotalShards) {
-        return "Found new collection " + collectionName + " with correct number of slices, but mismatch on number of shards. Expected: " + expectedTotalShards + ", actual: " + totalShards; 
+        return "Found new collection " + collectionName + " with correct number of slices, but mismatch on number of shards. Expected: " + expectedTotalShards + ", actual: " + totalShards;
         }
       return null;
     } else {
       return "Could not find new collection " + collectionName;
     }
   }
-  
+
   protected void checkForCollection(String collectionName,
       List<Integer> numShardsNumReplicaList,
       List<String> nodesAllowedToRunShards) throws Exception {
@@ -1780,7 +1775,7 @@ public abstract class AbstractFullDistri
     }
     return commondCloudSolrClient;
   }
-  
+
   public static String getUrlFromZk(ClusterState clusterState, String collection) {
     Map<String,Slice> slices = clusterState.getCollection(collection).getSlicesMap();
 
@@ -1889,9 +1884,9 @@ public abstract class AbstractFullDistri
 
   protected List<Replica> ensureAllReplicasAreActive(String testCollectionName, String shardId, int shards, int rf, int maxWaitSecs) throws Exception {
     long startMs = System.currentTimeMillis();
-    
+
     Map<String,Replica> notLeaders = new HashMap<String,Replica>();
-    
+
     ZkStateReader zkr = cloudClient.getZkStateReader();
     zkr.updateClusterState(true); // force the state to be fresh
 
@@ -1906,7 +1901,7 @@ public abstract class AbstractFullDistri
       // refresh state every 2 secs
       if (waitMs % 2000 == 0)
         cloudClient.getZkStateReader().updateClusterState(true);
-      
+
       cs = cloudClient.getZkStateReader().getClusterState();
       assertNotNull(cs);
       Slice shard = cs.getSlice(testCollectionName, shardId);
@@ -1919,7 +1914,7 @@ public abstract class AbstractFullDistri
       assertNotNull(leader);
       log.info("Found "+replicas.size()+" replicas and leader on "+
         leader.getNodeName()+" for "+shardId+" in "+testCollectionName);
-      
+
       // ensure all replicas are "active" and identify the non-leader replica
       for (Replica replica : replicas) {
         String replicaState = replica.getStr(ZkStateReader.STATE_PROP);
@@ -1927,11 +1922,11 @@ public abstract class AbstractFullDistri
           log.info("Replica " + replica.getName() + " is currently " + replicaState);
           allReplicasUp = false;
         }
-        
-        if (!leader.equals(replica)) 
+
+        if (!leader.equals(replica))
           notLeaders.put(replica.getName(), replica);
       }
-      
+
       if (!allReplicasUp) {
         try {
           Thread.sleep(500L);
@@ -1939,22 +1934,22 @@ public abstract class AbstractFullDistri
         waitMs += 500L;
       }
     } // end while
-    
-    if (!allReplicasUp) 
+
+    if (!allReplicasUp)
       fail("Didn't see all replicas for shard "+shardId+" in "+testCollectionName+
           " come up within " + maxWaitMs + " ms! ClusterState: " + printClusterStateInfo());
-    
-    if (notLeaders.isEmpty()) 
+
+    if (notLeaders.isEmpty())
       fail("Didn't isolate any replicas that are not the leader! ClusterState: " + printClusterStateInfo());
-    
+
     long diffMs = (System.currentTimeMillis() - startMs);
     log.info("Took " + diffMs + " ms to see all replicas become active.");
-    
+
     List<Replica> replicas = new ArrayList<Replica>();
     replicas.addAll(notLeaders.values());
     return replicas;
-  }  
-  
+  }
+
   protected String printClusterStateInfo() throws Exception {
     return printClusterStateInfo(null);
   }

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java?rev=1654256&r1=1654255&r2=1654256&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java Fri Jan 23 16:13:32 2015
@@ -424,7 +424,7 @@ public class ZkTestServer {
 
   public void run() throws InterruptedException {
     log.info("STARTING ZK TEST SERVER");
-    // we don't call super.setUp
+    // we don't call super.distribSetUp
     zooThread = new Thread() {
       
       @Override