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 2013/07/30 18:57:37 UTC

svn commit: r1508521 - in /lucene/dev/trunk: lucene/test-framework/src/java/org/apache/lucene/util/ solr/ solr/core/src/test-files/solr/collection1/conf/ solr/core/src/test/org/apache/solr/core/ solr/core/src/test/org/apache/solr/util/ solr/test-framew...

Author: hossman
Date: Tue Jul 30 16:57:37 2013
New Revision: 1508521

URL: http://svn.apache.org/r1508521
Log:
SOLR-4951: Better randomization of MergePolicy in Solr tests

Added:
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/TestRandomMergePolicy.java   (with props)
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java   (with props)
Modified:
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1508521&r1=1508520&r2=1508521&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Tue Jul 30 16:57:37 2013
@@ -793,15 +793,8 @@ public abstract class LuceneTestCase ext
       }
     }
 
-    if (rarely(r)) {
-      c.setMergePolicy(new MockRandomMergePolicy(r));
-    } else if (r.nextBoolean()) {
-      c.setMergePolicy(newTieredMergePolicy());
-    } else if (r.nextInt(5) == 0) { 
-      c.setMergePolicy(newAlcoholicMergePolicy());
-    } else {
-      c.setMergePolicy(newLogMergePolicy());
-    }
+    c.setMergePolicy(newMergePolicy(r));
+
     if (rarely(r)) {
       c.setMergedSegmentWarmer(new SimpleMergedSegmentWarmer(c.getInfoStream()));
     }
@@ -810,6 +803,21 @@ public abstract class LuceneTestCase ext
     return c;
   }
 
+  public static MergePolicy newMergePolicy(Random r) {
+    if (rarely(r)) {
+      return new MockRandomMergePolicy(r);
+    } else if (r.nextBoolean()) {
+      return newTieredMergePolicy(r);
+    } else if (r.nextInt(5) == 0) { 
+      return newAlcoholicMergePolicy(r, classEnvRule.timeZone);
+    }
+    return newLogMergePolicy(r);
+  }
+
+  public static MergePolicy newMergePolicy() {
+    return newMergePolicy(random());
+  }
+
   public static LogMergePolicy newLogMergePolicy() {
     return newLogMergePolicy(random());
   }

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1508521&r1=1508520&r2=1508521&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Tue Jul 30 16:57:37 2013
@@ -100,6 +100,8 @@ Other Changes
 
 * SOLR-5056: Improve type safety of ConfigSolr class. (Alan Woodward)
 
+* SOLR-4951: Better randomization of MergePolicy in Solr tests (hossman)
+
 ==================  4.4.0 ==================
 
 Versions of Major Components

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml?rev=1508521&r1=1508520&r2=1508521&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml Tue Jul 30 16:57:37 2013
@@ -25,5 +25,6 @@
     <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
     <maxIndexingThreads>123</maxIndexingThreads>
     <infoStream>true</infoStream>
+    <mergePolicy class="org.apache.solr.util.RandomMergePolicy" />
   </indexConfig>
 </config>

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml?rev=1508521&r1=1508520&r2=1508521&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml Tue Jul 30 16:57:37 2013
@@ -23,4 +23,5 @@ A solrconfig.xml snippet containing inde
 -->
 <indexConfig>
   <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+  <mergePolicy class="org.apache.solr.util.RandomMergePolicy" />
 </indexConfig>

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java?rev=1508521&r1=1508520&r2=1508521&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestSolrIndexConfig.java Tue Jul 30 16:57:37 2013
@@ -17,8 +17,14 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LiveIndexWriterConfig;
+
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.RandomMergePolicy;
 import org.apache.solr.update.LoggingInfoStream;
 import org.junit.BeforeClass;
 
@@ -28,11 +34,31 @@ public class TestSolrIndexConfig extends
   public static void beforeClass() throws Exception {
     initCore("solrconfig-indexconfig.xml","schema.xml");
   }
+
+  public void testLiveWriter() throws Exception {
+    SolrCore core = h.getCore();
+    RefCounted<IndexWriter> iw = core.getUpdateHandler().getSolrCoreState().getIndexWriter(core);
+    try {
+      checkIndexWriterConfig(iw.get().getConfig());
+    } finally {
+      if (null != iw) iw.decref();
+    }
+  }
+
   
