You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2016/04/21 23:49:22 UTC

[1/7] lucene-solr:branch_5x: SOLR-8599: After a failed connection during construction of SolrZkClient attempt to retry until a connection can be made

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_5x 0b7b28a04 -> 9e77319ab


SOLR-8599: After a failed connection during construction of SolrZkClient attempt to retry until a connection can be made


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d9875832
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d9875832
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d9875832

Branch: refs/heads/branch_5x
Commit: d9875832f4798e5f732f4ae5627c7b306ccafa9c
Parents: 0b7b28a
Author: Dennis Gove <dp...@gmail.com>
Authored: Wed Feb 10 15:02:18 2016 -0500
Committer: anshum <an...@apache.org>
Committed: Thu Apr 21 13:50:32 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +
 .../solr/cloud/ConnectionManagerTest.java       | 46 ++++++++++
 .../solr/common/cloud/ConnectionManager.java    | 96 +++++++++++++-------
 3 files changed, 110 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9875832/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3dd5b15..07efaa3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -95,6 +95,9 @@ Bug Fixes
   and other small LeaderElector improvements. (Scott Blum via Mark Miller, Alan
   Woodward)
 
+* SOLR-8599: After a failed connection during construction of SolrZkClient attempt to retry until a connection
+  can be made. (Keith Laban, Dennis Gove)
+
 ======================= 5.5.0 =======================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9875832/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
index 15baaca..c16d758 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
@@ -16,14 +16,20 @@
  */
 package org.apache.solr.cloud;
 
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.ConnectionManager;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.junit.Ignore;
+import org.junit.Test;
 
 @Slow
 public class ConnectionManagerTest extends SolrTestCaseJ4 {
@@ -108,4 +114,44 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
       server.shutdown();
     }
   }
+  
+  @Test
+  public void testReconnectWhenZkDisappeared() throws Exception {
+    ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new DefaultSolrThreadFactory("connectionManagerTest"));
+    
+    // setup a SolrZkClient to do some getBaseUrlForNodeName testing
+    String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+    try {
+      server.run();
+
+      AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
+      AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
+      
+      SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
+      ConnectionManager cm = zkClient.getConnectionManager();
+      try {
+        assertFalse(cm.isLikelyExpired());
+        assertTrue(cm.isConnected());
+        
+        
+        cm.setZkServerAddress("http://BADADDRESS");
+        executor.schedule(() -> {
+          cm.setZkServerAddress(server.getZkAddress()); 
+        }, 5, TimeUnit.SECONDS);
+        
+        // reconnect -- should no longer be likely expired
+        cm.process(new WatchedEvent(EventType.None, KeeperState.Expired, ""));
+        assertFalse(cm.isLikelyExpired());
+        assertTrue(cm.isConnected());
+      } finally {
+        cm.close();
+        zkClient.close();
+        executor.shutdown();
+      }
+    } finally {
+      server.shutdown();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9875832/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
index c3cf768..cc9327a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
@@ -37,7 +37,10 @@ public class ConnectionManager implements Watcher {
 
   private final ZkClientConnectionStrategy connectionStrategy;
 
-  private final String zkServerAddress;
+  //expert: mutable for testing
+  private String zkServerAddress;
+
+  
 
   private final SolrZkClient client;
 
@@ -128,40 +131,50 @@ public class ConnectionManager implements Watcher {
         }
       }
       
-      try {
-        connectionStrategy.reconnect(zkServerAddress,
-            client.getZkClientTimeout(), this,
-            new ZkClientConnectionStrategy.ZkUpdate() {
-              @Override
-              public void update(SolrZooKeeper keeper) {
-                try {
-                  waitForConnected(Long.MAX_VALUE);
-                } catch (Exception e1) {
-                  closeKeeper(keeper);
-                  throw new RuntimeException(e1);
-                }
-
-                log.info("Connection with ZooKeeper reestablished.");
-                try {
-                  client.updateKeeper(keeper);
-                } catch (InterruptedException e) {
-                  closeKeeper(keeper);
-                  Thread.currentThread().interrupt();
-                  // we must have been asked to stop
-                  throw new RuntimeException(e);
-                } catch (Exception t) {
-                  closeKeeper(keeper);
-                  throw new RuntimeException(t);
-                }
-
-                if (onReconnect != null) {
-                  onReconnect.command();
+      do {
+        // This loop will break iff a valid connection is made. If a connection is not made then it will repeat and
+        // try again to create a new connection.
+        try {
+          connectionStrategy.reconnect(zkServerAddress,
+              client.getZkClientTimeout(), this,
+              new ZkClientConnectionStrategy.ZkUpdate() {
+                @Override
+                public void update(SolrZooKeeper keeper) {
+                  try {
+                    waitForConnected(Long.MAX_VALUE);
+                  } catch (Exception e1) {
+                    closeKeeper(keeper);
+                    throw new RuntimeException(e1);
+                  }
+  
+                  log.info("Connection with ZooKeeper reestablished.");
+                  try {
+                    client.updateKeeper(keeper);
+                  } catch (InterruptedException e) {
+                    closeKeeper(keeper);
+                    Thread.currentThread().interrupt();
+                    // we must have been asked to stop
+                    throw new RuntimeException(e);
+                  } catch (Exception t) {
+                    closeKeeper(keeper);
+                    throw new RuntimeException(t);
+                  }
+  
+                  if (onReconnect != null) {
+                    onReconnect.command();
+                  }
                 }
-              }
-            });
-      } catch (Exception e) {
-        SolrException.log(log, "", e);
-      }
+              });
+          
+          break;
+          
+        } catch (Exception e) {
+          SolrException.log(log, "", e);
+          log.info("Could not connect due to error, sleeping for 5s and trying agian");
+          waitSleep(1000);
+        }
+        
+      } while (!isClosed);
       log.info("Connected:" + connected);
     } else if (state == KeeperState.Disconnected) {
       log.info("zkClient has disconnected");
@@ -186,6 +199,14 @@ public class ConnectionManager implements Watcher {
   public boolean isLikelyExpired() {
     return isClosed || likelyExpiredState.isLikelyExpired((long) (client.getZkClientTimeout() * 0.90));
   }
+  
+  public synchronized void waitSleep(long waitFor) {
+    try {
+      wait(waitFor);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    } 
+  }
 
   public synchronized void waitForConnected(long waitForConnection)
       throws TimeoutException {
@@ -234,4 +255,9 @@ public class ConnectionManager implements Watcher {
           "", e);
     }
   }
-}
\ No newline at end of file
+  
+  //expert: mutable for testing
+  public void setZkServerAddress(String zkServerAddress) {
+    this.zkServerAddress = zkServerAddress;
+  }
+}


