You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2013/05/30 09:53:46 UTC

svn commit: r1487777 [33/50] - in /lucene/dev/branches/security: ./ dev-tools/ dev-tools/eclipse/dot.settings/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/replicator/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/ma...

Modified: lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java Thu May 30 07:53:18 2013
@@ -43,6 +43,50 @@ public class MockIndexOutputWrapper exte
     this.delegate = delegate;
   }
 
+  private void checkCrashed() throws IOException {
+    // If MockRAMDir crashed since we were opened, then don't write anything
+    if (dir.crashed) {
+      throw new IOException("MockRAMDirectory was crashed; cannot write to " + name);
+    }
+  }
+  
+  private void checkDiskFull(byte[] b, int offset, DataInput in, long len) throws IOException {
+    long freeSpace = dir.maxSize == 0 ? 0 : dir.maxSize - dir.sizeInBytes();
+    long realUsage = 0;
+
+    // Enforce disk full:
+    if (dir.maxSize != 0 && freeSpace <= len) {
+      // Compute the real disk free.  This will greatly slow
+      // down our test but makes it more accurate:
+      realUsage = dir.getRecomputedActualSizeInBytes();
+      freeSpace = dir.maxSize - realUsage;
+    }
+
+    if (dir.maxSize != 0 && freeSpace <= len) {
+      if (freeSpace > 0) {
+        realUsage += freeSpace;
+        if (b != null) {
+          delegate.writeBytes(b, offset, (int) freeSpace);
+        } else {
+          delegate.copyBytes(in, len);
+        }
+      }
+      if (realUsage > dir.maxUsedSize) {
+        dir.maxUsedSize = realUsage;
+      }
+      String message = "fake disk full at " + dir.getRecomputedActualSizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.length();
+      if (freeSpace > 0) {
+        message += "; wrote " + freeSpace + " of " + len + " bytes";
+      }
+      message += ")";
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println(Thread.currentThread().getName() + ": MDW: now throw fake disk full");
+        new Throwable().printStackTrace(System.out);
+      }
+      throw new IOException(message);
+    }
+  }
+  
   @Override
   public void close() throws IOException {
     try {
@@ -75,48 +119,16 @@ public class MockIndexOutputWrapper exte
   
   @Override
   public void writeBytes(byte[] b, int offset, int len) throws IOException {
-    long freeSpace = dir.maxSize == 0 ? 0 : dir.maxSize - dir.sizeInBytes();
-    long realUsage = 0;
-    // If MockRAMDir crashed since we were opened, then
-    // don't write anything:
-    if (dir.crashed)
-      throw new IOException("MockRAMDirectory was crashed; cannot write to " + name);
-
-    // Enforce disk full:
-    if (dir.maxSize != 0 && freeSpace <= len) {
-      // Compute the real disk free.  This will greatly slow
-      // down our test but makes it more accurate:
-      realUsage = dir.getRecomputedActualSizeInBytes();
-      freeSpace = dir.maxSize - realUsage;
-    }
-
-    if (dir.maxSize != 0 && freeSpace <= len) {
-      if (freeSpace > 0) {
-        realUsage += freeSpace;
-        delegate.writeBytes(b, offset, (int) freeSpace);
-      }
-      if (realUsage > dir.maxUsedSize) {
-        dir.maxUsedSize = realUsage;
-      }
-      String message = "fake disk full at " + dir.getRecomputedActualSizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.length();
-      if (freeSpace > 0) {
-        message += "; wrote " + freeSpace + " of " + len + " bytes";
-      }
-      message += ")";
-      if (LuceneTestCase.VERBOSE) {
-        System.out.println(Thread.currentThread().getName() + ": MDW: now throw fake disk full");
-        new Throwable().printStackTrace(System.out);
-      }
-      throw new IOException(message);
+    checkCrashed();
+    checkDiskFull(b, offset, null, len);
+    
+    if (dir.randomState.nextInt(200) == 0) {
+      final int half = len/2;
+      delegate.writeBytes(b, offset, half);
+      Thread.yield();
+      delegate.writeBytes(b, offset+half, len-half);
     } else {
-      if (dir.randomState.nextInt(200) == 0) {
-        final int half = len/2;
-        delegate.writeBytes(b, offset, half);
-        Thread.yield();
-        delegate.writeBytes(b, offset+half, len-half);
-      } else {
-        delegate.writeBytes(b, offset, len);
-      }
+      delegate.writeBytes(b, offset, len);
     }
 
     dir.maybeThrowDeterministicException();
@@ -146,8 +158,10 @@ public class MockIndexOutputWrapper exte
 
   @Override
   public void copyBytes(DataInput input, long numBytes) throws IOException {
+    checkCrashed();
+    checkDiskFull(null, 0, input, numBytes);
+    
     delegate.copyBytes(input, numBytes);
-    // TODO: we may need to check disk full here as well
     dir.maybeThrowDeterministicException();
   }
 

Modified: lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Thu May 30 07:53:18 2013
@@ -23,6 +23,7 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.util.*;
 import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.logging.Logger;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -34,6 +35,7 @@ import org.apache.lucene.document.String
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.*;
 import org.apache.lucene.index.IndexReader.ReaderClosedListener;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.FieldCache.CacheEntry;
 import org.apache.lucene.search.QueryUtils.FCInvisibleMultiReader;
@@ -41,6 +43,9 @@ import org.apache.lucene.store.*;
 import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
 import org.apache.lucene.util.FieldCacheSanityChecker.Insanity;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RegExp;
 import org.junit.*;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
@@ -144,10 +149,10 @@ public abstract class LuceneTestCase ext
   public static final String SYSPROP_BADAPPLES = "tests.badapples";
 
   /** @see #ignoreAfterMaxFailures*/
-  private static final String SYSPROP_MAXFAILURES = "tests.maxfailures";
+  public static final String SYSPROP_MAXFAILURES = "tests.maxfailures";
 
   /** @see #ignoreAfterMaxFailures*/
-  private static final String SYSPROP_FAILFAST = "tests.failfast";
+  public static final String SYSPROP_FAILFAST = "tests.failfast";
 
   /**
    * Annotation for tests that should only be run during nightly builds.
@@ -352,9 +357,17 @@ public abstract class LuceneTestCase ext
       new TestRuleMarkFailure();
 
   /**
-   * Ignore tests after hitting a designated number of initial failures.
+   * Ignore tests after hitting a designated number of initial failures. This
+   * is truly a "static" global singleton since it needs to span the lifetime of all
+   * test classes running inside this JVM (it cannot be part of a class rule).
+   * 
+   * <p>This poses some problems for the test framework's tests because these sometimes
+   * trigger intentional failures which add up to the global count. This field contains
+   * a (possibly) changing reference to {@link TestRuleIgnoreAfterMaxFailures} and we
+   * dispatch to its current value from the {@link #classRules} chain using {@link TestRuleDelegate}.  
    */
-  final static TestRuleIgnoreAfterMaxFailures ignoreAfterMaxFailures; 
+  private static final AtomicReference<TestRuleIgnoreAfterMaxFailures> ignoreAfterMaxFailuresDelegate;
+  private static final TestRule ignoreAfterMaxFailures;
   static {
     int maxFailures = systemPropertyAsInt(SYSPROP_MAXFAILURES, Integer.MAX_VALUE);
     boolean failFast = systemPropertyAsBoolean(SYSPROP_FAILFAST, false);
@@ -369,7 +382,19 @@ public abstract class LuceneTestCase ext
       }
     }
 
-    ignoreAfterMaxFailures = new TestRuleIgnoreAfterMaxFailures(maxFailures);
+    ignoreAfterMaxFailuresDelegate = 
+        new AtomicReference<TestRuleIgnoreAfterMaxFailures>(
+            new TestRuleIgnoreAfterMaxFailures(maxFailures));
+    ignoreAfterMaxFailures = TestRuleDelegate.of(ignoreAfterMaxFailuresDelegate);
+  }
+
+  /**
+   * Temporarily substitute the global {@link TestRuleIgnoreAfterMaxFailures}. See
+   * {@link #ignoreAfterMaxFailuresDelegate} for some explanation why this method 
+   * is needed.
+   */
+  public static TestRuleIgnoreAfterMaxFailures replaceMaxFailureRule(TestRuleIgnoreAfterMaxFailures newValue) {
+    return ignoreAfterMaxFailuresDelegate.getAndSet(newValue);
   }
 
   /**
@@ -621,7 +646,7 @@ public abstract class LuceneTestCase ext
    * is active and {@link #RANDOM_MULTIPLIER}.
    */
   public static boolean rarely(Random random) {
-    int p = TEST_NIGHTLY ? 10 : 5;
+    int p = TEST_NIGHTLY ? 10 : 1;
     p += (p * Math.log(RANDOM_MULTIPLIER));
     int min = 100 - Math.min(p, 50); // never more than 50
     return random.nextInt(100) >= min;
@@ -655,6 +680,7 @@ public abstract class LuceneTestCase ext
    * Return <code>args</code> as a {@link Set} instance. The order of elements is not
    * preserved in iterators.
    */
+  @SafeVarargs @SuppressWarnings("varargs")
   public static <T> Set<T> asSet(T... args) {
     return new HashSet<T>(Arrays.asList(args));
   }
@@ -699,6 +725,16 @@ public abstract class LuceneTestCase ext
   public static IndexWriterConfig newIndexWriterConfig(Random r, Version v, Analyzer a) {
     IndexWriterConfig c = new IndexWriterConfig(v, a);
     c.setSimilarity(classEnvRule.similarity);
+    if (VERBOSE) {
+      // Even though TestRuleSetupAndRestoreClassEnv calls
+      // InfoStream.setDefault, we do it again here so that
+      // the PrintStreamInfoStream.messageID increments so
+      // that when there are separate instances of
+      // IndexWriter created we see "IW 0", "IW 1", "IW 2",
+      // ... instead of just always "IW 0":
+      c.setInfoStream(new TestRuleSetupAndRestoreClassEnv.ThreadNameFixingPrintStreamInfoStream(System.out));
+    }
+
     if (r.nextBoolean()) {
       c.setMergeScheduler(new SerialMergeScheduler());
     }
@@ -769,6 +805,9 @@ public abstract class LuceneTestCase ext
     } else {
       c.setMergePolicy(newLogMergePolicy());
     }
+    if (rarely(r)) {
+      c.setMergedSegmentWarmer(new SimpleMergedSegmentWarmer(c.getInfoStream()));
+    }
     c.setReaderPooling(r.nextBoolean());
     c.setReaderTermsIndexDivisor(_TestUtil.nextInt(r, 1, 4));
     return c;
@@ -1232,7 +1271,7 @@ public abstract class LuceneTestCase ext
    * Create a new searcher over the reader. This searcher might randomly use
    * threads.
    */
-  public static IndexSearcher newSearcher(IndexReader r) throws IOException {
+  public static IndexSearcher newSearcher(IndexReader r) {
     return newSearcher(r, true);
   }
   
@@ -1241,16 +1280,26 @@ public abstract class LuceneTestCase ext
    * threads. if <code>maybeWrap</code> is true, this searcher might wrap the
    * reader with one that returns null for getSequentialSubReaders.
    */
-  public static IndexSearcher newSearcher(IndexReader r, boolean maybeWrap) throws IOException {
+  public static IndexSearcher newSearcher(IndexReader r, boolean maybeWrap) {
     Random random = random();
     if (usually()) {
       if (maybeWrap) {
-        r = maybeWrapReader(r);
+        try {
+          r = maybeWrapReader(r);
+        } catch (IOException e) {
+          throw new AssertionError(e);
+        }
       }
-      if (rarely() && r instanceof AtomicReader) {
+      // TODO: this whole check is a coverage hack, we should move it to tests for various filterreaders.
+      // ultimately whatever you do will be checkIndex'd at the end anyway. 
+      if (random.nextInt(500) == 0 && r instanceof AtomicReader) {
         // TODO: not useful to check DirectoryReader (redundant with checkindex)
         // but maybe sometimes run this on the other crazy readers maybeWrapReader creates?
-        _TestUtil.checkReader(r);
+        try {
+          _TestUtil.checkReader(r);
+        } catch (IOException e) {
+          throw new AssertionError(e);
+        }
       }
       IndexSearcher ret = random.nextBoolean() ? new AssertingIndexSearcher(random, r) : new AssertingIndexSearcher(random, r.getContext());
       ret.setSimilarity(classEnvRule.similarity);
@@ -1308,4 +1357,648 @@ public abstract class LuceneTestCase ext
     }
     return true;
   }
+
+  public void assertReaderEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    assertReaderStatisticsEquals(info, leftReader, rightReader);
+    assertFieldsEquals(info, leftReader, MultiFields.getFields(leftReader), MultiFields.getFields(rightReader), true);
+    assertNormsEquals(info, leftReader, rightReader);
+    assertStoredFieldsEquals(info, leftReader, rightReader);
+    assertTermVectorsEquals(info, leftReader, rightReader);
+    assertDocValuesEquals(info, leftReader, rightReader);
+    assertDeletedDocsEquals(info, leftReader, rightReader);
+    assertFieldInfosEquals(info, leftReader, rightReader);
+  }
+
+  /** 
+   * checks that reader-level statistics are the same 
+   */
+  public void assertReaderStatisticsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    // Somewhat redundant: we never delete docs
+    assertEquals(info, leftReader.maxDoc(), rightReader.maxDoc());
+    assertEquals(info, leftReader.numDocs(), rightReader.numDocs());
+    assertEquals(info, leftReader.numDeletedDocs(), rightReader.numDeletedDocs());
+    assertEquals(info, leftReader.hasDeletions(), rightReader.hasDeletions());
+  }
+
+  /** 
+   * Fields api equivalency 
+   */
+  public void assertFieldsEquals(String info, IndexReader leftReader, Fields leftFields, Fields rightFields, boolean deep) throws IOException {
+    // Fields could be null if there are no postings,
+    // but then it must be null for both
+    if (leftFields == null || rightFields == null) {
+      assertNull(info, leftFields);
+      assertNull(info, rightFields);
+      return;
+    }
+    assertFieldStatisticsEquals(info, leftFields, rightFields);
+    
+    Iterator<String> leftEnum = leftFields.iterator();
+    Iterator<String> rightEnum = rightFields.iterator();
+    
+    while (leftEnum.hasNext()) {
+      String field = leftEnum.next();
+      assertEquals(info, field, rightEnum.next());
+      assertTermsEquals(info, leftReader, leftFields.terms(field), rightFields.terms(field), deep);
+    }
+    assertFalse(rightEnum.hasNext());
+  }
+
+  /** 
+   * checks that top-level statistics on Fields are the same 
+   */
+  public void assertFieldStatisticsEquals(String info, Fields leftFields, Fields rightFields) throws IOException {
+    if (leftFields.size() != -1 && rightFields.size() != -1) {
+      assertEquals(info, leftFields.size(), rightFields.size());
+    }
+  }
+
+  /** 
+   * Terms api equivalency 
+   */
+  public void assertTermsEquals(String info, IndexReader leftReader, Terms leftTerms, Terms rightTerms, boolean deep) throws IOException {
+    if (leftTerms == null || rightTerms == null) {
+      assertNull(info, leftTerms);
+      assertNull(info, rightTerms);
+      return;
+    }
+    assertTermsStatisticsEquals(info, leftTerms, rightTerms);
+    assertEquals(leftTerms.hasOffsets(), rightTerms.hasOffsets());
+    assertEquals(leftTerms.hasPositions(), rightTerms.hasPositions());
+    assertEquals(leftTerms.hasPayloads(), rightTerms.hasPayloads());
+
+    TermsEnum leftTermsEnum = leftTerms.iterator(null);
+    TermsEnum rightTermsEnum = rightTerms.iterator(null);
+    assertTermsEnumEquals(info, leftReader, leftTermsEnum, rightTermsEnum, true);
+    
+    assertTermsSeekingEquals(info, leftTerms, rightTerms);
+    
+    if (deep) {
+      int numIntersections = atLeast(3);
+      for (int i = 0; i < numIntersections; i++) {
+        String re = AutomatonTestUtil.randomRegexp(random());
+        CompiledAutomaton automaton = new CompiledAutomaton(new RegExp(re, RegExp.NONE).toAutomaton());
+        if (automaton.type == CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+          // TODO: test start term too
+          TermsEnum leftIntersection = leftTerms.intersect(automaton, null);
+          TermsEnum rightIntersection = rightTerms.intersect(automaton, null);
+          assertTermsEnumEquals(info, leftReader, leftIntersection, rightIntersection, rarely());
+        }
+      }
+    }
+  }
+
+  /** 
+   * checks collection-level statistics on Terms 
+   */
+  public void assertTermsStatisticsEquals(String info, Terms leftTerms, Terms rightTerms) throws IOException {
+    assert leftTerms.getComparator() == rightTerms.getComparator();
+    if (leftTerms.getDocCount() != -1 && rightTerms.getDocCount() != -1) {
+      assertEquals(info, leftTerms.getDocCount(), rightTerms.getDocCount());
+    }
+    if (leftTerms.getSumDocFreq() != -1 && rightTerms.getSumDocFreq() != -1) {
+      assertEquals(info, leftTerms.getSumDocFreq(), rightTerms.getSumDocFreq());
+    }
+    if (leftTerms.getSumTotalTermFreq() != -1 && rightTerms.getSumTotalTermFreq() != -1) {
+      assertEquals(info, leftTerms.getSumTotalTermFreq(), rightTerms.getSumTotalTermFreq());
+    }
+    if (leftTerms.size() != -1 && rightTerms.size() != -1) {
+      assertEquals(info, leftTerms.size(), rightTerms.size());
+    }
+  }
+
+  private static class RandomBits implements Bits {
+    FixedBitSet bits;
+    
+    RandomBits(int maxDoc, double pctLive, Random random) {
+      bits = new FixedBitSet(maxDoc);
+      for (int i = 0; i < maxDoc; i++) {
+        if (random.nextDouble() <= pctLive) {        
+          bits.set(i);
+        }
+      }
+    }
+    
+    @Override
+    public boolean get(int index) {
+      return bits.get(index);
+    }
+
+    @Override
+    public int length() {
+      return bits.length();
+    }
+  }
+
+  /** 
+   * checks the terms enum sequentially
+   * if deep is false, it does a 'shallow' test that doesnt go down to the docsenums
+   */
+  public void assertTermsEnumEquals(String info, IndexReader leftReader, TermsEnum leftTermsEnum, TermsEnum rightTermsEnum, boolean deep) throws IOException {
+    BytesRef term;
+    Bits randomBits = new RandomBits(leftReader.maxDoc(), random().nextDouble(), random());
+    DocsAndPositionsEnum leftPositions = null;
+    DocsAndPositionsEnum rightPositions = null;
+    DocsEnum leftDocs = null;
+    DocsEnum rightDocs = null;
+    
+    while ((term = leftTermsEnum.next()) != null) {
+      assertEquals(info, term, rightTermsEnum.next());
+      assertTermStatsEquals(info, leftTermsEnum, rightTermsEnum);
+      if (deep) {
+        assertDocsAndPositionsEnumEquals(info, leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
+                                   rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
+        assertDocsAndPositionsEnumEquals(info, leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
+                                   rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
+
+        assertPositionsSkippingEquals(info, leftReader, leftTermsEnum.docFreq(), 
+                                leftPositions = leftTermsEnum.docsAndPositions(null, leftPositions),
+                                rightPositions = rightTermsEnum.docsAndPositions(null, rightPositions));
+        assertPositionsSkippingEquals(info, leftReader, leftTermsEnum.docFreq(), 
+                                leftPositions = leftTermsEnum.docsAndPositions(randomBits, leftPositions),
+                                rightPositions = rightTermsEnum.docsAndPositions(randomBits, rightPositions));
+
+        // with freqs:
+        assertDocsEnumEquals(info, leftDocs = leftTermsEnum.docs(null, leftDocs),
+            rightDocs = rightTermsEnum.docs(null, rightDocs),
+            true);
+        assertDocsEnumEquals(info, leftDocs = leftTermsEnum.docs(randomBits, leftDocs),
+            rightDocs = rightTermsEnum.docs(randomBits, rightDocs),
+            true);
+
+        // w/o freqs:
+        assertDocsEnumEquals(info, leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE),
+            false);
+        assertDocsEnumEquals(info, leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE),
+            false);
+        
+        // with freqs:
+        assertDocsSkippingEquals(info, leftReader, leftTermsEnum.docFreq(), 
+            leftDocs = leftTermsEnum.docs(null, leftDocs),
+            rightDocs = rightTermsEnum.docs(null, rightDocs),
+            true);
+        assertDocsSkippingEquals(info, leftReader, leftTermsEnum.docFreq(), 
+            leftDocs = leftTermsEnum.docs(randomBits, leftDocs),
+            rightDocs = rightTermsEnum.docs(randomBits, rightDocs),
+            true);
+
+        // w/o freqs:
+        assertDocsSkippingEquals(info, leftReader, leftTermsEnum.docFreq(), 
+            leftDocs = leftTermsEnum.docs(null, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(null, rightDocs, DocsEnum.FLAG_NONE),
+            false);
+        assertDocsSkippingEquals(info, leftReader, leftTermsEnum.docFreq(), 
+            leftDocs = leftTermsEnum.docs(randomBits, leftDocs, DocsEnum.FLAG_NONE),
+            rightDocs = rightTermsEnum.docs(randomBits, rightDocs, DocsEnum.FLAG_NONE),
+            false);
+      }
+    }
+    assertNull(info, rightTermsEnum.next());
+  }
+
+
+  /**
+   * checks docs + freqs + positions + payloads, sequentially
+   */
+  public void assertDocsAndPositionsEnumEquals(String info, DocsAndPositionsEnum leftDocs, DocsAndPositionsEnum rightDocs) throws IOException {
+    if (leftDocs == null || rightDocs == null) {
+      assertNull(leftDocs);
+      assertNull(rightDocs);
+      return;
+    }
+    assertEquals(info, -1, leftDocs.docID());
+    assertEquals(info, -1, rightDocs.docID());
+    int docid;
+    while ((docid = leftDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      assertEquals(info, docid, rightDocs.nextDoc());
+      int freq = leftDocs.freq();
+      assertEquals(info, freq, rightDocs.freq());
+      for (int i = 0; i < freq; i++) {
+        assertEquals(info, leftDocs.nextPosition(), rightDocs.nextPosition());
+        assertEquals(info, leftDocs.getPayload(), rightDocs.getPayload());
+        assertEquals(info, leftDocs.startOffset(), rightDocs.startOffset());
+        assertEquals(info, leftDocs.endOffset(), rightDocs.endOffset());
+      }
+    }
+    assertEquals(info, DocIdSetIterator.NO_MORE_DOCS, rightDocs.nextDoc());
+  }
+  
+  /**
+   * checks docs + freqs, sequentially
+   */
+  public void assertDocsEnumEquals(String info, DocsEnum leftDocs, DocsEnum rightDocs, boolean hasFreqs) throws IOException {
+    if (leftDocs == null) {
+      assertNull(rightDocs);
+      return;
+    }
+    assertEquals(info, -1, leftDocs.docID());
+    assertEquals(info, -1, rightDocs.docID());
+    int docid;
+    while ((docid = leftDocs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      assertEquals(info, docid, rightDocs.nextDoc());
+      if (hasFreqs) {
+        assertEquals(info, leftDocs.freq(), rightDocs.freq());
+      }
+    }
+    assertEquals(info, DocIdSetIterator.NO_MORE_DOCS, rightDocs.nextDoc());
+  }
+  
+  /**
+   * checks advancing docs
+   */
+  public void assertDocsSkippingEquals(String info, IndexReader leftReader, int docFreq, DocsEnum leftDocs, DocsEnum rightDocs, boolean hasFreqs) throws IOException {
+    if (leftDocs == null) {
+      assertNull(rightDocs);
+      return;
+    }
+    int docid = -1;
+    int averageGap = leftReader.maxDoc() / (1+docFreq);
+    int skipInterval = 16;
+
+    while (true) {
+      if (random().nextBoolean()) {
+        // nextDoc()
+        docid = leftDocs.nextDoc();
+        assertEquals(info, docid, rightDocs.nextDoc());
+      } else {
+        // advance()
+        int skip = docid + (int) Math.ceil(Math.abs(skipInterval + random().nextGaussian() * averageGap));
+        docid = leftDocs.advance(skip);
+        assertEquals(info, docid, rightDocs.advance(skip));
+      }
+      
+      if (docid == DocIdSetIterator.NO_MORE_DOCS) {
+        return;
+      }
+      if (hasFreqs) {
+        assertEquals(info, leftDocs.freq(), rightDocs.freq());
+      }
+    }
+  }
+  
+  /**
+   * checks advancing docs + positions
+   */
+  public void assertPositionsSkippingEquals(String info, IndexReader leftReader, int docFreq, DocsAndPositionsEnum leftDocs, DocsAndPositionsEnum rightDocs) throws IOException {
+    if (leftDocs == null || rightDocs == null) {
+      assertNull(leftDocs);
+      assertNull(rightDocs);
+      return;
+    }
+    
+    int docid = -1;
+    int averageGap = leftReader.maxDoc() / (1+docFreq);
+    int skipInterval = 16;
+
+    while (true) {
+      if (random().nextBoolean()) {
+        // nextDoc()
+        docid = leftDocs.nextDoc();
+        assertEquals(info, docid, rightDocs.nextDoc());
+      } else {
+        // advance()
+        int skip = docid + (int) Math.ceil(Math.abs(skipInterval + random().nextGaussian() * averageGap));
+        docid = leftDocs.advance(skip);
+        assertEquals(info, docid, rightDocs.advance(skip));
+      }
+      
+      if (docid == DocIdSetIterator.NO_MORE_DOCS) {
+        return;
+      }
+      int freq = leftDocs.freq();
+      assertEquals(info, freq, rightDocs.freq());
+      for (int i = 0; i < freq; i++) {
+        assertEquals(info, leftDocs.nextPosition(), rightDocs.nextPosition());
+        assertEquals(info, leftDocs.getPayload(), rightDocs.getPayload());
+      }
+    }
+  }
+
+  
+  private void assertTermsSeekingEquals(String info, Terms leftTerms, Terms rightTerms) throws IOException {
+    TermsEnum leftEnum = null;
+    TermsEnum rightEnum = null;
+
+    // just an upper bound
+    int numTests = atLeast(20);
+    Random random = random();
+
+    // collect this number of terms from the left side
+    HashSet<BytesRef> tests = new HashSet<BytesRef>();
+    int numPasses = 0;
+    while (numPasses < 10 && tests.size() < numTests) {
+      leftEnum = leftTerms.iterator(leftEnum);
+      BytesRef term = null;
+      while ((term = leftEnum.next()) != null) {
+        int code = random.nextInt(10);
+        if (code == 0) {
+          // the term
+          tests.add(BytesRef.deepCopyOf(term));
+        } else if (code == 1) {
+          // truncated subsequence of term
+          term = BytesRef.deepCopyOf(term);
+          if (term.length > 0) {
+            // truncate it
+            term.length = random.nextInt(term.length);
+          }
+        } else if (code == 2) {
+          // term, but ensure a non-zero offset
+          byte newbytes[] = new byte[term.length+5];
+          System.arraycopy(term.bytes, term.offset, newbytes, 5, term.length);
+          tests.add(new BytesRef(newbytes, 5, term.length));
+        } else if (code == 3) {
+          switch (random().nextInt(3)) {
+            case 0:
+              tests.add(new BytesRef()); // before the first term
+              break;
+            case 1:
+              tests.add(new BytesRef(new byte[] {(byte) 0xFF, (byte) 0xFF})); // past the last term
+              break;
+            case 2:
+              tests.add(new BytesRef(_TestUtil.randomSimpleString(random()))); // random term
+              break;
+            default:
+              throw new AssertionError();
+          }
+        }
+      }
+      numPasses++;
+    }
+
+    rightEnum = rightTerms.iterator(rightEnum);
+
+    ArrayList<BytesRef> shuffledTests = new ArrayList<BytesRef>(tests);
+    Collections.shuffle(shuffledTests, random);
+
+    for (BytesRef b : shuffledTests) {
+      if (rarely()) {
+        // reuse the enums
+        leftEnum = leftTerms.iterator(leftEnum);
+        rightEnum = rightTerms.iterator(rightEnum);
+      }
+
+      final boolean useCache = random().nextBoolean();
+      final boolean seekExact = random().nextBoolean();
+
+      if (seekExact) {
+        assertEquals(info, leftEnum.seekExact(b, useCache), rightEnum.seekExact(b, useCache));
+      } else {
+        SeekStatus leftStatus = leftEnum.seekCeil(b, useCache);
+        SeekStatus rightStatus = rightEnum.seekCeil(b, useCache);
+        assertEquals(info, leftStatus, rightStatus);
+        if (leftStatus != SeekStatus.END) {
+          assertEquals(info, leftEnum.term(), rightEnum.term());
+          assertTermStatsEquals(info, leftEnum, rightEnum);
+        }
+      }
+    }
+  }
+  
+  /**
+   * checks term-level statistics
+   */
+  public void assertTermStatsEquals(String info, TermsEnum leftTermsEnum, TermsEnum rightTermsEnum) throws IOException {
+    assertEquals(info, leftTermsEnum.docFreq(), rightTermsEnum.docFreq());
+    if (leftTermsEnum.totalTermFreq() != -1 && rightTermsEnum.totalTermFreq() != -1) {
+      assertEquals(info, leftTermsEnum.totalTermFreq(), rightTermsEnum.totalTermFreq());
+    }
+  }
+  
+  /** 
+   * checks that norms are the same across all fields 
+   */
+  public void assertNormsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    Fields leftFields = MultiFields.getFields(leftReader);
+    Fields rightFields = MultiFields.getFields(rightReader);
+    // Fields could be null if there are no postings,
+    // but then it must be null for both
+    if (leftFields == null || rightFields == null) {
+      assertNull(info, leftFields);
+      assertNull(info, rightFields);
+      return;
+    }
+    
+    for (String field : leftFields) {
+      NumericDocValues leftNorms = MultiDocValues.getNormValues(leftReader, field);
+      NumericDocValues rightNorms = MultiDocValues.getNormValues(rightReader, field);
+      if (leftNorms != null && rightNorms != null) {
+        assertDocValuesEquals(info, leftReader.maxDoc(), leftNorms, rightNorms);
+      } else {
+        assertNull(info, leftNorms);
+        assertNull(info, rightNorms);
+      }
+    }
+  }
+  
+  /** 
+   * checks that stored fields of all documents are the same 
+   */
+  public void assertStoredFieldsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    assert leftReader.maxDoc() == rightReader.maxDoc();
+    for (int i = 0; i < leftReader.maxDoc(); i++) {
+      StoredDocument leftDoc = leftReader.document(i);
+      StoredDocument rightDoc = rightReader.document(i);
+      
+      // TODO: I think this is bogus because we don't document what the order should be
+      // from these iterators, etc. I think the codec/IndexReader should be free to order this stuff
+      // in whatever way it wants (e.g. maybe it packs related fields together or something)
+      // To fix this, we sort the fields in both documents by name, but
+      // we still assume that all instances with same name are in order:
+      Comparator<StorableField> comp = new Comparator<StorableField>() {
+        @Override
+        public int compare(StorableField arg0, StorableField arg1) {
+          return arg0.name().compareTo(arg1.name());
+        }        
+      };
+      Collections.sort(leftDoc.getFields(), comp);
+      Collections.sort(rightDoc.getFields(), comp);
+
+      Iterator<StorableField> leftIterator = leftDoc.iterator();
+      Iterator<StorableField> rightIterator = rightDoc.iterator();
+      while (leftIterator.hasNext()) {
+        assertTrue(info, rightIterator.hasNext());
+        assertStoredFieldEquals(info, leftIterator.next(), rightIterator.next());
+      }
+      assertFalse(info, rightIterator.hasNext());
+    }
+  }
+  
+  /** 
+   * checks that two stored fields are equivalent 
+   */
+  public void assertStoredFieldEquals(String info, StorableField leftField, StorableField rightField) {
+    assertEquals(info, leftField.name(), rightField.name());
+    assertEquals(info, leftField.binaryValue(), rightField.binaryValue());
+    assertEquals(info, leftField.stringValue(), rightField.stringValue());
+    assertEquals(info, leftField.numericValue(), rightField.numericValue());
+    // TODO: should we check the FT at all?
+  }
+  
+  /** 
+   * checks that term vectors across all fields are equivalent 
+   */
+  public void assertTermVectorsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    assert leftReader.maxDoc() == rightReader.maxDoc();
+    for (int i = 0; i < leftReader.maxDoc(); i++) {
+      Fields leftFields = leftReader.getTermVectors(i);
+      Fields rightFields = rightReader.getTermVectors(i);
+      assertFieldsEquals(info, leftReader, leftFields, rightFields, rarely());
+    }
+  }
+
+  private static Set<String> getDVFields(IndexReader reader) {
+    Set<String> fields = new HashSet<String>();
+    for(FieldInfo fi : MultiFields.getMergedFieldInfos(reader)) {
+      if (fi.hasDocValues()) {
+        fields.add(fi.name);
+      }
+    }
+
+    return fields;
+  }
+  
+  /**
+   * checks that docvalues across all fields are equivalent
+   */
+  public void assertDocValuesEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    Set<String> leftFields = getDVFields(leftReader);
+    Set<String> rightFields = getDVFields(rightReader);
+    assertEquals(info, leftFields, rightFields);
+
+    for (String field : leftFields) {
+      // TODO: clean this up... very messy
+      {
+        NumericDocValues leftValues = MultiDocValues.getNumericValues(leftReader, field);
+        NumericDocValues rightValues = MultiDocValues.getNumericValues(rightReader, field);
+        if (leftValues != null && rightValues != null) {
+          assertDocValuesEquals(info, leftReader.maxDoc(), leftValues, rightValues);
+        } else {
+          assertNull(info, leftValues);
+          assertNull(info, rightValues);
+        }
+      }
+
+      {
+        BinaryDocValues leftValues = MultiDocValues.getBinaryValues(leftReader, field);
+        BinaryDocValues rightValues = MultiDocValues.getBinaryValues(rightReader, field);
+        if (leftValues != null && rightValues != null) {
+          BytesRef scratchLeft = new BytesRef();
+          BytesRef scratchRight = new BytesRef();
+          for(int docID=0;docID<leftReader.maxDoc();docID++) {
+            leftValues.get(docID, scratchLeft);
+            rightValues.get(docID, scratchRight);
+            assertEquals(info, scratchLeft, scratchRight);
+          }
+        } else {
+          assertNull(info, leftValues);
+          assertNull(info, rightValues);
+        }
+      }
+      
+      {
+        SortedDocValues leftValues = MultiDocValues.getSortedValues(leftReader, field);
+        SortedDocValues rightValues = MultiDocValues.getSortedValues(rightReader, field);
+        if (leftValues != null && rightValues != null) {
+          // numOrds
+          assertEquals(info, leftValues.getValueCount(), rightValues.getValueCount());
+          // ords
+          BytesRef scratchLeft = new BytesRef();
+          BytesRef scratchRight = new BytesRef();
+          for (int i = 0; i < leftValues.getValueCount(); i++) {
+            leftValues.lookupOrd(i, scratchLeft);
+            rightValues.lookupOrd(i, scratchRight);
+            assertEquals(info, scratchLeft, scratchRight);
+          }
+          // bytes
+          for(int docID=0;docID<leftReader.maxDoc();docID++) {
+            leftValues.get(docID, scratchLeft);
+            rightValues.get(docID, scratchRight);
+            assertEquals(info, scratchLeft, scratchRight);
+          }
+        } else {
+          assertNull(info, leftValues);
+          assertNull(info, rightValues);
+        }
+      }
+      
+      {
+        SortedSetDocValues leftValues = MultiDocValues.getSortedSetValues(leftReader, field);
+        SortedSetDocValues rightValues = MultiDocValues.getSortedSetValues(rightReader, field);
+        if (leftValues != null && rightValues != null) {
+          // numOrds
+          assertEquals(info, leftValues.getValueCount(), rightValues.getValueCount());
+          // ords
+          BytesRef scratchLeft = new BytesRef();
+          BytesRef scratchRight = new BytesRef();
+          for (int i = 0; i < leftValues.getValueCount(); i++) {
+            leftValues.lookupOrd(i, scratchLeft);
+            rightValues.lookupOrd(i, scratchRight);
+            assertEquals(info, scratchLeft, scratchRight);
+          }
+          // ord lists
+          for(int docID=0;docID<leftReader.maxDoc();docID++) {
+            leftValues.setDocument(docID);
+            rightValues.setDocument(docID);
+            long ord;
+            while ((ord = leftValues.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+              assertEquals(info, ord, rightValues.nextOrd());
+            }
+            assertEquals(info, SortedSetDocValues.NO_MORE_ORDS, rightValues.nextOrd());
+          }
+        } else {
+          assertNull(info, leftValues);
+          assertNull(info, rightValues);
+        }
+      }
+    }
+  }
+  
+  public void assertDocValuesEquals(String info, int num, NumericDocValues leftDocValues, NumericDocValues rightDocValues) throws IOException {
+    assertNotNull(info, leftDocValues);
+    assertNotNull(info, rightDocValues);
+    for(int docID=0;docID<num;docID++) {
+      assertEquals(leftDocValues.get(docID),
+                   rightDocValues.get(docID));
+    }
+  }
+  
+  // TODO: this is kinda stupid, we don't delete documents in the test.
+  public void assertDeletedDocsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    assert leftReader.numDeletedDocs() == rightReader.numDeletedDocs();
+    Bits leftBits = MultiFields.getLiveDocs(leftReader);
+    Bits rightBits = MultiFields.getLiveDocs(rightReader);
+    
+    if (leftBits == null || rightBits == null) {
+      assertNull(info, leftBits);
+      assertNull(info, rightBits);
+      return;
+    }
+    
+    assert leftReader.maxDoc() == rightReader.maxDoc();
+    assertEquals(info, leftBits.length(), rightBits.length());
+    for (int i = 0; i < leftReader.maxDoc(); i++) {
+      assertEquals(info, leftBits.get(i), rightBits.get(i));
+    }
+  }
+  
+  public void assertFieldInfosEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
+    FieldInfos leftInfos = MultiFields.getMergedFieldInfos(leftReader);
+    FieldInfos rightInfos = MultiFields.getMergedFieldInfos(rightReader);
+    
+    // TODO: would be great to verify more than just the names of the fields!
+    TreeSet<String> left = new TreeSet<String>();
+    TreeSet<String> right = new TreeSet<String>();
+    
+    for (FieldInfo fi : leftInfos) {
+      left.add(fi.name);
+    }
+    
+    for (FieldInfo fi : rightInfos) {
+      right.add(fi.name);
+    }
+    
+    assertEquals(info, left, right);
+  }
 }

Modified: lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/branches/security/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Thu May 30 07:53:18 2013
@@ -17,6 +17,7 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.io.PrintStream;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.HashMap;
@@ -33,21 +34,17 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.asserting.AssertingCodec;
 import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
 import org.apache.lucene.codecs.compressing.CompressingCodec;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
 import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
-import org.apache.lucene.codecs.lucene41.Lucene41Codec;
 import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
 import org.apache.lucene.codecs.lucene42.Lucene42Codec;
-import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
 import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.index.RandomCodec;
 import org.apache.lucene.search.RandomSimilarityProvider;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;  // javadocs
 import org.junit.internal.AssumptionViolatedException;
-
 import com.carrotsearch.randomizedtesting.RandomizedContext;
 
 import static org.apache.lucene.util.LuceneTestCase.*;
@@ -79,6 +76,25 @@ final class TestRuleSetupAndRestoreClass
    */
   HashSet<String> avoidCodecs;
 
+  static class ThreadNameFixingPrintStreamInfoStream extends PrintStreamInfoStream {
+
+    public ThreadNameFixingPrintStreamInfoStream(PrintStream out) {
+      super(out);
+    }
+
+    @Override
+    public void message(String component, String message) {
+      final String name;
+      if (Thread.currentThread().getName().startsWith("TEST-")) {
+        // The name of the main thread is way too
+        // long when looking at IW verbose output...
+        name = "main";
+      } else {
+        name = Thread.currentThread().getName();
+      }
+      stream.println(component + " " + messageID + " [" + new Date() + "; " + name + "]: " + message);    
+    }
+  }
 
   @Override
   protected void before() throws Exception {
@@ -111,29 +127,14 @@ final class TestRuleSetupAndRestoreClass
     final Random random = RandomizedContext.current().getRandom();
     final boolean v = random.nextBoolean();
     if (INFOSTREAM) {
-      InfoStream.setDefault(new PrintStreamInfoStream(System.out) {
-          @Override
-          public void message(String component, String message) {
-            final String name;
-            if (Thread.currentThread().getName().startsWith("TEST-")) {
-              // The name of the main thread is way too
-              // long when looking at IW verbose output...
-              name = "main";
-            } else {
-              name = Thread.currentThread().getName();
-            }
-            stream.println(component + " " + messageID + " [" + new Date() + "; " + name + "]: " + message);    
-          }
-        });
+      InfoStream.setDefault(new ThreadNameFixingPrintStreamInfoStream(System.out));
     } else if (v) {
       InfoStream.setDefault(new NullInfoStream());
     }
 
     Class<?> targetClass = RandomizedContext.current().getTargetClass();
     avoidCodecs = new HashSet<String>();
-    // TODO: Fix below code to use c.isAnnotationPresent(). It was changed
-    // to the null check to work around a bug in JDK 8 b78 (see LUCENE-4808).
-    if (targetClass.getAnnotation(SuppressCodecs.class) != null) {
+    if (targetClass.isAnnotationPresent(SuppressCodecs.class)) {
       SuppressCodecs a = targetClass.getAnnotation(SuppressCodecs.class);
       avoidCodecs.addAll(Arrays.asList(a.value()));
     }
@@ -191,7 +192,7 @@ final class TestRuleSetupAndRestoreClass
           return super.toString() + ": " + format.toString() + ", " + dvFormat.toString();
         }
       };
-    } else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9 && !shouldAvoidCodec("SimpleText"))) {
+    } else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9 && LuceneTestCase.rarely(random) && !shouldAvoidCodec("SimpleText"))) {
       codec = new SimpleTextCodec();
     } else if ("CheapBastard".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 8 && !shouldAvoidCodec("CheapBastard") && !shouldAvoidCodec("Lucene41"))) {
       // we also avoid this codec if Lucene41 is avoided, since thats the postings format it uses.

Modified: lucene/dev/branches/security/lucene/tools/clover/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/clover/README.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/clover/README.txt (original)
+++ lucene/dev/branches/security/lucene/tools/clover/README.txt Thu May 30 07:53:18 2013
@@ -2,6 +2,26 @@
   DOCUMENTATION ABOUT ATLASSIAN CLOVER LICENSE
 ################################################
 
+From: Nicholas Muldoon <nm...@atlassian.com>
+To: Ted Yu <yu...@gmail.com>
+Cc: Enis Soztutar <en...@hortonworks.com>, Todd Lipcon <to...@cloudera.com>
+Content-Type: multipart/alternative; boundary=047d7b10cff34cee0f04c376df70
+X-Gm-Message-State: ALoCoQmzowRKrtL3txnON+W+U2vn7gwwWSn/U5dvchuTV0Nn3xaMoAojvbPal5TBkC6foBnuHPWU
+
+--047d7b10cff34cee0f04c376df70
+Content-Type: text/plain; charset=ISO-8859-1
+
+Hi Ted,
+
+Please find a renewed license below - Clover 3 compatible. Should you have
+any difficulty please let me know.
+
+[license]
+
+Thanks Ted, have a great day, thanks for your support,
+Nicholas
+
+
 On Fri, Dec 18, 2009 at 1:33 AM, Nicholas Muldoon <nm...@atlassian.com> wrote:
 ---------------------------------------------------------------------------------
 

Modified: lucene/dev/branches/security/lucene/tools/clover/clover.license
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/clover/clover.license?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/clover/clover.license (original)
+++ lucene/dev/branches/security/lucene/tools/clover/clover.license Thu May 30 07:53:18 2013
@@ -1,7 +1,5 @@
-AAABLQ0ODAoPeNptkF1rwjAUhu/zKwK7rjRVhwiBSdsLQVtZdVeDkaXHGkzTkA+n/36xtbiLXeQmh
-+d533NeKq/wyjeYzHCSLOfxkiQ4zfY4iQlBGVhuhHaiUzSV3QXM5xKXGhSuOm84oNQAu08z5oDek
-SheRGSO0k45xl3BWqBbcQa85ZypWoK1iPeiSeHbbzDl8WDBWBqRkcm3TEjaBqjtmTemGT/BpDMNy
-i9M+j6QHpm0MLoCJi5AnfHPLym7H6j1udEGjuIKlgbDZJAhKTgoCx8h+i5LUAhVDhRTHPKrFub23
-IgkEVmM2s0A7m8a+uXKXV58VeXhPc1RaRqmhB36rYagKi9oeBGZvs4W0yl68OuMbtbZP4M/4V5J0
-QoHNdqFW5+YhbFT/Oj0C1FClngwLgIVAJJTCumwsuBLa8eB0JcPVMcTcRiKAhUAgS9VbXCNOqGDY
-ooFNN9744FAK2s=X02f7
\ No newline at end of file
+RMnmmVojSVTAxfPXiUgnIWdgpmMJbqADeQonukSmgbVIhd
+mi2K<6pTN6gi825uWHxIERlJ2KvVSx3LOJtfGCJvZnLyCp
+qOPqVTPpPqNRmNomonNUXqmqqrrpQMNpNPuVuutwVsxspP
+MmonmqmUUnpqmspoummmmmUUnpqmspoummmmmUU1mXZebU
+Unmmmm

Modified: lucene/dev/branches/security/lucene/tools/forbiddenApis/executors.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/forbiddenApis/executors.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/forbiddenApis/executors.txt (original)
+++ lucene/dev/branches/security/lucene/tools/forbiddenApis/executors.txt Thu May 30 07:53:18 2013
@@ -13,11 +13,11 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-# These methods spawn threads with vague names. Use a custom thread factory and name
-# threads so that you can tell (by its name) which executor it is associated with.
 # see Solr's DefaultSolrThreadFactory
 # see Lucene's NamedThreadFactory
 
+@defaultMessage spawns threads with vague names; use a custom thread factory and name threads so that you can tell (by its name) which executor it is associated with
+
 java.util.concurrent.Executors#newFixedThreadPool(int)
 java.util.concurrent.Executors#newSingleThreadExecutor()
 java.util.concurrent.Executors#newCachedThreadPool()

Modified: lucene/dev/branches/security/lucene/tools/forbiddenApis/servlet-api.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/forbiddenApis/servlet-api.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/forbiddenApis/servlet-api.txt (original)
+++ lucene/dev/branches/security/lucene/tools/forbiddenApis/servlet-api.txt Thu May 30 07:53:18 2013
@@ -13,17 +13,19 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-# These methods from the Servlet API should not be used, because they are
-# either broken and slow in some environments (e.g., Jetty's UTF-8 readers),
-# or the parsing of request parameters is not using the correct encoding
-# without extra configuration in the servlet container:
-javax.servlet.ServletRequest#getReader()
+@defaultMessage Servlet API method is parsing request parameters without using the correct encoding if no extra configuration is given in the servlet container
+
 javax.servlet.ServletRequest#getParameter(java.lang.String) 
 javax.servlet.ServletRequest#getParameterMap() 
 javax.servlet.ServletRequest#getParameterNames() 
 javax.servlet.ServletRequest#getParameterValues(java.lang.String) 
-javax.servlet.ServletResponse#getWriter()
 
+javax.servlet.http.HttpServletRequest#getSession() @ Servlet API getter has side effect of creating sessions
+
+@defaultMessage Servlet API method is broken and slow in some environments (e.g., Jetty's UTF-8 readers)
+
+javax.servlet.ServletRequest#getReader()
+javax.servlet.ServletResponse#getWriter()
 javax.servlet.ServletInputStream#readLine(byte[],int,int) 
 javax.servlet.ServletOutputStream#print(boolean)
 javax.servlet.ServletOutputStream#print(char)

Modified: lucene/dev/branches/security/lucene/tools/forbiddenApis/tests.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/forbiddenApis/tests.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/forbiddenApis/tests.txt (original)
+++ lucene/dev/branches/security/lucene/tools/forbiddenApis/tests.txt Thu May 30 07:53:18 2013
@@ -13,13 +13,10 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-# All classes should derive from LuceneTestCase
-junit.framework.TestCase
+junit.framework.TestCase @ All classes should derive from LuceneTestCase
 
-# Use RandomizedRunner's random instead
-java.util.Random#<init>()
+java.util.Random#<init>() @ Use RandomizedRunner's random instead
 
-# Don't depend on wall clock times
 # TODO: fix tests that do this!
-#java.lang.System#currentTimeMillis()
-#java.lang.System#nanoTime()
+#java.lang.System#currentTimeMillis() @ Don't depend on wall clock times
+#java.lang.System#nanoTime() @ Don't depend on wall clock times

Modified: lucene/dev/branches/security/lucene/tools/javadoc/ecj.javadocs.prefs
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/javadoc/ecj.javadocs.prefs?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/javadoc/ecj.javadocs.prefs (original)
+++ lucene/dev/branches/security/lucene/tools/javadoc/ecj.javadocs.prefs Thu May 30 07:53:18 2013
@@ -1,7 +1,7 @@
 #Sun Sep 23 20:55:03 EDT 2012
 eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
-org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
+org.eclipse.jdt.core.compiler.compliance=1.7
 org.eclipse.jdt.core.compiler.doc.comment.support=enabled
 org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=ignore
 org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
@@ -88,4 +88,4 @@ org.eclipse.jdt.core.compiler.problem.un
 org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=ignore
 org.eclipse.jdt.core.compiler.problem.unusedWarningToken=ignore
 org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=ignore
-org.eclipse.jdt.core.compiler.source=1.6
+org.eclipse.jdt.core.compiler.source=1.7

Modified: lucene/dev/branches/security/lucene/tools/junit4/cached-timehints.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/junit4/cached-timehints.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/junit4/cached-timehints.txt (original)
+++ lucene/dev/branches/security/lucene/tools/junit4/cached-timehints.txt Thu May 30 07:53:18 2013
@@ -929,7 +929,7 @@ org.apache.solr.core.TestQuerySenderList
 org.apache.solr.core.TestQuerySenderNoQuery=392,616,406,406,413,413,356
 org.apache.solr.core.TestSolrDeletionPolicy1=636,736,868,732,627,1449,788
 org.apache.solr.core.TestSolrDeletionPolicy2=377,450,376,388,413,459,373
-org.apache.solr.core.TestSolrDiscoveryProperties=1183,1243,5175,1041,1490,2090,1181
+org.apache.solr.core.TestCoreDiscovery=1183,1243,5175,1041,1490,2090,1181
 org.apache.solr.core.TestSolrIndexConfig=491,411,360,426,407,414,369
 org.apache.solr.core.TestSolrXMLSerializer=29,24,33,39,164,65,46
 org.apache.solr.core.TestXIncludeConfig=158,139,147,115,1494,112,318

Modified: lucene/dev/branches/security/lucene/tools/junit4/tests.policy
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/tools/junit4/tests.policy?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/tools/junit4/tests.policy (original)
+++ lucene/dev/branches/security/lucene/tools/junit4/tests.policy Thu May 30 07:53:18 2013
@@ -54,6 +54,7 @@ grant {
 
   // Solr needs those:
   permission java.net.NetPermission "*";
+  permission java.sql.SQLPermission "*";
   permission java.util.logging.LoggingPermission "control";
   permission javax.management.MBeanPermission "*", "*";
   permission javax.management.MBeanServerPermission "*";

Modified: lucene/dev/branches/security/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/CHANGES.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/CHANGES.txt (original)
+++ lucene/dev/branches/security/solr/CHANGES.txt Thu May 30 07:53:18 2013
@@ -12,7 +12,7 @@ See http://lucene.apache.org/solr for mo
 
 Getting Started
 ---------------
-You need a Java 1.6 VM or later installed.
+You need a Java 1.7 VM or later installed.
 In this release, there is an example Solr server including a bundled 
 servlet container in the directory named "example".
 See the tutorial at http://lucene.apache.org/solr/tutorial.html
@@ -38,6 +38,552 @@ TBD...
 Detailed Change List
 ----------------------
 
+Other Changes
+----------------------
+
+* SOLR-4622: Hardcoded SolrCloud defaults for hostContext and hostPort that
+  were deprecated in 4.3 have been removed completely. (hossman)
+
+==================  4.4.0 ==================
+
+Versions of Major Components
+---------------------
+Apache Tika 1.3
+Carrot2 3.6.2
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.5
+
+Upgrading from Solr 4.3.0
+----------------------
+
+* SOLR-4778: The signature of LogWatcher.registerListener has changed, from
+  (ListenerConfig, CoreContainer) to (ListenerConfig).  Users implementing their
+  own LogWatcher classes will need to change their code accordingly.
+  
+Detailed Change List
+----------------------
+
+New Features
+----------------------
+
+* SOLR-3251: Dynamically add fields to schema. (Steve Rowe, Robert Muir, yonik)   
+
+* SOLR-4761: Add option to plugin a merged segment warmer into solrconfig.xml
+  (Mark Miller, Mike McCandless, Robert Muir)
+
+* SOLR-3240: Add "spellcheck.collateMaxCollectDocs" option so that when testing
+  potential Collations against the index, SpellCheckComponent will only collect
+  n documents, thereby estimating the hit-count.  This is a performance optimization
+  in cases where exact hit-counts are unnecessary.  Also, when "collateExtendedResults"
+  is false, this optimization is always made (James Dyer).
+
+* SOLR-4785: New MaxScoreQParserPlugin returning max() instead of sum() of terms (janhoy)
+
+* SOLR-4234: Add support for binary files in ZooKeeper. (Eric Pugh via Mark Miller)
+
+* SOLR-4048: Add findRecursive method to NamedList. (Shawn Heisey)
+
+* SOLR-4228: SolrJ's SolrPing object has new methods for ping, enable, and
+  disable. (Shawn Heisey, hossman, Steve Rowe)
+
+Bug Fixes
+----------------------
+
+* SOLR-4333: edismax parser to not double-escape colons if already escaped by
+  the client application (James Dyer, Robert J. van der Boon)
+
+* SOLR-4776: Solrj doesn't return "between" count in range facets
+  (Philip K. Warren via shalin)
+
+* SOLR-4616: HitRatio on caches is now exposed over JMX MBeans as a float.
+  (Greg Bowyer)
+  
+* SOLR-4803: Fixed core discovery mode (ie: new style solr.xml) to treat 
+  'collection1' as the default core name. (hossman)
+  
+* SOLR-4790: Throw an error if a core has the same name as another core, both old and
+  new style solr.xml
+
+* SOLR-4842: Fix facet.field local params from affecting other facet.field's.
+  (ehatcher, hossman)
+
+* SOLR-4814: If a SolrCore cannot be created it should remove any information it 
+  published about itself from ZooKeeper. (Mark Miller)
+
+* SOLR-4863: Removed non-existent attribute sourceId from dynamic JMX stats
+  to fix AttributeNotFoundException (suganuma, hossman via shalin)
+
+Other Changes
+----------------------
+
+* SOLR-4737: Update Guava to 14.0.1 (Mark Miller)
+
+* SOLR-2079: Add option to pass HttpServletRequest in the SolrQueryRequest context map.
+  (Tomás Fernández Löbbe via Robert Muir)
+
+* SOLR-4738: Update Jetty to 8.1.10.v20130312 (Mark Miller, Robert Muir)
+
+* SOLR-4749: Clean up and refactor CoreContainer code around solr.xml and SolrCore
+  management. (Mark Miller)
+
+* SOLR-4547: Move logging of filenames on commit from INFO to DEBUG.
+  (Shawn Heisey, hossman)
+
+* SOLR-4757: Change the example to use the new solr.xml format and core 
+  discovery by directory structure. (Mark Miller)
+
+* SOLR-4759: Velocity (/browse) template cosmetic cleanup.
+  (Mark Bennett, ehatcher)
+
+* SOLR-4778: LogWatcher init code moved out of CoreContainer (Alan Woodward)
+
+* SOLR-4784: Make class LuceneQParser public (janhoy)
+
+* SOLR-4448: Allow the solr internal load balancer to be more easily pluggable.
+  (Philip Hoy via Robert Muir)
+
+==================  4.3.1 ==================
+
+Versions of Major Components
+---------------------
+Apache Tika 1.3
+Carrot2 3.6.2
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.5
+
+Detailed Change List
+----------------------
+
+Bug Fixes
+----------------------
+
+* SOLR-4795: Sub shard leader should not accept any updates from parent after
+  it goes active (shalin)
+
+* SOLR-4798: shard splitting does not respect the router for the collection
+  when executing the index split.   One effect of this is that documents
+  may be placed in the wrong shard when the default compositeId router
+  is used in conjunction with IDs containing "!". (yonik)
+
+* SOLR-4797: Shard splitting creates sub shards which have the wrong hash
+  range in cluster state. This happens when numShards is not a power of two
+  and router is compositeId. (shalin)
+  
+* SOLR-4791: solr.xml sharedLib does not work in 4.3.0 (Ryan Ernst, Jan Høydahl via 
+  Erick Erickson)
+
+* SOLR-4806: Shard splitting does not abort if WaitForState times out (shalin)
+
+* SOLR-4807: The zkcli script now works with log4j. The zkcli.bat script
+  was broken on Windows in 4.3.0, now it works. (Shawn Heisey)
+
+* SOLR-4813: Fix SynonymFilterFactory to allow init parameters for
+  tokenizer factory used when parsing synonyms file.  (Shingo Sasaki, hossman)
+
+* SOLR-4829: Fix transaction log leaks (a failure to clean up some old logs)
+  on a shard leader, or when unexpected exceptions are thrown during log
+  recovery.  (Steven Bower, Mark Miller, yonik)
+
+* SOLR-4751: Fix replication problem of files in sub directory of conf directory.
+  (Minoru Osuka via Koji)
+
+* SOLR-4741: Deleting a collection should set DELETE_DATA_DIR to true.
+  (Mark Miller)
+
+* SOLR-4752: There are some minor bugs in the Collections API parameter
+  validation. (Mark Miller)
+
+* SOLR-4563: RSS DIH-example not working (janhoy)
+
+* SOLR-4796: zkcli.sh should honor JAVA_HOME (Roman Shaposhnik via Mark Miller)
+
+* SOLR-4734: Leader election fails with an NPE if there is no UpdateLog.
+ (Mark Miller, Alexander Eibner)
+
+* SOLR-4868: Setting the log level for the log4j root category results in
+  adding a new category, the empty string. (Shawn Heisey)
+
+* SOLR-4855: DistributedUpdateProcessor doesn't check for peer sync requests (shalin)
+
+* SOLR-4867: Admin UI - setting loglevel on root throws RangeError (steffkes)
+
+Other Changes
+----------------------
+
+* SOLR-4760: Include core name in logs when loading schema.
+  (Shawn Heisey)
+
+==================  4.3.0 ==================
+
+Versions of Major Components
+---------------------
+Apache Tika 1.3
+Carrot2 3.6.2
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.5
+
+Upgrading from Solr 4.2.0
+----------------------
+
+* In the schema REST API, the output path for copyFields and dynamicFields 
+  has been changed from all lowercase "copyfields" and "dynamicfields" to 
+  camelCase "copyFields" and "dynamicFields", respectively, to align with all 
+  other schema REST API outputs, which use camelCase.  The URL format remains 
+  the same: all resource names are lowercase.  See SOLR-4623 for details.
+  
+* Slf4j/logging jars are no longer included in the Solr webapp. All logging 
+  jars are now in example/lib/ext. Changing logging impls is now as easy as 
+  updating the jars in this folder with those necessary for the logging impl 
+  you would like. If you are using another webapp container, these jars will 
+  need to go in the corresponding location for that container. 
+  In conjunction, the dist-excl-slf4j and dist-war-excl-slf4 build targets 
+  have been removed since they are redundent.  See the Slf4j documentation, 
+  SOLR-3706, and SOLR-4651 for more details.
+
+* The hardcoded SolrCloud defaults for 'hostContext="solr"' and 
+  'hostPort="8983"' have been deprecated and will be removed in Solr 5.0.  
+  Existing solr.xml files that do not have these options explicitly specified 
+  should be updated accordingly.  See SOLR-4622 for more details.
+
+
+Detailed Change List
+----------------------
+
+New Features
+----------------------
+
+* SOLR-4648 PreAnalyzedUpdateProcessorFactory allows using the functionality
+  of PreAnalyzedField with other field types. See javadoc for details and
+  examples. (Andrzej Bialecki)
+  
+* SOLR-4623: Provide REST API read access to all elements of the live schema.
+  Add a REST API request to return the entire live schema, in JSON, XML, and
+  schema.xml formats.  Move REST API methods from package org.apache.solr.rest
+  to org.apache.solr.rest.schema, and rename base functionality REST API
+  classes to remove the current schema focus, to prepare for other non-schema
+  REST APIs.  Change output path for copyFields and dynamicFields from
+  "copyfields" and "dynamicfields" (all lowercase) to "copyFields" and
+  "dynamicFields", respectively, to align with all other REST API outputs, which
+  use camelCase.
+  (Steve Rowe)
+    
+* SOLR-4658: In preparation for REST API requests that can modify the schema,
+  a "managed schema" is introduced.  
+  Add '<schemaFactory class="ManagedSchemaFactory" mutable="true"/>' to solrconfig.xml
+  in order to use it, and to enable schema modifications via REST API requests.
+  (Steve Rowe, Robert Muir)
+
+* SOLR-4656: Added two new highlight parameters, hl.maxMultiValuedToMatch and 
+  hl.maxMultiValuedToExamine. maxMultiValuedToMatch stops looking for snippets after 
+  finding the specified number of matches, no matter how far into the multivalued field
+  you've gone. maxMultiValuedToExamine stops looking for matches after the specified
+  number of multiValued entries have been examined. If both are specified, the limit
+  hit first stops the loop. Also this patch cuts down on the copying of the document 
+  entries during highlighting. These optimizations are probably unnoticeable unless
+  there are a large number of entries in the multiValued field. Conspicuously, this will
+  prevent the "best" match from being found if it appears later in the MV list than the
+  cutoff specified by either of these params. (Erick Erickson)
+
+* SOLR-4675: Improve PostingsSolrHighlighter to support per-field/query-time overrides
+  and add additional configuration parameters. See the javadocs for more details and
+  examples. (Robert Muir)
+
+* SOLR-3755: A new collections api to add additional shards dynamically by splitting
+  existing shards. (yonik, Anshum Gupta, shalin)
+
+* SOLR-4530: DIH: Provide configuration to use Tika's IdentityHtmlMapper
+  (Alexandre Rafalovitch via shalin)
+  
+* SOLR-4662: Discover SolrCores by directory structure rather than defining them
+  in solr.xml. Also, change the format of solr.xml to be closer to that of solrconfig.xml.
+  This version of Solr will ship the example in the old style, but you can manually
+  try the new style. Solr 4.4 will ship with the new style, and Solr 5.0 will remove
+  support for the old style. (Erick Erickson, Mark Miller)
+  Additional Work:
+  - SOLR-4347: Ensure that newly-created cores via Admin handler are persisted in solr.xml
+  (Erick Erickson)
+  - SOLR-1905: Cores created by the admin request handler should be persisted to solr.xml.
+  Also fixed a problem whereby properties like solr.solr.datadir would be persisted
+  to solr.xml. Also, cores that didn't happen to be loaded were not persisted. 
+  (Erick Erickson)
+
+* SOLR-4717/SOLR-1351: SimpleFacets now work with localParams allowing faceting on the 
+  same field multiple ways (ryan, Uri Boness)
+
+* SOLR-4671: CSVResponseWriter now supports pseudo fields. (ryan, nihed mbarek)
+
+* SOLR-4358: HttpSolrServer sends the stream name and exposes 'useMultiPartPost'
+  (Karl Wright via ryan)
+   
+
+Bug Fixes
+----------------------
+
+* SOLR-4543: setting shardHandlerFactory in solr.xml/solr.properties does not work.
+  (Ryan Ernst, Robert Muir via Erick Erickson)
+
+* SOLR-4634: Fix scripting engine tests to work with Java 8's "Nashorn" Javascript
+  implementation.  (Uwe Schindler)
+
+* SOLR-4636: If opening a reader fails for some reason when opening a SolrIndexSearcher,
+  a Directory can be left unreleased. (Mark Miller)
+
+* SOLR-4405: Admin UI - admin-extra files are not rendered into the core-menu (steffkes)
+
+* SOLR-3956: Fixed group.facet=true to work with negative facet.limit
+  (Chris van der Merwe, hossman)
+  
+* SOLR-4650: copyField doesn't work with source globs that don't match any
+  explicit or dynamic fields.  This regression was introduced in Solr 4.2.
+  (Daniel Collins, Steve Rowe)
+  
+* SOLR-4641: Schema now throws exception on illegal field parameters.  (Robert Muir)
+
+* SOLR-3758: Fixed SpellCheckComponent to work consistently with distributed grouping
+  (James Dyer)
+
+* SOLR-4652: Fix broken behavior with shared libraries in resource loader for
+  solr.xml plugins.  (Ryan Ernst, Robert Muir, Uwe Schindler)
+
+* SOLR-4664: ZkStateReader should update aliases on construction. 
+  (Mark Miller, Elodie Sannier)
+
+* SOLR-4682: CoreAdminRequest.mergeIndexes can not merge multiple cores or indexDirs.
+  (Jason.D.Cao via shalin)
+
+* SOLR-4581: When faceting on numeric fields in Solr 4.2, negative values (constraints)
+  were sorted incorrectly. (Alexander Buhr, shalin, yonik)
+
+* SOLR-4699: The System admin handler should not assume a file system based data directory 
+  location. (Mark Miller)
+
+* SOLR-4695: Fix core admin SPLIT action to be useful with non-cloud setups (shalin)
+
+* SOLR-4680: Correct example spellcheck configuration's queryAnalyzerFieldType and
+  use "text" field instead of narrower "name" field (ehatcher, Mark Bennett)
+
+* SOLR-4702: Fix example /browse "Did you mean?" suggestion feature. (ehatcher, Mark Bennett)
+
+* SOLR-4710: You cannot delete a collection fully from ZooKeeper unless all nodes are up and 
+  functioning correctly. (Mark Miller)
+
+* SOLR-4487: SolrExceptions thrown by HttpSolrServer will now contain the 
+  proper HTTP status code returned by the remote server, even if that status 
+  code is not something Solr itself returned -- eg: from the Servlet Container, 
+  or an intermediate HTTP Proxy (hossman)
+
+* SOLR-4661: Admin UI Replication details now correctly displays the current
+  replicable generation/version of the master. (hossman)
+
+* SOLR-4716,SOLR-4584: SolrCloud request proxying does not work on Tomcat and
+  perhaps other non Jetty containers. (Po Rui, Yago Riveiro via Mark Miller)
+
+* SOLR-4746: Distributed grouping used a NamedList instead of a SimpleOrderedMap
+  for the top level group commands, causing output formatting differences
+  compared to non-distributed grouping. (yonik)
+
+* SOLR-4705: Fixed bug causing NPE when querying a single replica in SolrCloud 
+  using the shards param (Raintung Li, hossman)
+
+* SOLR-4729: LukeRequestHandler: Using a dynamic copyField source that is
+  not also a dynamic field triggers error message 'undefined field: "(glob)"'.
+  (Adam Hahn, hossman, Steve Rowe)
+
+Optimizations
+----------------------
+
+Other Changes
+----------------------
+
+* SOLR-4653: Solr configuration should log inaccessible/ non-existent relative paths in lib 
+  dir=... (Dawid Weiss)
+
+* SOLR-4317: SolrTestCaseJ4: Can't avoid "collection1" convention (Tricia Jenkins, via Erick Erickson)
+
+* SOLR-4571: SolrZkClient#setData should return Stat object. (Mark Miller)
+
+* SOLR-4603: CachingDirectoryFactory should use an IdentityHashMap for 
+  byDirectoryCache. (Mark Miller)
+
+* SOLR-4544: Refactor HttpShardHandlerFactory so load-balancing logic can be customized.
+  (Ryan Ernst via Robert Muir)
+
+* SOLR-4607: Use noggit 0.5 release jar rather than a forked copy.  (Yonik Seeley, Robert Muir)
+  
+* SOLR-3706: Ship setup to log with log4j. (ryan, Mark Miller)
+
+* SOLR-4651: Remove dist-excl-slf4j build target. (Shawn Heisey)
+
+* SOLR-4622: The hardcoded SolrCloud defaults for 'hostContext="solr"' and 
+  'hostPort="8983"' have been deprecated and will be removed in Solr 5.0.  
+  Existing solr.xml files that do not have these options explicitly specified 
+  should be updated accordingly.  (hossman)
+
+* SOLR-4672: Requests attempting to use SolrCores which had init failures 
+  (that would be reported by CoreAdmin STATUS requests) now result in 500 
+  error responses with the details about the init failure, instead of 404 
+  error responses.  (hossman)
+
+* SOLR-4730: Make the wiki link more prominent in the release documentation.
+  (Uri Laserson via Robert Muir)
+  
+
+==================  4.2.1 ==================
+
+Versions of Major Components
+---------------------
+Apache Tika 1.3
+Carrot2 3.6.2
+Velocity 1.7 and Velocity Tools 2.0
+Apache UIMA 2.3.1
+Apache ZooKeeper 3.4.5
+
+Detailed Change List
+----------------------
+
+Bug Fixes
+----------------------
+
+* SOLR-4567: copyField source glob matching explicit field(s) stopped working
+  in Solr 4.2. (Alexandre Rafalovitch, Steve Rowe)
+  
+* SOLR-4475: Fix various places that still assume File based paths even when 
+  not using a file based DirectoryFactory. (Mark Miller)
+
+* SOLR-4551: CachingDirectoryFactory needs to create CacheEntry's with the 
+  fullpath not path. (Mark Miller)
+
+* SOLR-4555: When forceNew is used with CachingDirectoryFactory#get, the old
+  CachValue should give up it's path as it will be used by a new Directory
+  instance. (Mark Miller)
+
+* SOLR-4578: CoreAdminHandler#handleCreateAction gets a SolrCore and does not
+  close it in SolrCloud mode when a core with the same name already exists.
+  (Mark Miller)
+
+* SOLR-4574: The Collections API will silently return success on an unknown
+  ACTION parameter. (Mark Miller)
+
+* SOLR-4576: Collections API validation errors should cause an exception on
+  clients and otherwise act as validation errors with the Core Admin API.
+  (Mark Miller)
+
+* SOLR-4577: The collections API should return responses (success or failure)
+  for each node it attempts to work with. (Mark Miller)
+  
+* SOLR-4568: The lastPublished state check before becoming a leader is not 
+  working correctly. (Mark Miller)
+
+* SOLR-4570: Even if an explicit shard id is used, ZkController#preRegister 
+  should still wait to see the shard id in it's current ClusterState.
+  (Mark Miller)
+
+* SOLR-4585: The Collections API validates numShards with < 0 but should use 
+  <= 0. (Mark Miller)
+
+* SOLR-4592: DefaultSolrCoreState#doRecovery needs to check the CoreContainer
+  shutdown flag inside the recoveryLock sync block. (Mark Miller)
+
+* SOLR-4595: CachingDirectoryFactory#close can throw a concurrent 
+  modification exception. (Mark Miller)
+
+* SOLR-4573: Accessing Admin UI files in SolrCloud mode logs warnings.
+  (Mark Miller, Phil John)
+
+* SOLR-4594: StandardDirectoryFactory#remove accesses byDirectoryCache 
+  without a lock. (Mark Miller)
+
+* SOLR-4597: CachingDirectoryFactory#remove should not attempt to empty/remove 
+  the index right away but flag for removal after close. (Mark Miller)
+
+* SOLR-4598: The Core Admin unload command's option 'deleteDataDir', should use 
+  the DirectoryFactory API to remove the data dir. (Mark Miller)
+
+* SOLR-4599: CachingDirectoryFactory calls close(Directory) on forceNew if the 
+  Directory has a refCnt of 0, but it should call closeDirectory(CacheValue).
+  (Mark Miller)
+
+* SOLR-4602: ZkController#unregister should cancel it's election participation 
+  before asking the Overseer to delete the SolrCore information. (Mark Miller)
+
+* SOLR-4601: A Collection that is only partially created and then deleted will 
+  leave pre allocated shard information in ZooKeeper. (Mark Miller)
+  
+* SOLR-4604: UpdateLog#init is over called on SolrCore#reload. (Mark Miller)
+
+* SOLR-4605: Rollback does not work correctly. (Mark S, Mark Miller)
+
+* SOLR-4609: The Collections API should only send the reload command to ACTIVE 
+  cores. (Mark Miller)     
+           
+* SOLR-4297: Atomic update request containing null=true sets all subsequent
+  fields to null (Ben Pennell, Rob, shalin)
+  
+* SOLR-4371: Admin UI - Analysis Screen shows empty result (steffkes)
+
+* SOLR-4318: NPE encountered with querying with wildcards on a field that uses
+  the DefaultAnalyzer (i.e. no analysis chain defined). (Erick Erickson)
+
+* SOLR-4361: DataImportHandler would throw UnsupportedOperationException if
+  handler-level parameters were specified containing periods in the name
+  (James Dyer)
+
+* SOLR-4538: Date Math expressions were being truncated to 32 characters
+  when used in field:value queries in the lucene QParser.  (hossman, yonik)
+
+* SOLR-4617: SolrCore#reload needs to pass the deletion policy to the next 
+  SolrCore through it's constructor rather than setting a field after.
+  (Mark Miller)
+    
+* SOLR-4589: Fixed CPU spikes and poor performance in lazy field loading 
+  of multivalued fields. (hossman)
+
+* SOLR-4608: Update Log replay and PeerSync replay should use the default
+  processor chain to update the index. (Ludovic Boutros, yonik)
+
+* SOLR-4625: The solr (lucene syntax) query parser lost top-level boost
+  values and top-level phrase slops on queries produced by nested
+  sub-parsers. (yonik)
+
+* SOLR-4624: CachingDirectoryFactory does not need to support forceNew any 
+  longer and it appears to be causing a missing close directory bug. forceNew
+  is no longer respected and will be removed in 4.3. (Mark Miller)
+
+* SOLR-3819: Grouped faceting (group.facet=true) did not respect filter
+  exclusions. (Petter Remen, yonik)
+
+* SOLR-4637: Replication can sometimes wait until shutdown or core unload until
+  removing some tmp directories. (Mark Miller)
+  
+* SOLR-4638: DefaultSolrCoreState#getIndexWriter(null) is a way to avoid
+  creating the IndexWriter earlier than necessary, but it's not 
+  implemented quite right. (Mark Miller)
+
+* SOLR-4640: CachingDirectoryFactory can fail to close directories in some race
+  conditions. (Mark Miller)
+
+* SOLR-4642: QueryResultKey is not calculating the correct hashCode for filters.
+  (Joel Bernstein via Mark Miller)
+
+Optimizations
+----------------------
+
+* SOLR-4569: waitForReplicasToComeUp should bail right away if it doesn't see the
+  expected slice in the clusterstate rather than waiting. (Mark Miller)
+
+* SOLR-4311: Admin UI - Optimize Caching Behaviour (steffkes)
+      
+Other Changes
+----------------------
+
+* SOLR-4537: Clean up schema information REST API. (Steve Rowe)
+
+* SOLR-4596: DistributedQueue should ensure its full path exists in the constructor.
+  (Mark Miller)
+  
 ==================  4.2.0 ==================
 
 Versions of Major Components
@@ -218,6 +764,8 @@ Bug Fixes
   dynamic field references that aren't string-equal to the name of
   the referenced dynamic field. (Steve Rowe)
 
+* SOLR-4497: Collection Aliasing. (Mark Miller)
+
 Optimizations
 ----------------------
 
@@ -291,6 +839,9 @@ Other Changes
 
 * SOLR-4416: Upgrade to Tika 1.3. (Markus Jelsma via Mark Miller)
 
+* SOLR-4200: Reduce INFO level logging from CachingDirectoryFactory
+  (Shawn Heisey via hossman)
+
 ==================  4.1.0 ==================
 
 Versions of Major Components
@@ -877,7 +1428,7 @@ Other Changes
 
 * SOLR-4208: ExtendedDismaxQParserPlugin has been refactored to make 
   subclassing easier. (Tomás Fernández Löbbe, hossman)
-
+  
 * SOLR-3735: Relocate the example mime-to-extension mapping, and
   upgrade Velocity Engine to 1.7 (ehatcher)
 

Modified: lucene/dev/branches/security/solr/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/NOTICE.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/NOTICE.txt (original)
+++ lucene/dev/branches/security/solr/NOTICE.txt Thu May 30 07:53:18 2013
@@ -69,11 +69,6 @@ Jean-Philippe Barrette-LaPierre. This li
 see http://sites.google.com/site/rrettesite/moman and 
 http://bitbucket.org/jpbarrette/moman/overview/
 
-The class org.apache.lucene.util.SorterTemplate was inspired by CGLIB's class
-with the same name. The implementation part is mainly done using pre-existing
-Lucene sorting code. In-place stable mergesort was borrowed from CGLIB,
-which is Apache-licensed.
-
 The class org.apache.lucene.util.WeakIdentityMap was derived from
 the Apache CXF project and is Apache License 2.0.
 
@@ -151,8 +146,8 @@ LGPL and Creative Commons ShareAlike.
 Morfologic includes data from BSD-licensed dictionary of Polish (SGJP)
 (http://sgjp.pl/morfeusz/)
 
-Servlet-api.jar is under the CDDL license, the original source
-code for this can be found at http://www.eclipse.org/jetty/downloads.php
+Servlet-api.jar and javax.servlet-*.jar are under the CDDL license, the original
+source code for this can be found at http://www.eclipse.org/jetty/downloads.php
 
 ===========================================================================
 Kuromoji Japanese Morphological Analyzer - Apache Lucene Integration

Modified: lucene/dev/branches/security/solr/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/README.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/README.txt (original)
+++ lucene/dev/branches/security/solr/README.txt Thu May 30 07:53:18 2013
@@ -62,10 +62,10 @@ docs/index.html
 Instructions for Building Apache Solr from Source
 -------------------------------------------------
 
-1. Download the Java SE 6 JDK (Java Development Kit) or later from http://java.sun.com/
+1. Download the Java SE 7 JDK (Java Development Kit) or later from http://java.sun.com/
    You will need the JDK installed, and the $JAVA_HOME/bin (Windows: %JAVA_HOME%\bin) 
    folder included on your command path. To test this, issue a "java -version" command 
-   from your shell (command prompt) and verify that the Java version is 1.6 or later.
+   from your shell (command prompt) and verify that the Java version is 1.7 or later.
 
 2. Download the Apache Ant binary distribution (1.8.2+) from 
    http://ant.apache.org/  You will need Ant installed and the $ANT_HOME/bin (Windows: 

Modified: lucene/dev/branches/security/solr/SYSTEM_REQUIREMENTS.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/SYSTEM_REQUIREMENTS.txt?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/SYSTEM_REQUIREMENTS.txt (original)
+++ lucene/dev/branches/security/solr/SYSTEM_REQUIREMENTS.txt Thu May 30 07:53:18 2013
@@ -1,16 +1,13 @@
 # System Requirements 
 
-Apache Solr runs of Java 6 or greater. When using Java 7, be sure to 
+Apache Solr runs of Java 7 or greater. When using Java 7, be sure to 
 install at least Update 1! With all Java versions it is strongly 
 recommended to not use experimental `-XX` JVM options. It is also 
 recommended to always use the latest update version of your Java VM, 
 because bugs may affect Solr. An overview of known JVM bugs can be 
-found on http://wiki.apache.org/lucene-java/SunJavaBugs. 
+found on http://wiki.apache.org/lucene-java/JavaBugs. 
 
 CPU, disk and memory requirements are based on the many choices made in 
 implementing Solr (document size, number of documents, and number of 
 hits retrieved to name a few). The benchmarks page has some information 
 related to performance on particular platforms. 
-
-*To build Apache Solr from source, refer to the `BUILD.txt` file in 
-the distribution directory.* 

Modified: lucene/dev/branches/security/solr/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/build.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/build.xml (original)
+++ lucene/dev/branches/security/solr/build.xml Thu May 30 07:53:18 2013
@@ -59,7 +59,6 @@
   <target name="run-example" depends="example"
           description="Run Solr interactively, via Jetty.  -Dexample.debug=true to enable JVM debugger">
     <property name="example.solr.home" location="example/solr"/>
-    <property name="example.data.dir" location="example/solr/data"/>
     <property name="example.debug.suspend" value="n"/>
     <property name="example.jetty.port" value="8983"/>
     <condition property="example.jvm.line" value="-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=${example.debug.suspend},address=5005">
@@ -70,7 +69,6 @@
     <java jar="${example}/start.jar" fork="true" dir="${example}" maxmemory="${example.heap.size}">
       <jvmarg line="${example.jvm.line}"/>
       <sysproperty key="solr.solr.home" file="${example.solr.home}"/>
-      <sysproperty key="solr.data.dir" file="${example.data.dir}"/>
       <sysproperty key="jetty.port" value="${example.jetty.port}"/>
     </java>
   </target>
@@ -338,18 +336,7 @@
   
   <target name="dist"
           description="Creates the Solr distribution files."
-          depends="-dist-common, dist-war" />
- 
-  <target name="dist-excl-slf4j"
-          description="Creates the Solr distribution files without slf4j API or bindings."
-          depends="-dist-common, dist-war-excl-slf4j" />
- 
-  <target name="-dist-common"
-          depends="dist-solrj, dist-core, dist-test-framework, dist-contrib">
-    <!-- Targets common to dist and dist-excl-slf4j.
-         Only usable as a dependency
-    -->
-  </target>
+          depends="dist-solrj, dist-core, dist-test-framework, dist-contrib, dist-war" />
  
   <target name="dist-test-framework" depends="init-dist"
           description="Creates the Solr test-framework JAR.">
@@ -366,20 +353,10 @@
   <target name="dist-war"
           description="Creates the Solr WAR Distribution file.">
     <ant dir="webapp" target="dist" inheritall="false">
-      <property name="exclude.from.war" value="log4j-1.*" />
       <propertyset refid="uptodate.and.compiled.properties"/>
     </ant>
   </target>
   
-  <target name="dist-war-excl-slf4j"
-          description="Creates a Solr WAR Distribution file, excluding slf4j API and bindings.">
-    <ant dir="webapp" target="dist" inheritall="false">
-      <propertyset refid="uptodate.and.compiled.properties"/>
-      <property name="exclude.from.war" value="*slf4j*,log4j-*" />
-      <property name="solr.war.suffix" value="-excl-slf4j" />
-    </ant>
-  </target>
-
   <target name="prepare-release-no-sign" depends="clean, package, generate-maven-artifacts"/>
   <target name="prepare-release" depends="prepare-release-no-sign, sign-artifacts"/>
  
@@ -409,7 +386,7 @@
     <svn-export-source source.dir=".."/>
 
     <!-- Exclude javadoc package-list files under licenses incompatible with the ASL -->
-    <delete dir="${svn.export.dir}/lucene/tools/javadoc/java6"/>
+    <delete dir="${svn.export.dir}/lucene/tools/javadoc/java7"/>
     <!-- Exclude clover license files incompatible with the ASL -->
     <delete dir="${svn.export.dir}/lucene/tools/clover"/>
 
@@ -587,7 +564,7 @@
     <check-missing-javadocs dir="${javadoc.dir}" level="package"/>
   </target>
  
-  <target name="-ecj-javadoc-lint" depends="compile,compile-test,jar-test-framework,-ecj-resolve">
+  <target name="-ecj-javadoc-lint" depends="compile,compile-test,jar-test-framework,-ecj-javadoc-lint-unsupported,-ecj-resolve" if="ecj-javadoc-lint.supported">
     <subant target="-ecj-javadoc-lint" failonerror="true" inheritall="false">
       <propertyset refid="uptodate.and.compiled.properties"/>
       <fileset dir="core" includes="build.xml"/>

Modified: lucene/dev/branches/security/solr/cloud-dev/solrcloud-extzk-start.sh
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/cloud-dev/solrcloud-extzk-start.sh?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/cloud-dev/solrcloud-extzk-start.sh (original)
+++ lucene/dev/branches/security/solr/cloud-dev/solrcloud-extzk-start.sh Thu May 30 07:53:18 2013
@@ -24,7 +24,7 @@ cp -r -f example example4
 cp -r -f example example5
 cp -r -f example example6
 
-java -classpath lib/*:dist/*:build/lucene-libs/* org.apache.solr.cloud.ZkController "$zkaddress" 8983 example/solr/conf conf1
+java -classpath "example/solr-webapp/webapp/WEB-INF/lib/*:example/lib/ext/" org.apache.solr.cloud.ZkController "$zkaddress" 8983 example/solr/conf conf1
 
 cd example
 java -DzkHost="$zkaddress" -DnumShards=2 -DSTOP.PORT=7983 -DSTOP.KEY=key -jar start.jar 1>example.log 2>&1 &

Modified: lucene/dev/branches/security/solr/cloud-dev/solrcloud-multi-start.sh
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/cloud-dev/solrcloud-multi-start.sh?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/cloud-dev/solrcloud-multi-start.sh (original)
+++ lucene/dev/branches/security/solr/cloud-dev/solrcloud-multi-start.sh Thu May 30 07:53:18 2013
@@ -24,7 +24,7 @@ cp -r -f example example4
 cp -r -f example example5
 cp -r -f example example6
 
-java -classpath lib/*:dist/*:build/lucene-libs/* org.apache.solr.cloud.ZkCLI -cmd upconf -zkhost 127.0.0.1:9983 -solrhome example/multicore -runzk 8983
+java -classpath "example/solr-webapp/webapp/WEB-INF/lib/*:example/lib/ext/" org.apache.solr.cloud.ZkCLI -cmd upconf -zkhost 127.0.0.1:9983 -solrhome example/multicore -runzk 8983
 
 cd example
 java -DzkRun -DnumShards=2 -DSTOP.PORT=7983 -DSTOP.KEY=key -Dsolr.solr.home=multicore -jar start.jar 1>example.log 2>&1 &