You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2016/12/09 00:17:54 UTC

lucene-solr:jira/solr-5944: SOLR-5944: refactor randomized index building code into buildRandomIndex

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-5944 4aeae85fb -> b0059ff31


SOLR-5944: refactor randomized index building code into buildRandomIndex

NOTE: not currently used by docValuesUpdateTest because of other bugs/assumptions in that test


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

Branch: refs/heads/jira/solr-5944
Commit: b0059ff31a85087523b499078db4f14265df65cd
Parents: 4aeae85
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Dec 8 16:47:53 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Dec 8 16:47:53 2016 -0700

----------------------------------------------------------------------
 .../solr/update/TestInPlaceUpdatesDistrib.java  | 130 ++++++++++---------
 1 file changed, 66 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b0059ff3/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
index 0f5f876..8959587 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
@@ -32,6 +32,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
@@ -166,14 +167,11 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
 
   // The following should work: full update to doc 0, in-place update for doc 0, delete doc 0
   private void outOfOrderDBQsTest() throws Exception {
-    // nocommit: need to build randomized index with doc we care about mixed into the index randomly
-    // nocommit: see jira comments for in-depth suggestions
     
     del("*:*");
     commit();
-
-    index("id", 0, "title_s", "title0", "id_i", 0);
-    commit();
+    
+    buildRandomIndex(0);
 
     float inplace_updatable_float = 1;
 
@@ -241,9 +239,10 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     int numDocs = atLeast(100);
     log.info("Trying num docs = " + numDocs);
 
-    // nocommit: reuse randomized index building code refactored out of ensureRtgWorksWithPartialUpdatesTest ?
+    // nocommit: reuse buildRandomIndex here (ideally with randomized initFloat)
+    // nocommit: switching to it isn't trivial, because many places in test assume no docs in index other then ones being udpated
     //
-    // nocommit: doing that and changing rest of method to only use a subset of docs would help
+    // nocommit: doing that and changing rest of method to only look at specialIds (positive ids)
     // nocommit: to ensure we don't hit have any edge case bugs where this test only passes because
     // nocommit: all docs are updated (or all in a segment, etc...)
     for (int i = 0; i < numDocs; i++) {
@@ -264,7 +263,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
       valuesList.add(r.nextFloat()*5.0f);
     }
     log.info("inplace_updatable_float: "+valuesList);
-    // update doc, set
+    // update doc w/ set
     for (int i = numDocs - 1; i >= 0; i--) {
       index("id", i, "inplace_updatable_float", map("set", valuesList.get(i)));
     }
@@ -281,6 +280,8 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
 
       // nocommit: useless validation query since every doc starts off with a value when index is built
       // nocommit: see jira comment for more detailed suggestion on fix
+      //
+      // nocommit: inplace_updatable_float has a schema default="0" -- which means even if we switch to using buildRandomIndex, this check is useless
       numFound = clientToTest.query(
           params("q", "inplace_updatable_float:[* TO *]")).getResults().getNumFound();
       if (numFound != numDocs) {
@@ -376,26 +377,8 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     String title = "title100";
     long version = 0, currentVersion;
 
-    // nocommit: BEGIN: refactor this "build random index" code into a helper method for use by all test methods
-    // Adding random number of docs before adding the doc to be tested
-    int numDocsBefore = random().nextInt(1000);
-    for (int i=0; i<numDocsBefore; i++) {
-      index("id", 1000 + i, "title_s", "title" + (1000+i), "id_i", 1000 + i);
-    }
-    
-    // Document to be tested
-    currentVersion = addDocAndGetVersion("id", 100, "title_s", "title100", "id_i", 100);
+    currentVersion = buildRandomIndex(100).get(0);
     assertTrue(currentVersion > version);
-    version = currentVersion;
-    
-    // Adding random number of docs after adding the doc to be tested
-    int numDocsAfter = random().nextInt(1000);
-    for (int i=0; i<numDocsAfter; i++) {
-      index("id", 5000 + i, "title_s", "title" + (5000+i), "id_i", 5000 + i);
-    }
-    
-    LEADER.commit();
-    // nocommit: END: refactor this "build random index" code into a helper method for use by all test methods
     
     // get the internal docids of id=100 document from the three replicas
     List<Integer> docids = getInternalDocIds("100");
@@ -473,32 +456,16 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
   }
 
   private void outOfOrderUpdatesIndividualReplicaTest() throws Exception {
-    // nocommit: adding doc we're going to update before doing index randomization defeats the point
-    // nocommit: see jira comments for in-depth suggestions
     
     del("*:*");
     commit();
 
-    index("id", 0, "title_s", "title0", "id_i", 0);
-    commit();
+    buildRandomIndex(0);
 
     float inplace_updatable_float = 1;
-
-    // Adding random number of docs before adding the doc to be tested
-    int numDocsBefore = random().nextInt(1000);
-    for (int i=0; i<numDocsBefore; i++) {
-      index("id", 1000 + i, "title_s", "title" + (1000+i), "id_i", 1000 + i);
-    }
-
     // update doc, set
     index("id", 0, "inplace_updatable_float", map("set", inplace_updatable_float));
 
-    // Adding random number of docs after adding the doc to be tested
-    int numDocsAfter = random().nextInt(1000);
-    for (int i=0; i<numDocsAfter; i++) {
-      index("id", 5000 + i, "title_s", "title" + (5000+i), "id_i", 5000 + i);
-    }
-
     LEADER.commit();
     SolrDocument sdoc = LEADER.getById("0");  // RTG straight from the index
     assertEquals(inplace_updatable_float, sdoc.get("inplace_updatable_float"));
@@ -561,17 +528,13 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
   
   // The following should work: full update to doc 0, in-place update for doc 0, delete doc 0
   private void outOfOrderDeleteUpdatesIndividualReplicaTest() throws Exception {
-    // nocommit: need to build randomized index with doc we care about mixed into the index randomly
-    // nocommit: see jira comments for in-depth suggestions
     
     del("*:*");
     commit();
 
-    index("id", 0, "title_s", "title0", "id_i", 0);
-    commit();
+    buildRandomIndex(0);
 
     float inplace_updatable_float = 1;
-
     // update doc, set
     index("id", 0, "inplace_updatable_float", map("set", inplace_updatable_float));
 
@@ -639,16 +602,11 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
         DV(id=x, val=5, ver=3)
    */
   private void reorderedDBQsWithInPlaceUpdatesShouldNotThrowReplicaInLIRTest() throws Exception {
-    // nocommit: need to build randomized index with doc we care about mixed into the index randomly
-    // nocommit: see jira comments for in-depth suggestions
-    
     del("*:*");
     commit();
 
-    index("id", 0, "title_s", "title0", "id_i", 0);
-    commit();
+    buildRandomIndex(0);
 
-    LEADER.commit();
     SolrDocument sdoc = LEADER.getById("0");  // RTG straight from the index
     //assertEquals(value, sdoc.get("inplace_updatable_float"));
     assertEquals("title0", sdoc.get("title_s"));
@@ -737,18 +695,13 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
   }
   
   private void delayedReorderingFetchesMissingUpdateFromLeaderTest() throws Exception {
-    // nocommit: need to build randomized index with doc we care about mixed into the index randomly
-    // nocommit: see jira comments for in-depth suggestions
-    
     del("*:*");
     commit();
+    
+    float inplace_updatable_float = 1F;
+    buildRandomIndex(inplace_updatable_float, false, 1);
 
-    float inplace_updatable_float = 1;
-
-    index("id", 1, "title_s", "title1", "id_i", 1, "inplace_updatable_float", inplace_updatable_float);
-    commit();
-
-    float newinplace_updatable_float = 100;
+    float newinplace_updatable_float = 100F;
     List<UpdateRequest> updates = new ArrayList<>();
     updates.add(regularUpdateRequest("id", 1, "title_s", "title1_new", "id_i", 1, "inplace_updatable_float", newinplace_updatable_float));
     updates.add(regularUpdateRequest("id", 1, "inplace_updatable_float", map("inc", 1)));
@@ -995,4 +948,53 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
         + " worked around that problem here.", returnedVersion > 0);
     return returnedVersion;
   }
+
+  /**
+   * Convinience method variant that never uses <code>initFloat</code>
+   * @see #buildRandomIndex(Float,boolean,int[])
+   */
+  protected List<Long> buildRandomIndex(int... specialIds) throws Exception {
+    return buildRandomIndex(null, false, specialIds);
+  }
+                                        
+  /** 
+   * Helper method to build a randomized index with the fields needed for all test methods in this class.
+   * At a minimum, this index will contain 1 doc per "special" (non-negative) document id.  These special documents will be added with the <code>initFloat</code> specified in the "inplace_updatable_float" based on the <code>useFloatRandomly</code> param.
+   *
+   * A random number of documents (with negative ids) will be indexed in between each of the 
+   * "special" documents, as well as before/after the first/last special document.
+   *
+   * @param initFloat Value to use in the "inplace_updatable_float" for some of the special documents, based on the <code>useFloatRandomly</code> param; will never be used if null
+   * @param useFloatRandomly  If false, all special docs will get the <code>initFloat</code> value; if true, only a random subset of the special docs will get a value.
+   * @param specialIds The ids to use for the special documents, all values must be non-negative
+   * @return the versions of each of the specials document returned when indexing it
+   */
+  protected List<Long> buildRandomIndex(Float initFloat, boolean useFloatRandomly,
+                                        int... specialIds) throws Exception {
+    int id = -1; // used for non special docs
+    final int numPreDocs = rarely() ? TestUtil.nextInt(random(),0,9) : atLeast(10);
+    for (int i = 1; i <= numPreDocs; i++) {
+      addDocAndGetVersion("id", id, "title_s", "title" + id, "id_i", id);
+      id--;
+    }
+    final List<Long> versions = new ArrayList<>(specialIds.length);
+    for (int special : specialIds) {
+      if (null == initFloat || (useFloatRandomly && random().nextBoolean()) ) {
+        versions.add(addDocAndGetVersion("id", special, "title_s", "title" + special, "id_i", special));
+      } else {
+        versions.add(addDocAndGetVersion("id", special, "title_s", "title" + special, "id_i", special,
+                                         "inplace_updatable_float", initFloat));
+      }
+      final int numPostDocs = rarely() ? TestUtil.nextInt(random(),0,9) : atLeast(10);
+      for (int i = 1; i <= numPostDocs; i++) {
+        addDocAndGetVersion("id", id, "title_s", "title" + id, "id_i", id);
+        id--;
+      }
+    }
+    LEADER.commit();
+    
+    assert specialIds.length == versions.size();
+    return versions;
+  }
+  
 }