[4/7] lucene-solr:branch_5x: SOLR-8748: OverseerTaskProcessor limits number of concurrent tasks to just 10 even though the thread pool size is 100. The limit has now been increased to 100.

Posted by an...@apache.org.
SOLR-8748: OverseerTaskProcessor limits number of concurrent tasks to just 10 even though the thread pool size is 100. The limit has now been increased to 100.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/95394918
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/95394918
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/95394918

Branch: refs/heads/branch_5x
Commit: 953949181992351cdad417d5fff05f1fcc5ee510
Parents: 2beccf4
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Sat Feb 27 14:05:07 2016 +0530
Committer: anshum <an...@apache.org>
Committed: Thu Apr 21 14:02:52 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                      |  3 +++
 .../org/apache/solr/cloud/OverseerTaskProcessor.java  | 14 +++++++++-----
 2 files changed, 12 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95394918/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0904b6f..37674e6 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -101,6 +101,9 @@ Bug Fixes
 * SOLR-8420: Fix long overflow in sumOfSquares for Date statistics. (Tom Hill, Christine Poerschke,
   Tomás Fernández Löbbe)
 
+* SOLR-8748: OverseerTaskProcessor limits number of concurrent tasks to just 10 even though the thread pool
+  size is 100. The limit has now been increased to 100. (Scott Blum, shalin)
+
 ======================= 5.5.0 =======================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95394918/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