-  public void testIndexConfig() throws Exception {
+  public void testIndexConfigParsing() throws Exception {
     IndexWriterConfig iwc = solrConfig.indexConfig.toIndexWriterConfig(h.getCore().getLatestSchema());
 
+    checkIndexWriterConfig(iwc);
+  }
+
+  private void checkIndexWriterConfig(LiveIndexWriterConfig iwc) {
+
     assertEquals(123, iwc.getMaxThreadStates());
     assertTrue(iwc.getInfoStream() instanceof LoggingInfoStream);
+    assertTrue(iwc.getMergePolicy().getClass().toString(),
+               iwc.getMergePolicy() instanceof RandomMergePolicy);
+
   }
+
 }

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/TestRandomMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/TestRandomMergePolicy.java?rev=1508521&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/TestRandomMergePolicy.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/TestRandomMergePolicy.java Tue Jul 30 16:57:37 2013
@@ -0,0 +1,73 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.InvocationTargetException;
+
+/** 
+ * A "test the test" sanity check using reflection to ensure that 
+ * {@linke RandomMergePolicy} is working as expected
+ */
+public class TestRandomMergePolicy extends LuceneTestCase {  
+
+  /**
+   * Ensure every MP method is overridden by RMP 
+   * (future proof ourselves against new methods being added to MP)
+   */
+  public void testMethodOverride() {
+    Class rmp = RandomMergePolicy.class;
+    for (Method meth : rmp.getMethods()) {
+      if (// ignore things like hashCode, equals, etc...
+          meth.getDeclaringClass().equals(Object.class)
+          // can't do anything about it regardless of what class declares it
+          || Modifier.isFinal(meth.getModifiers())) {
+        continue;
+      }
+      assertEquals("method not overridden by RandomMergePolicy: " + 
+                   meth.toGenericString(), 
+                   rmp, meth.getDeclaringClass());
+    }
+  }
+
+  /**
+   * Ensure any "getter" methods return the same value as
+   * the wrapped MP
+   * (future proof ourselves against new final getter/setter pairs being 
+   * added to MP w/o dealing with them in the RMP Constructor)
+   */
+  public void testGetters() throws IllegalAccessException, InvocationTargetException {
+    final int iters = atLeast(20);
+    for (int i = 0; i < iters; i++) {
+      RandomMergePolicy rmp = new RandomMergePolicy();
+      Class mp = MergePolicy.class;
+      for (Method meth : mp.getDeclaredMethods()) {
+        if (meth.getName().startsWith("get") &&
+            (0 == meth.getParameterTypes().length)) {
+
+          assertEquals("MergePolicy getter gave diff results for RandomMergePolicy and the policy it wrapped: " + meth.toGenericString(),
+                       meth.invoke(rmp), meth.invoke(rmp.inner));
+        }
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java?rev=1508521&view=auto
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java (added)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java Tue Jul 30 16:57:37 2013
@@ -0,0 +1,91 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.index.*;
+import org.apache.lucene.index.MergePolicy.MergeSpecification;
+import org.apache.lucene.util.LuceneTestCase;
+
+import java.util.Map;
+import java.io.IOException;
+
+/**
+ * A {@link MergePolicy} with a no-arg constructor that proxies to a 
+ * wrapped instance retrieved from {@link LuceneTestCase#newMergePolicy}.
+ * Solr tests utilizing the Lucene randomized test framework can refer 
+ * to this class in solrconfig.xml to get a fully randomized merge policy.
+ */
+public final class RandomMergePolicy extends MergePolicy {
+
+  /** 
+   * Not private so tests can inspect it, 
+   * Not final so it can be set on clone
+   */
+  MergePolicy inner;
+
+  public RandomMergePolicy() {
+    this(LuceneTestCase.newMergePolicy());
+  }
+  private RandomMergePolicy(MergePolicy inner) {
+    super(inner.getNoCFSRatio(), 
+          (long) (inner.getMaxCFSSegmentSizeMB() * 1024 * 1024));
+    this.inner = inner;
+  }
+
+  public RandomMergePolicy clone() {
+    RandomMergePolicy clone = (RandomMergePolicy) super.clone();
+    clone.inner = this.inner.clone();
+    return clone;
+  }
+
+  public void close() {
+    inner.close();
+  }
+
+  public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos) 
+    throws IOException {
+
+    return inner.findForcedDeletesMerges(segmentInfos);
+  }
+  public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, 
+                                             int maxSegmentCount, 
+                                             Map<SegmentInfoPerCommit,Boolean> segmentsToMerge) 
+    throws IOException {
+    
+    return inner.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge);
+  }
+
+  public MergeSpecification findMerges(MergeTrigger mergeTrigger, 
+                                       SegmentInfos segmentInfos)
+    throws IOException {
+
+    return inner.findMerges(mergeTrigger, segmentInfos);
+  }
+
+  public void setIndexWriter(IndexWriter writer) {
+    inner.setIndexWriter(writer);
+  }
+
+  public boolean useCompoundFile(SegmentInfos infos,
+                                 SegmentInfoPerCommit mergedInfo)
+    throws IOException {
+    
+    return inner.useCompoundFile(infos, mergedInfo);
+  }
+
+}