You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2019/03/07 12:18:45 UTC

[lucene-solr] branch master updated: SOLR-13302: Adding HttpPartitionTest and ForceLeaderTest for TLOG replicas

This is an automated email from the ASF dual-hosted git repository.

datcm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 002a4aa  SOLR-13302: Adding HttpPartitionTest and ForceLeaderTest for TLOG replicas
002a4aa is described below

commit 002a4aac3ea412583030db67004d48c0cd42f90e
Author: Cao Manh Dat <da...@apache.org>
AuthorDate: Thu Mar 7 12:18:41 2019 +0000

    SOLR-13302: Adding HttpPartitionTest and ForceLeaderTest for TLOG replicas
---
 .../org/apache/solr/cloud/ForceLeaderTest.java     | 30 ++++++++++++--------
 .../cloud/ForceLeaderWithTlogReplicasTest.java     | 26 ++++++++++++++++++
 .../org/apache/solr/cloud/HttpPartitionTest.java   | 13 ++++-----
 .../cloud/HttpPartitionWithTlogReplicasTest.java   | 32 ++++++++++++++++++++++
 4 files changed, 82 insertions(+), 19 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
index 26164ee..7509e62 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
@@ -20,7 +20,9 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.carrotsearch.randomizedtesting.annotations.Nightly;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.SocketProxy;
@@ -33,19 +35,17 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.util.TimeOut;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.carrotsearch.randomizedtesting.annotations.Nightly;
-
 @Nightly // this test is currently too slow for non nightly
 public class ForceLeaderTest extends HttpPartitionTest {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  // TODO: SOLR-12313 tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
-  private final boolean onlyLeaderIndexes = random().nextBoolean() && false; // consume same amount of random
 
   @BeforeClass
   public static void beforeClassSetup() {
@@ -55,11 +55,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
     System.setProperty("solr.retries.on.forward", "0");
     System.setProperty("solr.retries.to.followers", "0"); 
   }
-  
-  @Override
-  protected boolean useTlogReplicas() {
-    return onlyLeaderIndexes;
-  }
 
   @Test
   @Override
@@ -73,7 +68,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
    */
   @Test
   @Slow
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
   public void testReplicasInLowerTerms() throws Exception {
     handle.put("maxScore", SKIPVAL);
     handle.put("timestamp", SKIPVAL);
@@ -150,11 +144,25 @@ public class ForceLeaderTest extends HttpPartitionTest {
       assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, 1);
       assertDocsExistInAllReplicas(notLeaders, testCollectionName, 4, 4);
 
+      if (useTlogReplicas()) {
+
+      }
       // Docs 1 and 4 should be here. 2 was lost during the partition, 3 had failed to be indexed.
       log.info("Checking doc counts...");
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.add("q", "*:*");
-      assertEquals("Expected only 2 documents in the index", 2, cloudClient.query(params).getResults().getNumFound());
+      if (useTlogReplicas()) {
+        TimeOut timeOut = new TimeOut(15, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+        timeOut.waitFor("Expected only 2 documents in the index", () -> {
+          try {
+            return 2 == cloudClient.query(params).getResults().getNumFound();
+          } catch (Exception e) {
+            return false;
+          }
+        });
+      } else {
+        assertEquals("Expected only 2 documents in the index", 2, cloudClient.query(params).getResults().getNumFound());
+      }
 
       bringBackOldLeaderAndSendDoc(testCollectionName, leader, notLeaders, 5);
     } finally {
diff --git a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderWithTlogReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderWithTlogReplicasTest.java
new file mode 100644
index 0000000..fb32b01
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderWithTlogReplicasTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+public class ForceLeaderWithTlogReplicasTest extends ForceLeaderTest {
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return true;
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index c7c0e34..7a11262 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -84,8 +84,6 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
   // give plenty of time for replicas to recover when running in slow Jenkins test envs
   protected static final int maxWaitSecsToSeeAllActive = 90;
 
-  private final boolean onlyLeaderIndexes = random().nextBoolean();
-
   @BeforeClass
   public static void setupSysProps() {
     System.setProperty("socketTimeout", "10000");
@@ -101,11 +99,6 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     fixShardCount(3);
   }
 
-  @Override
-  protected boolean useTlogReplicas() {
-    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
-  }
-
   /**
    * We need to turn off directUpdatesToLeadersOnly due to SOLR-9512
    */
@@ -489,8 +482,12 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
     List<Replica> replicas = new ArrayList<Replica>();
     replicas.addAll(activeReplicas.values());
     return replicas;
-  }  
+  }
 
+  /**
+   * Assert docs exists in {@code notLeaders} replicas, docs must also exist in the shard1 leader as well.
+   * This method uses RTG for validation therefore it must work for asserting both TLOG and NRT replicas.
+   */
   protected void assertDocsExistInAllReplicas(List<Replica> notLeaders,
       String testCollectionName, int firstDocId, int lastDocId)
       throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionWithTlogReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionWithTlogReplicasTest.java
new file mode 100644
index 0000000..7e101a3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionWithTlogReplicasTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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 org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+
+@LuceneTestCase.Slow
+@SolrTestCaseJ4.SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
+public class HttpPartitionWithTlogReplicasTest extends HttpPartitionTest {
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return true;
+  }
+
+}
\ No newline at end of file