index 1bda80c..26a90cb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
@@ -58,9 +58,13 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
  */
 public class OverseerTaskProcessor implements Runnable, Closeable {
 
-  public int maxParallelThreads = 10;
+  /**
+   * Maximum number of overseer collection operations which can be
+   * executed concurrently
+   */
+  public static final int MAX_PARALLEL_TASKS = 100;
 
-  public ExecutorService tpe ;
+  public ExecutorService tpe;
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
@@ -162,7 +166,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
     // TODO: Make maxThreads configurable.
 
-    this.tpe = new ExecutorUtil.MDCAwareThreadPoolExecutor(5, 100, 0L, TimeUnit.MILLISECONDS,
+    this.tpe = new ExecutorUtil.MDCAwareThreadPoolExecutor(5, MAX_PARALLEL_TASKS, 0L, TimeUnit.MILLISECONDS,
         new SynchronousQueue<Runnable>(),
         new DefaultSolrThreadFactory("OverseerThreadFactory"));
     try {
@@ -183,7 +187,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
           boolean waited = false;
 
-          while (runningTasks.size() > maxParallelThreads) {
+          while (runningTasks.size() > MAX_PARALLEL_TASKS) {
             synchronized (waitLock) {
               waitLock.wait(100);//wait for 100 ms or till a task is complete
             }
@@ -193,7 +197,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
           if (waited)
             cleanUpWorkQueue();
 
-          List<QueueEvent> heads = workQueue.peekTopN(maxParallelThreads, runningZKTasks, 2000L);
+          List<QueueEvent> heads = workQueue.peekTopN(MAX_PARALLEL_TASKS, runningZKTasks, 2000L);
 
           if (heads == null)
             continue;


[7/7] lucene-solr:branch_5x: SOLR-8738: Fixed false success response when invalid deleteByQuery requests intially hit non-leader cloud nodes

Posted by an...@apache.org.
SOLR-8738: Fixed false success response when invalid deleteByQuery requests intially hit non-leader cloud nodes


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/9e77319a
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/9e77319a
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/9e77319a

Branch: refs/heads/branch_5x
Commit: 9e77319abcf3ff372b86cec4d66bac11f7e038b6
Parents: 27e284b
Author: Chris Hostetter <ho...@apache.org>
Authored: Tue Mar 1 10:02:07 2016 -0700
Committer: anshum <an...@apache.org>
Committed: Thu Apr 21 14:25:42 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../processor/DistributedUpdateProcessor.java   |   2 +-
 .../solr/cloud/TestCloudDeleteByQuery.java      | 245 +++++++++++++++++++
 3 files changed, 248 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9e77319a/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6f9d428..89a3bc5 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -106,6 +106,8 @@ Bug Fixes
 
 * SOLR-8375: ReplicaAssigner rejects valid nodes (Kelvin Tan, noble)
 
+* SOLR-8738: Fixed false success response when invalid deleteByQuery requests intially hit non-leader
+  cloud nodes (hossman)
 
 Other Changes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9e77319a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 0b936b2..4af8277 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -1282,7 +1282,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
           // don't forward to ourself
           leaderForAnyShard = true;
         } else {
-          leaders.add(new StdNode(coreLeaderProps, collection, sliceName));
+          leaders.add(new RetryNode(coreLeaderProps, zkController.getZkStateReader(), collection, sliceName));
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9e77319a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
new file mode 100644
index 0000000..a0bb42a
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud;
+
+import java.io.File;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestCloudDeleteByQuery extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final int NUM_SHARDS = 2; 
+  private static final int REPLICATION_FACTOR = 2; 
+  private static final int NUM_SERVERS = 5; 
+  
+  private static final String COLLECTION_NAME = "test_col";
+  
+  /** A basic client for operations at the cloud level, default collection will be set */
+  private static CloudSolrClient CLOUD_CLIENT;
+
+  /** A client for talking directly to the leader of shard1 */
+  private static HttpSolrClient S_ONE_LEADER_CLIENT;
+  
+  /** A client for talking directly to the leader of shard2 */
+  private static HttpSolrClient S_TWO_LEADER_CLIENT;
+
+  /** A client for talking directly to a passive replica of shard1 */
+  private static HttpSolrClient S_ONE_NON_LEADER_CLIENT;
+  
+  /** A client for talking directly to a passive replica of shard2 */
+  private static HttpSolrClient S_TWO_NON_LEADER_CLIENT;
+
+  /** A client for talking directly to a node that has no piece of the collection */
+  private static HttpSolrClient NO_COLLECTION_CLIENT;
+  
+  /** id field doc routing prefix for shard1 */
+  private static final String S_ONE_PRE = "abc!";
+  
+  /** id field doc routing prefix for shard2 */
+  private static final String S_TWO_PRE = "XYZ!";
+  
+  @BeforeClass
+  private static void createMiniSolrCloudCluster() throws Exception {
+    
+    final String configName = "solrCloudCollectionConfig";
+    File configDir = new File(TEST_HOME() + File.separator + "collection1" + File.separator + "conf");
+    
+    configureCluster(NUM_SERVERS)
+      .addConfig(configName, configDir.toPath())
+      .configure();
+    
+    Map<String, String> collectionProperties = new HashMap<>();
+    collectionProperties.put("config", "solrconfig-tlog.xml");
+    collectionProperties.put("schema", "schema15.xml"); // string id for doc routing prefix
+
+    assertNotNull(cluster.createCollection(COLLECTION_NAME, NUM_SHARDS, REPLICATION_FACTOR,
+                                           configName, null, null, collectionProperties));
+    
+    CLOUD_CLIENT = cluster.getSolrClient();
+    CLOUD_CLIENT.setDefaultCollection(COLLECTION_NAME);
+    
+    ZkStateReader zkStateReader = CLOUD_CLIENT.getZkStateReader();
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION_NAME, zkStateReader, true, true, 330);
+
+
+    // really hackish way to get a URL for specific nodes based on shard/replica hosting
+    // inspired by TestMiniSolrCloudCluster
+    HashMap<String, String> urlMap = new HashMap<>();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      URL jettyURL = jetty.getBaseUrl();
+      String nodeKey = jettyURL.getHost() + ":" + jettyURL.getPort() + jettyURL.getPath().replace("/","_");
+      urlMap.put(nodeKey, jettyURL.toString());
+    }
+    zkStateReader.updateClusterState();
+    ClusterState clusterState = zkStateReader.getClusterState();
+    for (Slice slice : clusterState.getSlices(COLLECTION_NAME)) {
+      String shardName = slice.getName();
+      Replica leader = slice.getLeader();
+      assertNotNull("slice has null leader: " + slice.toString(), leader);
+      assertNotNull("slice leader has null node name: " + slice.toString(), leader.getNodeName());
+      String leaderUrl = urlMap.remove(leader.getNodeName());
+      assertNotNull("could not find URL for " + shardName + " leader: " + leader.getNodeName(),
+                    leaderUrl);
+      assertEquals("expected two total replicas for: " + slice.getName(),
+                   2, slice.getReplicas().size());
+      
+      String passiveUrl = null;
+      
+      for (Replica replica : slice.getReplicas()) {
+        if ( ! replica.equals(leader)) {
+          passiveUrl = urlMap.remove(replica.getNodeName());
+          assertNotNull("could not find URL for " + shardName + " replica: " + replica.getNodeName(),
+                        passiveUrl);
+        }
+      }
+      assertNotNull("could not find URL for " + shardName + " replica", passiveUrl);
+
+      if (shardName.equals("shard1")) {
+        S_ONE_LEADER_CLIENT = new HttpSolrClient(leaderUrl + "/" + COLLECTION_NAME + "/");
+        S_ONE_NON_LEADER_CLIENT = new HttpSolrClient(passiveUrl + "/" + COLLECTION_NAME + "/");
+      } else if (shardName.equals("shard2")) {
+        S_TWO_LEADER_CLIENT = new HttpSolrClient(leaderUrl + "/" + COLLECTION_NAME + "/");
+        S_TWO_NON_LEADER_CLIENT = new HttpSolrClient(passiveUrl + "/" + COLLECTION_NAME + "/");
+      } else {
+        fail("unexpected shard: " + shardName);
+      }
+    }
+    assertEquals("Should be exactly one server left (nost hosting either shard)", 1, urlMap.size());
+    NO_COLLECTION_CLIENT = new HttpSolrClient(urlMap.values().iterator().next() +
+                                              "/" + COLLECTION_NAME + "/");
+    
+    assertNotNull(S_ONE_LEADER_CLIENT);
+    assertNotNull(S_TWO_LEADER_CLIENT);
+    assertNotNull(S_ONE_NON_LEADER_CLIENT);
+    assertNotNull(S_TWO_NON_LEADER_CLIENT);
+    assertNotNull(NO_COLLECTION_CLIENT);
+
+    // sanity check that our S_ONE_PRE & S_TWO_PRE really do map to shard1 & shard2 with default routing
+    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_ONE_PRE + random().nextInt()),
+                                         f("expected_shard_s", "shard1"))).getStatus());
+    assertEquals(0, CLOUD_CLIENT.add(doc(f("id", S_TWO_PRE + random().nextInt()),
+                                         f("expected_shard_s", "shard2"))).getStatus());
+    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+    SolrDocumentList docs = CLOUD_CLIENT.query(params("q", "*:*",
+                                                      "fl","id,expected_shard_s,[shard]")).getResults();
+    assertEquals(2, docs.getNumFound());
+    assertEquals(2, docs.size());
+    for (SolrDocument doc : docs) {
+      String expected = COLLECTION_NAME + "_" + doc.getFirstValue("expected_shard_s") + "_replica";
+      String docShard = doc.getFirstValue("[shard]").toString();
+      assertTrue("shard routing prefixes don't seem to be aligned anymore, " +
+                 "did someone change the default routing rules? " +
+                 "and/or the the default core name rules? " +
+                 "and/or the numShards used by this test? ... " +
+                 "couldn't find " + expected + " as substring of [shard] == '" + docShard +
+                 "' ... for docId == " + doc.getFirstValue("id"),
+                 docShard.contains(expected));
+    }
+  }
+  
+  @Before
+  private void clearCloudCollection() throws Exception {
+    assertEquals(0, CLOUD_CLIENT.deleteByQuery("*:*").getStatus());
+    assertEquals(0, CLOUD_CLIENT.commit().getStatus());
+  }
+
+  public void testMalformedDBQ(SolrClient client) throws Exception {
+    assertNotNull("client not initialized", client);
+    try {
+      UpdateResponse rsp = update(params()).deleteByQuery("foo_i:not_a_num").process(client);
+      fail("Expected DBQ failure: " + rsp.toString());
+    } catch (SolrException e) {
+      assertEquals("not the expected DBQ failure: " + e.getMessage(), 400, e.code());
+    }
+  }
+
+  //
+  public void testMalformedDBQViaCloudClient() throws Exception {
+    testMalformedDBQ(CLOUD_CLIENT);
+  }
+  public void testMalformedDBQViaShard1LeaderClient() throws Exception {
+    testMalformedDBQ(S_ONE_LEADER_CLIENT);
+  }
+  public void testMalformedDBQViaShard2LeaderClient() throws Exception {
+    testMalformedDBQ(S_TWO_LEADER_CLIENT);
+  }
+  public void testMalformedDBQViaShard1NonLeaderClient() throws Exception {
+    testMalformedDBQ(S_ONE_NON_LEADER_CLIENT);
+  }
+  public void testMalformedDBQViaShard2NonLeaderClient() throws Exception {
+    testMalformedDBQ(S_TWO_NON_LEADER_CLIENT);
+  }
+  public void testMalformedDBQViaNoCollectionClient() throws Exception {
+    testMalformedDBQ(NO_COLLECTION_CLIENT);
+  }
+
+  public static UpdateRequest update(SolrParams params, SolrInputDocument... docs) {
+    UpdateRequest r = new UpdateRequest();
+    r.setParams(new ModifiableSolrParams(params));
+    r.add(Arrays.asList(docs));
+    return r;
+  }
+  
+  public static SolrInputDocument doc(SolrInputField... fields) {
+    SolrInputDocument doc = new SolrInputDocument();
+    for (SolrInputField f : fields) {
+      doc.put(f.getName(), f);
+    }
+    return doc;
+  }
+  
+  public static SolrInputField f(String fieldName, Object... values) {
+    SolrInputField f = new SolrInputField(fieldName);
+    f.setValue(values, 1.0F);
+    return f;
+  }
+}


