You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/10/29 18:26:21 UTC

svn commit: r1194922 - in /lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene: index/RandomCodec.java index/codecs/preflexrw/PreFlexRWCodec.java util/LuceneTestCase.java util/_TestUtil.java

Author: rmuir
Date: Sat Oct 29 16:26:21 2011
New Revision: 1194922

URL: http://svn.apache.org/viewvc?rev=1194922&view=rev
Log:
LUCENE-3490: try to fix LTC

Added:
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java   (with props)
Modified:
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java?rev=1194922&r1=1194921&r2=1194922&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java Sat Oct 29 16:26:21 2011
@@ -29,6 +29,10 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
+import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
+import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
+import org.apache.lucene.index.codecs.mockrandom.MockRandomPostingsFormat;
+import org.apache.lucene.index.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
 import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
 import org.apache.lucene.util._TestUtil;
@@ -44,12 +48,13 @@ import org.apache.lucene.util._TestUtil;
  */
 public class RandomCodec extends Lucene40Codec {
   /** name->postingsformat mappings */
-  private Map<String,PostingsFormat> codecNames = new HashMap<String,PostingsFormat>();
+  private Map<String,PostingsFormat> formatNames = new HashMap<String,PostingsFormat>();
   /** shuffled list of postingsformats to use for new mappings */
-  private List<PostingsFormat> knownCodecs = new ArrayList<PostingsFormat>();
+  private List<PostingsFormat> knownFormats = new ArrayList<PostingsFormat>();
   /** memorized field->postingsformat mappings */
   private Map<String,PostingsFormat> previousMappings = new HashMap<String,PostingsFormat>();
   private final int perFieldSeed;
+  private final String configuration;
   
   @Override
   public PostingsFormat getPostingsFormat(String formatName) {
@@ -58,19 +63,24 @@ public class RandomCodec extends Lucene4
 
   @Override
   public synchronized String getPostingsFormatForField(String name) {
-    PostingsFormat codec = previousMappings.get(name);
-    if (codec == null) {
-      codec = knownCodecs.get(Math.abs(perFieldSeed ^ name.hashCode()) % knownCodecs.size());
-      if (codec instanceof SimpleTextPostingsFormat && perFieldSeed % 5 != 0) {
-        // make simpletext rarer, choose again
-        codec = knownCodecs.get(Math.abs(perFieldSeed ^ name.toUpperCase(Locale.ENGLISH).hashCode()) % knownCodecs.size());
+    if ("random".equals(configuration)) {
+      PostingsFormat codec = previousMappings.get(name);
+      if (codec == null) {
+        codec = knownFormats.get(Math.abs(perFieldSeed ^ name.hashCode()) % knownFormats.size());
+        if (codec instanceof SimpleTextPostingsFormat && perFieldSeed % 5 != 0) {
+          // make simpletext rarer, choose again
+          codec = knownFormats.get(Math.abs(perFieldSeed ^ name.toUpperCase(Locale.ENGLISH).hashCode()) % knownFormats.size());
+        }
+        previousMappings.put(name, codec);
       }
-      previousMappings.put(name, codec);
+      return codec.name;
+    } else {
+      return configuration;
     }
-    return codec.name;
   }
 
-  public RandomCodec(Random random, boolean useNoMemoryExpensiveCodec) {
+  public RandomCodec(Random random, boolean useNoMemoryExpensiveCodec, String configuration) {
+    this.configuration = configuration;
     this.perFieldSeed = random.nextInt();
     // TODO: make it possible to specify min/max iterms per
     // block via CL:
@@ -82,20 +92,25 @@ public class RandomCodec extends Lucene4
     minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
     maxItemsPerBlock = 2*(Math.max(1, minItemsPerBlock-1)) + random.nextInt(100);
     register(new PulsingPostingsFormat( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock));
+    register(new MockSepPostingsFormat());
+    register(new MockFixedIntBlockPostingsFormat(_TestUtil.nextInt(random, 1, 2000)));
+    register(new MockVariableIntBlockPostingsFormat( _TestUtil.nextInt(random, 1, 127)));
+    register(new MockRandomPostingsFormat(random));
     if (!useNoMemoryExpensiveCodec) {
       register(new SimpleTextPostingsFormat());
       register(new MemoryPostingsFormat());
     }
-    Collections.shuffle(knownCodecs, random);
+    Collections.shuffle(knownFormats, random);
   }
   
-  public synchronized void register(PostingsFormat codec) {
-    codecNames.put(codec.name, codec);
+  public synchronized void register(PostingsFormat format) {
+    formatNames.put(format.name, format);
+    knownFormats.add(format);
   }
   
   // TODO: needed anymore? I don't think so
-  public synchronized void unregister(PostingsFormat codec) {
-    knownCodecs.remove(codec);
+  public synchronized void unregister(PostingsFormat format) {
+    knownFormats.remove(format);
   }
   
   @Override

Added: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java?rev=1194922&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWCodec.java Sat Oct 29 16:26:21 2011
@@ -0,0 +1,54 @@
+package org.apache.lucene.index.codecs.preflexrw;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DefaultFieldsFormat;
+import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldsFormat;
+import org.apache.lucene.index.codecs.PostingsFormat;
+
+/**
+ * Writes 3.x-like indexes (not perfect emulation yet) for testing only!
+ * @lucene.experimental
+ */
+public class PreFlexRWCodec extends Codec {
+  private final PostingsFormat postings = new PreFlexRWPostingsFormat();
+  // TODO: we should emulate 3.x here as well
+  private final FieldsFormat fields = new DefaultFieldsFormat();
+  
+  // TODO: really this should take a Version param, and emulate that version of lucene exactly...
+  public PreFlexRWCodec() {
+    super("Lucene3x"); // impersonate
+  }
+
+  @Override
+  public PostingsFormat postingsFormat() {
+    return postings;
+  }
+
+  @Override
+  public DocValuesFormat docValuesFormat() {
+    return null; // unsupported
+  }
+
+  @Override
+  public FieldsFormat fieldsFormat() {
+    return fields;
+  }
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1194922&r1=1194921&r2=1194922&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Sat Oct 29 16:26:21 2011
@@ -40,8 +40,10 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.*;
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CoreCodecProvider;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xCodec;
 import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
@@ -137,8 +139,10 @@ public abstract class LuceneTestCase ext
   // by default we randomly pick a different codec for
   // each test case (non-J4 tests) and each test class (J4
   // tests)
+  /** Gets the postingsFormat to run tests with. */
+  public static final String TEST_POSTINGSFORMAT = System.getProperty("tests.postingsFormat", "random");
   /** Gets the codec to run tests with. */
-  public static final String TEST_CODEC = System.getProperty("tests.codec", "randomPerField");
+  public static final String TEST_CODEC = System.getProperty("tests.codec", "random");
   /** Gets the codecprovider to run tests with */
   public static final String TEST_CODECPROVIDER = System.getProperty("tests.codecprovider", "random");
   /** Gets the locale to run tests with */
@@ -210,10 +214,6 @@ public abstract class LuceneTestCase ext
   }
   private List<UncaughtExceptionEntry> uncaughtExceptions = Collections.synchronizedList(new ArrayList<UncaughtExceptionEntry>());
 
-  // saves default codec: we do this statically as many build indexes in @beforeClass
-  private static String savedDefaultCodec;
-  // default codec: not set when we use a per-field provider.
-  private static PostingsFormat codec;
   // default codec provider
   private static CodecProvider savedCodecProvider;
   
@@ -226,84 +226,6 @@ public abstract class LuceneTestCase ext
 
   protected static Map<MockDirectoryWrapper,StackTraceElement[]> stores;
 
-  private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock", "MockRandom"};
-
-  // returns current default codec
-  static PostingsFormat installTestCodecs(String codec, CodecProvider cp) {
-    savedDefaultCodec = cp.getDefaultFieldCodec();
-
-    final boolean codecHasParam;
-    int codecParam = 0;
-    if (codec.equals("randomPerField")) {
-      // lie
-      codec = "Standard";
-      codecHasParam = false;
-    } else if (codec.equals("random")) {
-      codec = pickRandomCodec(random);
-      codecHasParam = false;
-    } else {
-      Matcher m = codecWithParam.matcher(codec);
-      if (m.matches()) {
-        // codec has a fixed param
-        codecHasParam = true;
-        codec = m.group(1);
-        codecParam = Integer.parseInt(m.group(2));
-      } else {
-        codecHasParam = false;
-      }
-    }
-
-    cp.setDefaultFieldCodec(codec);
-
-    if (codec.equals("PreFlex")) {
-      // If we're running w/ PreFlex codec we must swap in the
-      // test-only PreFlexRW codec (since core PreFlex can
-      // only read segments):
-      swapCodec(new PreFlexRWPostingsFormat(), cp);
-    }
-
-    swapCodec(new MockSepPostingsFormat(), cp);
-    // TODO: make it possible to specify min/max iterms per
-    // block via CL:
-    int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
-    int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
-    swapCodec(new PulsingPostingsFormat(codecHasParam && "Pulsing".equals(codec) ? codecParam : 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock), cp);
-    swapCodec(new MockFixedIntBlockPostingsFormat(codecHasParam && "MockFixedIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 2000)), cp);
-    // baseBlockSize cannot be over 127:
-    swapCodec(new MockVariableIntBlockPostingsFormat(codecHasParam && "MockVariableIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 127)), cp);
-    swapCodec(new MockRandomPostingsFormat(random), cp);
-
-    return cp.lookup(codec);
-  }
-  
-  // returns current PreFlex codec
-  static void removeTestCodecs(PostingsFormat codec, CodecProvider cp) {
-    if (codec.name.equals("PreFlex")) {
-      final PostingsFormat preFlex = cp.lookup("PreFlex");
-      if (preFlex != null) {
-        cp.unregister(preFlex);
-      }
-      cp.register(new Lucene3xPostingsFormat());
-    }
-    cp.unregister(cp.lookup("MockSep"));
-    cp.unregister(cp.lookup("MockFixedIntBlock"));
-    cp.unregister(cp.lookup("MockVariableIntBlock"));
-    cp.unregister(cp.lookup("MockRandom"));
-    swapCodec(new PulsingPostingsFormat(), cp);
-    cp.setDefaultFieldCodec(savedDefaultCodec);
-  }
-
-  // randomly picks from core and test codecs
-  static String pickRandomCodec(Random rnd) {
-    int idx = rnd.nextInt(CodecProvider.CORE_CODECS.length +
-                          TEST_CODECS.length);
-    if (idx < CodecProvider.CORE_CODECS.length) {
-      return CodecProvider.CORE_CODECS[idx];
-    } else {
-      return TEST_CODECS[idx - CodecProvider.CORE_CODECS.length];
-    }
-  }
-
   /** @deprecated (4.0) until we fix no-fork problems in solr tests */
   @Deprecated
   static List<String> testClassesRun = new ArrayList<String>();
@@ -333,39 +255,49 @@ public abstract class LuceneTestCase ext
     }
     
     savedCodecProvider = CodecProvider.getDefault();
+    final CodecProvider cp;
     if ("random".equals(TEST_CODECPROVIDER)) {
-      if ("randomPerField".equals(TEST_CODEC)) {
-        if (random.nextInt(4) == 0) { // preflex-only setup
-          codec = installTestCodecs("PreFlex", CodecProvider.getDefault());
-        } else { // per-field setup
-          CodecProvider.setDefault(new RandomCodec(random, useNoMemoryExpensiveCodec));
-          codec = installTestCodecs(TEST_CODEC, CodecProvider.getDefault());
-        }
-      } else { // ordinary setup
-        codec = installTestCodecs(TEST_CODEC, CodecProvider.getDefault());
+      if ("random".equals(TEST_POSTINGSFORMAT) && random.nextInt(4) == 0) { // preflex-only setup
+        cp = new CoreCodecProvider() {
+          final Codec preflexRW = new Lucene3xCodec();
+          @Override
+          public Codec getDefaultCodec() {
+            return preflexRW;
+          }
+
+          @Override
+          public Codec lookup(String name) {
+            if ("Lucene3x".equals(name)) // impersonation!
+              return preflexRW;
+            else
+              return super.lookup(name);
+          }
+        };
+      } else { // per-field setup, or specified postingsformat
+        final Codec randomPerField = new RandomCodec(random, useNoMemoryExpensiveCodec, TEST_POSTINGSFORMAT);
+        cp = new CoreCodecProvider() {
+
+          @Override
+          public Codec lookup(String name) {
+            if ("Lucene40".equals(name)) // impersonation!
+              return randomPerField;
+            else
+              return super.lookup(name);
+          }
+        };
       }
     } else {
       // someone specified their own codecprovider by class
       try {
         Class<? extends CodecProvider> cpClazz = Class.forName(TEST_CODECPROVIDER).asSubclass(CodecProvider.class);
-        CodecProvider cp = cpClazz.newInstance();
-        String codecName;
-        if (TEST_CODEC.startsWith("random")) { // TODO: somehow do random per-field?!
-          Set<String> codecSet = cp.listAll();
-          String availableCodecs[] = codecSet.toArray(new String[codecSet.size()]);
-          codecName = availableCodecs[random.nextInt(availableCodecs.length)];
-        } else {
-          codecName = TEST_CODEC;
-        }
-        
-        codec = cp.lookup(codecName);
-        cp.setDefaultFieldCodec(codecName);
-        CodecProvider.setDefault(cp);
+        cp = cpClazz.newInstance();
       } catch (Exception e) {
         System.err.println("Could not instantiate CodecProvider: " + TEST_CODECPROVIDER);
         throw new RuntimeException(e);
       }
     }
+
+    CodecProvider.setDefault(cp);
     
     savedLocale = Locale.getDefault();
     
@@ -415,7 +347,8 @@ public abstract class LuceneTestCase ext
       }
     }
     
-    String codecDescription = uninstallCodecsAfterClass();
+    String codecDescription = CodecProvider.getDefault().toString();
+    CodecProvider.setDefault(savedCodecProvider);
     Locale.setDefault(savedLocale);
     TimeZone.setDefault(savedTimeZone);
     System.clearProperty("solr.solr.home");
@@ -480,24 +413,6 @@ public abstract class LuceneTestCase ext
         + "total=" + Runtime.getRuntime().totalMemory());
   }
   
-  /** uninstalls test codecs, returns description of the codec used for debugging */
-  private static String uninstallCodecsAfterClass() {
-    String codecDescription;
-    CodecProvider cp = CodecProvider.getDefault();
-
-    if ("randomPerField".equals(TEST_CODEC) && cp instanceof RandomCodec) {
-      codecDescription = cp.toString();
-    } else {
-      codecDescription = codec.toString();
-    }
-
-    if ("random".equals(TEST_CODECPROVIDER) && CodecProvider.getDefault() == savedCodecProvider)
-      removeTestCodecs(codec, CodecProvider.getDefault());
-    CodecProvider.setDefault(savedCodecProvider);
-
-    return codecDescription;
-  }
-  
   /** check that directories and their resources were closed */
   private static void checkResourcesAfterClass() {
     for (MockDirectoryWrapper d : stores.keySet()) {
@@ -630,16 +545,13 @@ public abstract class LuceneTestCase ext
     }
     
     if (useNoMemoryExpensiveCodec) {
-      PostingsFormat p = CodecProvider.getDefault().getDefaultCodec().postingsFormat();
-      if (p instanceof PerFieldPostingsFormat) {
-        String defCodec = ((PerFieldPostingsFormat)p).getPostingsFormatForField("thisCodeMakesAbsolutelyNoSenseCanWeDeleteIt");
-        // Stupid: assumeFalse in setUp() does not print any information, because
-        // TestWatchman does not watch test during setUp() - getName() is also not defined...
-        // => print info directly and use assume without message:
-        if ("SimpleText".equals(defCodec) || "Memory".equals(defCodec)) {
-          System.err.println("NOTE: A test method in " + getClass().getSimpleName() + " was ignored, as it uses too much memory with " + defCodec + ".");
-          Assume.assumeTrue(false);
-        }
+      String defFormat = _TestUtil.getPostingsFormat("thisCodeMakesAbsolutelyNoSenseCanWeDeleteIt");
+      // Stupid: assumeFalse in setUp() does not print any information, because
+      // TestWatchman does not watch test during setUp() - getName() is also not defined...
+      // => print info directly and use assume without message:
+      if ("SimpleText".equals(defFormat) || "Memory".equals(defFormat)) {
+        System.err.println("NOTE: A test method in " + getClass().getSimpleName() + " was ignored, as it uses too much memory with " + defFormat + ".");
+        Assume.assumeTrue(false);
       }
     }
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1194922&r1=1194921&r2=1194922&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Sat Oct 29 16:26:21 2011
@@ -48,6 +48,7 @@ import org.apache.lucene.index.codecs.Co
 import org.apache.lucene.index.codecs.CoreCodecProvider;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
@@ -381,6 +382,15 @@ public class _TestUtil {
   public static CodecProvider alwaysCodec(final String codec) {
     return alwaysCodec(CodecProvider.getDefault().lookup(codec));
   }
+  
+  public static String getPostingsFormat(String field) {
+    PostingsFormat p = CodecProvider.getDefault().getDefaultCodec().postingsFormat();
+    if (p instanceof PerFieldPostingsFormat) {
+      return ((PerFieldPostingsFormat)p).getPostingsFormatForField(field);
+    } else {
+      return p.name;
+    }
+  }
 
   public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {
     String[] files = dir.listAll();