[3/7] lucene-solr:branch_5x: SOLR-8420: Fix long overflow in sumOfSquares for Date statistics

Posted by an...@apache.org.
SOLR-8420: Fix long overflow in sumOfSquares for Date statistics

Casted operations to double. Changed the test to support a percentage error given the FUZZY flag in doubles


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2beccf46
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2beccf46
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2beccf46

Branch: refs/heads/branch_5x
Commit: 2beccf469f9e07eb5a05fef9ec3f869d6da4008a
Parents: 20e2cab
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Wed Feb 24 19:02:17 2016 -0800
Committer: anshum <an...@apache.org>
Committed: Thu Apr 21 14:00:11 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../handler/component/StatsValuesFactory.java   |  4 +-
 .../org/apache/solr/TestDistributedSearch.java  | 11 ++++-
 .../handler/component/StatsComponentTest.java   | 42 +++++++++++++++++++-
 .../solr/BaseDistributedSearchTestCase.java     | 32 +++++++++++++++
 5 files changed, 88 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2beccf46/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 07efaa3..0904b6f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -98,6 +98,9 @@ Bug Fixes
 * SOLR-8599: After a failed connection during construction of SolrZkClient attempt to retry until a connection
   can be made. (Keith Laban, Dennis Gove)
 
+* SOLR-8420: Fix long overflow in sumOfSquares for Date statistics. (Tom Hill, Christine Poerschke,
+  Tomás Fernández Löbbe)
+
 ======================= 5.5.0 =======================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2beccf46/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
index 6565b6e..5003709 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
@@ -751,7 +751,7 @@ class DateStatsValues extends AbstractStatsValues<Date> {
   public void updateTypeSpecificStats(Date v, int count) {
     long value = v.getTime();
     if (computeSumOfSquares) {
-      sumOfSquares += (value * value * count); // for std deviation
+      sumOfSquares += ((double)value * value * count); // for std deviation
     }
     if (computeSum) {
       sum += value * count;
@@ -807,7 +807,7 @@ class DateStatsValues extends AbstractStatsValues<Date> {
     if (count <= 1) {
       return 0.0D;
     }
-    return Math.sqrt(((count * sumOfSquares) - (sum * sum))
+    return Math.sqrt(((count * sumOfSquares) - (sum * (double)sum))
         / (count * (count - 1.0D)));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2beccf46/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/TestDistributedSearch.java b/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
index b6750c2..4b83801 100644
--- a/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
@@ -425,8 +425,13 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
     
     query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", "stats_dt");
     query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1);
+
+    handle.put("stddev", FUZZY);
+    handle.put("sumOfSquares", FUZZY);
     query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", tdate_a);
     query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", tdate_b);
+    handle.remove("stddev");
+    handle.remove("sumOfSquares");
 
 
     rsp = query("q", "*:*", "sort", i1 + " desc", "stats", "true", 
@@ -522,6 +527,8 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
           "stats.field", "{!key=special_key}stats_dt",
           "stats.field", "{!ex=xxx}stats_dt");
 
+    handle.put("stddev", FUZZY);
+    handle.put("sumOfSquares", FUZZY);
     query("q","*:*", "sort",i1+" desc", "stats", "true",
           // do a really simple query so distributed IDF doesn't cause problems
           // when comparing with control collection
@@ -838,7 +845,7 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
             
             paras.append("}").append(field);
             numTotalStatQueries++;
-            rsp = query("q","*:*", "rows", "0", "stats", "true",
+            rsp = query("q", q, "rows", "0", "stats", "true",
                         "stats.field", paras.toString());
             // simple assert, mostly relying on comparison with single shard
             FieldStatsInfo s = rsp.getFieldStatsInfo().get("k");
@@ -850,6 +857,8 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
         }
       }
     }
+    handle.remove("stddev");
+    handle.remove("sumOfSquares");
     assertEquals("Sanity check failed: either test broke, or test changed, or you adjusted Stat enum" + 
                  " (adjust constant accordingly if intentional)",
                  5082, numTotalStatQueries);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2beccf46/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
index 33dd13b..db6d137 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
@@ -54,7 +54,6 @@ import org.apache.solr.util.AbstractSolrTestCase;
 
 import org.apache.commons.math3.util.Combinations;
 import com.tdunning.math.stats.AVLTreeDigest;
-import com.google.common.hash.Hashing; 
 import com.google.common.hash.HashFunction;
 import org.apache.solr.util.hll.HLL;
 
@@ -465,6 +464,47 @@ public class StatsComponentTest extends AbstractSolrTestCase {
 
   }
 
+  // Check for overflow of sumOfSquares
+  public void testFieldStatisticsResultsDateFieldOverflow() throws Exception {
+    SolrCore core = h.getCore();
+
+    assertU(adoc("id", "1", "active_dt", "2015-12-14T09:00:00Z"));
+    assertU(commit());
+
+    Map<String, String> args = new HashMap<>();
+    args.put(CommonParams.Q, "*:*");
+    args.put(StatsParams.STATS, "true");
+    args.put(StatsParams.STATS_FIELD, "active_dt");
+    args.put("indent", "true");
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new MapSolrParams(args));
+
+    assertQ("test date statistics values", req,
+            "//long[@name='count'][.='1']",
+            "//date[@name='min'][.='2015-12-14T09:00:00Z']",
+            "//date[@name='max'][.='2015-12-14T09:00:00Z']",
+            "//date[@name='sum'][.='2015-12-14T09:00:00Z']",
+            "//date[@name='mean'][.='2015-12-14T09:00:00Z']",
+            "//double[@name='sumOfSquares'][.='" + Double.toString(2102742446988960000000000.0)+"']"
+            );
+
+    assertU(adoc("id", "2", "active_dt", "2115-12-14T09:00:00Z"));
+    assertU(adoc("id", "3", "active_dt", "2215-12-14T09:00:00Z"));
+    assertU(commit());
+
+    assertQ("test date statistics values", req,
+        "//long[@name='count'][.='3']",
+        "//date[@name='min'][.='2015-12-14T09:00:00Z']",
+        "//date[@name='max'][.='2215-12-14T09:00:00Z']",
+        "//date[@name='sum'][.='2407-11-08T03:00:00Z']",
+        "//date[@name='mean'][.='2115-12-14T09:00:00Z']",
+        "//double[@name='sumOfSquares'][.='" + Double.toString(83555549895529430000000000.0)+"']",
+        // The following number matches the number returned by the current solr
+        // implementation of standard deviation. Should be 3155673600000.
+        // That number is not precise, and the implementation should be fixed.
+        "//double[@name='stddev'][.='" + Double.toString(3155673599999.999)+"']"
+        );
+  }
+
 
   public void doTestFieldStatisticsMissingResult(String f, SolrParams[] baseParamsSet) throws Exception {
     assertU(adoc("id", "1", f, "-10"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2beccf46/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index ad006a4..812663e 100644
--- a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -240,6 +240,13 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
   public static int SKIP = 2;
   public static int SKIPVAL = 4;
   public static int UNORDERED = 8;
+  
+  /**
+   * When this flag is set, Double values will be allowed a difference ratio of 1E-8
+   * between the non-distributed and the distributed returned values
+   */
+  public static int FUZZY = 16;
+  private static final double DOUBLE_RATIO_LIMIT = 1E-8;
 
   protected int flags;
   protected Map<String, Integer> handle = new HashMap<>();
@@ -876,6 +883,31 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
 
     }
 
+    if ((flags & FUZZY) != 0) {
+      if ((a instanceof Double && b instanceof Double)) {
+        double aaa = ((Double) a).doubleValue();
+        double bbb = ((Double) b).doubleValue();
+        if (aaa == bbb || ((Double) a).isNaN() && ((Double) b).isNaN()) {
+          return null;
+        }
+        if ((aaa == 0.0) || (bbb == 0.0)) {
+            return ":" + a + "!=" + b;
+        }
+
+        double diff = Math.abs(aaa - bbb);
+        // When stats computations are done on multiple shards, there may
+        // be small differences in the results. Allow a small difference
+        // between the result of the computations.
+
+        double ratio = Math.max(Math.abs(diff / aaa), Math.abs(diff / bbb));
+        if (ratio > DOUBLE_RATIO_LIMIT) {
+          return ":" + a + "!=" + b;
+        } else {
+          return null;// close enough.
+        }
+      }
+    }
+
     if (!(a.equals(b))) {
       return ":" + a + "!=" + b;
     }


[6/7] lucene-solr:branch_5x: SOLR-8758: Add SolrCloudTestCase base class

Posted by an...@apache.org.
SOLR-8758: Add SolrCloudTestCase base class


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/27e284bb
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/27e284bb
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/27e284bb

Branch: refs/heads/branch_5x
Commit: 27e284bb7bcd6536b3c017d76e675f24397cce9c
Parents: 3c6ef10
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Feb 29 18:16:36 2016 +0000
Committer: anshum <an...@apache.org>
Committed: Thu Apr 21 14:24:25 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   7 +
 .../apache/solr/cloud/SolrCloudTestCase.java    | 163 +++++++++++++++++++
 2 files changed, 170 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/27e284bb/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 07c81fc..6f9d428 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -106,6 +106,13 @@ Bug Fixes
 
 * SOLR-8375: ReplicaAssigner rejects valid nodes (Kelvin Tan, noble)
 
+
+Other Changes
+----------------------
+
+* SOLR-8758: Add a new SolrCloudTestCase class, using MiniSolrCloudCluster (Alan
+  Woodward)
+
 ======================= 5.5.0 =======================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/27e284bb/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
new file mode 100644
index 0000000..615365e
--- /dev/null
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.embedded.JettyConfig;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.junit.AfterClass;
+import org.junit.Before;
+
+/**
+ * Base class for SolrCloud tests
+ *
+ * Derived tests should call {@link #configureCluster(int)} in a {@code BeforeClass}
+ * static method.  This configures and starts a {@link MiniSolrCloudCluster}, available
+ * via the {@code cluster} variable.  Cluster shutdown is handled automatically.
+ *
+ * <pre>
+ *   <code>
+ *   {@literal @}BeforeClass
+ *   public static void setupCluster() {
+ *     configureCluster(NUM_NODES)
+ *        .addConfig("configname", pathToConfig)
+ *        .configure();
+ *   }
+ *   </code>
+ * </pre>
+ */
+public class SolrCloudTestCase extends SolrTestCaseJ4 {
+
+  private static class Config {
+    final String name;
+    final Path path;
+
+    private Config(String name, Path path) {
+      this.name = name;
+      this.path = path;
+    }
+  }
+
+  /**
+   * Builder class for a MiniSolrCloudCluster
+   */
+  protected static class Builder {
+
+    private final int nodeCount;
+    private final Path baseDir;
+    private String solrxml = MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML;
+    private JettyConfig jettyConfig = buildJettyConfig("/solr");
+
+    private List<Config> configs = new ArrayList<>();
+
+    /**
+     * Create a builder
+     * @param nodeCount the number of nodes in the cluster
+     * @param baseDir   a base directory for the cluster
+     */
+    public Builder(int nodeCount, Path baseDir) {
+      this.nodeCount = nodeCount;
+      this.baseDir = baseDir;
+    }
+
+    /**
+     * Use a {@link JettyConfig} to configure the cluster's jetty servers
+     */
+    public Builder withJettyConfig(JettyConfig jettyConfig) {
+      this.jettyConfig = jettyConfig;
+      return this;
+    }
+
+    /**
+     * Use the provided string as solr.xml content
+     */
+    public Builder withSolrXml(String solrXml) {
+      this.solrxml = solrXml;
+      return this;
+    }
+
+    /**
+     * Read solr.xml from the provided path
+     */
+    public Builder withSolrXml(Path solrXml) {
+      try {
+        this.solrxml = new String(Files.readAllBytes(solrXml), Charset.defaultCharset());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      return this;
+    }
+
+    /**
+     * Upload a collection config before tests start
+     * @param configName the config name
+     * @param configPath the path to the config files
+     */
+    public Builder addConfig(String configName, Path configPath) {
+      this.configs.add(new Config(configName, configPath));
+      return this;
+    }
+
+    /**
+     * Configure and run the {@link MiniSolrCloudCluster}
+     * @throws Exception if an error occurs on startup
+     */
+    public void configure() throws Exception {
+      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml, jettyConfig);
+      CloudSolrClient client = cluster.getSolrClient();
+      for (Config config : configs) {
+        client.uploadConfig(config.path, config.name);
+      }
+    }
+
+  }
+
+  /** The cluster */
+  protected static MiniSolrCloudCluster cluster;
+
+  /**
+   * Call this to configure a cluster of n nodes.
+   *
+   * NB you must call {@link Builder#configure()} to start the cluster
+   *
+   * @param nodeCount the number of nodes
+   */
+  protected static Builder configureCluster(int nodeCount) {
+    return new Builder(nodeCount, createTempDir());
+  }
+
+  @AfterClass
+  public static void shutdownCluster() throws Exception {
+    if (cluster != null)
+      cluster.shutdown();
+  }
+
+  @Before
+  public void checkClusterConfiguration() {
+    if (cluster == null)
+      throw new RuntimeException("MiniSolrCloudCluster not configured - have you called configureCluster().configure()?");
+  }
+
+}


[5/7] lucene-solr:branch_5x: SOLR-8375: ReplicaAssigner rejects valid nodes

Posted by an...@apache.org.
SOLR-8375: ReplicaAssigner rejects valid nodes


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/3c6ef10e
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/3c6ef10e
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/3c6ef10e

Branch: refs/heads/branch_5x
Commit: 3c6ef10e9e3455fc5027a2f45b889dd1c025d055
Parents: 9539491
Author: Noble Paul <no...@apache.org>
Authored: Tue Mar 1 10:49:47 2016 +0530
Committer: anshum <an...@apache.org>
Committed: Thu Apr 21 14:07:18 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                            | 2 ++
 .../java/org/apache/solr/cloud/rule/ReplicaAssigner.java    | 9 +++------
 2 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c6ef10e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 37674e6..07c81fc 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -104,6 +104,8 @@ Bug Fixes
 * SOLR-8748: OverseerTaskProcessor limits number of concurrent tasks to just 10 even though the thread pool
   size is 100. The limit has now been increased to 100. (Scott Blum, shalin)
 
+* SOLR-8375: ReplicaAssigner rejects valid nodes (Kelvin Tan, noble)
+
 ======================= 5.5.0 =======================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3c6ef10e/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
index 7dfeaac..57922a1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
@@ -44,11 +44,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.singletonList;
-import static org.apache.solr.cloud.rule.Rule.MatchStatus.NODE_CAN_BE_ASSIGNED;
-import static org.apache.solr.cloud.rule.Rule.Phase.ASSIGN;
-import static org.apache.solr.cloud.rule.Rule.Phase.FUZZY_ASSIGN;
-import static org.apache.solr.cloud.rule.Rule.Phase.FUZZY_VERIFY;
-import static org.apache.solr.cloud.rule.Rule.Phase.VERIFY;
+import static org.apache.solr.cloud.rule.Rule.MatchStatus.*;
+import static org.apache.solr.cloud.rule.Rule.Phase.*;
 import static org.apache.solr.common.util.StrUtils.formatString;
 import static org.apache.solr.common.util.Utils.getDeepCopy;
 
@@ -282,7 +279,7 @@ public class ReplicaAssigner {
         Rule rule = rules.get(rulePermutation[i]);
         Rule.MatchStatus matchStatus = rule.tryAssignNodeToShard(e.getValue(),
             copyOfCurrentState, nodeVsTagsCopy, e.getKey().shard, fuzzyPhase ? FUZZY_VERIFY : VERIFY);
-        if (matchStatus != NODE_CAN_BE_ASSIGNED) return null;
+        if (matchStatus != NODE_CAN_BE_ASSIGNED && matchStatus != NOT_APPLICABLE) return null;
       }
     }
     return result;


[2/7] lucene-solr:branch_5x: SOLR-8599: Improved the tests for this issue to avoid changing a variable to non-final

Posted by an...@apache.org.
SOLR-8599: Improved the tests for this issue to avoid changing a variable to non-final


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/20e2caba
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/20e2caba
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/20e2caba

Branch: refs/heads/branch_5x
Commit: 20e2caba9615e19f84fbcc59a950fb197385592e
Parents: d987583
Author: Dennis Gove <dp...@gmail.com>
Authored: Sun Mar 20 11:13:56 2016 -0400
Committer: anshum <an...@apache.org>
Committed: Thu Apr 21 13:50:51 2016 -0700

----------------------------------------------------------------------
 .../solr/cloud/ConnectionManagerTest.java       | 40 ++++++++++++++++----
 .../solr/common/cloud/ConnectionManager.java    |  8 +---
 2 files changed, 33 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/20e2caba/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
index c16d758..7eec2c0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.cloud;
 
+import java.io.IOException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -23,9 +24,14 @@ import java.util.concurrent.TimeUnit;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.cloud.ConnectionManager;
+import org.apache.solr.common.cloud.DefaultConnectionStrategy;
+import org.apache.solr.common.cloud.OnReconnect;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkClientConnectionStrategy;
+import org.apache.solr.common.cloud.ZkClientConnectionStrategy.ZkUpdate;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.junit.Ignore;
@@ -129,22 +135,19 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
       AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
       AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
       
-      SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
+      MockZkClientConnectionStrategy strat = new MockZkClientConnectionStrategy();
+      SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT, strat , null);
       ConnectionManager cm = zkClient.getConnectionManager();
+      
       try {
         assertFalse(cm.isLikelyExpired());
         assertTrue(cm.isConnected());
-        
-        
-        cm.setZkServerAddress("http://BADADDRESS");
-        executor.schedule(() -> {
-          cm.setZkServerAddress(server.getZkAddress()); 
-        }, 5, TimeUnit.SECONDS);
-        
+               
         // reconnect -- should no longer be likely expired
         cm.process(new WatchedEvent(EventType.None, KeeperState.Expired, ""));
         assertFalse(cm.isLikelyExpired());
         assertTrue(cm.isConnected());
+        assertTrue(strat.isExceptionThrow());
       } finally {
         cm.close();
         zkClient.close();
@@ -154,4 +157,25 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 {
       server.shutdown();
     }
   }
+  
+  private class MockZkClientConnectionStrategy extends DefaultConnectionStrategy {
+    int called = 0;
+    boolean exceptionThrown = false;
+    
+    @Override
+    public void reconnect(final String serverAddress, final int zkClientTimeout,
+        final Watcher watcher, final ZkUpdate updater) throws IOException {
+      
+      if(called++ < 1) {
+        exceptionThrown = true;
+        throw new IOException("Testing");
+      }
+      
+      super.reconnect(serverAddress, zkClientTimeout, watcher, updater);
+    }
+    
+    public boolean isExceptionThrow() {
+      return exceptionThrown;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/20e2caba/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
index cc9327a..1754a2d 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
@@ -37,8 +37,7 @@ public class ConnectionManager implements Watcher {
 
   private final ZkClientConnectionStrategy connectionStrategy;
 
-  //expert: mutable for testing
-  private String zkServerAddress;
+  private final String zkServerAddress;
 
   
 
@@ -255,9 +254,4 @@ public class ConnectionManager implements Watcher {
           "", e);
     }
   }
-  
-  //expert: mutable for testing
-  public void setZkServerAddress(String zkServerAddress) {
-    this.zkServerAddress = zkServerAddress;
-  }
 }